From f1f4b6618334767d0da0f47965309b21676e7e9f Mon Sep 17 00:00:00 2001 From: Andrew Purtell Date: Mon, 30 Mar 2015 17:12:04 -0700 Subject: [PATCH] HBASE-12972 Region, a supportable public/evolving subset of HRegion --- .../org/apache/hadoop/hbase/Stoppable.java | 4 +- .../example/BulkDeleteEndpoint.java | 7 +- .../tmpl/regionserver/RegionListTmpl.jamon | 4 +- .../java/org/apache/hadoop/hbase/Server.java | 4 +- .../ZkSplitLogWorkerCoordination.java | 4 +- .../coprocessor/AggregateImplementation.java | 8 +- .../hbase/coprocessor/BaseRegionObserver.java | 6 +- .../coprocessor/BaseRegionServerObserver.java | 18 +- .../coprocessor/BaseRowProcessorEndpoint.java | 8 +- .../RegionCoprocessorEnvironment.java | 4 +- .../hbase/coprocessor/RegionObserver.java | 14 +- .../coprocessor/RegionServerObserver.java | 14 +- .../hadoop/hbase/master/RegionStateStore.java | 4 +- .../flush/FlushTableSubprocedure.java | 15 +- ...egionServerFlushTableProcedureManager.java | 6 +- .../quotas/RegionServerQuotaManager.java | 8 +- .../AnnotationReadingPriorityFunction.java | 5 +- .../regionserver/CompactSplitThread.java | 48 +- .../regionserver/CompactionRequestor.java | 14 +- .../ConstantSizeRegionSplitPolicy.java | 2 +- .../regionserver/FavoredNodesForRegion.java | 7 +- .../regionserver/FlushRequestListener.java | 2 +- .../hbase/regionserver/FlushRequester.java | 8 +- .../hadoop/hbase/regionserver/HRegion.java | 893 ++++++------------ .../hbase/regionserver/HRegionServer.java | 189 ++-- .../hbase/regionserver/HeapMemoryManager.java | 2 +- ...creasingToUpperBoundRegionSplitPolicy.java | 4 +- .../hadoop/hbase/regionserver/LogRoller.java | 2 +- .../hbase/regionserver/MemStoreFlusher.java | 96 +- .../MetricsRegionServerWrapperImpl.java | 22 +- .../hbase/regionserver/OnlineRegions.java | 27 +- .../hbase/regionserver/RSRpcServices.java | 145 +-- .../hadoop/hbase/regionserver/Region.java | 680 +++++++++++++ .../regionserver/RegionCoprocessorHost.java | 16 +- .../regionserver/RegionMergeRequest.java | 11 +- .../regionserver/RegionMergeTransaction.java | 35 +- .../regionserver/RegionServerServices.java | 13 +- .../hbase/regionserver/RegionSplitPolicy.java | 6 +- .../hbase/regionserver/SplitRequest.java | 18 +- .../hbase/regionserver/SplitTransaction.java | 36 +- .../hadoop/hbase/regionserver/Store.java | 7 +- .../regionserver/StorefileRefresherChore.java | 10 +- .../handler/CloseRegionHandler.java | 4 +- .../FinishRegionRecoveringHandler.java | 5 +- .../snapshot/FlushSnapshotSubprocedure.java | 16 +- .../snapshot/RegionServerSnapshotManager.java | 12 +- .../security/access/AccessControlLists.java | 7 +- .../security/access/AccessController.java | 34 +- .../access/SecureBulkLoadEndpoint.java | 8 +- .../DefaultVisibilityLabelServiceImpl.java | 9 +- .../visibility/VisibilityController.java | 8 +- .../security/visibility/VisibilityUtils.java | 4 +- .../hbase/snapshot/SnapshotManifest.java | 2 +- .../hbase/tool/WriteSinkCoprocessor.java | 2 +- .../org/apache/hadoop/hbase/util/HMerge.java | 15 +- .../org/apache/hadoop/hbase/util/Merge.java | 4 +- .../apache/hadoop/hbase/HBaseTestCase.java | 12 +- .../hadoop/hbase/HBaseTestingUtility.java | 35 +- .../apache/hadoop/hbase/MiniHBaseCluster.java | 30 +- .../hbase/MockRegionServerServices.java | 17 +- .../hadoop/hbase/TestGlobalMemStoreSize.java | 16 +- .../apache/hadoop/hbase/TestIOFencing.java | 8 +- .../hbase/backup/TestHFileArchiving.java | 31 +- .../TestZooKeeperTableArchiveClient.java | 16 +- .../hadoop/hbase/client/TestAdmin2.java | 9 +- .../hbase/client/TestClientPushback.java | 11 +- .../hbase/client/TestFromClientSide.java | 16 +- .../apache/hadoop/hbase/client/TestHCM.java | 6 +- ...ColumnAggregationEndpointNullResponse.java | 8 +- .../ColumnAggregationEndpointWithErrors.java | 6 +- .../coprocessor/SimpleRegionObserver.java | 10 +- .../hbase/coprocessor/TestClassLoading.java | 33 +- .../coprocessor/TestCoprocessorInterface.java | 52 +- .../TestRegionObserverScannerOpenHook.java | 17 +- .../coprocessor/TestRegionServerObserver.java | 22 +- .../hadoop/hbase/filter/TestFilter.java | 25 +- .../filter/TestInvocationRecordFilter.java | 11 +- .../hadoop/hbase/fs/TestBlockReorder.java | 11 +- .../hbase/io/encoding/TestEncodedSeekers.java | 13 +- .../hbase/io/encoding/TestPrefixTree.java | 8 +- .../hbase/io/hfile/TestCacheOnWrite.java | 9 +- .../hfile/TestForceCacheImportantBlocks.java | 8 +- .../TestScannerSelectionUsingKeyRange.java | 6 +- .../hfile/TestScannerSelectionUsingTTL.java | 12 +- .../TestImportTSVWithOperationAttributes.java | 9 +- .../mapreduce/TestImportTSVWithTTLs.java | 5 +- .../hadoop/hbase/master/MockRegionServer.java | 12 +- .../hbase/master/TestAssignmentListener.java | 8 +- .../master/TestDistributedLogSplitting.java | 7 +- .../master/TestGetLastFlushedSequenceId.java | 6 +- .../hbase/master/TestMasterFailover.java | 5 +- .../hbase/master/TestRegionPlacement.java | 11 +- .../hbase/master/TestTableLockManager.java | 2 +- .../hbase/namespace/TestNamespaceAuditor.java | 6 +- .../regionserver/NoOpScanPolicyObserver.java | 2 +- .../regionserver/TestAtomicOperation.java | 50 +- .../hbase/regionserver/TestBlocksRead.java | 26 +- .../hbase/regionserver/TestBlocksScanned.java | 5 +- .../hbase/regionserver/TestBulkLoad.java | 20 +- .../hbase/regionserver/TestColumnSeeking.java | 16 +- .../hbase/regionserver/TestCompaction.java | 6 +- .../regionserver/TestCompactionState.java | 10 +- .../regionserver/TestDefaultMemStore.java | 5 +- .../TestEncryptionKeyRotation.java | 5 +- .../TestEncryptionRandomKeying.java | 5 +- .../TestEndToEndSplitTransaction.java | 8 +- .../TestGetClosestAtOrBefore.java | 35 +- .../hbase/regionserver/TestHRegion.java | 173 ++-- .../regionserver/TestHRegionReplayEvents.java | 94 +- .../regionserver/TestHeapMemoryManager.java | 4 +- .../hbase/regionserver/TestKeepDeletes.java | 114 +-- .../regionserver/TestMajorCompaction.java | 65 +- .../hbase/regionserver/TestMinVersions.java | 36 +- .../regionserver/TestMinorCompaction.java | 12 +- .../regionserver/TestMultiColumnScanner.java | 4 +- .../TestPerColumnFamilyFlush.java | 80 +- .../regionserver/TestRegionFavoredNodes.java | 10 +- .../TestRegionMergeTransaction.java | 22 +- .../TestRegionReplicaFailover.java | 6 +- .../regionserver/TestRegionReplicas.java | 30 +- .../TestRegionServerNoMaster.java | 4 +- .../TestRegionServerOnlineConfigChange.java | 2 +- .../regionserver/TestRegionSplitPolicy.java | 20 +- .../regionserver/TestResettingCounters.java | 10 +- .../regionserver/TestReversibleScanners.java | 6 +- .../hbase/regionserver/TestRowTooBig.java | 10 +- .../regionserver/TestScanWithBloomError.java | 4 +- .../hbase/regionserver/TestScanner.java | 14 +- .../regionserver/TestSeekOptimizations.java | 4 +- .../regionserver/TestSplitTransaction.java | 20 +- .../TestSplitTransactionOnCluster.java | 43 +- .../TestStoreFileRefresherChore.java | 23 +- ...estCompactionWithThroughputController.java | 7 +- .../hbase/regionserver/wal/TestFSHLog.java | 3 +- .../regionserver/wal/TestLogRolling.java | 18 +- .../hbase/regionserver/wal/TestWALReplay.java | 45 +- .../TestMultiSlaveReplication.java | 2 +- .../TestRegionReplicaReplicationEndpoint.java | 15 +- ...ionReplicaReplicationEndpointNoMaster.java | 7 +- .../hbase/security/access/SecureTestUtil.java | 10 +- .../security/access/TestAccessController.java | 5 +- ...ExpAsStringVisibilityLabelServiceImpl.java | 4 +- .../visibility/TestVisibilityLabels.java | 8 +- .../hbase/snapshot/SnapshotTestingUtils.java | 6 +- .../hadoop/hbase/util/TestHBaseFsck.java | 4 +- .../hbase/util/TestHBaseFsckEncryption.java | 8 +- .../hadoop/hbase/util/TestMergeTable.java | 5 +- .../hadoop/hbase/wal/TestWALFiltering.java | 6 +- 148 files changed, 2325 insertions(+), 1916 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Stoppable.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Stoppable.java index b87b7649d34..fc83ba355d2 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Stoppable.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Stoppable.java @@ -19,11 +19,13 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; /** * Implementers are Stoppable. */ -@InterfaceAudience.Private +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving public interface Stoppable { /** * Stop this service. diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java index 90ef6372df8..e0c3baed7e0 100644 --- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java +++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java @@ -46,9 +46,9 @@ import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.Bu import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.OperationStatus; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.util.Bytes; @@ -113,7 +113,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor RpcCallback done) { long totalRowsDeleted = 0L; long totalVersionsDeleted = 0L; - HRegion region = env.getRegion(); + Region region = env.getRegion(); int rowBatchSize = request.getRowBatchSize(); Long timestamp = null; if (request.hasTimestamp()) { @@ -152,7 +152,8 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor for (List deleteRow : deleteRows) { deleteArr[i++] = createDeleteMutation(deleteRow, deleteType, timestamp); } - OperationStatus[] opStatus = region.batchMutate(deleteArr); + OperationStatus[] opStatus = region.batchMutate(deleteArr, HConstants.NO_NONCE, + HConstants.NO_NONCE); for (i = 0; i < opStatus.length; i++) { if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) { break; diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon index 6ca8ec6dd72..9d219f585d4 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon @@ -26,7 +26,7 @@ org.apache.hadoop.hbase.regionserver.HRegionServer; org.apache.hadoop.hbase.util.Bytes; org.apache.hadoop.hbase.HRegionInfo; - org.apache.hadoop.hbase.regionserver.HRegion; + org.apache.hadoop.hbase.regionserver.Region; org.apache.hadoop.hbase.ServerName; org.apache.hadoop.hbase.HBaseConfiguration; org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -254,7 +254,7 @@ <%for HRegionInfo r: onlineRegions %> <%java> - HRegion region = regionServer.getFromOnlineRegions(r.getEncodedName()); + Region region = regionServer.getFromOnlineRegions(r.getEncodedName()); MetricsRegionWrapper mWrap = region == null ? null: region.getMetrics().getRegionWrapper(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java index 85f847114df..365c0b845cb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; @@ -28,7 +29,8 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; * Defines the set of shared functions implemented by HBase servers (Masters * and RegionServers). */ -@InterfaceAudience.Private +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving public interface Server extends Abortable, Stoppable { /** * Gets the configuration object for this server. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java index 3fdfa5ce8ac..637920bd6cc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor; @@ -445,7 +445,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements taskReadyLock.wait(checkInterval); if (server != null) { // check to see if we have stale recovering regions in our internal memory state - Map recoveringRegions = server.getRecoveringRegions(); + Map recoveringRegions = server.getRecoveringRegions(); if (!recoveringRegions.isEmpty()) { // Make a local copy to prevent ConcurrentModificationException when other threads // modify recoveringRegions diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java index b5d16baa170..b6f834e8063 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/AggregateImplementation.java @@ -115,7 +115,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { } } log.info("Maximum from this region is " - + env.getRegion().getRegionNameAsString() + ": " + max); + + env.getRegion().getRegionInfo().getRegionNameAsString() + ": " + max); done.run(response); } @@ -168,7 +168,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { } } log.info("Minimum from this region is " - + env.getRegion().getRegionNameAsString() + ": " + min); + + env.getRegion().getRegionInfo().getRegionNameAsString() + ": " + min); done.run(response); } @@ -223,7 +223,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { } } log.debug("Sum from this region is " - + env.getRegion().getRegionNameAsString() + ": " + sum); + + env.getRegion().getRegionInfo().getRegionNameAsString() + ": " + sum); done.run(response); } @@ -274,7 +274,7 @@ extends AggregateService implements CoprocessorService, Coprocessor { } } log.info("Row counter from this region is " - + env.getRegion().getRegionNameAsString() + ": " + counter); + + env.getRegion().getRegionInfo().getRegionNameAsString() + ": " + counter); done.run(response); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java index 215ff16696b..9e0cb9b63d8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java @@ -42,11 +42,11 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.regionserver.DeleteTracker; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HRegion.Operation; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.Region.Operation; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; @@ -152,7 +152,7 @@ public abstract class BaseRegionObserver implements RegionObserver { } @Override - public void postSplit(ObserverContext e, HRegion l, HRegion r) + public void postSplit(ObserverContext e, Region l, Region r) throws IOException { } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionServerObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionServerObserver.java index 1f34f884e67..9fc130f66d0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionServerObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionServerObserver.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; /** @@ -48,28 +48,28 @@ public class BaseRegionServerObserver implements RegionServerObserver { public void stop(CoprocessorEnvironment env) throws IOException { } @Override - public void preMerge(ObserverContext ctx, HRegion regionA, - HRegion regionB) throws IOException { } + public void preMerge(ObserverContext ctx, Region regionA, + Region regionB) throws IOException { } @Override - public void postMerge(ObserverContext c, HRegion regionA, - HRegion regionB, HRegion mergedRegion) throws IOException { } + public void postMerge(ObserverContext c, Region regionA, + Region regionB, Region mergedRegion) throws IOException { } @Override public void preMergeCommit(ObserverContext ctx, - HRegion regionA, HRegion regionB, List metaEntries) throws IOException { } + Region regionA, Region regionB, List metaEntries) throws IOException { } @Override public void postMergeCommit(ObserverContext ctx, - HRegion regionA, HRegion regionB, HRegion mergedRegion) throws IOException { } + Region regionA, Region regionB, Region mergedRegion) throws IOException { } @Override public void preRollBackMerge(ObserverContext ctx, - HRegion regionA, HRegion regionB) throws IOException { } + Region regionA, Region regionB) throws IOException { } @Override public void postRollBackMerge(ObserverContext ctx, - HRegion regionA, HRegion regionB) throws IOException { } + Region regionA, Region regionB) throws IOException { } @Override public void preRollWALWriterRequest(ObserverContext ctx) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRowProcessorEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRowProcessorEndpoint.java index 7b841aab74a..ab5fc78c891 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRowProcessorEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRowProcessorEndpoint.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest; import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse; import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.RowProcessorService; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RowProcessor; import com.google.protobuf.ByteString; @@ -42,7 +42,7 @@ import com.google.protobuf.Service; /** * This class demonstrates how to implement atomic read-modify-writes - * using {@link HRegion#processRowsWithLocks} and Coprocessor endpoints. + * using {@link Region#processRowsWithLocks} and Coprocessor endpoints. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving @@ -50,7 +50,7 @@ public abstract class BaseRowProcessorEndpoint processor = constructRowProcessorFromRequest(request); - HRegion region = env.getRegion(); + Region region = env.getRegion(); long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE; long nonce = request.hasNonce() ? request.getNonce() : HConstants.NO_NONCE; region.processRowsWithLocks(processor, nonceGroup, nonce); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java index ccb16bf0148..a577748fc69 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java @@ -26,14 +26,14 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving public interface RegionCoprocessorEnvironment extends CoprocessorEnvironment { /** @return the region associated with this coprocessor */ - HRegion getRegion(); + Region getRegion(); /** @return region information for the region this coprocessor is running on */ HRegionInfo getRegionInfo(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index 9fede5258a5..7ee5a99451c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -42,8 +42,8 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.regionserver.DeleteTracker; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HRegion.Operation; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.Region.Operation; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; @@ -368,8 +368,8 @@ public interface RegionObserver extends Coprocessor { * @deprecated Use postCompleteSplit() instead */ @Deprecated - void postSplit(final ObserverContext c, final HRegion l, - final HRegion r) throws IOException; + void postSplit(final ObserverContext c, final Region l, + final Region r) throws IOException; /** * This will be called before PONR step as part of split transaction. Calling @@ -617,7 +617,7 @@ public interface RegionObserver extends Coprocessor { * called after acquiring the locks on the mutating rows and after applying the proper timestamp * for each Mutation at the server. The batch may contain Put/Delete. By setting OperationStatus * of Mutations ({@link MiniBatchOperationInProgress#setOperationStatus(int, OperationStatus)}), - * {@link RegionObserver} can make HRegion to skip these Mutations. + * {@link RegionObserver} can make Region to skip these Mutations. * @param c the environment provided by the region server * @param miniBatchOp batch of Mutations getting applied to region. * @throws IOException if an error occurred on the coprocessor @@ -637,7 +637,7 @@ public interface RegionObserver extends Coprocessor { /** * This will be called for region operations where read lock is acquired in - * {@link HRegion#startRegionOperation()}. + * {@link Region#startRegionOperation()}. * @param ctx * @param operation The operation is about to be taken on the region * @throws IOException @@ -646,7 +646,7 @@ public interface RegionObserver extends Coprocessor { Operation operation) throws IOException; /** - * Called after releasing read lock in {@link HRegion#closeRegionOperation(Operation)}. + * Called after releasing read lock in {@link Region#closeRegionOperation()}. * @param ctx * @param operation * @throws IOException diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java index 316bad50881..b1b94ff918e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.MetaMutationAnnotation; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; public interface RegionServerObserver extends Coprocessor { @@ -50,7 +50,7 @@ public interface RegionServerObserver extends Coprocessor { * @throws IOException */ void preMerge(final ObserverContext ctx, - final HRegion regionA, final HRegion regionB) throws IOException; + final Region regionA, final Region regionB) throws IOException; /** * called after the regions merge. @@ -61,7 +61,7 @@ public interface RegionServerObserver extends Coprocessor { * @throws IOException */ void postMerge(final ObserverContext c, - final HRegion regionA, final HRegion regionB, final HRegion mergedRegion) throws IOException; + final Region regionA, final Region regionB, final Region mergedRegion) throws IOException; /** * This will be called before PONR step as part of regions merge transaction. Calling @@ -74,7 +74,7 @@ public interface RegionServerObserver extends Coprocessor { * @throws IOException */ void preMergeCommit(final ObserverContext ctx, - final HRegion regionA, final HRegion regionB, + final Region regionA, final Region regionB, @MetaMutationAnnotation List metaEntries) throws IOException; /** @@ -86,7 +86,7 @@ public interface RegionServerObserver extends Coprocessor { * @throws IOException */ void postMergeCommit(final ObserverContext ctx, - final HRegion regionA, final HRegion regionB, final HRegion mergedRegion) throws IOException; + final Region regionA, final Region regionB, final Region mergedRegion) throws IOException; /** * This will be called before the roll back of the regions merge. @@ -96,7 +96,7 @@ public interface RegionServerObserver extends Coprocessor { * @throws IOException */ void preRollBackMerge(final ObserverContext ctx, - final HRegion regionA, final HRegion regionB) throws IOException; + final Region regionA, final Region regionB) throws IOException; /** * This will be called after the roll back of the regions merge. @@ -106,7 +106,7 @@ public interface RegionServerObserver extends Coprocessor { * @throws IOException */ void postRollBackMerge(final ObserverContext ctx, - final HRegion regionA, final HRegion regionB) throws IOException; + final Region regionA, final Region regionB) throws IOException; /** * This will be called before executing user request to roll a region server WAL. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java index df61b458d68..ba76115c3da 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.master.RegionState.State; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.MultiHConnection; @@ -56,7 +56,7 @@ public class RegionStateStore { /** The delimiter for meta columns for replicaIds > 0 */ protected static final char META_REPLICA_ID_DELIMITER = '_'; - private volatile HRegion metaRegion; + private volatile Region metaRegion; private volatile boolean initialized; private MultiHConnection multiHConnection; private final Server server; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java index d032ebadb43..8d64f2aafc7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java @@ -28,7 +28,8 @@ import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.procedure.ProcedureMember; import org.apache.hadoop.hbase.procedure.Subprocedure; import org.apache.hadoop.hbase.procedure.flush.RegionServerFlushTableProcedureManager.FlushTableSubprocedurePool; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.Region.Operation; /** * This flush region implementation uses the distributed procedure framework to flush @@ -40,12 +41,12 @@ public class FlushTableSubprocedure extends Subprocedure { private static final Log LOG = LogFactory.getLog(FlushTableSubprocedure.class); private final String table; - private final List regions; + private final List regions; private final FlushTableSubprocedurePool taskManager; public FlushTableSubprocedure(ProcedureMember member, ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout, - List regions, String table, + List regions, String table, FlushTableSubprocedurePool taskManager) { super(member, table, errorListener, wakeFrequency, timeout); this.table = table; @@ -54,8 +55,8 @@ public class FlushTableSubprocedure extends Subprocedure { } private static class RegionFlushTask implements Callable { - HRegion region; - RegionFlushTask(HRegion region) { + Region region; + RegionFlushTask(Region region) { this.region = region; } @@ -65,7 +66,7 @@ public class FlushTableSubprocedure extends Subprocedure { region.startRegionOperation(); try { LOG.debug("Flush region " + region.toString() + " started..."); - region.flushcache(); + region.flush(true); } finally { LOG.debug("Closing region operation on " + region); region.closeRegionOperation(); @@ -89,7 +90,7 @@ public class FlushTableSubprocedure extends Subprocedure { } // Add all hfiles already existing in region. - for (HRegion region : regions) { + for (Region region : regions) { // submit one task per region for parallelize by region. taskManager.submitTask(new RegionFlushTask(region)); monitor.rethrowException(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java index e6f074df43f..7664deedaca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java @@ -44,8 +44,8 @@ import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager; import org.apache.hadoop.hbase.procedure.Subprocedure; import org.apache.hadoop.hbase.procedure.SubprocedureFactory; import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -137,7 +137,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur } // check to see if this server is hosting any regions for the table - List involvedRegions; + List involvedRegions; try { involvedRegions = getRegionsToFlush(table); } catch (IOException e1) { @@ -172,7 +172,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur * @return the list of online regions. Empty list is returned if no regions. * @throws IOException */ - private List getRegionsToFlush(String table) throws IOException { + private List getRegionsToFlush(String table) throws IOException { return rss.getOnlineRegions(TableName.valueOf(table)); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerQuotaManager.java index 836025f87dc..a6c3e847837 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerQuotaManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/RegionServerQuotaManager.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.RpcScheduler; import org.apache.hadoop.hbase.ipc.RequestContext; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.security.UserGroupInformation; @@ -129,7 +129,7 @@ public class RegionServerQuotaManager { * @return the OperationQuota * @throws ThrottlingException if the operation cannot be executed due to quota exceeded. */ - public OperationQuota checkQuota(final HRegion region, + public OperationQuota checkQuota(final Region region, final OperationQuota.OperationType type) throws IOException, ThrottlingException { switch (type) { case SCAN: return checkQuota(region, 0, 0, 1); @@ -148,7 +148,7 @@ public class RegionServerQuotaManager { * @return the OperationQuota * @throws ThrottlingException if the operation cannot be executed due to quota exceeded. */ - public OperationQuota checkQuota(final HRegion region, + public OperationQuota checkQuota(final Region region, final List actions) throws IOException, ThrottlingException { int numWrites = 0; int numReads = 0; @@ -173,7 +173,7 @@ public class RegionServerQuotaManager { * @return the OperationQuota * @throws ThrottlingException if the operation cannot be executed due to quota exceeded. */ - private OperationQuota checkQuota(final HRegion region, + private OperationQuota checkQuota(final Region region, final int numWrites, final int numReads, final int numScans) throws IOException, ThrottlingException { UserGroupInformation ugi; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java index ddeabfa8190..29228db7c18 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java @@ -176,10 +176,11 @@ class AnnotationReadingPriorityFunction implements PriorityFunction { if (hasRegion != null && (Boolean)hasRegion.invoke(param, (Object[])null)) { Method getRegion = methodMap.get("getRegion").get(rpcArgClass); regionSpecifier = (RegionSpecifier)getRegion.invoke(param, (Object[])null); - HRegion region = rpcServices.getRegion(regionSpecifier); + Region region = rpcServices.getRegion(regionSpecifier); if (region.getRegionInfo().isSystemTable()) { if (LOG.isTraceEnabled()) { - LOG.trace("High priority because region=" + region.getRegionNameAsString()); + LOG.trace("High priority because region=" + + region.getRegionInfo().getRegionNameAsString()); } return HConstants.SYSTEMTABLE_QOS; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java index e891d007e6d..dbe4d0103d2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java @@ -218,8 +218,8 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi return queueLists.toString(); } - public synchronized void requestRegionsMerge(final HRegion a, - final HRegion b, final boolean forcible) { + public synchronized void requestRegionsMerge(final Region a, + final Region b, final boolean forcible) { try { mergePool.execute(new RegionMergeRequest(a, b, this.server, forcible)); if (LOG.isDebugEnabled()) { @@ -232,10 +232,10 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi } } - public synchronized boolean requestSplit(final HRegion r) { + public synchronized boolean requestSplit(final Region r) { // don't split regions that are blocking - if (shouldSplitRegion() && r.getCompactPriority() >= Store.PRIORITY_USER) { - byte[] midKey = r.checkSplit(); + if (shouldSplitRegion() && ((HRegion)r).getCompactPriority() >= Store.PRIORITY_USER) { + byte[] midKey = ((HRegion)r).checkSplit(); if (midKey != null) { requestSplit(r, midKey); return true; @@ -244,12 +244,12 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi return false; } - public synchronized void requestSplit(final HRegion r, byte[] midKey) { + public synchronized void requestSplit(final Region r, byte[] midKey) { if (midKey == null) { - LOG.debug("Region " + r.getRegionNameAsString() + + LOG.debug("Region " + r.getRegionInfo().getRegionNameAsString() + " not splittable because midkey=null"); - if (r.shouldForceSplit()) { - r.clearSplit(); + if (((HRegion)r).shouldForceSplit()) { + ((HRegion)r).clearSplit(); } return; } @@ -264,36 +264,36 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi } @Override - public synchronized List requestCompaction(final HRegion r, final String why) + public synchronized List requestCompaction(final Region r, final String why) throws IOException { return requestCompaction(r, why, null); } @Override - public synchronized List requestCompaction(final HRegion r, final String why, + public synchronized List requestCompaction(final Region r, final String why, List> requests) throws IOException { return requestCompaction(r, why, Store.NO_PRIORITY, requests); } @Override - public synchronized CompactionRequest requestCompaction(final HRegion r, final Store s, + public synchronized CompactionRequest requestCompaction(final Region r, final Store s, final String why, CompactionRequest request) throws IOException { return requestCompaction(r, s, why, Store.NO_PRIORITY, request); } @Override - public synchronized List requestCompaction(final HRegion r, final String why, + public synchronized List requestCompaction(final Region r, final String why, int p, List> requests) throws IOException { return requestCompactionInternal(r, why, p, requests, true); } - private List requestCompactionInternal(final HRegion r, final String why, + private List requestCompactionInternal(final Region r, final String why, int p, List> requests, boolean selectNow) throws IOException { // not a special compaction request, so make our own list List ret = null; if (requests == null) { ret = selectNow ? new ArrayList(r.getStores().size()) : null; - for (Store s : r.getStores().values()) { + for (Store s : r.getStores()) { CompactionRequest cr = requestCompactionInternal(r, s, why, p, null, selectNow); if (selectNow) ret.add(cr); } @@ -307,30 +307,30 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi return ret; } - public CompactionRequest requestCompaction(final HRegion r, final Store s, + public CompactionRequest requestCompaction(final Region r, final Store s, final String why, int priority, CompactionRequest request) throws IOException { return requestCompactionInternal(r, s, why, priority, request, true); } public synchronized void requestSystemCompaction( - final HRegion r, final String why) throws IOException { + final Region r, final String why) throws IOException { requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false); } public void requestSystemCompaction( - final HRegion r, final Store s, final String why) throws IOException { + final Region r, final Store s, final String why) throws IOException { requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false); } /** - * @param r HRegion store belongs to + * @param r region store belongs to * @param s Store to request compaction on * @param why Why compaction requested -- used in debug messages * @param priority override the default priority (NO_PRIORITY == decide) * @param request custom compaction request. Can be null in which case a simple * compaction will be used. */ - private synchronized CompactionRequest requestCompactionInternal(final HRegion r, final Store s, + private synchronized CompactionRequest requestCompactionInternal(final Region r, final Store s, final String why, int priority, CompactionRequest request, boolean selectNow) throws IOException { if (this.server.isStopped() @@ -358,12 +358,12 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi return selectNow ? compaction.getRequest() : null; } - private CompactionContext selectCompaction(final HRegion r, final Store s, + private CompactionContext selectCompaction(final Region r, final Store s, int priority, CompactionRequest request) throws IOException { CompactionContext compaction = s.requestCompaction(priority, request); if (compaction == null) { if(LOG.isDebugEnabled()) { - LOG.debug("Not compacting " + r.getRegionNameAsString() + + LOG.debug("Not compacting " + r.getRegionInfo().getRegionNameAsString() + " because compaction request was cancelled"); } return null; @@ -454,11 +454,11 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi private int queuedPriority; private ThreadPoolExecutor parent; - public CompactionRunner(Store store, HRegion region, + public CompactionRunner(Store store, Region region, CompactionContext compaction, ThreadPoolExecutor parent) { super(); this.store = store; - this.region = region; + this.region = (HRegion)region; this.compaction = compaction; this.queuedPriority = (this.compaction == null) ? store.getCompactPriority() : compaction.getRequest().getPriority(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java index 93a73e96970..d40b21d74b9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java @@ -34,7 +34,7 @@ public interface CompactionRequestor { * compactions were started * @throws IOException */ - List requestCompaction(final HRegion r, final String why) + List requestCompaction(final Region r, final String why) throws IOException; /** @@ -48,7 +48,7 @@ public interface CompactionRequestor { * @throws IOException */ List requestCompaction( - final HRegion r, final String why, List> requests + final Region r, final String why, List> requests ) throws IOException; @@ -56,13 +56,13 @@ public interface CompactionRequestor { * @param r Region to compact * @param s Store within region to compact * @param why Why compaction was requested -- used in debug messages - * @param request custom compaction request for the {@link HRegion} and {@link Store}. Custom + * @param request custom compaction request for the {@link Region} and {@link Store}. Custom * request must be null or be constructed with matching region and store. * @return The created {@link CompactionRequest} or null if no compaction was started. * @throws IOException */ CompactionRequest requestCompaction( - final HRegion r, final Store s, final String why, CompactionRequest request + final Region r, final Store s, final String why, CompactionRequest request ) throws IOException; /** @@ -77,7 +77,7 @@ public interface CompactionRequestor { * @throws IOException */ List requestCompaction( - final HRegion r, final String why, int pri, List> requests + final Region r, final String why, int pri, List> requests ) throws IOException; /** @@ -85,12 +85,12 @@ public interface CompactionRequestor { * @param s Store within region to compact * @param why Why compaction was requested -- used in debug messages * @param pri Priority of this compaction. minHeap. <=0 is critical - * @param request custom compaction request to run. {@link Store} and {@link HRegion} for the + * @param request custom compaction request to run. {@link Store} and {@link Region} for the * request must match the region and store specified here. * @return The created {@link CompactionRequest} or null if no compaction was started * @throws IOException */ CompactionRequest requestCompaction( - final HRegion r, final Store s, final String why, int pri, CompactionRequest request + final Region r, final Store s, final String why, int pri, CompactionRequest request ) throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java index fba5b2a2544..2459ae6a4e3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java @@ -55,7 +55,7 @@ public class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy { boolean force = region.shouldForceSplit(); boolean foundABigStore = false; - for (Store store : region.getStores().values()) { + for (Store store : region.getStores()) { // If any of the stores are unable to split (eg they contain reference files) // then don't split if ((!store.canSplit())) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FavoredNodesForRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FavoredNodesForRegion.java index d978a2d11f1..f516ecd8f01 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FavoredNodesForRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FavoredNodesForRegion.java @@ -21,15 +21,18 @@ package org.apache.hadoop.hbase.regionserver; import java.net.InetSocketAddress; import java.util.List; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName; /** * Abstraction that allows different modules in RegionServer to update/get * the favored nodes information for regions. */ -@InterfaceAudience.Private -interface FavoredNodesForRegion { +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving +public interface FavoredNodesForRegion { /** * Used to update the favored nodes mapping when required. * @param encodedRegionName diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequestListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequestListener.java index 80ac07ba24c..0e6bc4fb736 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequestListener.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequestListener.java @@ -32,5 +32,5 @@ public interface FlushRequestListener { * @param type The type of flush. (ie. Whether a normal flush or flush because of global heap preassure) * @param region The region for which flush is requested */ - void flushRequested(FlushType type, HRegion region); + void flushRequested(FlushType type, Region region); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java index 75174544877..c7e155a3bbc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java @@ -29,21 +29,21 @@ public interface FlushRequester { /** * Tell the listener the cache needs to be flushed. * - * @param region the HRegion requesting the cache flush + * @param region the Region requesting the cache flush * @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log * rolling. */ - void requestFlush(HRegion region, boolean forceFlushAllStores); + void requestFlush(Region region, boolean forceFlushAllStores); /** * Tell the listener the cache needs to be flushed after a delay * - * @param region the HRegion requesting the cache flush + * @param region the Region requesting the cache flush * @param delay after how much time should the flush happen * @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log * rolling. */ - void requestDelayedFlush(HRegion region, long delay, boolean forceFlushAllStores); + void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores); /** * Register a FlushRequestListener diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index bd25a8241c4..83127d2aa3e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -144,6 +144,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl.WriteEntry; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputControllerFactory; import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.ReplayHLogKey; @@ -189,44 +190,8 @@ import com.google.protobuf.RpcController; import com.google.protobuf.Service; import com.google.protobuf.TextFormat; -/** - * HRegion stores data for a certain region of a table. It stores all columns - * for each row. A given table consists of one or more HRegions. - * - *

We maintain multiple HStores for a single HRegion. - * - *

An Store is a set of rows with some column data; together, - * they make up all the data for the rows. - * - *

Each HRegion has a 'startKey' and 'endKey'. - *

The first is inclusive, the second is exclusive (except for - * the final region) The endKey of region 0 is the same as - * startKey for region 1 (if it exists). The startKey for the - * first region is null. The endKey for the final region is null. - * - *

Locking at the HRegion level serves only one purpose: preventing the - * region from being closed (and consequently split) while other operations - * are ongoing. Each row level operation obtains both a row lock and a region - * read lock for the duration of the operation. While a scanner is being - * constructed, getScanner holds a read lock. If the scanner is successfully - * constructed, it holds a read lock until it is closed. A close takes out a - * write lock and consequently will block for ongoing operations and will block - * new operations from starting while the close is in progress. - * - *

An HRegion is defined by its table and its key extent. - * - *

It consists of at least one Store. The number of Stores should be - * configurable, so that data which is accessed together is stored in the same - * Store. Right now, we approximate that by building a single Store for - * each column family. (This config info will be communicated via the - * tabledesc.) - * - *

The HTableDescriptor contains metainfo about the HRegion's table. - * regionName is a unique identifier for this HRegion. (startKey, endKey] - * defines the keyspace for this HRegion. - */ @InterfaceAudience.Private -public class HRegion implements HeapSize, PropagatingConfigurationObserver { // , Writable{ +public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region { public static final Log LOG = LogFactory.getLog(HRegion.class); public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY = @@ -278,17 +243,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // protected volatile long lastReplayedOpenRegionSeqId = -1L; protected volatile long lastReplayedCompactionSeqId = -1L; - /** - * Operation enum is used in {@link HRegion#startRegionOperation} to provide operation context for - * startRegionOperation to possibly invoke different checks before any region operations. Not all - * operations have to be defined here. It's only needed when a special check is need in - * startRegionOperation - */ - public enum Operation { - ANY, GET, PUT, DELETE, SCAN, APPEND, INCREMENT, SPLIT_REGION, MERGE_REGION, BATCH_MUTATE, - REPLAY_BATCH_MUTATE, COMPACT_REGION, REPLAY_EVENT - } - ////////////////////////////////////////////////////////////////////////////// // Members ////////////////////////////////////////////////////////////////////////////// @@ -324,19 +278,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // Number of requests blocked by memstore size. private final Counter blockedRequestsCount = new Counter(); - /** - * @return the number of blocked requests count. - */ - public long getBlockedRequestsCount() { - return this.blockedRequestsCount.get(); - } - // Compaction counters final AtomicLong compactionsFinished = new AtomicLong(0L); final AtomicLong compactionNumFilesCompacted = new AtomicLong(0L); final AtomicLong compactionNumBytesCompacted = new AtomicLong(0L); - private final WAL wal; private final HRegionFileSystem fs; protected final Configuration conf; @@ -425,6 +371,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } return minimumReadPoint; } + /* * Data structure of write state flags used coordinating flushes, * compactions and closes. @@ -476,17 +423,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * be specified if the flush was successful, and the failure message should only be specified * if it didn't flush. */ - public static class FlushResult { - enum Result { - FLUSHED_NO_COMPACTION_NEEDED, - FLUSHED_COMPACTION_NEEDED, - // Special case where a flush didn't run because there's nothing in the memstores. Used when - // bulk loading to know when we can still load even if a flush didn't happen. - CANNOT_FLUSH_MEMSTORE_EMPTY, - CANNOT_FLUSH - // Be careful adding more to this enum, look at the below methods to make sure - } - + public static class FlushResultImpl implements FlushResult { final Result result; final String failureReason; final long flushSequenceId; @@ -499,7 +436,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * @param flushSequenceId Generated sequence id that comes right after the edits in the * memstores. */ - FlushResult(Result result, long flushSequenceId) { + FlushResultImpl(Result result, long flushSequenceId) { this(result, flushSequenceId, null, false); assert result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result .FLUSHED_COMPACTION_NEEDED; @@ -510,7 +447,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * @param result Expecting CANNOT_FLUSH_MEMSTORE_EMPTY or CANNOT_FLUSH. * @param failureReason Reason why we couldn't flush. */ - FlushResult(Result result, String failureReason, boolean wroteFlushMarker) { + FlushResultImpl(Result result, String failureReason, boolean wroteFlushMarker) { this(result, -1, failureReason, wroteFlushMarker); assert result == Result.CANNOT_FLUSH_MEMSTORE_EMPTY || result == Result.CANNOT_FLUSH; } @@ -521,7 +458,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * @param flushSequenceId Generated sequence id if the memstores were flushed else -1. * @param failureReason Reason why we couldn't flush, or null. */ - FlushResult(Result result, long flushSequenceId, String failureReason, + FlushResultImpl(Result result, long flushSequenceId, String failureReason, boolean wroteFlushMarker) { this.result = result; this.flushSequenceId = flushSequenceId; @@ -554,6 +491,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // .append("failureReason:").append(failureReason).append(",") .append("flush seq id").append(flushSequenceId).toString(); } + + @Override + public Result getResult() { + return result; + } } /** A result object from prepare flush cache stage */ @@ -594,6 +536,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // this.flushedSeqId = flushedSeqId; this.totalFlushableSize = totalFlushableSize; } + + public FlushResult getResult() { + return this.result; + } } final WriteState writestate = new WriteState(); @@ -755,7 +701,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this); this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper); - Map recoveringRegions = rsServices.getRecoveringRegions(); + Map recoveringRegions = rsServices.getRecoveringRegions(); String encodedName = getRegionInfo().getEncodedName(); if (recoveringRegions != null && recoveringRegions.containsKey(encodedName)) { this.isRecovering = true; @@ -827,8 +773,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // nextSeqid will be -1 if the initialization fails. // At least it will be 0 otherwise. if (nextSeqId == -1) { - status - .abort("Exception during region " + this.getRegionNameAsString() + " initialization."); + status.abort("Exception during region " + getRegionInfo().getRegionNameAsString() + + " initialization."); } } } @@ -941,7 +887,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // for (int i = 0; i < htableDescriptor.getFamilies().size(); i++) { Future future = completionService.take(); HStore store = future.get(); - this.stores.put(store.getColumnFamilyName().getBytes(), store); + this.stores.put(store.getFamily().getName(), store); long storeMaxSequenceId = store.getMaxSequenceId(); maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), @@ -993,15 +939,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } private void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException { - Map> storeFiles - = new TreeMap>(Bytes.BYTES_COMPARATOR); - for (Map.Entry entry : getStores().entrySet()) { - Store store = entry.getValue(); + Map> storeFiles = new TreeMap>(Bytes.BYTES_COMPARATOR); + for (Store store: getStores()) { ArrayList storeFileNames = new ArrayList(); for (StoreFile storeFile: store.getStorefiles()) { storeFileNames.add(storeFile.getPath()); } - storeFiles.put(entry.getKey(), storeFileNames); + storeFiles.put(store.getFamily().getName(), storeFileNames); } RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor( @@ -1012,15 +956,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } private void writeRegionCloseMarker(WAL wal) throws IOException { - Map> storeFiles - = new TreeMap>(Bytes.BYTES_COMPARATOR); - for (Map.Entry entry : getStores().entrySet()) { - Store store = entry.getValue(); + Map> storeFiles = new TreeMap>(Bytes.BYTES_COMPARATOR); + for (Store store: getStores()) { ArrayList storeFileNames = new ArrayList(); for (StoreFile storeFile: store.getStorefiles()) { storeFileNames.add(storeFile.getPath()); } - storeFiles.put(entry.getKey(), storeFileNames); + storeFiles.put(store.getFamily().getName(), storeFileNames); } RegionEventDescriptor regionEventDesc = ProtobufUtil.toRegionEventDescriptor( @@ -1048,11 +990,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return false; } - /** - * This function will return the HDFS blocks distribution based on the data - * captured when HFile is created - * @return The HDFS blocks distribution for the region. - */ + @Override public HDFSBlocksDistribution getHDFSBlocksDistribution() { HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); @@ -1109,10 +1047,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return hdfsBlocksDistribution; } - public AtomicLong getMemstoreSize() { - return memstoreSize; - } - /** * Increase the size of mem store in this region and the size of global mem * store @@ -1125,7 +1059,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return this.memstoreSize.addAndGet(memStoreSize); } - /** @return a HRegionInfo object for this region */ + @Override public HRegionInfo getRegionInfo() { return this.fs.getRegionInfo(); } @@ -1138,32 +1072,76 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return this.rsServices; } - /** @return readRequestsCount for this region */ - long getReadRequestsCount() { - return this.readRequestsCount.get(); + @Override + public long getReadRequestsCount() { + return readRequestsCount.get(); } - /** @return writeRequestsCount for this region */ - long getWriteRequestsCount() { - return this.writeRequestsCount.get(); + @Override + public void updateReadRequestsCount(long i) { + readRequestsCount.add(i); } + @Override + public long getWriteRequestsCount() { + return writeRequestsCount.get(); + } + + @Override + public void updateWriteRequestsCount(long i) { + writeRequestsCount.add(i); + } + + @Override + public long getMemstoreSize() { + return memstoreSize.get(); + } + + @Override + public long getNumMutationsWithoutWAL() { + return numMutationsWithoutWAL.get(); + } + + @Override + public long getDataInMemoryWithoutWAL() { + return dataInMemoryWithoutWAL.get(); + } + + @Override + public long getBlockedRequestsCount() { + return blockedRequestsCount.get(); + } + + @Override + public long getCheckAndMutateChecksPassed() { + return checkAndMutateChecksPassed.get(); + } + + @Override + public long getCheckAndMutateChecksFailed() { + return checkAndMutateChecksFailed.get(); + } + + @Override public MetricsRegion getMetrics() { return metricsRegion; } - /** @return true if region is closed */ + @Override public boolean isClosed() { return this.closed.get(); } - /** - * @return True if closing process has started. - */ + @Override public boolean isClosing() { return this.closing.get(); } + @Override + public boolean isReadOnly() { + return this.writestate.isReadOnly(); + } + /** * Reset recovering state of current region */ @@ -1216,14 +1194,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - /** - * @return True if current region is in recovering - */ + @Override public boolean isRecovering() { return this.isRecovering; } - /** @return true if region is available (not closed and not closing) */ + @Override public boolean isAvailable() { return !isClosed() && !isClosing(); } @@ -1238,12 +1214,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // */ public boolean isMergeable() { if (!isAvailable()) { - LOG.debug("Region " + this.getRegionNameAsString() + LOG.debug("Region " + getRegionInfo().getRegionNameAsString() + " is not mergeable because it is closing or closed"); return false; } if (hasReferences()) { - LOG.debug("Region " + this.getRegionNameAsString() + LOG.debug("Region " + getRegionInfo().getRegionNameAsString() + " is not mergeable because it has references"); return false; } @@ -1261,9 +1237,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return mvcc; } - /* - * Returns readpoint considering given IsolationLevel - */ + @Override + public long getMaxFlushedSeqId() { + return maxFlushedSeqId; + } + + @Override public long getReadpoint(IsolationLevel isolationLevel) { if (isolationLevel == IsolationLevel.READ_UNCOMMITTED) { // This scan can read even uncommitted transactions @@ -1272,6 +1251,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return mvcc.memstoreReadPoint(); } + @Override public boolean isLoadingCfsOnDemandDefault() { return this.isLoadingCfsOnDemandDefault; } @@ -1370,7 +1350,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // the close flag? if (!abort && worthPreFlushing() && canFlush) { status.setStatus("Pre-flushing region before close"); - LOG.info("Running close preflush of " + this.getRegionNameAsString()); + LOG.info("Running close preflush of " + getRegionInfo().getRegionNameAsString()); try { internalFlushcache(status); } catch (IOException ioe) { @@ -1393,7 +1373,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // Don't flush the cache if we are aborting if (!abort && canFlush) { int flushCount = 0; - while (this.getMemstoreSize().get() > 0) { + while (this.memstoreSize.get() > 0) { try { if (flushCount++ > 0) { int actualFlushes = flushCount - 1; @@ -1401,7 +1381,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // If we tried 5 times and are unable to clear memory, abort // so we do not lose data throw new DroppedSnapshotException("Failed clearing memory after " + - actualFlushes + " attempts on region: " + Bytes.toStringBinary(getRegionName())); + actualFlushes + " attempts on region: " + + Bytes.toStringBinary(getRegionInfo().getRegionName())); } LOG.info("Running extra flush, " + actualFlushes + " (carrying snapshot?) " + this); @@ -1423,7 +1404,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // if (!stores.isEmpty()) { // initialize the thread pool for closing stores in parallel. ThreadPoolExecutor storeCloserThreadPool = - getStoreOpenAndCloseThreadPool("StoreCloserThread-" + this.getRegionNameAsString()); + getStoreOpenAndCloseThreadPool("StoreCloserThread-" + + getRegionInfo().getRegionNameAsString()); CompletionService>> completionService = new ExecutorCompletionService>>(storeCloserThreadPool); @@ -1488,11 +1470,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - /** - * Wait for all current flushes and compactions of the region to complete. - *

- * Exposed for TESTING. - */ + @Override public void waitForFlushesAndCompactions() { synchronized (writestate) { if (this.writestate.readOnly) { @@ -1565,32 +1543,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // HRegion accessors ////////////////////////////////////////////////////////////////////////////// - /** @return start key for region */ - public byte [] getStartKey() { - return this.getRegionInfo().getStartKey(); - } - - /** @return end key for region */ - public byte [] getEndKey() { - return this.getRegionInfo().getEndKey(); - } - - /** @return region id */ - public long getRegionId() { - return this.getRegionInfo().getRegionId(); - } - - /** @return region name */ - public byte [] getRegionName() { - return this.getRegionInfo().getRegionName(); - } - - /** @return region name as string for logging */ - public String getRegionNameAsString() { - return this.getRegionInfo().getRegionNameAsString(); - } - - /** @return HTableDescriptor for this region */ + @Override public HTableDescriptor getTableDesc() { return this.htableDescriptor; } @@ -1628,25 +1581,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return this.fs; } - /** - * @return Returns the earliest time a store in the region was flushed. All - * other stores in the region would have been flushed either at, or - * after this time. - */ - @VisibleForTesting + @Override public long getEarliestFlushTimeForAllStores() { return lastStoreFlushTimeMap.isEmpty() ? Long.MAX_VALUE : Collections.min(lastStoreFlushTimeMap .values()); } - /** - * This can be used to determine the last time all files of this region were major compacted. - * @param majorCompactioOnly Only consider HFile that are the result of major compaction - * @return the timestamp of the oldest HFile for all stores of this region - */ + @Override public long getOldestHfileTs(boolean majorCompactioOnly) throws IOException { long result = Long.MAX_VALUE; - for (Store store : getStores().values()) { + for (Store store : getStores()) { for (StoreFile file : store.getStorefiles()) { HFile.Reader reader = file.getReader().getHFileReader(); if (majorCompactioOnly) { @@ -1711,25 +1655,31 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // protected void doRegionCompactionPrep() throws IOException { } - void triggerMajorCompaction() { - for (Store h : stores.values()) { - h.triggerMajorCompaction(); + @Override + public void triggerMajorCompaction() throws IOException { + for (Store s : getStores()) { + s.triggerMajorCompaction(); } } - /** - * This is a helper function that compact all the stores synchronously - * It is used by utilities and testing - * - * @param majorCompaction True to force a major compaction regardless of thresholds - * @throws IOException e - */ - public void compactStores(final boolean majorCompaction) - throws IOException { + @Override + public void compact(final boolean majorCompaction) throws IOException { if (majorCompaction) { - this.triggerMajorCompaction(); + triggerMajorCompaction(); + } + for (Store s : getStores()) { + CompactionContext compaction = s.requestCompaction(); + if (compaction != null) { + CompactionThroughputController controller = null; + if (rsServices != null) { + controller = CompactionThroughputControllerFactory.create(rsServices, conf); + } + if (controller == null) { + controller = NoLimitCompactionThroughputController.INSTANCE; + } + compact(compaction, s, controller); + } } - compactStores(); } /** @@ -1739,7 +1689,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * @throws IOException e */ public void compactStores() throws IOException { - for (Store s : getStores().values()) { + for (Store s : getStores()) { CompactionContext compaction = s.requestCompaction(); if (compaction != null) { compact(compaction, s, NoLimitCompactionThroughputController.INSTANCE); @@ -1856,43 +1806,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - /** - * Flush all stores. - *

- * See {@link #flushcache(boolean)}. - * - * @return whether the flush is success and whether the region needs compacting - * @throws IOException - */ - public FlushResult flushcache() throws IOException { - return flushcache(true, false); + @Override + public FlushResult flush(boolean force) throws IOException { + return flushcache(force, false); } - /** - * Flush the cache. - * - * When this method is called the cache will be flushed unless: - *

    - *
  1. the cache is empty
  2. - *
  3. the region is closed.
  4. - *
  5. a flush is already in progress
  6. - *
  7. writes are disabled
  8. - *
- * - *

This method may block for some time, so it should not be called from a - * time-sensitive thread. - * @param forceFlushAllStores whether we want to flush all stores - * @return whether the flush is success and whether the region needs compacting - * - * @throws IOException general io exceptions - * @throws DroppedSnapshotException Thrown when replay of wal is required - * because a Snapshot was not properly persisted. - */ - public FlushResult flushcache(boolean forceFlushAllStores) throws IOException { - return flushcache(forceFlushAllStores, false); - } - - /** * Flush the cache. * @@ -1920,7 +1838,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // if (this.closing.get()) { String msg = "Skipping flush on " + this + " because closing"; LOG.debug(msg); - return new FlushResult(FlushResult.Result.CANNOT_FLUSH, msg, false); + return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); } MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this); status.setStatus("Acquiring readlock on region"); @@ -1931,7 +1849,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // String msg = "Skipping flush on " + this + " because closed"; LOG.debug(msg); status.abort(msg); - return new FlushResult(FlushResult.Result.CANNOT_FLUSH, msg, false); + return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); } if (coprocessorHost != null) { status.setStatus("Running coprocessor pre-flush hooks"); @@ -1956,7 +1874,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // + (writestate.flushing ? "already flushing" : "writes not enabled"); status.abort(msg); - return new FlushResult(FlushResult.Result.CANNOT_FLUSH, msg, false); + return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); } } @@ -2045,7 +1963,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } //since we didn't flush in the recent past, flush now if certain conditions //are met. Return true on first such memstore hit. - for (Store s : this.getStores().values()) { + for (Store s : getStores()) { if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) { // we have an old enough edit in the memstore, flush return true; @@ -2142,7 +2060,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // if (wal != null) { w = mvcc.beginMemstoreInsert(); long flushOpSeqId = getNextSequenceId(wal); - FlushResult flushResult = new FlushResult( + FlushResult flushResult = new FlushResultImpl( FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, flushOpSeqId, "Nothing to flush", writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker)); w.setWriteNumber(flushOpSeqId); @@ -2151,8 +2069,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return new PrepareFlushResult(flushResult, myseqid); } else { return new PrepareFlushResult( - new FlushResult(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, "Nothing to flush", - false), + new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, + "Nothing to flush", false), myseqid); } } @@ -2220,7 +2138,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // + this.getRegionInfo().getEncodedName() + "] - because the WAL is closing."; status.setStatus(msg); return new PrepareFlushResult( - new FlushResult(FlushResult.Result.CANNOT_FLUSH, msg, false), + new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false), myseqid); } flushOpSeqId = getNextSequenceId(wal); @@ -2407,7 +2325,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes()); } DroppedSnapshotException dse = new DroppedSnapshotException("region: " + - Bytes.toStringBinary(getRegionName())); + Bytes.toStringBinary(getRegionInfo().getRegionName())); dse.initCause(t); status.abort("Flush failed: " + StringUtils.stringifyException(t)); throw dse; @@ -2447,8 +2365,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // LOG.info(msg); status.setStatus(msg); - return new FlushResult(compactionRequested ? FlushResult.Result.FLUSHED_COMPACTION_NEEDED : - FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, flushOpSeqId); + return new FlushResultImpl(compactionRequested ? + FlushResult.Result.FLUSHED_COMPACTION_NEEDED : + FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, + flushOpSeqId); } /** @@ -2465,32 +2385,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // ////////////////////////////////////////////////////////////////////////////// // get() methods for client use. ////////////////////////////////////////////////////////////////////////////// - /** - * Return all the data for the row that matches row exactly, - * or the one that immediately preceeds it, at or immediately before - * ts. - * - * @param row row key - * @return map of values - * @throws IOException - */ - Result getClosestRowBefore(final byte [] row) - throws IOException{ - return getClosestRowBefore(row, HConstants.CATALOG_FAMILY); - } - /** - * Return all the data for the row that matches row exactly, - * or the one that immediately precedes it, at or immediately before - * ts. - * - * @param row row key - * @param family column family to find on - * @return map of values - * @throws IOException read exceptions - */ - public Result getClosestRowBefore(final byte [] row, final byte [] family) - throws IOException { + @Override + public Result getClosestRowBefore(final byte [] row, final byte [] family) throws IOException { if (coprocessorHost != null) { Result result = new Result(); if (coprocessorHost.preGetClosestRowBefore(row, family, result)) { @@ -2521,37 +2418,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - /** - * Return an iterator that scans over the HRegion, returning the indicated - * columns and rows specified by the {@link Scan}. - *

- * This Iterator must be closed by the caller. - * - * @param scan configured {@link Scan} - * @return RegionScanner - * @throws IOException read exceptions - */ + @Override public RegionScanner getScanner(Scan scan) throws IOException { return getScanner(scan, null); } - void prepareScanner(Scan scan) { - if(!scan.hasFamilies()) { - // Adding all families to scanner - for(byte[] family: this.htableDescriptor.getFamiliesKeys()){ - scan.addFamily(family); - } - } - } - protected RegionScanner getScanner(Scan scan, List additionalScanners) throws IOException { startRegionOperation(Operation.SCAN); try { // Verify families are all valid - prepareScanner(scan); - if(scan.hasFamilies()) { - for(byte [] family : scan.getFamilyMap().keySet()) { + if (!scan.hasFamilies()) { + // Adding all families to scanner + for (byte[] family: this.htableDescriptor.getFamiliesKeys()) { + scan.addFamily(family); + } + } else { + for (byte [] family : scan.getFamilyMap().keySet()) { checkFamily(family); } } @@ -2572,10 +2455,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return new RegionScannerImpl(scan, additionalScanners, this); } - /* - * @param delete The passed delete is modified by this method. WARNING! - */ - void prepareDelete(Delete delete) throws IOException { + @Override + public void prepareDelete(Delete delete) throws IOException { // Check to see if this is a deleteRow insert if(delete.getFamilyCellMap().isEmpty()){ for(byte [] family : this.htableDescriptor.getFamiliesKeys()){ @@ -2592,15 +2473,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - ////////////////////////////////////////////////////////////////////////////// - // set() methods for client use. - ////////////////////////////////////////////////////////////////////////////// - /** - * @param delete delete object - * @throws IOException read exceptions - */ - public void delete(Delete delete) - throws IOException { + @Override + public void delete(Delete delete) throws IOException { checkReadOnly(); checkResources(); startRegionOperation(Operation.DELETE); @@ -2617,6 +2491,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * Row needed by below method. */ private static final byte [] FOR_UNIT_TESTS_ONLY = Bytes.toBytes("ForUnitTestsOnly"); + /** * This is used only by unit tests. Not required to be a public API. * @param familyMap map of family to edits for the given family. @@ -2630,15 +2505,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // doBatchMutate(delete); } - /** - * Setup correct timestamps in the KVs in Delete object. - * Caller should have the row and region locks. - * @param mutation - * @param familyMap - * @param byteNow - * @throws IOException - */ - void prepareDeleteTimestamps(Mutation mutation, Map> familyMap, + @Override + public void prepareDeleteTimestamps(Mutation mutation, Map> familyMap, byte[] byteNow) throws IOException { for (Map.Entry> e : familyMap.entrySet()) { @@ -2698,11 +2566,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // CellUtil.setTimestamp(cell, getCell.getTimestamp()); } - /** - * @throws IOException - */ - public void put(Put put) - throws IOException { + @Override + public void put(Put put) throws IOException { checkReadOnly(); // Do a rough check that we have resources to accept a write. The check is @@ -2829,16 +2694,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - /** - * Perform a batch of mutations. - * It supports only Put and Delete mutations and will ignore other types passed. - * @param mutations the list of mutations - * @return an array of OperationStatus which internally contains the - * OperationStatusCode and the exceptionMessage if any. - * @throws IOException - */ - public OperationStatus[] batchMutate( - Mutation[] mutations, long nonceGroup, long nonce) throws IOException { + @Override + public OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce) + throws IOException { // As it stands, this is used for 3 things // * batchMutate with single mutation - put/delete, separate or from checkAndMutate. // * coprocessor calls (see ex. BulkDeleteEndpoint). @@ -2850,14 +2708,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE); } - /** - * Replay a batch of mutations. - * @param mutations mutations to replay. - * @param replaySeqId SeqId for current mutations - * @return an array of OperationStatus which internally contains the - * OperationStatusCode and the exceptionMessage if any. - * @throws IOException - */ + @Override public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId) throws IOException { if (!RegionReplicaUtil.isDefaultReplica(getRegionInfo()) @@ -3361,11 +3212,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // //the getting of the lock happens before, so that you would just pass it into //the methods. So in the case of checkAndMutate you could just do lockRow, //get, put, unlockRow or something - /** - * - * @throws IOException - * @return true if the new put was executed, false otherwise - */ + + @Override public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier, CompareOp compareOp, ByteArrayComparable comparator, Mutation w, boolean writeToWAL) @@ -3466,15 +3314,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // //the getting of the lock happens before, so that you would just pass it into //the methods. So in the case of checkAndMutate you could just do lockRow, //get, put, unlockRow or something - /** - * - * @throws IOException - * @return true if the new put was executed, false otherwise - */ + + @Override public boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier, CompareOp compareOp, ByteArrayComparable comparator, RowMutations rm, - boolean writeToWAL) - throws IOException{ + boolean writeToWAL) throws IOException { checkReadOnly(); //TODO, add check for value length or maybe even better move this to the //client if this becomes a global setting @@ -3545,10 +3389,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // closeRegionOperation(); } } + private void doBatchMutate(Mutation mutation) throws IOException { // Currently this is only called for puts and deletes, so no nonces. - OperationStatus[] batchMutate = this.batchMutate(new Mutation[] { mutation }, - HConstants.NO_NONCE, HConstants.NO_NONCE); + OperationStatus[] batchMutate = this.batchMutate(new Mutation[] { mutation }); if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) { throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg()); } else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) { @@ -3579,12 +3423,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // manifest.addRegion(this); } - /** - * Replaces any KV timestamps set to {@link HConstants#LATEST_TIMESTAMP} with the - * provided current timestamp. - * @throws IOException - */ - void updateCellTimestamps(final Iterable> cellItr, final byte[] now) + @Override + public void updateCellTimestamps(final Iterable> cellItr, final byte[] now) throws IOException { for (List cells: cellItr) { if (cells == null) continue; @@ -3676,7 +3516,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * @throws IOException Throws exception if region is in read-only mode. */ protected void checkReadOnly() throws IOException { - if (this.writestate.isReadOnly()) { + if (isReadOnly()) { throw new IOException("region is read only"); } } @@ -3767,12 +3607,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // LOG.debug("rollbackMemstore rolled back " + kvsRolledback); } - /** - * Check the collection of families for validity. - * @throws NoSuchColumnFamilyException if a family does not exist. - */ - void checkFamilies(Collection families) - throws NoSuchColumnFamilyException { + @Override + public void checkFamilies(Collection families) throws NoSuchColumnFamilyException { for (byte[] family : families) { checkFamily(family); } @@ -3802,8 +3638,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - void checkTimestamps(final Map> familyMap, - long now) throws FailedSanityCheckException { + @Override + public void checkTimestamps(final Map> familyMap, long now) + throws FailedSanityCheckException { if (timestampSlop == HConstants.LATEST_TIMESTAMP) { return; } @@ -3969,7 +3806,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // The edits size added into rsAccounting during this replaying will not // be required any more. So just clear it. if (this.rsAccounting != null) { - this.rsAccounting.clearRegionReplayEditsSize(this.getRegionName()); + this.rsAccounting.clearRegionReplayEditsSize(getRegionInfo().getRegionName()); } if (seqid > minSeqIdForTheRegion) { // Then we added some edits to memory. Flush and cleanup split edit files. @@ -4340,7 +4177,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } else { // special case empty memstore. We will still save the flush result in this case, since // our memstore ie empty, but the primary is still flushing - if (prepareResult.result.result == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) { + if (prepareResult.getResult().getResult() == + FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) { this.writestate.flushing = true; this.prepareFlushResult = prepareResult; if (LOG.isDebugEnabled()) { @@ -4852,15 +4690,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - /** - * Checks the underlying store files, and opens the files that have not - * been opened, and removes the store file readers for store files no longer - * available. Mainly used by secondary region replicas to keep up to date with - * the primary region files or open new flushed files and drop their memstore snapshots in case - * of memory pressure. - * @throws IOException - */ - boolean refreshStoreFiles() throws IOException { + @Override + public boolean refreshStoreFiles() throws IOException { if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { return false; // if primary nothing to do } @@ -4877,7 +4708,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // startRegionOperation(); // obtain region close lock try { synchronized (writestate) { - for (Store store : getStores().values()) { + for (Store store : getStores()) { // TODO: some stores might see new data from flush, while others do not which // MIGHT break atomic edits across column families. long maxSeqIdBefore = store.getMaxSequenceId(); @@ -4920,7 +4751,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // advance the mvcc read point so that the new flushed files are visible. // there may be some in-flight transactions, but they won't be made visible since they are // either greater than flush seq number or they were already picked up via flush. - for (Store s : getStores().values()) { + for (Store s : getStores()) { getMVCC().advanceMemstoreReadPointIfNeeded(s.getMaxMemstoreTS()); } @@ -4983,7 +4814,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // protected boolean restoreEdit(final Store s, final Cell cell) { long kvSize = s.add(cell).getFirst(); if (this.rsAccounting != null) { - rsAccounting.addAndGetRegionReplayEditsSize(this.getRegionName(), kvSize); + rsAccounting.addAndGetRegionReplayEditsSize(getRegionInfo().getRegionName(), kvSize); } return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize)); } @@ -5007,13 +4838,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return new HStore(this, family, this.conf); } - /** - * Return HStore instance. - * Use with caution. Exposed for use of fixup utilities. - * @param column Name of column family hosted by this region. - * @return Store that goes with the family on passed column. - * TODO: Make this lookup faster. - */ + @Override public Store getStore(final byte[] column) { return this.stores.get(column); } @@ -5034,17 +4859,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return null; } - public Map getStores() { - return this.stores; + @Override + public List getStores() { + List list = new ArrayList(stores.size()); + list.addAll(stores.values()); + return list; } - /** - * Return list of storeFiles for the set of CFs. - * Uses closeLock to prevent the race condition where a region closes - * in between the for loop - closing the stores one by one, some stores - * will return 0 files. - * @return List of storeFiles. - */ + @Override public List getStoreFileList(final byte [][] columns) throws IllegalArgumentException { List storeFileNames = new ArrayList(); @@ -5074,21 +4896,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // if (!rowIsInRange(getRegionInfo(), row)) { throw new WrongRegionException("Requested row out of range for " + op + " on HRegion " + this + ", startKey='" + - Bytes.toStringBinary(getStartKey()) + "', getEndKey()='" + - Bytes.toStringBinary(getEndKey()) + "', row='" + + Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" + + Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" + Bytes.toStringBinary(row) + "'"); } } - /** - * Tries to acquire a lock on the given row. - * @param waitForLock if true, will block until the lock is available. - * Otherwise, just tries to obtain the lock and returns - * false if unavailable. - * @return the row lock if acquired, - * null if waitForLock was false and the lock was not acquired - * @throws IOException if waitForLock was true and the lock could not be acquired after waiting - */ + @Override public RowLock getRowLock(byte[] row, boolean waitForLock) throws IOException { startRegionOperation(); try { @@ -5160,9 +4974,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return getRowLock(row, true); } - /** - * If the given list of row locks is not null, releases all locks. - */ + @Override public void releaseRowLocks(List rowLocks) { if (rowLocks != null) { for (RowLock rowLock : rowLocks) { @@ -5178,8 +4990,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * * @param familyPaths List of Pair */ - private static boolean hasMultipleColumnFamilies( - List> familyPaths) { + private static boolean hasMultipleColumnFamilies(Collection> familyPaths) { boolean multipleFamilies = false; byte[] family = null; for (Pair pair : familyPaths) { @@ -5194,36 +5005,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return multipleFamilies; } - /** - * Bulk load a/many HFiles into this region - * - * @param familyPaths A list which maps column families to the location of the HFile to load - * into that column family region. - * @param assignSeqId Force a flush, get it's sequenceId to preserve the guarantee that all the - * edits lower than the highest sequential ID from all the HFiles are flushed - * on disk. - * @return true if successful, false if failed recoverably - * @throws IOException if failed unrecoverably. - */ - public boolean bulkLoadHFiles(List> familyPaths, - boolean assignSeqId) throws IOException { - return bulkLoadHFiles(familyPaths, assignSeqId, null); - } - - /** - * Attempts to atomically load a group of hfiles. This is critical for loading - * rows with multiple column families atomically. - * - * @param familyPaths List of Pair - * @param bulkLoadListener Internal hooks enabling massaging/preparation of a - * file about to be bulk loaded - * @param assignSeqId Force a flush, get it's sequenceId to preserve the guarantee that - * all the edits lower than the highest sequential ID from all the - * HFiles are flushed on disk. - * @return true if successful, false if failed recoverably - * @throws IOException if failed unrecoverably. - */ - public boolean bulkLoadHFiles(List> familyPaths, boolean assignSeqId, + @Override + public boolean bulkLoadHFiles(Collection> familyPaths, boolean assignSeqId, BulkLoadListener bulkLoadListener) throws IOException { long seqId = -1; Map> storeFiles = new TreeMap>(Bytes.BYTES_COMPARATOR); @@ -5286,14 +5069,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // guaranteed to be one beyond the file made when we flushed (or if nothing to flush, it is // a sequence id that we can be sure is beyond the last hfile written). if (assignSeqId) { - FlushResult fs = this.flushcache(); + FlushResult fs = flushcache(true, false); if (fs.isFlushSucceeded()) { - seqId = fs.flushSequenceId; - } else if (fs.result == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) { - seqId = fs.flushSequenceId; + seqId = ((FlushResultImpl)fs).flushSequenceId; + } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) { + seqId = ((FlushResultImpl)fs).flushSequenceId; } else { - throw new IOException("Could not bulk load with an assigned sequential ID because the " + - "flush didn't run. Reason for not flushing: " + fs.failureReason); + throw new IOException("Could not bulk load with an assigned sequential ID because the "+ + "flush didn't run. Reason for not flushing: " + ((FlushResultImpl)fs).failureReason); } } @@ -5362,18 +5145,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // @Override public boolean equals(Object o) { - return o instanceof HRegion && Bytes.equals(this.getRegionName(), - ((HRegion) o).getRegionName()); + return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(), + ((HRegion) o).getRegionInfo().getRegionName()); } @Override public int hashCode() { - return Bytes.hashCode(this.getRegionName()); + return Bytes.hashCode(getRegionInfo().getRegionName()); } @Override public String toString() { - return this.getRegionNameAsString(); + return getRegionInfo().getRegionNameAsString(); } /** @@ -5703,7 +5486,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // long afterTime = rpcCall.disconnectSince(); if (afterTime >= 0) { throw new CallerDisconnectedException( - "Aborting on region " + getRegionNameAsString() + ", call " + + "Aborting on region " + getRegionInfo().getRegionNameAsString() + ", call " + this + " after " + afterTime + " ms, since " + "caller disconnected"); } @@ -6210,6 +5993,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return r.openHRegion(reporter); } + public static Region openHRegion(final Region other, final CancelableProgressable reporter) + throws IOException { + return openHRegion((HRegion)other, reporter); + } + /** * Open HRegion. * Calls initialize and sets sequenceId. @@ -6335,7 +6123,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // public static void addRegionToMETA(final HRegion meta, final HRegion r) throws IOException { meta.checkResources(); // The row key is the region name - byte[] row = r.getRegionName(); + byte[] row = r.getRegionInfo().getRegionName(); final long now = EnvironmentEdgeManager.currentTime(); final List cells = new ArrayList(2); cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY, @@ -6402,18 +6190,20 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // Make sure that srcA comes first; important for key-ordering during // write of the merged file. - if (srcA.getStartKey() == null) { - if (srcB.getStartKey() == null) { + if (srcA.getRegionInfo().getStartKey() == null) { + if (srcB.getRegionInfo().getStartKey() == null) { throw new IOException("Cannot merge two regions with null start key"); } // A's start key is null but B's isn't. Assume A comes before B - } else if ((srcB.getStartKey() == null) || - (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) { + } else if ((srcB.getRegionInfo().getStartKey() == null) || + (Bytes.compareTo(srcA.getRegionInfo().getStartKey(), + srcB.getRegionInfo().getStartKey()) > 0)) { a = srcB; b = srcA; } - if (!(Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0)) { + if (!(Bytes.compareTo(a.getRegionInfo().getEndKey(), + b.getRegionInfo().getStartKey()) == 0)) { throw new IOException("Cannot merge non-adjacent regions"); } return merge(a, b); @@ -6434,16 +6224,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // FileSystem fs = a.getRegionFileSystem().getFileSystem(); // Make sure each region's cache is empty - a.flushcache(); - b.flushcache(); + a.flush(true); + b.flush(true); // Compact each region so we only have one store file per family - a.compactStores(true); + a.compact(true); if (LOG.isDebugEnabled()) { LOG.debug("Files for region: " + a); a.getRegionFileSystem().logFileSystemState(LOG); } - b.compactStores(true); + b.compact(true); if (LOG.isDebugEnabled()) { LOG.debug("Files for region: " + b); b.getRegionFileSystem().logFileSystemState(LOG); @@ -6468,7 +6258,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // throw new IOException("Failed merging region " + a + " and " + b + ", and successfully rolled back"); } - dstRegion.compactStores(true); + dstRegion.compact(true); if (LOG.isDebugEnabled()) { LOG.debug("Files for new region"); @@ -6489,14 +6279,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return dstRegion; } - // - // HBASE-880 - // - /** - * @param get get object - * @return result - * @throws IOException read exceptions - */ + @Override public Result get(final Get get) throws IOException { checkRow(get.getRow(), "Get"); // Verify families are all valid @@ -6514,13 +6297,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale); } - /* - * Do a get based on the get parameter. - * @param withCoprocessor invoke coprocessor or not. We don't want to - * always invoke cp for this private method. - */ - public List get(Get get, boolean withCoprocessor) - throws IOException { + @Override + public List get(Get get, boolean withCoprocessor) throws IOException { List results = new ArrayList(); @@ -6605,27 +6383,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return stats.build(); } - /** - * Performs atomic multiple reads and writes on a given row. - * - * @param processor The object defines the reads and writes to a row. - * @param nonceGroup Optional nonce group of the operation (client Id) - * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") - */ + @Override + public void processRowsWithLocks(RowProcessor processor) throws IOException { + processRowsWithLocks(processor, rowProcessorTimeout, HConstants.NO_NONCE, + HConstants.NO_NONCE); + } + + @Override public void processRowsWithLocks(RowProcessor processor, long nonceGroup, long nonce) throws IOException { processRowsWithLocks(processor, rowProcessorTimeout, nonceGroup, nonce); } - /** - * Performs atomic multiple reads and writes on a given row. - * - * @param processor The object defines the reads and writes to a row. - * @param timeout The timeout of the processor.process() execution - * Use a negative number to switch off the time bound - * @param nonceGroup Optional nonce group of the operation (client Id) - * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") - */ + @Override public void processRowsWithLocks(RowProcessor processor, long timeout, long nonceGroup, long nonce) throws IOException { @@ -6838,14 +6608,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // TODO: There's a lot of boiler plate code identical to increment. // We should refactor append and increment as local get-mutate-put // transactions, so all stores only go through one code path for puts. - /** - * Perform one or more append operations on a row. - * - * @return new keyvalues after increment - * @throws IOException - */ - public Result append(Append append, long nonceGroup, long nonce) - throws IOException { + + @Override + public Result append(Append append, long nonceGroup, long nonce) throws IOException { byte[] row = append.getRow(); checkRow(row, "append"); boolean flush = false; @@ -7103,11 +6868,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // TODO: There's a lot of boiler plate code identical to append. // We should refactor append and increment as local get-mutate-put // transactions, so all stores only go through one code path for puts. - /** - * Perform one or more increment operations on a row. - * @return new keyvalues after increment - * @throws IOException - */ + + @Override public Result increment(Increment increment, long nonceGroup, long nonce) throws IOException { byte [] row = increment.getRow(); @@ -7405,22 +7167,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // System.exit(1); } - /** - * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to - * be available for handling - * {@link HRegion#execService(com.google.protobuf.RpcController, - * org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall)}} calls. - * - *

- * Only a single instance may be registered per region for a given {@link Service} subclass (the - * instances are keyed on {@link com.google.protobuf.Descriptors.ServiceDescriptor#getFullName()}. - * After the first registration, subsequent calls with the same service name will fail with - * a return value of {@code false}. - *

- * @param instance the {@code Service} subclass instance to expose as a coprocessor endpoint - * @return {@code true} if the registration was successful, {@code false} - * otherwise - */ + @Override public boolean registerService(Service instance) { /* * No stacking of instances is allowed for a single service name @@ -7435,26 +7182,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // coprocessorServiceHandlers.put(serviceDesc.getFullName(), instance); if (LOG.isDebugEnabled()) { - LOG.debug("Registered coprocessor service: region="+ - Bytes.toStringBinary(getRegionName())+" service="+serviceDesc.getFullName()); + LOG.debug("Registered coprocessor service: region=" + + Bytes.toStringBinary(getRegionInfo().getRegionName()) + + " service=" + serviceDesc.getFullName()); } return true; } - /** - * Executes a single protocol buffer coprocessor endpoint {@link Service} method using - * the registered protocol handlers. {@link Service} implementations must be registered via the - * {@link HRegion#registerService(com.google.protobuf.Service)} - * method before they are available. - * - * @param controller an {@code RpcController} implementation to pass to the invoked service - * @param call a {@code CoprocessorServiceCall} instance identifying the service, method, - * and parameters for the method invocation - * @return a protocol buffer {@code Message} instance containing the method's result - * @throws IOException if no registered service handler is found or an error - * occurs during the invocation - * @see org.apache.hadoop.hbase.regionserver.HRegion#registerService(com.google.protobuf.Service) - */ + @Override public Message execService(RpcController controller, CoprocessorServiceCall call) throws IOException { String serviceName = call.getServiceName(); @@ -7462,7 +7197,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // if (!coprocessorServiceHandlers.containsKey(serviceName)) { throw new UnknownProtocolException(null, "No registered coprocessor service found for name "+serviceName+ - " in region "+Bytes.toStringBinary(getRegionName())); + " in region "+Bytes.toStringBinary(getRegionInfo().getRegionName())); } Service service = coprocessorServiceHandlers.get(serviceName); @@ -7471,7 +7206,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // if (methodDesc == null) { throw new UnknownProtocolException(service.getClass(), "Unknown method "+methodName+" called on service "+serviceName+ - " in region "+Bytes.toStringBinary(getRegionName())); + " in region "+Bytes.toStringBinary(getRegionInfo().getRegionName())); } Message request = service.getRequestPrototype(methodDesc).newBuilderForType() @@ -7523,7 +7258,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // try { region.initialize(null); if (majorCompact) { - region.compactStores(true); + region.compact(true); } else { // Default behavior Scan scan = new Scan(); @@ -7636,22 +7371,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // this.coprocessorHost = coprocessorHost; } - /** - * This method needs to be called before any public call that reads or - * modifies data. It has to be called just before a try. - * #closeRegionOperation needs to be called in the try's finally block - * Acquires a read lock and checks if the region is closing or closed. - * @throws IOException - */ + @Override public void startRegionOperation() throws IOException { startRegionOperation(Operation.ANY); } - /** - * @param op The operation is about to be taken on the region - * @throws IOException - */ - protected void startRegionOperation(Operation op) throws IOException { + @Override + public void startRegionOperation(Operation op) throws IOException { switch (op) { case GET: // read operations case SCAN: @@ -7667,7 +7393,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // when a region is in recovering state, no read, split or merge is allowed if (isRecovering() && (this.disallowWritesInRecovering || (op != Operation.PUT && op != Operation.DELETE && op != Operation.BATCH_MUTATE))) { - throw new RegionInRecoveryException(this.getRegionNameAsString() + + throw new RegionInRecoveryException(getRegionInfo().getRegionNameAsString() + " is recovering; cannot take reads"); } break; @@ -7681,12 +7407,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return; } if (this.closing.get()) { - throw new NotServingRegionException(getRegionNameAsString() + " is closing"); + throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing"); } lock(lock.readLock()); if (this.closed.get()) { lock.readLock().unlock(); - throw new NotServingRegionException(getRegionNameAsString() + " is closed"); + throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed"); } try { if (coprocessorHost != null) { @@ -7698,11 +7424,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - /** - * Closes the lock. This needs to be called in the finally block corresponding - * to the try block of #startRegionOperation - * @throws IOException - */ + @Override public void closeRegionOperation() throws IOException { closeRegionOperation(Operation.ANY); } @@ -7731,14 +7453,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // private void startBulkRegionOperation(boolean writeLockNeeded) throws NotServingRegionException, RegionTooBusyException, InterruptedIOException { if (this.closing.get()) { - throw new NotServingRegionException(getRegionNameAsString() + " is closing"); + throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing"); } if (writeLockNeeded) lock(lock.writeLock()); else lock(lock.readLock()); if (this.closed.get()) { if (writeLockNeeded) lock.writeLock().unlock(); else lock.readLock().unlock(); - throw new NotServingRegionException(getRegionNameAsString() + " is closed"); + throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed"); } } @@ -7911,30 +7633,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - /** - * Gets the latest sequence number that was read from storage when this region was opened. - */ + @Override public long getOpenSeqNum() { return this.openSeqNum; } - /** - * Gets max sequence ids of stores that was read from storage when this region was opened. WAL - * Edits with smaller or equal sequence number will be skipped from replay. - */ - public Map getMaxStoreSeqIdForLogReplay() { + @Override + public Map getMaxStoreSeqId() { return this.maxSeqIdInStores; } - @VisibleForTesting + @Override public long getOldestSeqIdOfStore(byte[] familyName) { return wal.getEarliestMemstoreSeqNum(getRegionInfo() .getEncodedNameAsBytes(), familyName); } - /** - * @return if a given region is in compaction now. - */ + @Override public CompactionState getCompactionState() { boolean hasMajor = majorInProgress.get() > 0, hasMinor = minorInProgress.get() > 0; return (hasMajor ? (hasMinor ? CompactionState.MAJOR_AND_MINOR : CompactionState.MAJOR) @@ -7973,39 +7688,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // this.sequenceId.set(value); } - /** - * Listener class to enable callers of - * bulkLoadHFile() to perform any necessary - * pre/post processing of a given bulkload call - */ - public interface BulkLoadListener { - - /** - * Called before an HFile is actually loaded - * @param family family being loaded to - * @param srcPath path of HFile - * @return final path to be used for actual loading - * @throws IOException - */ - String prepareBulkLoad(byte[] family, String srcPath) throws IOException; - - /** - * Called after a successful HFile load - * @param family family being loaded to - * @param srcPath path of HFile - * @throws IOException - */ - void doneBulkLoad(byte[] family, String srcPath) throws IOException; - - /** - * Called after a failed HFile load - * @param family family being loaded to - * @param srcPath path of HFile - * @throws IOException - */ - void failedBulkLoad(byte[] family, String srcPath) throws IOException; - } - @VisibleForTesting class RowLockContext { private final HashedBytes row; private final CountDownLatch latch = new CountDownLatch(1); @@ -8023,7 +7705,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // RowLock newLock() { lockCount++; - return new RowLock(this); + RowLockImpl rl = new RowLockImpl(); + rl.setContext(this); + return rl; } @Override @@ -8051,29 +7735,26 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } - /** - * Row lock held by a given thread. - * One thread may acquire multiple locks on the same row simultaneously. - * The locks must be released by calling release() from the same thread. - */ - public static class RowLock { - @VisibleForTesting final RowLockContext context; + public static class RowLockImpl implements RowLock { + private RowLockContext context; private boolean released = false; - @VisibleForTesting RowLock(RowLockContext context) { + @VisibleForTesting + public RowLockContext getContext() { + return context; + } + + @VisibleForTesting + public void setContext(RowLockContext context) { this.context = context; } - /** - * Release the given lock. If there are no remaining locks held by the current thread - * then unlock the row and allow other threads to acquire the lock. - * @throws IllegalArgumentException if called by a different thread than the lock owning thread - */ + @Override public void release() { if (!released) { context.releaseLock(); - released = true; } + released = true; } } @@ -8097,16 +7778,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return key; } - /** - * Explicitly sync wal - * @throws IOException - */ - public void syncWal() throws IOException { - if(this.wal != null) { - this.wal.sync(); - } - } - /** * {@inheritDoc} */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 34c417d1f9f..a618a329201 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -253,8 +253,7 @@ public class HRegionServer extends HasThread implements * Map of regions currently being served by this region server. Key is the * encoded region name. All access should be synchronized. */ - protected final Map onlineRegions = - new ConcurrentHashMap(); + protected final Map onlineRegions = new ConcurrentHashMap(); /** * Map of encoded region names to the DataNode locations they should be hosted on @@ -272,8 +271,8 @@ public class HRegionServer extends HasThread implements * Set of regions currently being in recovering state which means it can accept writes(edits from * previous failed region server) but not reads. A recovering region is also an online region. */ - protected final Map recoveringRegions = Collections - .synchronizedMap(new HashMap()); + protected final Map recoveringRegions = Collections + .synchronizedMap(new HashMap()); // Leases protected Leases leases; @@ -1073,7 +1072,7 @@ public class HRegionServer extends HasThread implements private boolean areAllUserRegionsOffline() { if (getNumberOfOnlineRegions() > 2) return false; boolean allUserRegionsOffline = true; - for (Map.Entry e: this.onlineRegions.entrySet()) { + for (Map.Entry e: this.onlineRegions.entrySet()) { if (!e.getValue().getRegionInfo().isMetaTable()) { allUserRegionsOffline = false; break; @@ -1087,7 +1086,7 @@ public class HRegionServer extends HasThread implements */ private long getWriteRequestCount() { int writeCount = 0; - for (Map.Entry e: this.onlineRegions.entrySet()) { + for (Map.Entry e: this.onlineRegions.entrySet()) { writeCount += e.getValue().getWriteRequestsCount(); } return writeCount; @@ -1133,10 +1132,9 @@ public class HRegionServer extends HasThread implements // Instead they should be stored in an HBase table so that external visibility into HBase is // improved; Additionally the load balancer will be able to take advantage of a more complete // history. - MetricsRegionServerWrapper regionServerWrapper = this.metricsRegionServer.getRegionServerWrapper(); - Collection regions = getOnlineRegionsLocalContext(); - MemoryUsage memory = - ManagementFactory.getMemoryMXBean().getHeapMemoryUsage(); + MetricsRegionServerWrapper regionServerWrapper = metricsRegionServer.getRegionServerWrapper(); + Collection regions = getOnlineRegionsLocalContext(); + MemoryUsage memory = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage(); ClusterStatusProtos.ServerLoad.Builder serverLoad = ClusterStatusProtos.ServerLoad.newBuilder(); @@ -1151,7 +1149,7 @@ public class HRegionServer extends HasThread implements } RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder(); RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder(); - for (HRegion region : regions) { + for (Region region : regions) { serverLoad.addRegionLoads(createRegionLoad(region, regionLoadBldr, regionSpecifier)); for (String coprocessor : getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors()) { @@ -1186,7 +1184,7 @@ public class HRegionServer extends HasThread implements String getOnlineRegionsAsPrintableString() { StringBuilder sb = new StringBuilder(); - for (HRegion r: this.onlineRegions.values()) { + for (Region r: this.onlineRegions.values()) { if (sb.length() > 0) sb.append(", "); sb.append(r.getRegionInfo().getEncodedName()); } @@ -1222,7 +1220,7 @@ public class HRegionServer extends HasThread implements // Ensure all user regions have been sent a close. Use this to // protect against the case where an open comes in after we start the // iterator of onlineRegions to close all user regions. - for (Map.Entry e : this.onlineRegions.entrySet()) { + for (Map.Entry e : this.onlineRegions.entrySet()) { HRegionInfo hri = e.getValue().getRegionInfo(); if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes()) && !closedRegions.contains(hri.getEncodedName())) { @@ -1378,44 +1376,37 @@ public class HRegionServer extends HasThread implements * * @throws IOException */ - private RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, + private RegionLoad createRegionLoad(final Region r, RegionLoad.Builder regionLoadBldr, RegionSpecifier.Builder regionSpecifier) throws IOException { - byte[] name = r.getRegionName(); + byte[] name = r.getRegionInfo().getRegionName(); int stores = 0; int storefiles = 0; int storeUncompressedSizeMB = 0; int storefileSizeMB = 0; - int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024); + int memstoreSizeMB = (int) (r.getMemstoreSize() / 1024 / 1024); int storefileIndexSizeMB = 0; int rootIndexSizeKB = 0; int totalStaticIndexSizeKB = 0; int totalStaticBloomSizeKB = 0; long totalCompactingKVs = 0; long currentCompactedKVs = 0; - synchronized (r.stores) { - stores += r.stores.size(); - for (Store store : r.stores.values()) { - storefiles += store.getStorefilesCount(); - storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed() - / 1024 / 1024); - storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024); - storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024); - CompactionProgress progress = store.getCompactionProgress(); - if (progress != null) { - totalCompactingKVs += progress.totalCompactingKVs; - currentCompactedKVs += progress.currentCompactedKVs; - } - - rootIndexSizeKB += - (int) (store.getStorefilesIndexSize() / 1024); - - totalStaticIndexSizeKB += - (int) (store.getTotalStaticIndexSize() / 1024); - - totalStaticBloomSizeKB += - (int) (store.getTotalStaticBloomSize() / 1024); + List storeList = r.getStores(); + stores += storeList.size(); + for (Store store : storeList) { + storefiles += store.getStorefilesCount(); + storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed() / 1024 / 1024); + storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024); + storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024); + CompactionProgress progress = store.getCompactionProgress(); + if (progress != null) { + totalCompactingKVs += progress.totalCompactingKVs; + currentCompactedKVs += progress.currentCompactedKVs; } + rootIndexSizeKB += (int) (store.getStorefilesIndexSize() / 1024); + totalStaticIndexSizeKB += (int) (store.getTotalStaticIndexSize() / 1024); + totalStaticBloomSizeKB += (int) (store.getTotalStaticBloomSize() / 1024); } + float dataLocality = r.getHDFSBlocksDistribution().getBlockLocalityIndex(serverName.getHostname()); if (regionLoadBldr == null) { @@ -1426,8 +1417,7 @@ public class HRegionServer extends HasThread implements } regionSpecifier.setType(RegionSpecifierType.REGION_NAME); regionSpecifier.setValue(ByteStringer.wrap(name)); - r.setCompleteSequenceId(regionLoadBldr) - .setRegionSpecifier(regionSpecifier.build()) + regionLoadBldr.setRegionSpecifier(regionSpecifier.build()) .setStores(stores) .setStorefiles(storefiles) .setStoreUncompressedSizeMB(storeUncompressedSizeMB) @@ -1437,12 +1427,13 @@ public class HRegionServer extends HasThread implements .setRootIndexSizeKB(rootIndexSizeKB) .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB) .setTotalStaticBloomSizeKB(totalStaticBloomSizeKB) - .setReadRequestsCount(r.readRequestsCount.get()) - .setWriteRequestsCount(r.writeRequestsCount.get()) + .setReadRequestsCount(r.getReadRequestsCount()) + .setWriteRequestsCount(r.getWriteRequestsCount()) .setTotalCompactingKVs(totalCompactingKVs) .setCurrentCompactedKVs(currentCompactedKVs) .setDataLocality(dataLocality) .setLastMajorCompactionTs(r.getOldestHfileTs(true)); + ((HRegion)r).setCompleteSequenceId(regionLoadBldr); return regionLoadBldr.build(); } @@ -1452,8 +1443,7 @@ public class HRegionServer extends HasThread implements * @return An instance of RegionLoad. */ public RegionLoad createRegionLoad(final String encodedRegionName) throws IOException { - HRegion r = null; - r = this.onlineRegions.get(encodedRegionName); + Region r = onlineRegions.get(encodedRegionName); return r != null ? createRegionLoad(r, null, null) : null; } @@ -1482,10 +1472,10 @@ public class HRegionServer extends HasThread implements @Override protected void chore() { - for (HRegion r : this.instance.onlineRegions.values()) { + for (Region r : this.instance.onlineRegions.values()) { if (r == null) continue; - for (Store s : r.getStores().values()) { + for (Store s : r.getStores()) { try { long multiplier = s.getCompactionCheckMultiplier(); assert multiplier > 0; @@ -1496,7 +1486,7 @@ public class HRegionServer extends HasThread implements + " requests compaction"); } else if (s.isMajorCompaction()) { if (majorCompactPriority == DEFAULT_PRIORITY - || majorCompactPriority > r.getCompactPriority()) { + || majorCompactPriority > ((HRegion)r).getCompactPriority()) { this.instance.compactSplitThread.requestCompaction(r, s, getName() + " requests major compaction; use default priority", null); } else { @@ -1525,15 +1515,15 @@ public class HRegionServer extends HasThread implements @Override protected void chore() { - for (HRegion r : this.server.onlineRegions.values()) { + for (Region r : this.server.onlineRegions.values()) { if (r == null) continue; - if (r.shouldFlush()) { + if (((HRegion)r).shouldFlush()) { FlushRequester requester = server.getFlushRequester(); if (requester != null) { long randomDelay = RandomUtils.nextInt(RANGE_OF_DELAY) + MIN_DELAY_TIME; - LOG.info(getName() + " requesting flush for region " + r.getRegionNameAsString() + - " after a delay of " + randomDelay); + LOG.info(getName() + " requesting flush for region " + + r.getRegionInfo().getRegionNameAsString() + " after a delay of " + randomDelay); //Throttle the flushes by putting a delay. If we don't throttle, and there //is a balanced write-load on the regions in a table, we might end up //overwhelming the filesystem with too many flushes at once. @@ -1839,12 +1829,12 @@ public class HRegionServer extends HasThread implements } @Override - public void postOpenDeployTasks(final HRegion r) - throws KeeperException, IOException { + public void postOpenDeployTasks(final Region r) throws KeeperException, IOException { + Preconditions.checkArgument(r instanceof HRegion, "r must be an HRegion"); rpcServices.checkOpen(); - LOG.info("Post open deploy tasks for " + r.getRegionNameAsString()); + LOG.info("Post open deploy tasks for " + r.getRegionInfo().getRegionNameAsString()); // Do checks to see if we need to compact (references or too many files) - for (Store s : r.getStores().values()) { + for (Store s : r.getStores()) { if (s.hasReferences() || s.needsCompaction()) { this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region"); } @@ -1852,7 +1842,8 @@ public class HRegionServer extends HasThread implements long openSeqNum = r.getOpenSeqNum(); if (openSeqNum == HConstants.NO_SEQNUM) { // If we opened a region, we should have read some sequence number from it. - LOG.error("No sequence number found when opening " + r.getRegionNameAsString()); + LOG.error("No sequence number found when opening " + + r.getRegionInfo().getRegionNameAsString()); openSeqNum = 0; } @@ -1863,12 +1854,12 @@ public class HRegionServer extends HasThread implements if (!reportRegionStateTransition( TransitionCode.OPENED, openSeqNum, r.getRegionInfo())) { throw new IOException("Failed to report opened region to master: " - + r.getRegionNameAsString()); + + r.getRegionInfo().getRegionNameAsString()); } - triggerFlushInPrimaryRegion(r); + triggerFlushInPrimaryRegion((HRegion)r); - LOG.debug("Finished post open deploy task for " + r.getRegionNameAsString()); + LOG.debug("Finished post open deploy task for " + r.getRegionInfo().getRegionNameAsString()); } @Override @@ -2272,10 +2263,10 @@ public class HRegionServer extends HasThread implements * @param abort Whether we're running an abort. */ void closeMetaTableRegions(final boolean abort) { - HRegion meta = null; + Region meta = null; this.lock.writeLock().lock(); try { - for (Map.Entry e: onlineRegions.entrySet()) { + for (Map.Entry e: onlineRegions.entrySet()) { HRegionInfo hri = e.getValue().getRegionInfo(); if (hri.isMetaRegion()) { meta = e.getValue(); @@ -2297,8 +2288,8 @@ public class HRegionServer extends HasThread implements void closeUserRegions(final boolean abort) { this.lock.writeLock().lock(); try { - for (Map.Entry e: this.onlineRegions.entrySet()) { - HRegion r = e.getValue(); + for (Map.Entry e: this.onlineRegions.entrySet()) { + Region r = e.getValue(); if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) { // Don't update zk with this close transition; pass false. closeRegionIgnoreErrors(r.getRegionInfo(), abort); @@ -2328,7 +2319,7 @@ public class HRegionServer extends HasThread implements } @Override - public Map getRecoveringRegions() { + public Map getRecoveringRegions() { return this.recoveringRegions; } @@ -2359,13 +2350,13 @@ public class HRegionServer extends HasThread implements * This method will only work if HRegionServer is in the same JVM as client; * HRegion cannot be serialized to cross an rpc. */ - public Collection getOnlineRegionsLocalContext() { - Collection regions = this.onlineRegions.values(); + public Collection getOnlineRegionsLocalContext() { + Collection regions = this.onlineRegions.values(); return Collections.unmodifiableCollection(regions); } @Override - public void addToOnlineRegions(HRegion region) { + public void addToOnlineRegions(Region region) { this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region); configurationManager.registerObserver(region); } @@ -2375,9 +2366,9 @@ public class HRegionServer extends HasThread implements * biggest. If two regions are the same size, then the last one found wins; i.e. this method * may NOT return all regions. */ - SortedMap getCopyOfOnlineRegionsSortedBySize() { + SortedMap getCopyOfOnlineRegionsSortedBySize() { // we'll sort the regions in reverse - SortedMap sortedRegions = new TreeMap( + SortedMap sortedRegions = new TreeMap( new Comparator() { @Override public int compare(Long a, Long b) { @@ -2385,8 +2376,8 @@ public class HRegionServer extends HasThread implements } }); // Copy over all regions. Regions are sorted by size with biggest first. - for (HRegion region : this.onlineRegions.values()) { - sortedRegions.put(region.memstoreSize.get(), region); + for (Region region : this.onlineRegions.values()) { + sortedRegions.put(region.getMemstoreSize(), region); } return sortedRegions; } @@ -2412,7 +2403,7 @@ public class HRegionServer extends HasThread implements */ protected HRegionInfo[] getMostLoadedRegions() { ArrayList regions = new ArrayList(); - for (HRegion r : onlineRegions.values()) { + for (Region r : onlineRegions.values()) { if (!r.isAvailable()) { continue; } @@ -2608,10 +2599,10 @@ public class HRegionServer extends HasThread implements * @return Online regions from tableName */ @Override - public List getOnlineRegions(TableName tableName) { - List tableRegions = new ArrayList(); + public List getOnlineRegions(TableName tableName) { + List tableRegions = new ArrayList(); synchronized (this.onlineRegions) { - for (HRegion region: this.onlineRegions.values()) { + for (Region region: this.onlineRegions.values()) { HRegionInfo regionInfo = region.getRegionInfo(); if(regionInfo.getTable().equals(tableName)) { tableRegions.add(region); @@ -2630,7 +2621,7 @@ public class HRegionServer extends HasThread implements public Set getOnlineTables() { Set tables = new HashSet(); synchronized (this.onlineRegions) { - for (HRegion region: this.onlineRegions.values()) { + for (Region region: this.onlineRegions.values()) { tables.add(region.getTableDesc().getTableName()); } } @@ -2647,8 +2638,8 @@ public class HRegionServer extends HasThread implements "skipping."); LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception); } - Collection regions = getOnlineRegionsLocalContext(); - for (HRegion region: regions) { + Collection regions = getOnlineRegionsLocalContext(); + for (Region region: regions) { coprocessors.addAll(region.getCoprocessorHost().getCoprocessors()); try { coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors()); @@ -2698,7 +2689,7 @@ public class HRegionServer extends HasThread implements protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn) throws NotServingRegionException { //Check for permissions to close. - HRegion actualRegion = this.getFromOnlineRegions(encodedName); + Region actualRegion = this.getFromOnlineRegions(encodedName); if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) { try { actualRegion.getCoprocessorHost().preClose(false); @@ -2759,7 +2750,7 @@ public class HRegionServer extends HasThread implements * @return HRegion for the passed binary regionName or null if * named region is not member of the online regions. */ - public HRegion getOnlineRegion(final byte[] regionName) { + public Region getOnlineRegion(final byte[] regionName) { String encodedRegionName = HRegionInfo.encodeRegionName(regionName); return this.onlineRegions.get(encodedRegionName); } @@ -2769,14 +2760,14 @@ public class HRegionServer extends HasThread implements } @Override - public HRegion getFromOnlineRegions(final String encodedRegionName) { + public Region getFromOnlineRegions(final String encodedRegionName) { return this.onlineRegions.get(encodedRegionName); } @Override - public boolean removeFromOnlineRegions(final HRegion r, ServerName destination) { - HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName()); + public boolean removeFromOnlineRegions(final Region r, ServerName destination) { + Region toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName()); if (destination != null) { try { @@ -2808,20 +2799,20 @@ public class HRegionServer extends HasThread implements * @return {@link HRegion} for regionName * @throws NotServingRegionException */ - protected HRegion getRegion(final byte[] regionName) + protected Region getRegion(final byte[] regionName) throws NotServingRegionException { String encodedRegionName = HRegionInfo.encodeRegionName(regionName); return getRegionByEncodedName(regionName, encodedRegionName); } - public HRegion getRegionByEncodedName(String encodedRegionName) + public Region getRegionByEncodedName(String encodedRegionName) throws NotServingRegionException { return getRegionByEncodedName(null, encodedRegionName); } - protected HRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName) + protected Region getRegionByEncodedName(byte[] regionName, String encodedRegionName) throws NotServingRegionException { - HRegion region = this.onlineRegions.get(encodedRegionName); + Region region = this.onlineRegions.get(encodedRegionName); if (region == null) { MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName); if (moveInfo != null) { @@ -3077,17 +3068,17 @@ public class HRegionServer extends HasThread implements * @throws KeeperException * @throws IOException */ - private void updateRecoveringRegionLastFlushedSequenceId(HRegion r) throws KeeperException, + private void updateRecoveringRegionLastFlushedSequenceId(Region r) throws KeeperException, IOException { if (!r.isRecovering()) { // return immdiately for non-recovering regions return; } - HRegionInfo region = r.getRegionInfo(); + HRegionInfo regionInfo = r.getRegionInfo(); ZooKeeperWatcher zkw = getZooKeeper(); - String previousRSName = this.getLastFailedRSFromZK(region.getEncodedName()); - Map maxSeqIdInStores = r.getMaxStoreSeqIdForLogReplay(); + String previousRSName = this.getLastFailedRSFromZK(regionInfo.getEncodedName()); + Map maxSeqIdInStores = r.getMaxStoreSeqId(); long minSeqIdForLogReplay = -1; for (Long storeSeqIdForReplay : maxSeqIdInStores.values()) { if (minSeqIdForLogReplay == -1 || storeSeqIdForReplay < minSeqIdForLogReplay) { @@ -3098,7 +3089,7 @@ public class HRegionServer extends HasThread implements try { long lastRecordedFlushedSequenceId = -1; String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode, - region.getEncodedName()); + regionInfo.getEncodedName()); // recovering-region level byte[] data; try { @@ -3107,7 +3098,7 @@ public class HRegionServer extends HasThread implements throw new InterruptedIOException(); } if (data != null) { - lastRecordedFlushedSequenceId = ZKSplitLog.parseLastFlushedSequenceIdFrom(data); + lastRecordedFlushedSequenceId = ZKSplitLog.parseLastFlushedSequenceIdFrom(data); } if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) { ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay)); @@ -3117,14 +3108,14 @@ public class HRegionServer extends HasThread implements nodePath = ZKUtil.joinZNode(nodePath, previousRSName); ZKUtil.setData(zkw, nodePath, ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores)); - LOG.debug("Update last flushed sequence id of region " + region.getEncodedName() + " for " - + previousRSName); + LOG.debug("Update last flushed sequence id of region " + regionInfo.getEncodedName() + + " for " + previousRSName); } else { LOG.warn("Can't find failed region server for recovering region " + - region.getEncodedName()); + regionInfo.getEncodedName()); } } catch (NoNodeException ignore) { - LOG.debug("Region " + region.getEncodedName() + + LOG.debug("Region " + regionInfo.getEncodedName() + " must have completed recovery because its recovery znode has been removed", ignore); } } @@ -3241,8 +3232,8 @@ public class HRegionServer extends HasThread implements @Override public double getCompactionPressure() { double max = 0; - for (HRegion region : onlineRegions.values()) { - for (Store store : region.getStores().values()) { + for (Region region : onlineRegions.values()) { + for (Store store : region.getStores()) { double normCount = store.getCompactionPressure(); if (normCount > max) { max = normCount; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java index 43deb584f1e..a66a29ce530 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java @@ -324,7 +324,7 @@ public class HeapMemoryManager { } @Override - public void flushRequested(FlushType type, HRegion region) { + public void flushRequested(FlushType type, Region region) { switch (type) { case ABOVE_HIGHER_MARK: blockedFlushCount.incrementAndGet(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java index 18bb3760933..d7a9be56771 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java @@ -72,7 +72,7 @@ extends ConstantSizeRegionSplitPolicy { // Get size to check long sizeToCheck = getSizeToCheck(tableRegionsCount); - for (Store store : region.getStores().values()) { + for (Store store : region.getStores()) { // If any of the stores is unable to split (eg they contain reference files) // then don't split if ((!store.canSplit())) { @@ -114,7 +114,7 @@ extends ConstantSizeRegionSplitPolicy { TableName tablename = this.region.getTableDesc().getTableName(); int tableRegionsCount = 0; try { - List hri = rss.getOnlineRegions(tablename); + List hri = rss.getOnlineRegions(tablename); tableRegionsCount = hri == null || hri.isEmpty()? 0: hri.size(); } catch (IOException e) { LOG.debug("Failed getOnlineRegions " + tablename, e); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java index 71dea3b3667..de8ed8d7b5b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java @@ -164,7 +164,7 @@ public class LogRoller extends HasThread { */ private void scheduleFlush(final byte [] encodedRegionName) { boolean scheduled = false; - HRegion r = this.services.getFromOnlineRegions(Bytes.toString(encodedRegionName)); + Region r = this.services.getFromOnlineRegions(Bytes.toString(encodedRegionName)); FlushRequester requester = null; if (r != null) { requester = this.services.getFlushRequester(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java index 8bee0022e91..485d30f90bd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil; +import org.apache.hadoop.hbase.regionserver.Region.FlushResult; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Counter; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -79,8 +80,8 @@ class MemStoreFlusher implements FlushRequester { // a corresponding entry in the other. private final BlockingQueue flushQueue = new DelayQueue(); - private final Map regionsInQueue = - new HashMap(); + private final Map regionsInQueue = + new HashMap(); private AtomicBoolean wakeupPending = new AtomicBoolean(); private final long threadWakeFrequency; @@ -139,10 +140,8 @@ class MemStoreFlusher implements FlushRequester { * @return true if successful */ private boolean flushOneForGlobalPressure() { - SortedMap regionsBySize = - server.getCopyOfOnlineRegionsSortedBySize(); - - Set excludedRegions = new HashSet(); + SortedMap regionsBySize = server.getCopyOfOnlineRegionsSortedBySize(); + Set excludedRegions = new HashSet(); double secondaryMultiplier = ServerRegionReplicaUtil.getRegionReplicaStoreFileRefreshMultiplier(conf); @@ -151,13 +150,12 @@ class MemStoreFlusher implements FlushRequester { while (!flushedOne) { // Find the biggest region that doesn't have too many storefiles // (might be null!) - HRegion bestFlushableRegion = getBiggestMemstoreRegion( - regionsBySize, excludedRegions, true); + Region bestFlushableRegion = getBiggestMemstoreRegion(regionsBySize, excludedRegions, true); // Find the biggest region, total, even if it might have too many flushes. - HRegion bestAnyRegion = getBiggestMemstoreRegion( + Region bestAnyRegion = getBiggestMemstoreRegion( regionsBySize, excludedRegions, false); // Find the biggest region that is a secondary region - HRegion bestRegionReplica = getBiggestMemstoreOfRegionReplica(regionsBySize, + Region bestRegionReplica = getBiggestMemstoreOfRegionReplica(regionsBySize, excludedRegions); if (bestAnyRegion == null && bestRegionReplica == null) { @@ -165,19 +163,20 @@ class MemStoreFlusher implements FlushRequester { return false; } - HRegion regionToFlush; + Region regionToFlush; if (bestFlushableRegion != null && - bestAnyRegion.memstoreSize.get() > 2 * bestFlushableRegion.memstoreSize.get()) { + bestAnyRegion.getMemstoreSize() > 2 * bestFlushableRegion.getMemstoreSize()) { // Even if it's not supposed to be flushed, pick a region if it's more than twice // as big as the best flushable one - otherwise when we're under pressure we make // lots of little flushes and cause lots of compactions, etc, which just makes // life worse! if (LOG.isDebugEnabled()) { LOG.debug("Under global heap pressure: " + "Region " - + bestAnyRegion.getRegionNameAsString() + " has too many " + "store files, but is " - + TraditionalBinaryPrefix.long2String(bestAnyRegion.memstoreSize.get(), "", 1) + + bestAnyRegion.getRegionInfo().getRegionNameAsString() + + " has too many " + "store files, but is " + + TraditionalBinaryPrefix.long2String(bestAnyRegion.getMemstoreSize(), "", 1) + " vs best flushable region's " - + TraditionalBinaryPrefix.long2String(bestFlushableRegion.memstoreSize.get(), "", 1) + + TraditionalBinaryPrefix.long2String(bestFlushableRegion.getMemstoreSize(), "", 1) + ". Choosing the bigger."); } regionToFlush = bestAnyRegion; @@ -190,14 +189,14 @@ class MemStoreFlusher implements FlushRequester { } Preconditions.checkState( - (regionToFlush != null && regionToFlush.memstoreSize.get() > 0) || - (bestRegionReplica != null && bestRegionReplica.memstoreSize.get() > 0)); + (regionToFlush != null && regionToFlush.getMemstoreSize() > 0) || + (bestRegionReplica != null && bestRegionReplica.getMemstoreSize() > 0)); if (regionToFlush == null || (bestRegionReplica != null && ServerRegionReplicaUtil.isRegionReplicaStoreFileRefreshEnabled(conf) && - (bestRegionReplica.memstoreSize.get() - > secondaryMultiplier * regionToFlush.memstoreSize.get()))) { + (bestRegionReplica.getMemstoreSize() + > secondaryMultiplier * regionToFlush.getMemstoreSize()))) { LOG.info("Refreshing storefiles of region " + regionToFlush + " due to global heap pressure. memstore size=" + StringUtils.humanReadableInt( server.getRegionServerAccounting().getGlobalMemstoreSize())); @@ -212,7 +211,7 @@ class MemStoreFlusher implements FlushRequester { + "Total Memstore size=" + humanReadableInt(server.getRegionServerAccounting().getGlobalMemstoreSize()) + ", Region memstore size=" - + humanReadableInt(regionToFlush.memstoreSize.get())); + + humanReadableInt(regionToFlush.getMemstoreSize())); flushedOne = flushRegion(regionToFlush, true, true); if (!flushedOne) { @@ -289,17 +288,18 @@ class MemStoreFlusher implements FlushRequester { } } - private HRegion getBiggestMemstoreRegion( - SortedMap regionsBySize, - Set excludedRegions, + private Region getBiggestMemstoreRegion( + SortedMap regionsBySize, + Set excludedRegions, boolean checkStoreFileCount) { synchronized (regionsInQueue) { - for (HRegion region : regionsBySize.values()) { + for (Region region : regionsBySize.values()) { if (excludedRegions.contains(region)) { continue; } - if (region.writestate.flushing || !region.writestate.writesEnabled) { + if (((HRegion)region).writestate.flushing || + !((HRegion)region).writestate.writesEnabled) { continue; } @@ -312,10 +312,10 @@ class MemStoreFlusher implements FlushRequester { return null; } - private HRegion getBiggestMemstoreOfRegionReplica(SortedMap regionsBySize, - Set excludedRegions) { + private Region getBiggestMemstoreOfRegionReplica(SortedMap regionsBySize, + Set excludedRegions) { synchronized (regionsInQueue) { - for (HRegion region : regionsBySize.values()) { + for (Region region : regionsBySize.values()) { if (excludedRegions.contains(region)) { continue; } @@ -330,7 +330,7 @@ class MemStoreFlusher implements FlushRequester { return null; } - private boolean refreshStoreFilesAndReclaimMemory(HRegion region) { + private boolean refreshStoreFilesAndReclaimMemory(Region region) { try { return region.refreshStoreFiles(); } catch (IOException e) { @@ -356,7 +356,7 @@ class MemStoreFlusher implements FlushRequester { } @Override - public void requestFlush(HRegion r, boolean forceFlushAllStores) { + public void requestFlush(Region r, boolean forceFlushAllStores) { synchronized (regionsInQueue) { if (!regionsInQueue.containsKey(r)) { // This entry has no delay so it will be added at the top of the flush @@ -369,7 +369,7 @@ class MemStoreFlusher implements FlushRequester { } @Override - public void requestDelayedFlush(HRegion r, long delay, boolean forceFlushAllStores) { + public void requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) { synchronized (regionsInQueue) { if (!regionsInQueue.containsKey(r)) { // This entry has some delay @@ -435,19 +435,19 @@ class MemStoreFlusher implements FlushRequester { * not flushed. */ private boolean flushRegion(final FlushRegionEntry fqe) { - HRegion region = fqe.region; + Region region = fqe.region; if (!region.getRegionInfo().isMetaRegion() && isTooManyStoreFiles(region)) { if (fqe.isMaximumWait(this.blockingWaitTime)) { LOG.info("Waited " + (EnvironmentEdgeManager.currentTime() - fqe.createTime) + "ms on a compaction to clean up 'too many store files'; waited " + "long enough... proceeding with flush of " + - region.getRegionNameAsString()); + region.getRegionInfo().getRegionNameAsString()); } else { // If this is first time we've been put off, then emit a log message. if (fqe.getRequeueCount() <= 0) { // Note: We don't impose blockingStoreFiles constraint on meta regions - LOG.warn("Region " + region.getRegionNameAsString() + " has too many " + + LOG.warn("Region " + region.getRegionInfo().getRegionNameAsString() + " has too many " + "store files; delaying flush up to " + this.blockingWaitTime + "ms"); if (!this.server.compactSplitThread.requestSplit(region)) { try { @@ -456,9 +456,8 @@ class MemStoreFlusher implements FlushRequester { } catch (IOException e) { e = e instanceof RemoteException ? ((RemoteException)e).unwrapRemoteException() : e; - LOG.error( - "Cache flush failed for region " + Bytes.toStringBinary(region.getRegionName()), - e); + LOG.error("Cache flush failed for region " + + Bytes.toStringBinary(region.getRegionInfo().getRegionName()), e); } } } @@ -485,7 +484,7 @@ class MemStoreFlusher implements FlushRequester { * false, there will be accompanying log messages explaining why the region was * not flushed. */ - private boolean flushRegion(final HRegion region, final boolean emergencyFlush, + private boolean flushRegion(final Region region, final boolean emergencyFlush, boolean forceFlushAllStores) { long startTime = 0; synchronized (this.regionsInQueue) { @@ -509,10 +508,10 @@ class MemStoreFlusher implements FlushRequester { lock.readLock().lock(); try { notifyFlushRequest(region, emergencyFlush); - HRegion.FlushResult flushResult = region.flushcache(forceFlushAllStores); + FlushResult flushResult = region.flush(forceFlushAllStores); boolean shouldCompact = flushResult.isCompactionNeeded(); // We just want to check the size - boolean shouldSplit = region.checkSplit() != null; + boolean shouldSplit = ((HRegion)region).checkSplit() != null; if (shouldSplit) { this.server.compactSplitThread.requestSplit(region); } else if (shouldCompact) { @@ -535,8 +534,9 @@ class MemStoreFlusher implements FlushRequester { ex = ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex; LOG.error( "Cache flush failed" - + (region != null ? (" for region " + Bytes.toStringBinary(region.getRegionName())) - : ""), ex); + + (region != null ? (" for region " + + Bytes.toStringBinary(region.getRegionInfo().getRegionName())) + : ""), ex); if (!server.checkFileSystem()) { return false; } @@ -547,7 +547,7 @@ class MemStoreFlusher implements FlushRequester { return true; } - private void notifyFlushRequest(HRegion region, boolean emergencyFlush) { + private void notifyFlushRequest(Region region, boolean emergencyFlush) { FlushType type = FlushType.NORMAL; if (emergencyFlush) { type = isAboveHighWaterMark() ? FlushType.ABOVE_HIGHER_MARK : FlushType.ABOVE_LOWER_MARK; @@ -563,8 +563,8 @@ class MemStoreFlusher implements FlushRequester { } } - private boolean isTooManyStoreFiles(HRegion region) { - for (Store store : region.stores.values()) { + private boolean isTooManyStoreFiles(Region region) { + for (Store store : region.getStores()) { if (store.hasTooManyStoreFiles()) { return true; } @@ -719,7 +719,7 @@ class MemStoreFlusher implements FlushRequester { * a while. */ static class FlushRegionEntry implements FlushQueueEntry { - private final HRegion region; + private final Region region; private final long createTime; private long whenToExpire; @@ -727,7 +727,7 @@ class MemStoreFlusher implements FlushRequester { private boolean forceFlushAllStores; - FlushRegionEntry(final HRegion r, boolean forceFlushAllStores) { + FlushRegionEntry(final Region r, boolean forceFlushAllStores) { this.region = r; this.createTime = EnvironmentEdgeManager.currentTime(); this.whenToExpire = this.createTime; @@ -789,7 +789,7 @@ class MemStoreFlusher implements FlushRequester { @Override public String toString() { - return "[flush region " + Bytes.toStringBinary(region.getRegionName()) + "]"; + return "[flush region "+Bytes.toStringBinary(region.getRegionInfo().getRegionName())+"]"; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index 5e5590d47be..3111661a776 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.Collection; +import java.util.List; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -166,7 +167,7 @@ class MetricsRegionServerWrapperImpl @Override public long getNumOnlineRegions() { - Collection onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext(); + Collection onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext(); if (onlineRegionsLocalContext == null) { return 0; } @@ -452,16 +453,17 @@ class MetricsRegionServerWrapperImpl long tempMajorCompactedCellsSize = 0; long tempBlockedRequestsCount = 0L; - for (HRegion r : regionServer.getOnlineRegionsLocalContext()) { - tempNumMutationsWithoutWAL += r.numMutationsWithoutWAL.get(); - tempDataInMemoryWithoutWAL += r.dataInMemoryWithoutWAL.get(); - tempReadRequestsCount += r.readRequestsCount.get(); - tempWriteRequestsCount += r.writeRequestsCount.get(); - tempCheckAndMutateChecksFailed += r.checkAndMutateChecksFailed.get(); - tempCheckAndMutateChecksPassed += r.checkAndMutateChecksPassed.get(); + for (Region r : regionServer.getOnlineRegionsLocalContext()) { + tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL(); + tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL(); + tempReadRequestsCount += r.getReadRequestsCount(); + tempWriteRequestsCount += r.getWriteRequestsCount(); + tempCheckAndMutateChecksFailed += r.getCheckAndMutateChecksFailed(); + tempCheckAndMutateChecksPassed += r.getCheckAndMutateChecksPassed(); tempBlockedRequestsCount += r.getBlockedRequestsCount(); - tempNumStores += r.stores.size(); - for (Store store : r.stores.values()) { + List storeList = r.getStores(); + tempNumStores += storeList.size(); + for (Store store : storeList) { tempNumStoreFiles += store.getStorefilesCount(); tempMemstoreSize += store.getMemStoreSize(); tempStoreFileSize += store.getStorefilesSize(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java index 1cde7e394b9..60fc9fb8ade 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java @@ -22,46 +22,49 @@ import java.io.IOException; import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; /** * Interface to Map of online regions. In the Map, the key is the region's - * encoded name and the value is an {@link HRegion} instance. + * encoded name and the value is an {@link Region} instance. */ -@InterfaceAudience.Private -interface OnlineRegions extends Server { +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving +public interface OnlineRegions extends Server { /** * Add to online regions. * @param r */ - void addToOnlineRegions(final HRegion r); + void addToOnlineRegions(final Region r); /** - * This method removes HRegion corresponding to hri from the Map of onlineRegions. + * This method removes Region corresponding to hri from the Map of onlineRegions. * * @param r Region to remove. * @param destination Destination, if any, null otherwise. * @return True if we removed a region from online list. */ - boolean removeFromOnlineRegions(final HRegion r, ServerName destination); + boolean removeFromOnlineRegions(final Region r, ServerName destination); /** - * Return {@link HRegion} instance. - * Only works if caller is in same context, in same JVM. HRegion is not + * Return {@link Region} instance. + * Only works if caller is in same context, in same JVM. Region is not * serializable. * @param encodedRegionName - * @return HRegion for the passed encoded encodedRegionName or + * @return Region for the passed encoded encodedRegionName or * null if named region is not member of the online regions. */ - HRegion getFromOnlineRegions(String encodedRegionName); + Region getFromOnlineRegions(String encodedRegionName); /** * Get all online regions of a table in this RS. * @param tableName - * @return List of HRegion + * @return List of Region * @throws java.io.IOException */ - List getOnlineRegions(TableName tableName) throws IOException; + List getOnlineRegions(TableName tableName) throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 62921eec657..b0fd9eb72de 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -151,9 +151,10 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.quotas.OperationQuota; import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager; -import org.apache.hadoop.hbase.regionserver.HRegion.Operation; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException; +import org.apache.hadoop.hbase.regionserver.Region.FlushResult; +import org.apache.hadoop.hbase.regionserver.Region.Operation; import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -163,6 +164,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; import org.apache.hadoop.hbase.util.Strings; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; @@ -215,9 +217,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, private static class RegionScannerHolder { private RegionScanner s; private long nextCallSeq = 0L; - private HRegion r; + private Region r; - public RegionScannerHolder(RegionScanner s, HRegion r) { + public RegionScannerHolder(RegionScanner s, Region r) { this.s = s; this.r = r; } @@ -242,7 +244,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, LOG.info("Scanner " + this.scannerName + " lease expired on region " + s.getRegionInfo().getRegionNameAsString()); try { - HRegion region = regionServer.getRegion(s.getRegionInfo().getRegionName()); + Region region = regionServer.getRegion(s.getRegionInfo().getRegionName()); if (region != null && region.getCoprocessorHost() != null) { region.getCoprocessorHost().preScannerClose(s); } @@ -362,7 +364,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @param cellScanner if non-null, the mutation data -- the Cell content. * @throws IOException */ - private ClientProtos.RegionLoadStats mutateRows(final HRegion region, + private ClientProtos.RegionLoadStats mutateRows(final Region region, final List actions, final CellScanner cellScanner) throws IOException { if (!region.getRegionInfo().isMetaTable()) { @@ -390,7 +392,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } region.mutateRow(rm); - return region.getRegionStats(); + return ((HRegion)region).getRegionStats(); } /** @@ -405,7 +407,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @param compareOp * @param comparator @throws IOException */ - private boolean checkAndRowMutate(final HRegion region, final List actions, + private boolean checkAndRowMutate(final Region region, final List actions, final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier, CompareOp compareOp, ByteArrayComparable comparator) throws IOException { if (!region.getRegionInfo().isMetaTable()) { @@ -445,7 +447,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * bypassed as indicated by RegionObserver, null otherwise * @throws IOException */ - private Result append(final HRegion region, final OperationQuota quota, final MutationProto m, + private Result append(final Region region, final OperationQuota quota, final MutationProto m, final CellScanner cellScanner, long nonceGroup) throws IOException { long before = EnvironmentEdgeManager.currentTime(); Append append = ProtobufUtil.toAppend(m, cellScanner); @@ -482,7 +484,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @return the Result * @throws IOException */ - private Result increment(final HRegion region, final OperationQuota quota, + private Result increment(final Region region, final OperationQuota quota, final MutationProto mutation, final CellScanner cells, long nonceGroup) throws IOException { long before = EnvironmentEdgeManager.currentTime(); @@ -523,7 +525,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * method returns as a 'result'. * @return Return the cellScanner passed */ - private List doNonAtomicRegionMutation(final HRegion region, + private List doNonAtomicRegionMutation(final Region region, final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner, final RegionActionResult.Builder builder, List cellsToReturn, long nonceGroup) { // Gather up CONTIGUOUS Puts and Deletes in this mutations List. Idea is that rather than do @@ -622,7 +624,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @param region * @param mutations */ - private void doBatchOp(final RegionActionResult.Builder builder, final HRegion region, + private void doBatchOp(final RegionActionResult.Builder builder, final Region region, final OperationQuota quota, final List mutations, final CellScanner cells) { Mutation[] mArray = new Mutation[mutations.size()]; @@ -648,7 +650,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, regionServer.cacheFlusher.reclaimMemStoreMemory(); } - OperationStatus codes[] = region.batchMutate(mArray); + OperationStatus codes[] = region.batchMutate(mArray, HConstants.NO_NONCE, + HConstants.NO_NONCE); for (i = 0; i < codes.length; i++) { int index = mutations.get(i).getIndex(); Exception e = null; @@ -670,7 +673,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, case SUCCESS: builder.addResultOrException(getResultOrException( - ClientProtos.Result.getDefaultInstance(), index, region.getRegionStats())); + ClientProtos.Result.getDefaultInstance(), index, + ((HRegion)region).getRegionStats())); break; } } @@ -700,7 +704,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * exceptionMessage if any * @throws IOException */ - private OperationStatus [] doReplayBatchOp(final HRegion region, + private OperationStatus [] doReplayBatchOp(final Region region, final List mutations, long replaySeqId) throws IOException { long before = EnvironmentEdgeManager.currentTime(); boolean batchContainsPuts = false, batchContainsDelete = false; @@ -720,26 +724,27 @@ public class RSRpcServices implements HBaseRPCErrorHandler, for (Cell metaCell : metaCells) { CompactionDescriptor compactionDesc = WALEdit.getCompaction(metaCell); boolean isDefaultReplica = RegionReplicaUtil.isDefaultReplica(region.getRegionInfo()); + HRegion hRegion = (HRegion)region; if (compactionDesc != null) { // replay the compaction. Remove the files from stores only if we are the primary // region replica (thus own the files) - region.replayWALCompactionMarker(compactionDesc, !isDefaultReplica, isDefaultReplica, + hRegion.replayWALCompactionMarker(compactionDesc, !isDefaultReplica, isDefaultReplica, replaySeqId); continue; } FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(metaCell); if (flushDesc != null && !isDefaultReplica) { - region.replayWALFlushMarker(flushDesc, replaySeqId); + hRegion.replayWALFlushMarker(flushDesc, replaySeqId); continue; } RegionEventDescriptor regionEvent = WALEdit.getRegionEventDescriptor(metaCell); if (regionEvent != null && !isDefaultReplica) { - region.replayWALRegionEventMarker(regionEvent); + hRegion.replayWALRegionEventMarker(regionEvent); continue; } BulkLoadDescriptor bulkLoadEvent = WALEdit.getBulkLoadDescriptor(metaCell); if (bulkLoadEvent != null) { - region.replayWALBulkLoadEventMarker(bulkLoadEvent); + hRegion.replayWALBulkLoadEventMarker(bulkLoadEvent); continue; } } @@ -852,7 +857,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return 0L; } - long addScanner(RegionScanner s, HRegion r) throws LeaseStillHeldException { + long addScanner(RegionScanner s, Region r) throws LeaseStillHeldException { long scannerId = this.scannerIdGen.incrementAndGet(); String scannerName = String.valueOf(scannerId); @@ -873,7 +878,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @throws IOException if the specifier is not null, * but failed to find the region */ - HRegion getRegion( + Region getRegion( final RegionSpecifier regionSpecifier) throws IOException { return regionServer.getRegionByEncodedName(regionSpecifier.getValue().toByteArray(), ProtobufUtil.getRegionEncodedName(regionSpecifier)); @@ -1006,7 +1011,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion()); // Can be null if we're calling close on a region that's not online - final HRegion region = regionServer.getFromOnlineRegions(encodedRegionName); + final Region region = regionServer.getFromOnlineRegions(encodedRegionName); if ((region != null) && (region .getCoprocessorHost() != null)) { region.getCoprocessorHost().preClose(false); } @@ -1035,9 +1040,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - HRegion region = getRegion(request.getRegion()); + Region region = getRegion(request.getRegion()); region.startRegionOperation(Operation.COMPACT_REGION); - LOG.info("Compacting " + region.getRegionNameAsString()); + LOG.info("Compacting " + region.getRegionInfo().getRegionNameAsString()); boolean major = false; byte [] family = null; Store store = null; @@ -1046,7 +1051,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, store = region.getStore(family); if (store == null) { throw new ServiceException(new IOException("column family " + Bytes.toString(family) - + " does not exist in region " + region.getRegionNameAsString())); + + " does not exist in region " + region.getRegionInfo().getRegionNameAsString())); } } if (request.hasMajor()) { @@ -1063,7 +1068,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, String familyLogMsg = (family != null)?" for column family: " + Bytes.toString(family):""; if (LOG.isTraceEnabled()) { LOG.trace("User-triggered compaction requested for region " - + region.getRegionNameAsString() + familyLogMsg); + + region.getRegionInfo().getRegionNameAsString() + familyLogMsg); } String log = "User-triggered " + (major ? "major " : "") + "compaction" + familyLogMsg; if(family != null) { @@ -1093,8 +1098,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - HRegion region = getRegion(request.getRegion()); - LOG.info("Flushing " + region.getRegionNameAsString()); + Region region = getRegion(request.getRegion()); + LOG.info("Flushing " + region.getRegionInfo().getRegionNameAsString()); boolean shouldFlush = true; if (request.hasIfOlderThanTs()) { shouldFlush = region.getEarliestFlushTimeForAllStores() < request.getIfOlderThanTs(); @@ -1104,7 +1109,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, boolean writeFlushWalMarker = request.hasWriteFlushWalMarker() ? request.getWriteFlushWalMarker() : false; long startTime = EnvironmentEdgeManager.currentTime(); - HRegion.FlushResult flushResult = region.flushcache(true, writeFlushWalMarker); + // Go behind the curtain so we can manage writing of the flush WAL marker + HRegion.FlushResultImpl flushResult = (HRegion.FlushResultImpl) + ((HRegion)region).flushcache(true, writeFlushWalMarker); if (flushResult.isFlushSucceeded()) { long endTime = EnvironmentEdgeManager.currentTime(); regionServer.metricsRegionServer.updateFlushTime(endTime - startTime); @@ -1117,7 +1124,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, builder.setFlushed(flushResult.isFlushSucceeded()); builder.setWroteFlushWalMarker(flushResult.wroteFlushWalMarker); } - builder.setLastFlushTime( region.getEarliestFlushTimeForAllStores()); + builder.setLastFlushTime(region.getEarliestFlushTimeForAllStores()); return builder.build(); } catch (DroppedSnapshotException ex) { // Cache flush can fail in a few places. If it fails in a critical @@ -1138,9 +1145,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Map onlineRegions = regionServer.onlineRegions; + Map onlineRegions = regionServer.onlineRegions; List list = new ArrayList(onlineRegions.size()); - for (HRegion region: onlineRegions.values()) { + for (Region region: onlineRegions.values()) { list.add(region.getRegionInfo()); } Collections.sort(list); @@ -1157,7 +1164,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - HRegion region = getRegion(request.getRegion()); + Region region = getRegion(request.getRegion()); HRegionInfo info = region.getRegionInfo(); GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder(); builder.setRegionInfo(HRegionInfo.convert(info)); @@ -1198,11 +1205,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler, final GetStoreFileRequest request) throws ServiceException { try { checkOpen(); - HRegion region = getRegion(request.getRegion()); + Region region = getRegion(request.getRegion()); requestCount.increment(); Set columnFamilies; if (request.getFamilyCount() == 0) { - columnFamilies = region.getStores().keySet(); + columnFamilies = region.getTableDesc().getFamiliesKeys(); } else { columnFamilies = new TreeSet(Bytes.BYTES_RAWCOMPARATOR); for (ByteString cf: request.getFamilyList()) { @@ -1235,8 +1242,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - HRegion regionA = getRegion(request.getRegionA()); - HRegion regionB = getRegion(request.getRegionB()); + Region regionA = getRegion(request.getRegionA()); + Region regionB = getRegion(request.getRegionB()); boolean forcible = request.getForcible(); regionA.startRegionOperation(Operation.MERGE_REGION); regionB.startRegionOperation(Operation.MERGE_REGION); @@ -1247,13 +1254,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler, LOG.info("Receiving merging request for " + regionA + ", " + regionB + ",forcible=" + forcible); long startTime = EnvironmentEdgeManager.currentTime(); - HRegion.FlushResult flushResult = regionA.flushcache(); + FlushResult flushResult = regionA.flush(true); if (flushResult.isFlushSucceeded()) { long endTime = EnvironmentEdgeManager.currentTime(); regionServer.metricsRegionServer.updateFlushTime(endTime - startTime); } startTime = EnvironmentEdgeManager.currentTime(); - flushResult = regionB.flushcache(); + flushResult = regionB.flush(true); if (flushResult.isFlushSucceeded()) { long endTime = EnvironmentEdgeManager.currentTime(); regionServer.metricsRegionServer.updateFlushTime(endTime - startTime); @@ -1346,7 +1353,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { String encodedName = region.getEncodedName(); byte[] encodedNameBytes = region.getEncodedNameAsBytes(); - final HRegion onlineRegion = regionServer.getFromOnlineRegions(encodedName); + final Region onlineRegion = regionServer.getFromOnlineRegions(encodedName); if (onlineRegion != null) { // The region is already online. This should not happen any more. String error = "Received OPEN for the region:" @@ -1456,7 +1463,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { String encodedName = region.getEncodedName(); byte[] encodedNameBytes = region.getEncodedNameAsBytes(); - final HRegion onlineRegion = regionServer.getFromOnlineRegions(encodedName); + final Region onlineRegion = regionServer.getFromOnlineRegions(encodedName); if (onlineRegion != null) { LOG.info("Region already online. Skipping warming up " + region); @@ -1507,7 +1514,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return ReplicateWALEntryResponse.newBuilder().build(); } ByteString regionName = entries.get(0).getKey().getEncodedRegionName(); - HRegion region = regionServer.getRegionByEncodedName(regionName.toStringUtf8()); + Region region = regionServer.getRegionByEncodedName(regionName.toStringUtf8()); RegionCoprocessorHost coprocessorHost = ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo()) ? region.getCoprocessorHost() @@ -1558,12 +1565,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } //sync wal at the end because ASYNC_WAL is used above - region.syncWal(); + WAL wal = getWAL(region); + if (wal != null) { + wal.sync(); + } if (coprocessorHost != null) { - for (Pair wal : walEntries) { - coprocessorHost.postWALRestore(region.getRegionInfo(), wal.getFirst(), - wal.getSecond()); + for (Pair entry : walEntries) { + coprocessorHost.postWALRestore(region.getRegionInfo(), entry.getFirst(), + entry.getSecond()); } } return ReplicateWALEntryResponse.newBuilder().build(); @@ -1577,6 +1587,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } + WAL getWAL(Region region) { + return ((HRegion)region).getWAL(); + } + /** * Replicate WAL entries on the region server. * @@ -1640,15 +1654,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - HRegion region = getRegion(request.getRegion()); + Region region = getRegion(request.getRegion()); region.startRegionOperation(Operation.SPLIT_REGION); if (region.getRegionInfo().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) { throw new IOException("Can't split replicas directly. " + "Replicas are auto-split when their primary is split."); } - LOG.info("Splitting " + region.getRegionNameAsString()); + LOG.info("Splitting " + region.getRegionInfo().getRegionNameAsString()); long startTime = EnvironmentEdgeManager.currentTime(); - HRegion.FlushResult flushResult = region.flushcache(); + FlushResult flushResult = region.flush(true); if (flushResult.isFlushSucceeded()) { long endTime = EnvironmentEdgeManager.currentTime(); regionServer.metricsRegionServer.updateFlushTime(endTime - startTime); @@ -1657,8 +1671,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, if (request.hasSplitPoint()) { splitPoint = request.getSplitPoint().toByteArray(); } - region.forceSplit(splitPoint); - regionServer.compactSplitThread.requestSplit(region, region.checkSplit()); + ((HRegion)region).forceSplit(splitPoint); + regionServer.compactSplitThread.requestSplit(region, ((HRegion)region).checkSplit()); return SplitRegionResponse.newBuilder().build(); } catch (IOException ie) { throw new ServiceException(ie); @@ -1707,7 +1721,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - HRegion region = getRegion(request.getRegion()); + Region region = getRegion(request.getRegion()); List> familyPaths = new ArrayList>(); for (FamilyPath familyPath: request.getFamilyPathList()) { familyPaths.add(new Pair(familyPath.getFamily().toByteArray(), @@ -1719,7 +1733,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } boolean loaded = false; if (!bypass) { - loaded = region.bulkLoadHFiles(familyPaths, request.getAssignSeqNum()); + loaded = region.bulkLoadHFiles(familyPaths, request.getAssignSeqNum(), null); } if (region.getCoprocessorHost() != null) { loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded); @@ -1738,12 +1752,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - HRegion region = getRegion(request.getRegion()); + Region region = getRegion(request.getRegion()); Message result = execServiceOnRegion(region, request.getCall()); CoprocessorServiceResponse.Builder builder = CoprocessorServiceResponse.newBuilder(); builder.setRegion(RequestConverter.buildRegionSpecifier( - RegionSpecifierType.REGION_NAME, region.getRegionName())); + RegionSpecifierType.REGION_NAME, region.getRegionInfo().getRegionName())); builder.setValue( builder.getValueBuilder().setName(result.getClass().getName()) .setValue(result.toByteString())); @@ -1753,7 +1767,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } - private Message execServiceOnRegion(HRegion region, + private Message execServiceOnRegion(Region region, final ClientProtos.CoprocessorServiceCall serviceCall) throws IOException { // ignore the passed in controller (from the serialized call) ServerRpcController execController = new ServerRpcController(); @@ -1779,7 +1793,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - HRegion region = getRegion(request.getRegion()); + Region region = getRegion(request.getRegion()); GetResponse.Builder builder = GetResponse.newBuilder(); ClientProtos.Get get = request.getGet(); @@ -1871,7 +1885,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, for (RegionAction regionAction : request.getRegionActionList()) { this.requestCount.add(regionAction.getActionCount()); OperationQuota quota; - HRegion region; + Region region; regionActionResultBuilder.clear(); try { region = getRegion(regionAction.getRegion()); @@ -1946,14 +1960,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - HRegion region = getRegion(request.getRegion()); + Region region = getRegion(request.getRegion()); MutateResponse.Builder builder = MutateResponse.newBuilder(); MutationProto mutation = request.getMutation(); if (!region.getRegionInfo().isMetaTable()) { regionServer.cacheFlusher.reclaimMemStoreMemory(); } - long nonceGroup = request.hasNonceGroup() - ? request.getNonceGroup() : HConstants.NO_NONCE; + long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE; Result r = null; Boolean processed = null; MutationType type = mutation.getMutateType(); @@ -2090,7 +2103,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, requestCount.increment(); int ttl = 0; - HRegion region = null; + Region region = null; RegionScanner scanner = null; RegionScannerHolder rsh = null; boolean moreResults = true; @@ -2129,7 +2142,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } isSmallScan = scan.isSmall(); - region.prepareScanner(scan); + if (!scan.hasFamilies()) { + // Adding all families to scanner + for (byte[] family: region.getTableDesc().getFamiliesKeys()) { + scan.addFamily(family); + } + } + if (region.getCoprocessorHost() != null) { scanner = region.getCoprocessorHost().preScannerOpen(scan); } @@ -2273,7 +2292,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, builder.setMoreResultsInRegion(false); } } - region.readRequestsCount.add(i); + region.updateReadRequestsCount(i); region.getMetrics().updateScanNext(totalCellSize); } finally { region.closeRegionOperation(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java new file mode 100644 index 00000000000..441a93bfa7e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java @@ -0,0 +1,680 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.DroppedSnapshotException; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HDFSBlocksDistribution; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.IsolationLevel; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.RowMutations; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.conf.ConfigurationObserver; +import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; +import org.apache.hadoop.hbase.filter.ByteArrayComparable; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay; + +import com.google.protobuf.Message; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; + +/** + * Regions store data for a certain region of a table. It stores all columns + * for each row. A given table consists of one or more Regions. + * + *

An Region is defined by its table and its key extent. + * + *

Locking at the Region level serves only one purpose: preventing the + * region from being closed (and consequently split) while other operations + * are ongoing. Each row level operation obtains both a row lock and a region + * read lock for the duration of the operation. While a scanner is being + * constructed, getScanner holds a read lock. If the scanner is successfully + * constructed, it holds a read lock until it is closed. A close takes out a + * write lock and consequently will block for ongoing operations and will block + * new operations from starting while the close is in progress. + */ +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving +public interface Region extends ConfigurationObserver { + + /////////////////////////////////////////////////////////////////////////// + // Region state + + /** @return region information for this region */ + HRegionInfo getRegionInfo(); + + /** @return table descriptor for this region */ + HTableDescriptor getTableDesc(); + + /** @return true if region is available (not closed and not closing) */ + boolean isAvailable(); + + /** @return true if region is closed */ + boolean isClosed(); + + /** @return True if closing process has started */ + boolean isClosing(); + + /** @return True if region is in recovering state */ + boolean isRecovering(); + + /** @return True if region is read only */ + boolean isReadOnly(); + + /** + * Return the list of Stores managed by this region + *

Use with caution. Exposed for use of fixup utilities. + * @return a list of the Stores managed by this region + */ + List getStores(); + + /** + * Return the Store for the given family + *

Use with caution. Exposed for use of fixup utilities. + * @return the Store for the given family + */ + Store getStore(byte[] family); + + /** @return list of store file names for the given families */ + List getStoreFileList(byte [][] columns); + + /** + * Check the region's underlying store files, open the files that have not + * been opened yet, and remove the store file readers for store files no + * longer available. + * @throws IOException + */ + boolean refreshStoreFiles() throws IOException; + + /** @return the latest sequence number that was read from storage when this region was opened */ + long getOpenSeqNum(); + + /** @return the max sequence id of flushed data on this region */ + long getMaxFlushedSeqId(); + + /** @return the oldest sequence id found in the store for the given family */ + public long getOldestSeqIdOfStore(byte[] familyName); + + /** + * This can be used to determine the last time all files of this region were major compacted. + * @param majorCompactioOnly Only consider HFile that are the result of major compaction + * @return the timestamp of the oldest HFile for all stores of this region + */ + long getOldestHfileTs(boolean majorCompactioOnly) throws IOException; + + /** + * @return map of column family names to max sequence id that was read from storage when this + * region was opened + */ + public Map getMaxStoreSeqId(); + + /** @return true if loading column families on demand by default */ + boolean isLoadingCfsOnDemandDefault(); + + /** @return readpoint considering given IsolationLevel */ + long getReadpoint(IsolationLevel isolationLevel); + + /** + * @return The earliest time a store in the region was flushed. All + * other stores in the region would have been flushed either at, or + * after this time. + */ + long getEarliestFlushTimeForAllStores(); + + /////////////////////////////////////////////////////////////////////////// + // Metrics + + /** @return read requests count for this region */ + long getReadRequestsCount(); + + /** + * Update the read request count for this region + * @param i increment + */ + void updateReadRequestsCount(long i); + + /** @return write request count for this region */ + long getWriteRequestsCount(); + + /** + * Update the write request count for this region + * @param i increment + */ + void updateWriteRequestsCount(long i); + + /** @return memstore size for this region, in bytes */ + long getMemstoreSize(); + + /** @return the number of mutations processed bypassing the WAL */ + long getNumMutationsWithoutWAL(); + + /** @return the size of data processed bypassing the WAL, in bytes */ + long getDataInMemoryWithoutWAL(); + + /** @return the number of blocked requests */ + long getBlockedRequestsCount(); + + /** @return the number of checkAndMutate guards that passed */ + long getCheckAndMutateChecksPassed(); + + /** @return the number of failed checkAndMutate guards */ + long getCheckAndMutateChecksFailed(); + + /** @return the MetricsRegion for this region */ + MetricsRegion getMetrics(); + + /** @return the block distribution for all Stores managed by this region */ + HDFSBlocksDistribution getHDFSBlocksDistribution(); + + /////////////////////////////////////////////////////////////////////////// + // Locking + + // Region read locks + + /** + * Operation enum is used in {@link Region#startRegionOperation} to provide context for + * various checks before any region operation begins. + */ + enum Operation { + ANY, GET, PUT, DELETE, SCAN, APPEND, INCREMENT, SPLIT_REGION, MERGE_REGION, BATCH_MUTATE, + REPLAY_BATCH_MUTATE, COMPACT_REGION, REPLAY_EVENT + } + + /** + * This method needs to be called before any public call that reads or + * modifies data. + * Acquires a read lock and checks if the region is closing or closed. + *

{@link #closeRegionOperation} MUST then always be called after + * the operation has completed, whether it succeeded or failed. + * @throws IOException + */ + void startRegionOperation() throws IOException; + + /** + * This method needs to be called before any public call that reads or + * modifies data. + * Acquires a read lock and checks if the region is closing or closed. + *

{@link #closeRegionOperation} MUST then always be called after + * the operation has completed, whether it succeeded or failed. + * @param op The operation is about to be taken on the region + * @throws IOException + */ + void startRegionOperation(Operation op) throws IOException; + + /** + * Closes the region operation lock. + * @throws IOException + */ + void closeRegionOperation() throws IOException; + + // Row write locks + + /** + * Row lock held by a given thread. + * One thread may acquire multiple locks on the same row simultaneously. + * The locks must be released by calling release() from the same thread. + */ + public interface RowLock { + /** + * Release the given lock. If there are no remaining locks held by the current thread + * then unlock the row and allow other threads to acquire the lock. + * @throws IllegalArgumentException if called by a different thread than the lock owning + * thread + */ + void release(); + } + + /** + * Tries to acquire a lock on the given row. + * @param waitForLock if true, will block until the lock is available. + * Otherwise, just tries to obtain the lock and returns + * false if unavailable. + * @return the row lock if acquired, + * null if waitForLock was false and the lock was not acquired + * @throws IOException if waitForLock was true and the lock could not be acquired after waiting + */ + RowLock getRowLock(byte[] row, boolean waitForLock) throws IOException; + + /** + * If the given list of row locks is not null, releases all locks. + */ + void releaseRowLocks(List rowLocks); + + /////////////////////////////////////////////////////////////////////////// + // Region operations + + /** + * Perform one or more append operations on a row. + * @param append + * @param nonceGroup + * @param nonce + * @return result of the operation + * @throws IOException + */ + Result append(Append append, long nonceGroup, long nonce) throws IOException; + + /** + * Perform a batch of mutations. + *

+ * Note this supports only Put and Delete mutations and will ignore other types passed. + * @param mutations the list of mutations + * @param nonceGroup + * @param nonce + * @return an array of OperationStatus which internally contains the + * OperationStatusCode and the exceptionMessage if any. + * @throws IOException + */ + OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce) + throws IOException; + + /** + * Replay a batch of mutations. + * @param mutations mutations to replay. + * @param replaySeqId + * @return an array of OperationStatus which internally contains the + * OperationStatusCode and the exceptionMessage if any. + * @throws IOException + */ + OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId) throws IOException; + + /** + * Atomically checks if a row/family/qualifier value matches the expected val + * If it does, it performs the row mutations. If the passed value is null, t + * is for the lack of column (ie: non-existence) + * @param row to check + * @param family column family to check + * @param qualifier column qualifier to check + * @param compareOp the comparison operator + * @param comparator + * @param mutation + * @param writeToWAL + * @return true if mutation was applied, false otherwise + * @throws IOException + */ + boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier, CompareOp compareOp, + ByteArrayComparable comparator, Mutation mutation, boolean writeToWAL) throws IOException; + + /** + * Atomically checks if a row/family/qualifier value matches the expected val + * If it does, it performs the row mutations. If the passed value is null, t + * is for the lack of column (ie: non-existence) + * @param row to check + * @param family column family to check + * @param qualifier column qualifier to check + * @param compareOp the comparison operator + * @param comparator + * @param mutations + * @param writeToWAL + * @return true if mutation was applied, false otherwise + * @throws IOException + */ + boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier, CompareOp compareOp, + ByteArrayComparable comparator, RowMutations mutations, boolean writeToWAL) + throws IOException; + + /** + * Deletes the specified cells/row. + * @param delete + * @throws IOException + */ + void delete(Delete delete) throws IOException; + + /** + * Do a get based on the get parameter. + * @param get query parameters + * @return result of the operation + */ + Result get(Get get) throws IOException; + + /** + * Do a get based on the get parameter. + * @param get query parameters + * @param withCoprocessor invoke coprocessor or not. We don't want to + * always invoke cp. + * @return list of cells resulting from the operation + */ + List get(Get get, boolean withCoprocessor) throws IOException; + + /** + * Return all the data for the row that matches row exactly, + * or the one that immediately preceeds it, at or immediately before + * ts. + * @param row + * @param family + * @return result of the operation + * @throws IOException + */ + Result getClosestRowBefore(byte[] row, byte[] family) throws IOException; + + /** + * Return an iterator that scans over the HRegion, returning the indicated + * columns and rows specified by the {@link Scan}. + *

+ * This Iterator must be closed by the caller. + * + * @param scan configured {@link Scan} + * @return RegionScanner + * @throws IOException read exceptions + */ + RegionScanner getScanner(Scan scan) throws IOException; + + /** + * Perform one or more increment operations on a row. + * @param increment + * @param nonceGroup + * @param nonce + * @return result of the operation + * @throws IOException + */ + Result increment(Increment increment, long nonceGroup, long nonce) throws IOException; + + /** + * Performs multiple mutations atomically on a single row. Currently + * {@link Put} and {@link Delete} are supported. + * + * @param mutations object that specifies the set of mutations to perform atomically + * @throws IOException + */ + void mutateRow(RowMutations mutations) throws IOException; + + /** + * Perform atomic mutations within the region. + * + * @param mutations The list of mutations to perform. + * mutations can contain operations for multiple rows. + * Caller has to ensure that all rows are contained in this region. + * @param rowsToLock Rows to lock + * @param nonceGroup Optional nonce group of the operation (client Id) + * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") + * If multiple rows are locked care should be taken that + * rowsToLock is sorted in order to avoid deadlocks. + * @throws IOException + */ + void mutateRowsWithLocks(Collection mutations, Collection rowsToLock, + long nonceGroup, long nonce) throws IOException; + + /** + * Performs atomic multiple reads and writes on a given row. + * + * @param processor The object defines the reads and writes to a row. + */ + void processRowsWithLocks(RowProcessor processor) throws IOException; + + /** + * Performs atomic multiple reads and writes on a given row. + * + * @param processor The object defines the reads and writes to a row. + * @param nonceGroup Optional nonce group of the operation (client Id) + * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") + */ + void processRowsWithLocks(RowProcessor processor, long nonceGroup, long nonce) + throws IOException; + + /** + * Performs atomic multiple reads and writes on a given row. + * + * @param processor The object defines the reads and writes to a row. + * @param timeout The timeout of the processor.process() execution + * Use a negative number to switch off the time bound + * @param nonceGroup Optional nonce group of the operation (client Id) + * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") + */ + void processRowsWithLocks(RowProcessor processor, long timeout, long nonceGroup, long nonce) + throws IOException; + + /** + * Puts some data in the table. + * @param put + * @throws IOException + */ + void put(Put put) throws IOException; + + /** + * Listener class to enable callers of + * bulkLoadHFile() to perform any necessary + * pre/post processing of a given bulkload call + */ + interface BulkLoadListener { + + /** + * Called before an HFile is actually loaded + * @param family family being loaded to + * @param srcPath path of HFile + * @return final path to be used for actual loading + * @throws IOException + */ + String prepareBulkLoad(byte[] family, String srcPath) throws IOException; + + /** + * Called after a successful HFile load + * @param family family being loaded to + * @param srcPath path of HFile + * @throws IOException + */ + void doneBulkLoad(byte[] family, String srcPath) throws IOException; + + /** + * Called after a failed HFile load + * @param family family being loaded to + * @param srcPath path of HFile + * @throws IOException + */ + void failedBulkLoad(byte[] family, String srcPath) throws IOException; + } + + /** + * Attempts to atomically load a group of hfiles. This is critical for loading + * rows with multiple column families atomically. + * + * @param familyPaths List of Pair + * @param bulkLoadListener Internal hooks enabling massaging/preparation of a + * file about to be bulk loaded + * @param assignSeqId + * @return true if successful, false if failed recoverably + * @throws IOException if failed unrecoverably. + */ + boolean bulkLoadHFiles(Collection> familyPaths, boolean assignSeqId, + BulkLoadListener bulkLoadListener) throws IOException; + + /////////////////////////////////////////////////////////////////////////// + // Coprocessors + + /** @return the coprocessor host */ + RegionCoprocessorHost getCoprocessorHost(); + + /** + * Executes a single protocol buffer coprocessor endpoint {@link Service} method using + * the registered protocol handlers. {@link Service} implementations must be registered via the + * {@link Region#registerService(com.google.protobuf.Service)} + * method before they are available. + * + * @param controller an {@code RpcContoller} implementation to pass to the invoked service + * @param call a {@code CoprocessorServiceCall} instance identifying the service, method, + * and parameters for the method invocation + * @return a protocol buffer {@code Message} instance containing the method's result + * @throws IOException if no registered service handler is found or an error + * occurs during the invocation + * @see org.apache.hadoop.hbase.regionserver.Region#registerService(com.google.protobuf.Service) + */ + Message execService(RpcController controller, CoprocessorServiceCall call) throws IOException; + + /** + * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to + * be available for handling + * {@link Region#execService(com.google.protobuf.RpcController, + * org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall)}} calls. + * + *

+ * Only a single instance may be registered per region for a given {@link Service} subclass (the + * instances are keyed on {@link com.google.protobuf.Descriptors.ServiceDescriptor#getFullName()}. + * After the first registration, subsequent calls with the same service name will fail with + * a return value of {@code false}. + *

+ * @param instance the {@code Service} subclass instance to expose as a coprocessor endpoint + * @return {@code true} if the registration was successful, {@code false} + * otherwise + */ + boolean registerService(Service instance); + + /////////////////////////////////////////////////////////////////////////// + // RowMutation processor support + + /** + * Check the collection of families for validity. + * @param families + * @throws NoSuchColumnFamilyException + */ + void checkFamilies(Collection families) throws NoSuchColumnFamilyException; + + /** + * Check the collection of families for valid timestamps + * @param familyMap + * @param now current timestamp + * @throws FailedSanityCheckException + */ + void checkTimestamps(Map> familyMap, long now) + throws FailedSanityCheckException; + + /** + * Prepare a delete for a row mutation processor + * @param delete The passed delete is modified by this method. WARNING! + * @throws IOException + */ + void prepareDelete(Delete delete) throws IOException; + + /** + * Set up correct timestamps in the KVs in Delete object. + *

Caller should have the row and region locks. + * @param mutation + * @param familyCellMap + * @param now + * @throws IOException + */ + void prepareDeleteTimestamps(Mutation mutation, Map> familyCellMap, + byte[] now) throws IOException; + + /** + * Replace any cell timestamps set to {@link HConstants#LATEST_TIMESTAMP} with the + * provided current timestamp. + * @param values + * @param now + */ + void updateCellTimestamps(final Iterable> values, final byte[] now) + throws IOException; + + /////////////////////////////////////////////////////////////////////////// + // Flushes, compactions, splits, etc. + // Wizards only, please + + interface FlushResult { + enum Result { + FLUSHED_NO_COMPACTION_NEEDED, + FLUSHED_COMPACTION_NEEDED, + // Special case where a flush didn't run because there's nothing in the memstores. Used when + // bulk loading to know when we can still load even if a flush didn't happen. + CANNOT_FLUSH_MEMSTORE_EMPTY, + CANNOT_FLUSH + } + + /** @return the detailed result code */ + Result getResult(); + + /** @return true if the memstores were flushed, else false */ + boolean isFlushSucceeded(); + + /** @return True if the flush requested a compaction, else false */ + boolean isCompactionNeeded(); + } + + /** + * Flush the cache. + * + *

When this method is called the cache will be flushed unless: + *

    + *
  1. the cache is empty
  2. + *
  3. the region is closed.
  4. + *
  5. a flush is already in progress
  6. + *
  7. writes are disabled
  8. + *
+ * + *

This method may block for some time, so it should not be called from a + * time-sensitive thread. + * @param force whether we want to force a flush of all stores + * @return FlushResult indicating whether the flush was successful or not and if + * the region needs compacting + * + * @throws IOException general io exceptions + * @throws DroppedSnapshotException Thrown when abort is required + * because a snapshot was not properly persisted. + */ + FlushResult flush(boolean force) throws IOException; + + /** + * Synchronously compact all stores in the region. + *

This operation could block for a long time, so don't call it from a + * time-sensitive thread. + *

Note that no locks are taken to prevent possible conflicts between + * compaction and splitting activities. The regionserver does not normally compact + * and split in parallel. However by calling this method you may introduce + * unexpected and unhandled concurrency. Don't do this unless you know what + * you are doing. + * + * @param majorCompaction True to force a major compaction regardless of thresholds + * @throws IOException + */ + void compact(final boolean majorCompaction) throws IOException; + + /** + * Trigger major compaction on all stores in the region. + *

+ * Compaction will be performed asynchronously to this call by the RegionServer's + * CompactSplitThread. See also {@link Store#triggerMajorCompaction()} + * @throws IOException + */ + void triggerMajorCompaction() throws IOException; + + /** + * @return if a given region is in compaction now. + */ + CompactionState getCompactionState(); + + /** Wait for all current flushes and compactions of the region to complete */ + void waitForFlushesAndCompactions(); + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java index d81d96caded..6e239520dc6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java @@ -77,7 +77,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.regionserver.HRegion.Operation; +import org.apache.hadoop.hbase.regionserver.Region.Operation; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.wal.WALKey; @@ -88,7 +88,7 @@ import org.apache.hadoop.hbase.util.Pair; /** * Implements the coprocessor environment and runtime support for coprocessors - * loaded within a {@link HRegion}. + * loaded within a {@link Region}. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving @@ -106,7 +106,7 @@ public class RegionCoprocessorHost static class RegionEnvironment extends CoprocessorHost.Environment implements RegionCoprocessorEnvironment { - private HRegion region; + private Region region; private RegionServerServices rsServices; ConcurrentMap sharedData; private static final int LATENCY_BUFFER_SIZE = 100; @@ -121,7 +121,7 @@ public class RegionCoprocessorHost * @param priority chaining priority */ public RegionEnvironment(final Coprocessor impl, final int priority, - final int seq, final Configuration conf, final HRegion region, + final int seq, final Configuration conf, final Region region, final RegionServerServices services, final ConcurrentMap sharedData) { super(impl, priority, seq, conf); this.region = region; @@ -139,7 +139,7 @@ public class RegionCoprocessorHost /** @return the region */ @Override - public HRegion getRegion() { + public Region getRegion() { return region; } @@ -209,7 +209,7 @@ public class RegionCoprocessorHost /** The region server services */ RegionServerServices rsServices; /** The region */ - HRegion region; + Region region; /** * Constructor @@ -217,7 +217,7 @@ public class RegionCoprocessorHost * @param rsServices interface to available region server functionality * @param conf the configuration */ - public RegionCoprocessorHost(final HRegion region, + public RegionCoprocessorHost(final Region region, final RegionServerServices rsServices, final Configuration conf) { super(rsServices); this.conf = conf; @@ -707,7 +707,7 @@ public class RegionCoprocessorHost * @param r the new right-hand daughter region * @throws IOException */ - public void postSplit(final HRegion l, final HRegion r) throws IOException { + public void postSplit(final Region l, final Region r) throws IOException { execOperation(coprocessors.isEmpty() ? null : new RegionOperation() { @Override public void call(RegionObserver oserver, ObserverContext ctx) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java index 5226a984a39..ffa98cde82c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java @@ -42,10 +42,10 @@ class RegionMergeRequest implements Runnable { private final boolean forcible; private TableLock tableLock; - RegionMergeRequest(HRegion a, HRegion b, HRegionServer hrs, boolean forcible) { + RegionMergeRequest(Region a, Region b, HRegionServer hrs, boolean forcible) { Preconditions.checkNotNull(hrs); - this.region_a = a; - this.region_b = b; + this.region_a = (HRegion)a; + this.region_b = (HRegion)b; this.server = hrs; this.forcible = forcible; } @@ -71,7 +71,8 @@ class RegionMergeRequest implements Runnable { //acquire a shared read lock on the table, so that table schema modifications //do not happen concurrently tableLock = server.getTableLockManager().readLock(region_a.getTableDesc().getTableName() - , "MERGE_REGIONS:" + region_a.getRegionNameAsString() + ", " + region_b.getRegionNameAsString()); + , "MERGE_REGIONS:" + region_a.getRegionInfo().getRegionNameAsString() + ", " + + region_b.getRegionInfo().getRegionNameAsString()); try { tableLock.acquire(); } catch (IOException ex) { @@ -134,7 +135,7 @@ class RegionMergeRequest implements Runnable { LOG.error("Could not release the table lock (something is really wrong). " + "Aborting this server to avoid holding the lock forever."); this.server.abort("Abort; we got an error when releasing the table lock " - + "on " + region_a.getRegionNameAsString()); + + "on " + region_a.getRegionInfo().getRegionNameAsString()); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java index d478bfe518b..17ab887e8a1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java @@ -141,14 +141,14 @@ public class RegionMergeTransaction { * @param b region b to merge * @param forcible if false, we will only merge adjacent regions */ - public RegionMergeTransaction(final HRegion a, final HRegion b, + public RegionMergeTransaction(final Region a, final Region b, final boolean forcible) { if (a.getRegionInfo().compareTo(b.getRegionInfo()) <= 0) { - this.region_a = a; - this.region_b = b; + this.region_a = (HRegion)a; + this.region_b = (HRegion)b; } else { - this.region_a = b; - this.region_b = a; + this.region_a = (HRegion)b; + this.region_b = (HRegion)a; } this.forcible = forcible; this.mergesdir = region_a.getRegionFileSystem().getMergesDir(); @@ -173,8 +173,8 @@ public class RegionMergeTransaction { } if (!forcible && !HRegionInfo.areAdjacent(region_a.getRegionInfo(), region_b.getRegionInfo())) { - String msg = "Skip merging " + this.region_a.getRegionNameAsString() - + " and " + this.region_b.getRegionNameAsString() + String msg = "Skip merging " + this.region_a.getRegionInfo().getRegionNameAsString() + + " and " + this.region_b.getRegionInfo().getRegionNameAsString() + ", because they are not adjacent."; LOG.info(msg); return false; @@ -184,18 +184,19 @@ public class RegionMergeTransaction { } try { boolean regionAHasMergeQualifier = hasMergeQualifierInMeta(services, - region_a.getRegionName()); + region_a.getRegionInfo().getRegionName()); if (regionAHasMergeQualifier || - hasMergeQualifierInMeta(services, region_b.getRegionName())) { - LOG.debug("Region " + (regionAHasMergeQualifier ? region_a.getRegionNameAsString() - : region_b.getRegionNameAsString()) + hasMergeQualifierInMeta(services, region_b.getRegionInfo().getRegionName())) { + LOG.debug("Region " + (regionAHasMergeQualifier ? + region_a.getRegionInfo().getRegionNameAsString() : + region_b.getRegionInfo().getRegionNameAsString()) + " is not mergeable because it has merge qualifier in META"); return false; } } catch (IOException e) { LOG.warn("Failed judging whether merge transaction is available for " - + region_a.getRegionNameAsString() + " and " - + region_b.getRegionNameAsString(), e); + + region_a.getRegionInfo().getRegionNameAsString() + " and " + + region_b.getRegionInfo().getRegionNameAsString(), e); return false; } @@ -254,7 +255,7 @@ public class RegionMergeTransaction { HRegion createMergedRegion(final Server server, final RegionServerServices services) throws IOException { LOG.info("Starting merge of " + region_a + " and " - + region_b.getRegionNameAsString() + ", forcible=" + forcible); + + region_b.getRegionInfo().getRegionNameAsString() + ", forcible=" + forcible); if ((server != null && server.isStopped()) || (services != null && services.isStopping())) { throw new IOException("Server is stopped or stopping"); @@ -487,7 +488,7 @@ public class RegionMergeTransaction { boolean stopped = server != null && server.isStopped(); boolean stopping = services != null && services.isStopping(); if (stopped || stopping) { - LOG.info("Not opening merged region " + merged.getRegionNameAsString() + LOG.info("Not opening merged region " + merged.getRegionInfo().getRegionNameAsString() + " because stopping=" + stopping + ", stopped=" + stopped); return; } @@ -588,7 +589,7 @@ public class RegionMergeTransaction { this.region_a.initialize(); } catch (IOException e) { LOG.error("Failed rollbacking CLOSED_REGION_A of region " - + this.region_a.getRegionNameAsString(), e); + + this.region_a.getRegionInfo().getRegionNameAsString(), e); throw new RuntimeException(e); } break; @@ -603,7 +604,7 @@ public class RegionMergeTransaction { this.region_b.initialize(); } catch (IOException e) { LOG.error("Failed rollbacking CLOSED_REGION_A of region " - + this.region_b.getRegionNameAsString(), e); + + this.region_b.getRegionInfo().getRegionNameAsString(), e); throw new RuntimeException(e); } break; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java index c75ed53dadd..eaffa380e08 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java @@ -24,9 +24,11 @@ import java.util.Set; import java.util.concurrent.ConcurrentMap; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.master.TableLockManager; @@ -40,9 +42,9 @@ import com.google.protobuf.Service; /** * Services provided by {@link HRegionServer} */ -@InterfaceAudience.Private -public interface RegionServerServices - extends OnlineRegions, FavoredNodesForRegion { +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving +public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegion { /** * @return True if this regionserver is stopping. */ @@ -85,8 +87,7 @@ public interface RegionServerServices * @throws KeeperException * @throws IOException */ - void postOpenDeployTasks(final HRegion r) - throws KeeperException, IOException; + void postOpenDeployTasks(final Region r) throws KeeperException, IOException; /** * Notify master that a handler requests to change a region state @@ -127,7 +128,7 @@ public interface RegionServerServices /** * @return set of recovering regions on the hosting region server */ - Map getRecoveringRegions(); + Map getRecoveringRegions(); /** * Only required for "old" log replay; if it's removed, remove this. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java index ec7f9fe02ac..0e28ebbfafb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import java.util.Map; +import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -74,11 +74,11 @@ public abstract class RegionSplitPolicy extends Configured { if (explicitSplitPoint != null) { return explicitSplitPoint; } - Map stores = region.getStores(); + List stores = region.getStores(); byte[] splitPointFromLargestStore = null; long largestStoreSize = 0; - for (Store s : stores.values()) { + for (Store s : stores) { byte[] splitPoint = s.getSplitPoint(); long storeSize = s.getSize(); if (splitPoint != null && largestStoreSize < storeSize) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java index 1dfe4adac4d..9034a72a91f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java @@ -42,9 +42,9 @@ class SplitRequest implements Runnable { private final HRegionServer server; private TableLock tableLock; - SplitRequest(HRegion region, byte[] midKey, HRegionServer hrs) { + SplitRequest(Region region, byte[] midKey, HRegionServer hrs) { Preconditions.checkNotNull(hrs); - this.parent = region; + this.parent = (HRegion)region; this.midKey = midKey; this.server = hrs; } @@ -69,7 +69,7 @@ class SplitRequest implements Runnable { //acquire a shared read lock on the table, so that table schema modifications //do not happen concurrently tableLock = server.getTableLockManager().readLock(parent.getTableDesc().getTableName() - , "SPLIT_REGION:" + parent.getRegionNameAsString()); + , "SPLIT_REGION:" + parent.getRegionInfo().getRegionNameAsString()); try { tableLock.acquire(); } catch (IOException ex) { @@ -87,22 +87,22 @@ class SplitRequest implements Runnable { if (this.server.isStopping() || this.server.isStopped()) { LOG.info( "Skip rollback/cleanup of failed split of " - + parent.getRegionNameAsString() + " because server is" + + parent.getRegionInfo().getRegionNameAsString() + " because server is" + (this.server.isStopping() ? " stopping" : " stopped"), e); return; } try { LOG.info("Running rollback/cleanup of failed split of " + - parent.getRegionNameAsString() + "; " + e.getMessage(), e); + parent.getRegionInfo().getRegionNameAsString() + "; " + e.getMessage(), e); if (st.rollback(this.server, this.server)) { LOG.info("Successful rollback of failed split of " + - parent.getRegionNameAsString()); + parent.getRegionInfo().getRegionNameAsString()); } else { this.server.abort("Abort; we got an error after point-of-no-return"); } } catch (RuntimeException ee) { String msg = "Failed rollback of failed split of " + - parent.getRegionNameAsString() + " -- aborting server"; + parent.getRegionInfo().getRegionNameAsString() + " -- aborting server"; // If failed rollback, kill this server to avoid having a hole in table. LOG.info(msg, ee); this.server.abort(msg + " -- Cause: " + ee.getMessage()); @@ -133,7 +133,7 @@ class SplitRequest implements Runnable { server.metricsRegionServer.incrSplitSuccess(); // Log success LOG.info("Region split, hbase:meta updated, and report to master. Parent=" - + parent.getRegionNameAsString() + ", new regions: " + + parent.getRegionInfo().getRegionNameAsString() + ", new regions: " + st.getFirstDaughter().getRegionNameAsString() + ", " + st.getSecondDaughter().getRegionNameAsString() + ". Split took " + StringUtils.formatTimeDiff(EnvironmentEdgeManager.currentTime(), startTime)); @@ -151,7 +151,7 @@ class SplitRequest implements Runnable { LOG.error("Could not release the table lock (something is really wrong). " + "Aborting this server to avoid holding the lock forever."); this.server.abort("Abort; we got an error when releasing the table lock " - + "on " + parent.getRegionNameAsString()); + + "on " + parent.getRegionInfo().getRegionNameAsString()); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java index dbcf0333a5d..bbee93c5bfe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java @@ -197,8 +197,8 @@ public class SplitTransaction { * @param r Region to split * @param splitrow Row to split around */ - public SplitTransaction(final HRegion r, final byte [] splitrow) { - this.parent = r; + public SplitTransaction(final Region r, final byte [] splitrow) { + this.parent = (HRegion)r; this.splitrow = splitrow; this.journal.add(new JournalEntry(JournalEntryType.STARTED)); } @@ -259,7 +259,7 @@ public class SplitTransaction { * Call {@link #rollback(Server, RegionServerServices)} * @return Regions created */ - /* package */PairOfSameType createDaughters(final Server server, + /* package */PairOfSameType createDaughters(final Server server, final RegionServerServices services) throws IOException { LOG.info("Starting split of region " + this.parent); if ((server != null && server.isStopped()) || @@ -287,14 +287,14 @@ public class SplitTransaction { server.getConfiguration().getLong("hbase.regionserver.fileSplitTimeout", this.fileSplitTimeout); - PairOfSameType daughterRegions = stepsBeforePONR(server, services, testing); + PairOfSameType daughterRegions = stepsBeforePONR(server, services, testing); List metaEntries = new ArrayList(); if (this.parent.getCoprocessorHost() != null) { if (this.parent.getCoprocessorHost(). preSplitBeforePONR(this.splitrow, metaEntries)) { throw new IOException("Coprocessor bypassing region " - + this.parent.getRegionNameAsString() + " split."); + + this.parent.getRegionInfo().getRegionNameAsString() + " split."); } try { for (Mutation p : metaEntries) { @@ -338,12 +338,12 @@ public class SplitTransaction { return daughterRegions; } - public PairOfSameType stepsBeforePONR(final Server server, + public PairOfSameType stepsBeforePONR(final Server server, final RegionServerServices services, boolean testing) throws IOException { if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_SPLIT, parent.getRegionInfo(), hri_a, hri_b)) { throw new IOException("Failed to get ok from master to split " - + parent.getRegionNameAsString()); + + parent.getRegionInfo().getRegionNameAsString()); } this.journal.add(new JournalEntry(JournalEntryType.SET_SPLITTING)); @@ -392,7 +392,7 @@ public class SplitTransaction { this.journal.add(new JournalEntry(JournalEntryType.STARTED_REGION_A_CREATION)); assertReferenceFileCount(expectedReferences.getFirst(), this.parent.getRegionFileSystem().getSplitsDir(this.hri_a)); - HRegion a = this.parent.createDaughterRegionFromSplits(this.hri_a); + Region a = this.parent.createDaughterRegionFromSplits(this.hri_a); assertReferenceFileCount(expectedReferences.getFirst(), new Path(this.parent.getRegionFileSystem().getTableDir(), this.hri_a.getEncodedName())); @@ -400,11 +400,11 @@ public class SplitTransaction { this.journal.add(new JournalEntry(JournalEntryType.STARTED_REGION_B_CREATION)); assertReferenceFileCount(expectedReferences.getSecond(), this.parent.getRegionFileSystem().getSplitsDir(this.hri_b)); - HRegion b = this.parent.createDaughterRegionFromSplits(this.hri_b); + Region b = this.parent.createDaughterRegionFromSplits(this.hri_b); assertReferenceFileCount(expectedReferences.getSecond(), new Path(this.parent.getRegionFileSystem().getTableDir(), this.hri_b.getEncodedName())); - return new PairOfSameType(a, b); + return new PairOfSameType(a, b); } void assertReferenceFileCount(int expectedReferenceFileCount, Path dir) @@ -425,7 +425,7 @@ public class SplitTransaction { * Call {@link #rollback(Server, RegionServerServices)} */ /* package */void openDaughters(final Server server, - final RegionServerServices services, HRegion a, HRegion b) + final RegionServerServices services, Region a, Region b) throws IOException { boolean stopped = server != null && server.isStopped(); boolean stopping = services != null && services.isStopping(); @@ -438,8 +438,8 @@ public class SplitTransaction { " because stopping=" + stopping + ", stopped=" + stopped); } else { // Open daughters in parallel. - DaughterOpener aOpener = new DaughterOpener(server, a); - DaughterOpener bOpener = new DaughterOpener(server, b); + DaughterOpener aOpener = new DaughterOpener(server, (HRegion)a); + DaughterOpener bOpener = new DaughterOpener(server, (HRegion)b); aOpener.start(); bOpener.start(); try { @@ -485,18 +485,18 @@ public class SplitTransaction { * @throws IOException * @see #rollback(Server, RegionServerServices) */ - public PairOfSameType execute(final Server server, + public PairOfSameType execute(final Server server, final RegionServerServices services) throws IOException { - PairOfSameType regions = createDaughters(server, services); + PairOfSameType regions = createDaughters(server, services); if (this.parent.getCoprocessorHost() != null) { this.parent.getCoprocessorHost().preSplitAfterPONR(); } return stepsAfterPONR(server, services, regions); } - public PairOfSameType stepsAfterPONR(final Server server, - final RegionServerServices services, PairOfSameType regions) + public PairOfSameType stepsAfterPONR(final Server server, + final RegionServerServices services, PairOfSameType regions) throws IOException { openDaughters(server, services, regions.getFirst(), regions.getSecond()); journal.add(new JournalEntry(JournalEntryType.BEFORE_POST_SPLIT_HOOK)); @@ -749,7 +749,7 @@ public class SplitTransaction { this.parent.initialize(); } catch (IOException e) { LOG.error("Failed rollbacking CLOSED_PARENT_REGION of region " + - this.parent.getRegionNameAsString(), e); + this.parent.getRegionInfo().getRegionNameAsString(), e); throw new RuntimeException(e); } break; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index b638a8fbf61..a77fc0e85ca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; @@ -48,7 +49,7 @@ import org.apache.hadoop.hbase.util.Pair; * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or * more StoreFiles, which stretch backwards over time. */ -@InterfaceAudience.Private +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving public interface Store extends HeapSize, StoreConfigInformation, PropagatingConfigurationObserver { @@ -63,7 +64,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf Collection getStorefiles(); /** - * Close all the readers We don't need to worry about subsequent requests because the HRegion + * Close all the readers We don't need to worry about subsequent requests because the Region * holds a write lock that will prevent any more reads or writes. * @return the {@link StoreFile StoreFiles} that were previously being used. * @throws IOException on failure @@ -241,7 +242,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf void assertBulkLoadHFileOk(Path srcPath) throws IOException; /** - * This method should only be called from HRegion. It is assumed that the ranges of values in the + * This method should only be called from Region. It is assumed that the ranges of values in the * HFile fit within the stores assigned region. (assertBulkLoadHFileOk checks this) * * @param srcPathStr diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StorefileRefresherChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StorefileRefresherChore.java index 49183914a8a..a2a0dcccc44 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StorefileRefresherChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StorefileRefresherChore.java @@ -84,8 +84,8 @@ public class StorefileRefresherChore extends ScheduledChore { @Override protected void chore() { - for (HRegion r : regionServer.getOnlineRegionsLocalContext()) { - if (!r.writestate.isReadOnly()) { + for (Region r : regionServer.getOnlineRegionsLocalContext()) { + if (!r.isReadOnly()) { // skip checking for this region if it can accept writes continue; } @@ -98,7 +98,7 @@ public class StorefileRefresherChore extends ScheduledChore { lastRefreshTimes.put(encodedName, time); } try { - for (Store store : r.getStores().values()) { + for (Store store : r.getStores()) { // TODO: some stores might see new data from flush, while others do not which // MIGHT break atomic edits across column families. We can fix this with setting // mvcc read numbers that we know every store has seen @@ -110,12 +110,12 @@ public class StorefileRefresherChore extends ScheduledChore { // Store files have a TTL in the archive directory. If we fail to refresh for that long, we stop serving reads if (isRegionStale(encodedName, time)) { - r.setReadsEnabled(false); // stop serving reads + ((HRegion)r).setReadsEnabled(false); // stop serving reads } continue; } lastRefreshTimes.put(encodedName, time); - r.setReadsEnabled(true); // restart serving reads + ((HRegion)r).setReadsEnabled(true); // restart serving reads } // remove closed regions diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java index dbc45e7f8d9..26e824de6ae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java @@ -90,7 +90,7 @@ public class CloseRegionHandler extends EventHandler { LOG.debug("Processing close of " + name); String encodedRegionName = regionInfo.getEncodedName(); // Check that this region is being served here - HRegion region = this.rsServices.getFromOnlineRegions(encodedRegionName); + HRegion region = (HRegion)rsServices.getFromOnlineRegions(encodedRegionName); if (region == null) { LOG.warn("Received CLOSE for region " + name + " but currently not serving - ignoring"); // TODO: do better than a simple warning @@ -119,7 +119,7 @@ public class CloseRegionHandler extends EventHandler { rsServices.reportRegionStateTransition(TransitionCode.CLOSED, regionInfo); // Done! Region is closed on this RS - LOG.debug("Closed " + region.getRegionNameAsString()); + LOG.debug("Closed " + region.getRegionInfo().getRegionNameAsString()); } finally { this.rsServices.getRegionsInTransitionInRS(). remove(this.regionInfo.getEncodedNameAsBytes(), Boolean.FALSE); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/FinishRegionRecoveringHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/FinishRegionRecoveringHandler.java index 2ff3454fef6..19838d3648c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/FinishRegionRecoveringHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/FinishRegionRecoveringHandler.java @@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; public class FinishRegionRecoveringHandler extends EventHandler { @@ -45,9 +46,9 @@ public class FinishRegionRecoveringHandler extends EventHandler { @Override public void process() throws IOException { - HRegion region = this.rss.getRecoveringRegions().remove(regionName); + Region region = this.rss.getRecoveringRegions().remove(regionName); if (region != null) { - region.setRecovering(false); + ((HRegion)region).setRecovering(false); LOG.info(path + " deleted; " + regionName + " recovered."); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/FlushSnapshotSubprocedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/FlushSnapshotSubprocedure.java index 611e432ddc8..998c1fb3d7d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/FlushSnapshotSubprocedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/FlushSnapshotSubprocedure.java @@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.procedure.ProcedureMember; import org.apache.hadoop.hbase.procedure.Subprocedure; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.Region.Operation; import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager.SnapshotSubprocedurePool; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; @@ -45,14 +47,14 @@ import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; public class FlushSnapshotSubprocedure extends Subprocedure { private static final Log LOG = LogFactory.getLog(FlushSnapshotSubprocedure.class); - private final List regions; + private final List regions; private final SnapshotDescription snapshot; private final SnapshotSubprocedurePool taskManager; private boolean snapshotSkipFlush = false; public FlushSnapshotSubprocedure(ProcedureMember member, ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout, - List regions, SnapshotDescription snapshot, + List regions, SnapshotDescription snapshot, SnapshotSubprocedurePool taskManager) { super(member, snapshot.getName(), errorListener, wakeFrequency, timeout); this.snapshot = snapshot; @@ -68,8 +70,8 @@ public class FlushSnapshotSubprocedure extends Subprocedure { * Callable for adding files to snapshot manifest working dir. Ready for multithreading. */ private class RegionSnapshotTask implements Callable { - HRegion region; - RegionSnapshotTask(HRegion region) { + Region region; + RegionSnapshotTask(Region region) { this.region = region; } @@ -94,9 +96,9 @@ public class FlushSnapshotSubprocedure extends Subprocedure { LOG.debug("take snapshot without flush memstore first"); } else { LOG.debug("Flush Snapshotting region " + region.toString() + " started..."); - region.flushcache(); + region.flush(true); } - region.addRegionToSnapshot(snapshot, monitor); + ((HRegion)region).addRegionToSnapshot(snapshot, monitor); if (snapshotSkipFlush) { LOG.debug("... SkipFlush Snapshotting region " + region.toString() + " completed."); } else { @@ -126,7 +128,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure { } // Add all hfiles already existing in region. - for (HRegion region : regions) { + for (Region region : regions) { // submit one task per region for parallelize by region. taskManager.submitTask(new RegionSnapshotTask(region)); monitor.rethrowException(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java index 93d836d761d..021c16f0c5b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java @@ -49,8 +49,8 @@ import org.apache.hadoop.hbase.procedure.Subprocedure; import org.apache.hadoop.hbase.procedure.SubprocedureFactory; import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -160,7 +160,7 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager { // check to see if this server is hosting any regions for the snapshots // check to see if we have regions for the snapshot - List involvedRegions; + List involvedRegions; try { involvedRegions = getRegionsToSnapshot(snapshot); } catch (IOException e1) { @@ -220,12 +220,12 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager { * the given snapshot. * @throws IOException */ - private List getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException { - List onlineRegions = rss.getOnlineRegions(TableName.valueOf(snapshot.getTable())); - Iterator iterator = onlineRegions.iterator(); + private List getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException { + List onlineRegions = rss.getOnlineRegions(TableName.valueOf(snapshot.getTable())); + Iterator iterator = onlineRegions.iterator(); // remove the non-default regions while (iterator.hasNext()) { - HRegion r = iterator.next(); + Region r = iterator.next(); if (!RegionReplicaUtil.isDefaultReplica(r.getRegionInfo())) { iterator.remove(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java index b918354586c..a01f8a1a420 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java @@ -61,9 +61,9 @@ import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -328,7 +328,7 @@ public class AccessControlLists { * Returns {@code true} if the given region is part of the {@code _acl_} * metadata table. */ - static boolean isAclRegion(HRegion region) { + static boolean isAclRegion(Region region) { return ACL_TABLE_NAME.equals(region.getTableDesc().getTableName()); } @@ -347,8 +347,7 @@ public class AccessControlLists { * @return a map of the permissions for this table. * @throws IOException */ - static Map> loadAll( - HRegion aclRegion) + static Map> loadAll(Region aclRegion) throws IOException { if (!isAclRegion(aclRegion)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 140534df099..f5e2aa590f8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -86,10 +86,10 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; @@ -199,7 +199,7 @@ public class AccessController extends BaseMasterAndRegionObserver // This boolean having relevance only in the Master. private volatile boolean aclTabAvailable = false; - public HRegion getRegion() { + public Region getRegion() { return regionEnv != null ? regionEnv.getRegion() : null; } @@ -208,7 +208,7 @@ public class AccessController extends BaseMasterAndRegionObserver } void initialize(RegionCoprocessorEnvironment e) throws IOException { - final HRegion region = e.getRegion(); + final Region region = e.getRegion(); Configuration conf = e.getConfiguration(); Map> tables = AccessControlLists.loadAll(region); @@ -1347,7 +1347,7 @@ public class AccessController extends BaseMasterAndRegionObserver public void preOpen(ObserverContext e) throws IOException { RegionCoprocessorEnvironment env = e.getEnvironment(); - final HRegion region = env.getRegion(); + final Region region = env.getRegion(); if (region == null) { LOG.error("NULL region from RegionCoprocessorEnvironment in preOpen()"); } else { @@ -1363,7 +1363,7 @@ public class AccessController extends BaseMasterAndRegionObserver @Override public void postOpen(ObserverContext c) { RegionCoprocessorEnvironment env = c.getEnvironment(); - final HRegion region = env.getRegion(); + final Region region = env.getRegion(); if (region == null) { LOG.error("NULL region from RegionCoprocessorEnvironment in postOpen()"); return; @@ -1467,7 +1467,7 @@ public class AccessController extends BaseMasterAndRegionObserver throw new RuntimeException("Unhandled operation " + opType); } AuthResult authResult = permissionGranted(opType, user, env, families, Action.READ); - HRegion region = getRegion(env); + Region region = getRegion(env); TableName table = getTableName(region); Map cfVsMaxVersions = Maps.newHashMap(); for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) { @@ -2312,19 +2312,19 @@ public class AccessController extends BaseMasterAndRegionObserver return AccessControlProtos.AccessControlService.newReflectiveService(this); } - private HRegion getRegion(RegionCoprocessorEnvironment e) { + private Region getRegion(RegionCoprocessorEnvironment e) { return e.getRegion(); } private TableName getTableName(RegionCoprocessorEnvironment e) { - HRegion region = e.getRegion(); + Region region = e.getRegion(); if (region != null) { return getTableName(region); } return null; } - private TableName getTableName(HRegion region) { + private TableName getTableName(Region region) { HRegionInfo regionInfo = region.getRegionInfo(); if (regionInfo != null) { return regionInfo.getTable(); @@ -2428,31 +2428,31 @@ public class AccessController extends BaseMasterAndRegionObserver } @Override - public void preMerge(ObserverContext ctx, HRegion regionA, - HRegion regionB) throws IOException { + public void preMerge(ObserverContext ctx, Region regionA, + Region regionB) throws IOException { requirePermission("mergeRegions", regionA.getTableDesc().getTableName(), null, null, Action.ADMIN); } @Override - public void postMerge(ObserverContext c, HRegion regionA, - HRegion regionB, HRegion mergedRegion) throws IOException { } + public void postMerge(ObserverContext c, Region regionA, + Region regionB, Region mergedRegion) throws IOException { } @Override public void preMergeCommit(ObserverContext ctx, - HRegion regionA, HRegion regionB, List metaEntries) throws IOException { } + Region regionA, Region regionB, List metaEntries) throws IOException { } @Override public void postMergeCommit(ObserverContext ctx, - HRegion regionA, HRegion regionB, HRegion mergedRegion) throws IOException { } + Region regionA, Region regionB, Region mergedRegion) throws IOException { } @Override public void preRollBackMerge(ObserverContext ctx, - HRegion regionA, HRegion regionB) throws IOException { } + Region regionA, Region regionB) throws IOException { } @Override public void postRollBackMerge(ObserverContext ctx, - HRegion regionA, HRegion regionB) throws IOException { } + Region regionA, Region regionB) throws IOException { } @Override public void preRollWALWriterRequest(ObserverContext ctx) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java index 058992fc3e7..e1c49ecb5b4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.security.access; import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcController; import com.google.protobuf.Service; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -49,7 +50,8 @@ import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBu import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadResponse; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.Region.BulkLoadListener; import org.apache.hadoop.hbase.security.SecureBulkLoadUtil; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; @@ -236,7 +238,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService return; } - HRegion region = env.getRegion(); + Region region = env.getRegion(); boolean bypass = false; if (region.getCoprocessorHost() != null) { try { @@ -353,7 +355,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService return this; } - private static class SecureBulkLoadListener implements HRegion.BulkLoadListener { + private static class SecureBulkLoadListener implements BulkLoadListener { // Target filesystem private FileSystem fs; private String stagingDir; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java index 6b9a358a60c..34ccb4a25eb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java @@ -44,6 +44,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; @@ -55,8 +56,8 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.util.StreamUtils; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.OperationStatus; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.AccessControlLists; @@ -76,7 +77,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService private AtomicInteger ordinalCounter = new AtomicInteger(-1); private Configuration conf; - private HRegion labelsRegion; + private Region labelsRegion; private VisibilityLabelsCache labelsCache; private List scanLabelGenerators; private List superUsers; @@ -196,7 +197,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService return new Pair, Map>>(labels, userAuths); } - protected void addSystemLabel(HRegion region, Map labels, + protected void addSystemLabel(Region region, Map labels, Map> userAuths) throws IOException { if (!labels.containsKey(SYSTEM_LABEL)) { Put p = new Put(Bytes.toBytes(SYSTEM_LABEL_ORDINAL)); @@ -307,7 +308,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService private boolean mutateLabelsRegion(List mutations, OperationStatus[] finalOpStatus) throws IOException { OperationStatus[] opStatus = this.labelsRegion.batchMutate(mutations - .toArray(new Mutation[mutations.size()])); + .toArray(new Mutation[mutations.size()]), HConstants.NO_NONCE, HConstants.NO_NONCE); int i = 0; boolean updateZk = false; for (OperationStatus status : opStatus) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java index f65494c9da3..fbfc3336611 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java @@ -90,10 +90,10 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.Visibil import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.DeleteTracker; import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; import org.apache.hadoop.hbase.regionserver.OperationStatus; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.security.AccessDeniedException; @@ -513,7 +513,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements if (!initialized) { throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"); } - HRegion region = e.getEnvironment().getRegion(); + Region region = e.getEnvironment().getRegion(); Authorizations authorizations = null; try { authorizations = scan.getAuthorizations(); @@ -547,7 +547,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements public DeleteTracker postInstantiateDeleteTracker( ObserverContext ctx, DeleteTracker delTracker) throws IOException { - HRegion region = ctx.getEnvironment().getRegion(); + Region region = ctx.getEnvironment().getRegion(); TableName table = region.getRegionInfo().getTable(); if (table.isSystemTable()) { return delTracker; @@ -611,7 +611,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements if (!initialized) { throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!"); } - HRegion region = e.getEnvironment().getRegion(); + Region region = e.getEnvironment().getRegion(); Authorizations authorizations = null; try { authorizations = get.getAuthorizations(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java index ebff5ffa98a..d495b69e8e6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java @@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUs import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.AccessControlLists; @@ -308,7 +308,7 @@ public class VisibilityUtils { return false; } - public static Filter createVisibilityLabelFilter(HRegion region, Authorizations authorizations) + public static Filter createVisibilityLabelFilter(Region region, Authorizations authorizations) throws IOException { Map cfVsMaxVersions = new HashMap(); for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java index 330ead4918e..85f08af6d07 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java @@ -167,7 +167,7 @@ public class SnapshotManifest { // 2. iterate through all the stores in the region LOG.debug("Creating references for hfiles"); - for (Store store : region.getStores().values()) { + for (Store store : region.getStores()) { // 2.1. build the snapshot reference for the store Object familyData = visitor.familyOpen(regionData, store.getFamily().getName()); monitor.rethrowException(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/WriteSinkCoprocessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/WriteSinkCoprocessor.java index fff1374ab71..92ab4d1498c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/WriteSinkCoprocessor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/WriteSinkCoprocessor.java @@ -60,7 +60,7 @@ public class WriteSinkCoprocessor extends BaseRegionObserver { @Override public void preOpen(ObserverContext e) throws IOException { - regionName = e.getEnvironment().getRegion().getRegionNameAsString(); + regionName = e.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java index 96e0d48a8a8..fa1aa00e374 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java @@ -206,16 +206,17 @@ class HMerge { if ((currentSize + nextSize) <= (maxFilesize / 2)) { // We merge two adjacent regions if their total size is less than // one half of the desired maximum size - LOG.info("Merging regions " + currentRegion.getRegionNameAsString() + - " and " + nextRegion.getRegionNameAsString()); + LOG.info("Merging regions " + currentRegion.getRegionInfo().getRegionNameAsString() + + " and " + nextRegion.getRegionInfo().getRegionNameAsString()); HRegion mergedRegion = HRegion.mergeAdjacent(currentRegion, nextRegion); - updateMeta(currentRegion.getRegionName(), nextRegion.getRegionName(), - mergedRegion); + updateMeta(currentRegion.getRegionInfo().getRegionName(), + nextRegion.getRegionInfo().getRegionName(), mergedRegion); break; } - LOG.info("not merging regions " + Bytes.toStringBinary(currentRegion.getRegionName()) - + " and " + Bytes.toStringBinary(nextRegion.getRegionName())); + LOG.info("not merging regions " + + Bytes.toStringBinary(currentRegion.getRegionInfo().getRegionName()) + + " and " + Bytes.toStringBinary(nextRegion.getRegionInfo().getRegionName())); currentRegion.close(); currentRegion = nextRegion; currentSize = nextSize; @@ -342,7 +343,7 @@ class HMerge { if(LOG.isDebugEnabled()) { LOG.debug("updated columns in row: " - + Bytes.toStringBinary(newRegion.getRegionName())); + + Bytes.toStringBinary(newRegion.getRegionInfo().getRegionName())); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java index 6002f29c1d0..4f9658572e4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java @@ -173,11 +173,11 @@ public class Merge extends Configured implements Tool { throws IOException { if (info1 == null) { throw new IOException("Could not find " + Bytes.toStringBinary(region1) + " in " + - Bytes.toStringBinary(meta.getRegionName())); + Bytes.toStringBinary(meta.getRegionInfo().getRegionName())); } if (info2 == null) { throw new IOException("Could not find " + Bytes.toStringBinary(region2) + " in " + - Bytes.toStringBinary(meta.getRegionName())); + Bytes.toStringBinary(meta.getRegionInfo().getRegionName())); } HRegion merged = null; HRegion r1 = HRegion.openHRegion(info1, htd, utils.getLog(info1), getConf()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java index 8ea617887a0..7d1ff0d31d8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; @@ -231,7 +232,7 @@ public abstract class HBaseTestCase extends TestCase { * @throws IOException * @return count of what we added. */ - public static long addContent(final HRegion r, final byte [] columnFamily, final byte[] column) + public static long addContent(final Region r, final byte [] columnFamily, final byte[] column) throws IOException { byte [] startKey = r.getRegionInfo().getStartKey(); byte [] endKey = r.getRegionInfo().getEndKey(); @@ -243,8 +244,7 @@ public abstract class HBaseTestCase extends TestCase { startKeyBytes, endKey, -1); } - public static long addContent(final HRegion r, final byte [] columnFamily) - throws IOException { + public static long addContent(final Region r, final byte [] columnFamily) throws IOException { return addContent(r, columnFamily, null); } @@ -440,6 +440,10 @@ public abstract class HBaseTestCase extends TestCase { this.region = HRegion; } + public HRegionIncommon(final Region region) { + this.region = (HRegion)region; + } + public void put(Put put) throws IOException { region.put(put); } @@ -470,7 +474,7 @@ public abstract class HBaseTestCase extends TestCase { } public void flushcache() throws IOException { - this.region.flushcache(); + this.region.flush(true); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 0445cb012ed..ff795697338 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -90,6 +90,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL; @@ -286,6 +287,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return htu; } + /** + * Close both the region {@code r} and it's underlying WAL. For use in tests. + */ + public static void closeRegionAndWAL(final Region r) throws IOException { + closeRegionAndWAL((HRegion)r); + } + /** * Close both the HRegion {@code r} and it's underlying WAL. For use in tests. */ @@ -2131,6 +2139,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return loadRegion(r, f, false); } + public int loadRegion(final Region r, final byte[] f) throws IOException { + return loadRegion((HRegion)r, f); + } + /** * Load region with rows from 'aaa' to 'zzz'. * @param r Region @@ -2152,8 +2164,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { Put put = new Put(k); put.setDurability(Durability.SKIP_WAL); put.add(f, null, k); - if (r.getWAL() == null) put.setDurability(Durability.SKIP_WAL); - + if (r.getWAL() == null) { + put.setDurability(Durability.SKIP_WAL); + } int preRowCount = rowCount; int pause = 10; int maxPause = 1000; @@ -2169,7 +2182,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } } if (flush) { - r.flushcache(); + r.flush(true); } } return rowCount; @@ -2204,11 +2217,21 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } } + public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow) + throws IOException { + verifyNumericRows((HRegion)region, f, startRow, endRow); + } + public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow) throws IOException { verifyNumericRows(region, f, startRow, endRow, true); } + public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow, + final boolean present) throws IOException { + verifyNumericRows((HRegion)region, f, startRow, endRow, present); + } + public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow, final boolean present) throws IOException { for (int i = startRow; i < endRow; i++) { @@ -3755,10 +3778,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { if (server.equals(rs.getServerName())) { continue; } - Collection hrs = rs.getOnlineRegionsLocalContext(); - for (HRegion r: hrs) { + Collection hrs = rs.getOnlineRegionsLocalContext(); + for (Region r: hrs) { assertTrue("Region should not be double assigned", - r.getRegionId() != hri.getRegionId()); + r.getRegionInfo().getRegionId() != hri.getRegionId()); } } return; // good, we are happy diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java index 24b6e714dbb..4a02b043ae4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.util.JVMClusterUtil; @@ -535,8 +536,8 @@ public class MiniHBaseCluster extends HBaseCluster { public void flushcache() throws IOException { for (JVMClusterUtil.RegionServerThread t: this.hbaseCluster.getRegionServers()) { - for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { - r.flushcache(); + for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) { + r.flush(true); } } } @@ -548,9 +549,9 @@ public class MiniHBaseCluster extends HBaseCluster { public void flushcache(TableName tableName) throws IOException { for (JVMClusterUtil.RegionServerThread t: this.hbaseCluster.getRegionServers()) { - for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { + for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) { if(r.getTableDesc().getTableName().equals(tableName)) { - r.flushcache(); + r.flush(true); } } } @@ -563,8 +564,8 @@ public class MiniHBaseCluster extends HBaseCluster { public void compact(boolean major) throws IOException { for (JVMClusterUtil.RegionServerThread t: this.hbaseCluster.getRegionServers()) { - for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { - r.compactStores(major); + for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) { + r.compact(major); } } } @@ -576,9 +577,9 @@ public class MiniHBaseCluster extends HBaseCluster { public void compact(TableName tableName, boolean major) throws IOException { for (JVMClusterUtil.RegionServerThread t: this.hbaseCluster.getRegionServers()) { - for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { + for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) { if(r.getTableDesc().getTableName().equals(tableName)) { - r.compactStores(major); + r.compact(major); } } } @@ -615,9 +616,9 @@ public class MiniHBaseCluster extends HBaseCluster { List ret = new ArrayList(); for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) { HRegionServer hrs = rst.getRegionServer(); - for (HRegion region : hrs.getOnlineRegionsLocalContext()) { + for (Region region : hrs.getOnlineRegionsLocalContext()) { if (region.getTableDesc().getTableName().equals(tableName)) { - ret.add(region); + ret.add((HRegion)region); } } } @@ -643,8 +644,7 @@ public class MiniHBaseCluster extends HBaseCluster { int count = 0; for (JVMClusterUtil.RegionServerThread rst: getRegionServerThreads()) { HRegionServer hrs = rst.getRegionServer(); - HRegion metaRegion = - hrs.getOnlineRegion(regionName); + Region metaRegion = hrs.getOnlineRegion(regionName); if (metaRegion != null) { index = count; break; @@ -662,7 +662,7 @@ public class MiniHBaseCluster extends HBaseCluster { // should hold some regions. Please refer to #countServedRegions // to see how we find out all regions. HMaster master = getMaster(); - HRegion region = master.getOnlineRegion(regionName); + Region region = master.getOnlineRegion(regionName); if (region != null) { return master.getServerName(); } @@ -712,9 +712,9 @@ public class MiniHBaseCluster extends HBaseCluster { ArrayList ret = new ArrayList(); for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) { HRegionServer hrs = rst.getRegionServer(); - for (HRegion region : hrs.getOnlineRegions(tableName)) { + for (Region region : hrs.getOnlineRegions(tableName)) { if (region.getTableDesc().getTableName().equals(tableName)) { - ret.add(region); + ret.add((HRegion)region); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java index 6d0a3c7caa4..810ab9089f4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java @@ -40,9 +40,9 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager; import org.apache.hadoop.hbase.regionserver.CompactionRequestor; import org.apache.hadoop.hbase.regionserver.FlushRequester; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HeapMemoryManager; import org.apache.hadoop.hbase.regionserver.Leases; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.ServerNonceManager; @@ -59,7 +59,7 @@ import com.google.protobuf.Service; */ public class MockRegionServerServices implements RegionServerServices { protected static final Log LOG = LogFactory.getLog(MockRegionServerServices.class); - private final Map regions = new HashMap(); + private final Map regions = new HashMap(); private final ConcurrentSkipListMap rit = new ConcurrentSkipListMap(Bytes.BYTES_COMPARATOR); private HFileSystem hfs = null; @@ -90,17 +90,17 @@ public class MockRegionServerServices implements RegionServerServices { } @Override - public boolean removeFromOnlineRegions(HRegion r, ServerName destination) { + public boolean removeFromOnlineRegions(Region r, ServerName destination) { return this.regions.remove(r.getRegionInfo().getEncodedName()) != null; } @Override - public HRegion getFromOnlineRegions(String encodedRegionName) { + public Region getFromOnlineRegions(String encodedRegionName) { return this.regions.get(encodedRegionName); } @Override - public List getOnlineRegions(TableName tableName) throws IOException { + public List getOnlineRegions(TableName tableName) throws IOException { return null; } @@ -110,13 +110,12 @@ public class MockRegionServerServices implements RegionServerServices { } @Override - public void addToOnlineRegions(HRegion r) { + public void addToOnlineRegions(Region r) { this.regions.put(r.getRegionInfo().getEncodedName(), r); } @Override - public void postOpenDeployTasks(HRegion r) - throws KeeperException, IOException { + public void postOpenDeployTasks(Region r) throws KeeperException, IOException { addToOnlineRegions(r); } @@ -258,7 +257,7 @@ public class MockRegionServerServices implements RegionServerServices { } @Override - public Map getRecoveringRegions() { + public Map getRecoveringRegions() { // TODO Auto-generated method stub return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java index d8178f072d3..2d081647ff1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java @@ -31,8 +31,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; @@ -90,7 +90,7 @@ public class TestGlobalMemStoreSize { ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) { globalMemStoreSize += server.getFromOnlineRegions(regionInfo.getEncodedName()). - getMemstoreSize().get(); + getMemstoreSize(); } assertEquals(server.getRegionServerAccounting().getGlobalMemstoreSize(), globalMemStoreSize); @@ -104,7 +104,7 @@ public class TestGlobalMemStoreSize { for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) { - HRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName()); + Region r = server.getFromOnlineRegions(regionInfo.getEncodedName()); flush(r, server); } LOG.info("Post flush on " + server.getServerName()); @@ -120,14 +120,14 @@ public class TestGlobalMemStoreSize { // our test was running.... for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) { - HRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName()); - long l = r.getMemstoreSize().longValue(); + Region r = server.getFromOnlineRegions(regionInfo.getEncodedName()); + long l = r.getMemstoreSize(); if (l > 0) { // Only meta could have edits at this stage. Give it another flush // clear them. assertTrue(regionInfo.isMetaRegion()); LOG.info(r.toString() + " " + l + ", reflushing"); - r.flushcache(); + r.flush(true); } } } @@ -145,10 +145,10 @@ public class TestGlobalMemStoreSize { * @param server * @throws IOException */ - private void flush(final HRegion r, final HRegionServer server) + private void flush(final Region r, final HRegionServer server) throws IOException { LOG.info("Flush " + r.toString() + " on " + server.getServerName() + - ", " + r.flushcache() + ", size=" + + ", " + r.flush(true) + ", size=" + server.getRegionServerAccounting().getGlobalMemstoreSize()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java index f44eb7b9b3c..48a5dbd3d03 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; @@ -290,13 +291,14 @@ public class TestIOFencing { long startWaitTime = System.currentTimeMillis(); while (compactingRegion.getEarliestFlushTimeForAllStores() <= lastFlushTime || compactingRegion.countStoreFiles() <= 1) { - LOG.info("Waiting for the region to flush " + compactingRegion.getRegionNameAsString()); + LOG.info("Waiting for the region to flush " + + compactingRegion.getRegionInfo().getRegionNameAsString()); Thread.sleep(1000); assertTrue("Timed out waiting for the region to flush", System.currentTimeMillis() - startWaitTime < 30000); } assertTrue(compactingRegion.countStoreFiles() > 1); - final byte REGION_NAME[] = compactingRegion.getRegionName(); + final byte REGION_NAME[] = compactingRegion.getRegionInfo().getRegionName(); LOG.info("Asking for compaction"); ((HBaseAdmin)admin).majorCompact(TABLE_NAME.getName()); LOG.info("Waiting for compaction to be about to start"); @@ -314,7 +316,7 @@ public class TestIOFencing { Waiter.waitFor(c, 60000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { - HRegion newRegion = newServer.getOnlineRegion(REGION_NAME); + Region newRegion = newServer.getOnlineRegion(REGION_NAME); return newRegion != null && !newRegion.isRecovering(); } }); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java index 903ce0ec2eb..3fc7594675c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java @@ -42,8 +42,8 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; @@ -231,7 +231,7 @@ public class TestHFileArchiving { List servingRegions = UTIL.getHBaseCluster().getRegions(TABLE_NAME); // make sure we only have 1 region serving this table assertEquals(1, servingRegions.size()); - HRegion region = servingRegions.get(0); + Region region = servingRegions.get(0); // get the parent RS and monitor HRegionServer hrs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); @@ -242,7 +242,7 @@ public class TestHFileArchiving { UTIL.loadRegion(region, TEST_FAM); // get the hfiles in the region - List regions = hrs.getOnlineRegions(TABLE_NAME); + List regions = hrs.getOnlineRegions(TABLE_NAME); assertEquals("More that 1 region for test table.", 1, regions.size()); region = regions.get(0); @@ -257,7 +257,8 @@ public class TestHFileArchiving { clearArchiveDirectory(); // then get the current store files - List storeFiles = getRegionStoreFiles(region); + byte[][]columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]); + List storeFiles = region.getStoreFileList(columns); // then delete the table so the hfiles get archived UTIL.deleteTable(TABLE_NAME); @@ -310,7 +311,7 @@ public class TestHFileArchiving { List servingRegions = UTIL.getHBaseCluster().getRegions(TABLE_NAME); // make sure we only have 1 region serving this table assertEquals(1, servingRegions.size()); - HRegion region = servingRegions.get(0); + Region region = servingRegions.get(0); // get the parent RS and monitor HRegionServer hrs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); @@ -321,7 +322,7 @@ public class TestHFileArchiving { UTIL.loadRegion(region, TEST_FAM); // get the hfiles in the region - List regions = hrs.getOnlineRegions(TABLE_NAME); + List regions = hrs.getOnlineRegions(TABLE_NAME); assertEquals("More that 1 region for test table.", 1, regions.size()); region = regions.get(0); @@ -336,7 +337,8 @@ public class TestHFileArchiving { clearArchiveDirectory(); // then get the current store files - List storeFiles = getRegionStoreFiles(region); + byte[][]columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]); + List storeFiles = region.getStoreFileList(columns); // then delete the table so the hfiles get archived UTIL.getHBaseAdmin().deleteColumn(TABLE_NAME, TEST_FAM); @@ -449,19 +451,4 @@ public class TestHFileArchiving { } return fileNames; } - - private List getRegionStoreFiles(final HRegion region) throws IOException { - Path regionDir = region.getRegionFileSystem().getRegionDir(); - FileSystem fs = region.getRegionFileSystem().getFileSystem(); - List storeFiles = getAllFileNames(fs, regionDir); - // remove all the non-storefile named files for the region - for (int i = 0; i < storeFiles.size(); i++) { - String file = storeFiles.get(i); - if (file.contains(HRegionFileSystem.REGION_INFO_FILE) || file.contains("wal")) { - storeFiles.remove(i--); - } - } - storeFiles.remove(HRegionFileSystem.REGION_INFO_FILE); - return storeFiles; - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java index 772c345ccaa..eba3c0bd447 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -170,7 +170,7 @@ public class TestZooKeeperTableArchiveClient { // create the region HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM); - HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); + Region region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); loadFlushAndCompact(region, TEST_FAM); @@ -220,12 +220,12 @@ public class TestZooKeeperTableArchiveClient { // create the region HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM); - HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); + Region region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); loadFlushAndCompact(region, TEST_FAM); // create the another table that we don't archive hcd = new HColumnDescriptor(TEST_FAM); - HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd); + Region otherRegion = UTIL.createTestRegion(otherTable, hcd); loadFlushAndCompact(otherRegion, TEST_FAM); // get the current hfiles in the archive directory @@ -379,7 +379,7 @@ public class TestZooKeeperTableArchiveClient { return allFiles; } - private void loadFlushAndCompact(HRegion region, byte[] family) throws IOException { + private void loadFlushAndCompact(Region region, byte[] family) throws IOException { // create two hfiles in the region createHFileInRegion(region, family); createHFileInRegion(region, family); @@ -391,7 +391,7 @@ public class TestZooKeeperTableArchiveClient { // compact the two files into one file to get files in the archive LOG.debug("Compacting stores"); - region.compactStores(true); + region.compact(true); } /** @@ -400,13 +400,13 @@ public class TestZooKeeperTableArchiveClient { * @param columnFamily family for which to add data * @throws IOException */ - private void createHFileInRegion(HRegion region, byte[] columnFamily) throws IOException { + private void createHFileInRegion(Region region, byte[] columnFamily) throws IOException { // put one row in the region Put p = new Put(Bytes.toBytes("row")); p.add(columnFamily, Bytes.toBytes("Qual"), Bytes.toBytes("v1")); region.put(p); // flush the region to make a store file - region.flushcache(); + region.flush(true); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 460ac19cf67..710cf609989 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -53,6 +52,7 @@ import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -545,11 +545,8 @@ public class TestAdmin2 { + DefaultWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files"); // flush all regions - - List regions = new ArrayList(regionServer - .getOnlineRegionsLocalContext()); - for (HRegion r : regions) { - r.flushcache(); + for (Region r : regionServer.getOnlineRegionsLocalContext()) { + r.flush(true); } admin.rollWALWriter(regionServer.getServerName()); int count = DefaultWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java index 82f62e44cfb..953f64159c7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.backoff.ServerStatistics; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -91,7 +92,7 @@ public class TestClientPushback { HTable table = (HTable) conn.getTable(tablename); HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0); - HRegion region = rs.getOnlineRegions(tablename).get(0); + Region region = rs.getOnlineRegions(tablename).get(0); LOG.debug("Writing some data to "+tablename); // write some data @@ -101,7 +102,7 @@ public class TestClientPushback { table.flushCommits(); // get the current load on RS. Hopefully memstore isn't flushed since we wrote the the data - int load = (int)((region.addAndGetGlobalMemstoreSize(0) * 100) / flushSizeBytes); + int load = (int)((((HRegion)region).addAndGetGlobalMemstoreSize(0) * 100) / flushSizeBytes); LOG.debug("Done writing some data to "+tablename); // get the stats for the region hosting our table @@ -114,7 +115,7 @@ public class TestClientPushback { assertNotNull( "No stats configured for the client!", stats); // get the names so we can query the stats ServerName server = rs.getServerName(); - byte[] regionName = region.getRegionName(); + byte[] regionName = region.getRegionInfo().getRegionName(); // check to see we found some load on the memstore ServerStatistics serverStats = stats.getServerStatsForTesting(server); @@ -125,8 +126,8 @@ public class TestClientPushback { // check that the load reported produces a nonzero delay long backoffTime = backoffPolicy.getBackoffTime(server, regionName, serverStats); assertNotEquals("Reported load does not produce a backoff", backoffTime, 0); - LOG.debug("Backoff calculated for " + region.getRegionNameAsString() + " @ " + server + - " is " + backoffTime); + LOG.debug("Backoff calculated for " + region.getRegionInfo().getRegionNameAsString() + " @ " + + server + " is " + backoffTime); // Reach into the connection and submit work directly to AsyncProcess so we can // monitor how long the submission was delayed via a callback diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index f58bae1e4fb..18c74ac423a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateR import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -4180,7 +4181,7 @@ public class TestFromClientSide { // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow // in Store.rowAtOrBeforeFromStoreFile String regionName = table.getRegionLocations().firstKey().getEncodedName(); - HRegion region = + Region region = TEST_UTIL.getRSForFirstRegionInTable(tableAname).getFromOnlineRegions(regionName); Put put1 = new Put(firstRow); Put put2 = new Put(secondRow); @@ -4199,7 +4200,7 @@ public class TestFromClientSide { table.put(put2); table.put(put3); table.put(put4); - region.flushcache(); + region.flush(true); Result result = null; // Test before first that null is returned @@ -5051,8 +5052,9 @@ public class TestFromClientSide { HTable table = TEST_UTIL.createTable(tableName, FAMILY); // get the block cache and region String regionName = table.getRegionLocations().firstKey().getEncodedName(); - HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName); - Store store = region.getStores().values().iterator().next(); + Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName) + .getFromOnlineRegions(regionName); + Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); cacheConf.setCacheDataOnWrite(true); cacheConf.setEvictOnClose(true); @@ -5087,7 +5089,7 @@ public class TestFromClientSide { assertEquals(startBlockMiss, cache.getStats().getMissCount()); // flush the data System.out.println("Flushing cache"); - region.flushcache(); + region.flush(true); // expect one more block in cache, no change in hits/misses long expectedBlockCount = startBlockCount + 1; long expectedBlockHits = startBlockHits; @@ -5114,7 +5116,7 @@ public class TestFromClientSide { assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); // flush, one new block System.out.println("Flushing cache"); - region.flushcache(); + region.flush(true); assertEquals(++expectedBlockCount, cache.getBlockCount()); assertEquals(expectedBlockHits, cache.getStats().getHitCount()); assertEquals(expectedBlockMiss, cache.getStats().getMissCount()); @@ -5122,7 +5124,7 @@ public class TestFromClientSide { System.out.println("Compacting"); assertEquals(2, store.getStorefilesCount()); store.triggerMajorCompaction(); - region.compactStores(); + region.compact(true); waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max assertEquals(1, store.getStorefilesCount()); expectedBlockCount -= 2; // evicted two blocks, cached none diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index 0b08562df03..53b36e386d6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -64,8 +64,8 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.master.HMaster; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -989,9 +989,9 @@ public class TestHCM { ServerName destServerName = destServer.getServerName(); //find another row in the cur server that is less than ROW_X - List regions = curServer.getOnlineRegions(TABLE_NAME3); + List regions = curServer.getOnlineRegions(TABLE_NAME3); byte[] otherRow = null; - for (HRegion region : regions) { + for (Region region : regions) { if (!region.getRegionInfo().getEncodedName().equals(toMove.getRegionInfo().getEncodedName()) && Bytes.BYTES_COMPARATOR.compare(region.getRegionInfo().getStartKey(), ROW_X) < 0) { otherRow = region.getRegionInfo().getStartKey(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java index d85ef5db499..c9a628a0207 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java @@ -33,9 +33,9 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse; import org.apache.hadoop.hbase.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import com.google.protobuf.RpcCallback; @@ -87,9 +87,9 @@ implements Coprocessor, CoprocessorService { int sumResult = 0; InternalScanner scanner = null; try { - HRegion region = this.env.getRegion(); + Region region = this.env.getRegion(); // for the last region in the table, return null to test null handling - if (Bytes.equals(region.getEndKey(), HConstants.EMPTY_END_ROW)) { + if (Bytes.equals(region.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW)) { done.run(null); return; } @@ -123,6 +123,6 @@ implements Coprocessor, CoprocessorService { } done.run(SumResponse.newBuilder().setSum(sumResult).build()); LOG.info("Returning sum " + sumResult + " for region " + - Bytes.toStringBinary(env.getRegion().getRegionName())); + Bytes.toStringBinary(env.getRegion().getRegionInfo().getRegionName())); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java index 3a382972c00..0c4d07605d9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java @@ -34,9 +34,9 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest; import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse; import org.apache.hadoop.hbase.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import com.google.protobuf.RpcCallback; @@ -88,9 +88,9 @@ implements Coprocessor, CoprocessorService { int sumResult = 0; InternalScanner scanner = null; try { - HRegion region = this.env.getRegion(); + Region region = this.env.getRegion(); // throw an exception for requests to the last region in the table, to test error handling - if (Bytes.equals(region.getEndKey(), HConstants.EMPTY_END_ROW)) { + if (Bytes.equals(region.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW)) { throw new DoNotRetryIOException("An expected exception"); } scanner = region.getScanner(scan); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java index 7100ae7c4fd..f4981f17354 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java @@ -54,12 +54,12 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HRegion.Operation; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.Leases; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.Region.Operation; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; @@ -151,8 +151,8 @@ public class SimpleRegionObserver extends BaseRegionObserver { // from external packages RegionCoprocessorEnvironment re = (RegionCoprocessorEnvironment)e; Leases leases = re.getRegionServerServices().getLeases(); - leases.createLease(re.getRegion().getRegionNameAsString(), 2000, null); - leases.cancelLease(re.getRegion().getRegionNameAsString()); + leases.createLease(re.getRegion().getRegionInfo().getRegionNameAsString(), 2000, null); + leases.cancelLease(re.getRegion().getRegionInfo().getRegionNameAsString()); } @Override @@ -229,7 +229,7 @@ public class SimpleRegionObserver extends BaseRegionObserver { } @Override - public void postSplit(ObserverContext c, HRegion l, HRegion r) { + public void postSplit(ObserverContext c, Region l, Region r) { ctPostSplit.incrementAndGet(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java index 140c3b9d6c2..6a5080bf3c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java @@ -20,11 +20,10 @@ package org.apache.hadoop.hbase.coprocessor; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.TestServerCustomProtocol; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -163,12 +162,12 @@ public class TestClassLoading { // verify that the coprocessors were loaded boolean foundTableRegion=false; boolean found1 = true, found2 = true, found2_k1 = true, found2_k2 = true, found2_k3 = true; - Map> regionsActiveClassLoaders = - new HashMap>(); + Map> regionsActiveClassLoaders = + new HashMap>(); MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster(); - for (HRegion region: + for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { - if (region.getRegionNameAsString().startsWith(tableName.getNameAsString())) { + if (region.getRegionInfo().getRegionNameAsString().startsWith(tableName.getNameAsString())) { foundTableRegion = true; CoprocessorEnvironment env; env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName1); @@ -206,7 +205,7 @@ public class TestClassLoading { //check if region active classloaders are shared across all RS regions Set externalClassLoaders = new HashSet( CoprocessorClassLoader.getAllCached()); - for (Map.Entry> regionCP : regionsActiveClassLoaders.entrySet()) { + for (Map.Entry> regionCP : regionsActiveClassLoaders.entrySet()) { assertTrue("Some CP classloaders for region " + regionCP.getKey() + " are not cached." + " ClassLoader Cache:" + externalClassLoaders + " Region ClassLoaders:" + regionCP.getValue(), @@ -235,9 +234,8 @@ public class TestClassLoading { // verify that the coprocessor was loaded boolean found = false; MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster(); - for (HRegion region: - hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { - if (region.getRegionNameAsString().startsWith(cpName3)) { + for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { + if (region.getRegionInfo().getRegionNameAsString().startsWith(cpName3)) { found = (region.getCoprocessorHost().findCoprocessor(cpName3) != null); } } @@ -261,9 +259,8 @@ public class TestClassLoading { // verify that the coprocessor was loaded correctly boolean found = false; MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster(); - for (HRegion region: - hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { - if (region.getRegionNameAsString().startsWith(cpName4)) { + for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { + if (region.getRegionInfo().getRegionNameAsString().startsWith(cpName4)) { Coprocessor cp = region.getCoprocessorHost().findCoprocessor(cpName4); if (cp != null) { found = true; @@ -333,9 +330,8 @@ public class TestClassLoading { found6_k4 = false; MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster(); - for (HRegion region: - hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { - if (region.getRegionNameAsString().startsWith(tableName.getNameAsString())) { + for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { + if (region.getRegionInfo().getRegionNameAsString().startsWith(tableName.getNameAsString())) { found_1 = found_1 || (region.getCoprocessorHost().findCoprocessor(cpName1) != null); found_2 = found_2 || @@ -422,9 +418,8 @@ public class TestClassLoading { boolean found1 = false, found2 = false, found2_k1 = false, found2_k2 = false, found2_k3 = false; MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster(); - for (HRegion region: - hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { - if (region.getRegionNameAsString().startsWith(tableName.getNameAsString())) { + for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { + if (region.getRegionInfo().getRegionNameAsString().startsWith(tableName.getNameAsString())) { CoprocessorEnvironment env; env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName1); if (env != null) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java index 75fe93d5c9c..6deade88145 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; @@ -231,7 +232,7 @@ public class TestCoprocessorInterface { preSplitWithSplitRowCalled = true; } @Override - public void postSplit(ObserverContext e, HRegion l, HRegion r) { + public void postSplit(ObserverContext e, Region l, Region r) { postSplitCalled = true; } @@ -297,20 +298,19 @@ public class TestCoprocessorInterface { byte [][] families = { fam1, fam2, fam3 }; Configuration hc = initSplit(); - HRegion region = initHRegion(tableName, name.getMethodName(), hc, + Region region = initHRegion(tableName, name.getMethodName(), hc, new Class[]{}, families); for (int i = 0; i < 3; i++) { HBaseTestCase.addContent(region, fam3); - region.flushcache(); + region.flush(true); } - region.compactStores(); - - byte [] splitRow = region.checkSplit(); + region.compact(false); + byte [] splitRow = ((HRegion)region).checkSplit(); assertNotNull(splitRow); - HRegion [] regions = split(region, splitRow); + Region [] regions = split(region, splitRow); for (int i = 0; i < regions.length; i++) { regions[i] = reopenRegion(regions[i], CoprocessorImpl.class, CoprocessorII.class); } @@ -336,7 +336,7 @@ public class TestCoprocessorInterface { // now have all Environments fail for (int i = 0; i < regions.length; i++) { try { - byte [] r = regions[i].getStartKey(); + byte [] r = regions[i].getRegionInfo().getStartKey(); if (r == null || r.length <= 0) { // Its the start row. Can't ask for null. Ask for minimal key instead. r = new byte [] {0}; @@ -376,19 +376,19 @@ public class TestCoprocessorInterface { byte [][] families = { fam1, fam2, fam3 }; Configuration hc = initSplit(); - HRegion region = initHRegion(tableName, name.getMethodName(), hc, + Region region = initHRegion(tableName, name.getMethodName(), hc, new Class[]{CoprocessorImpl.class}, families); for (int i = 0; i < 3; i++) { HBaseTestCase.addContent(region, fam3); - region.flushcache(); + region.flush(true); } - region.compactStores(); + region.compact(false); - byte [] splitRow = region.checkSplit(); + byte [] splitRow = ((HRegion)region).checkSplit(); assertNotNull(splitRow); - HRegion [] regions = split(region, splitRow); + Region [] regions = split(region, splitRow); for (int i = 0; i < regions.length; i++) { regions[i] = reopenRegion(regions[i], CoprocessorImpl.class); } @@ -423,10 +423,10 @@ public class TestCoprocessorInterface { } } - HRegion reopenRegion(final HRegion closedRegion, Class ... implClasses) + Region reopenRegion(final Region closedRegion, Class ... implClasses) throws IOException { //HRegionInfo info = new HRegionInfo(tableName, null, null, false); - HRegion r = HRegion.openHRegion(closedRegion, null); + Region r = HRegion.openHRegion(closedRegion, null); // this following piece is a hack. currently a coprocessorHost // is secretly loaded at OpenRegionHandler. we don't really @@ -434,7 +434,7 @@ public class TestCoprocessorInterface { // and set it to region. Configuration conf = TEST_UTIL.getConfiguration(); RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf); - r.setCoprocessorHost(host); + ((HRegion)r).setCoprocessorHost(host); for (Class implClass : implClasses) { host.load(implClass, Coprocessor.PRIORITY_USER, conf); @@ -450,7 +450,7 @@ public class TestCoprocessorInterface { return r; } - HRegion initHRegion (TableName tableName, String callingMethod, + Region initHRegion (TableName tableName, String callingMethod, Configuration conf, Class [] implClasses, byte [][] families) throws IOException { HTableDescriptor htd = new HTableDescriptor(tableName); @@ -459,11 +459,11 @@ public class TestCoprocessorInterface { } HRegionInfo info = new HRegionInfo(tableName, null, null, false); Path path = new Path(DIR + callingMethod); - HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); + Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); // this following piece is a hack. RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf); - r.setCoprocessorHost(host); + ((HRegion)r).setCoprocessorHost(host); for (Class implClass : implClasses) { host.load(implClass, Coprocessor.PRIORITY_USER, conf); @@ -497,10 +497,8 @@ public class TestCoprocessorInterface { return TEST_UTIL.getConfiguration(); } - private HRegion [] split(final HRegion r, final byte [] splitRow) - throws IOException { - - HRegion[] regions = new HRegion[2]; + private Region [] split(final Region r, final byte [] splitRow) throws IOException { + Region[] regions = new Region[2]; SplitTransaction st = new SplitTransaction(r, splitRow); int i = 0; @@ -513,18 +511,18 @@ public class TestCoprocessorInterface { Server mockServer = Mockito.mock(Server.class); when(mockServer.getConfiguration()).thenReturn( TEST_UTIL.getConfiguration()); - PairOfSameType daughters = st.execute(mockServer, null); - for (HRegion each_daughter: daughters) { + PairOfSameType daughters = st.execute(mockServer, null); + for (Region each_daughter: daughters) { regions[i] = each_daughter; i++; } } catch (IOException ioe) { - LOG.info("Split transaction of " + r.getRegionNameAsString() + + LOG.info("Split transaction of " + r.getRegionInfo().getRegionNameAsString() + " failed:" + ioe.getMessage()); assertTrue(false); } catch (RuntimeException e) { LOG.info("Failed rollback of failed split of " + - r.getRegionNameAsString() + e.getMessage()); + r.getRegionInfo().getRegionNameAsString() + e.getMessage()); } assertTrue(i == 2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index 6c7552a950b..00808bde896 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.ScanType; @@ -143,7 +144,7 @@ public class TestRegionObserverScannerOpenHook { } } - HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf, + Region initHRegion(byte[] tableName, String callingMethod, Configuration conf, byte[]... families) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); for (byte[] family : families) { @@ -170,7 +171,7 @@ public class TestRegionObserverScannerOpenHook { byte[][] FAMILIES = new byte[][] { A }; Configuration conf = HBaseConfiguration.create(); - HRegion region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES); + Region region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES); RegionCoprocessorHost h = region.getCoprocessorHost(); h.load(NoDataFromScan.class, Coprocessor.PRIORITY_HIGHEST, conf); h.load(EmptyRegionObsever.class, Coprocessor.PRIORITY_USER, conf); @@ -195,7 +196,7 @@ public class TestRegionObserverScannerOpenHook { byte[][] FAMILIES = new byte[][] { A }; Configuration conf = HBaseConfiguration.create(); - HRegion region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES); + Region region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES); RegionCoprocessorHost h = region.getCoprocessorHost(); h.load(NoDataFromFlush.class, Coprocessor.PRIORITY_HIGHEST, conf); h.load(EmptyRegionObsever.class, Coprocessor.PRIORITY_USER, conf); @@ -204,7 +205,7 @@ public class TestRegionObserverScannerOpenHook { Put put = new Put(ROW); put.add(A, A, A); region.put(put); - region.flushcache(); + region.flush(true); Get get = new Get(ROW); Result r = region.get(get); assertNull( @@ -272,10 +273,10 @@ public class TestRegionObserverScannerOpenHook { table.put(put); HRegionServer rs = UTIL.getRSForFirstRegionInTable(desc.getTableName()); - List regions = rs.getOnlineRegions(desc.getTableName()); + List regions = rs.getOnlineRegions(desc.getTableName()); assertEquals("More than 1 region serving test table with 1 row", 1, regions.size()); - HRegion region = regions.get(0); - admin.flushRegion(region.getRegionName()); + Region region = regions.get(0); + admin.flushRegion(region.getRegionInfo().getRegionName()); CountDownLatch latch = ((CompactionCompletionNotifyingRegion)region) .getCompactionStateChangeLatch(); @@ -283,7 +284,7 @@ public class TestRegionObserverScannerOpenHook { put = new Put(Bytes.toBytes("anotherrow")); put.add(A, A, A); table.put(put); - admin.flushRegion(region.getRegionName()); + admin.flushRegion(region.getRegionInfo().getRegionName()); // run a compaction, which normally would should get rid of the data // wait for the compaction checker to complete diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java index 2e6eabc2648..de43feb76a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java @@ -36,10 +36,10 @@ import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; @@ -90,7 +90,7 @@ public class TestRegionServerObserver { desc.addFamily(new HColumnDescriptor(FAM)); admin.createTable(desc, new byte[][] { Bytes.toBytes("row") }); assertFalse(regionServerObserver.wasRegionMergeCalled()); - List regions = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)); + List regions = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)); admin.mergeRegions(regions.get(0).getRegionInfo().getEncodedNameAsBytes(), regions.get(1) .getRegionInfo().getEncodedNameAsBytes(), true); int regionsCount = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)).size(); @@ -130,18 +130,18 @@ public class TestRegionServerObserver { } @Override - public void preMerge(ObserverContext ctx, HRegion regionA, - HRegion regionB) throws IOException { + public void preMerge(ObserverContext ctx, Region regionA, + Region regionB) throws IOException { preMergeCalled = true; } @Override public void preMergeCommit(ObserverContext ctx, - HRegion regionA, HRegion regionB, List metaEntries) throws IOException { + Region regionA, Region regionB, List metaEntries) throws IOException { preMergeBeforePONRCalled = true; RegionServerCoprocessorEnvironment environment = ctx.getEnvironment(); HRegionServer rs = (HRegionServer) environment.getRegionServerServices(); - List onlineRegions = + List onlineRegions = rs.getOnlineRegions(TableName.valueOf("testRegionServerObserver_2")); rmt = new RegionMergeTransaction(onlineRegions.get(0), onlineRegions.get(1), true); if (!rmt.prepare(rs)) { @@ -159,7 +159,7 @@ public class TestRegionServerObserver { @Override public void postMergeCommit(ObserverContext ctx, - HRegion regionA, HRegion regionB, HRegion mr) throws IOException { + Region regionA, Region regionB, Region mr) throws IOException { preMergeAfterPONRCalled = true; RegionServerCoprocessorEnvironment environment = ctx.getEnvironment(); HRegionServer rs = (HRegionServer) environment.getRegionServerServices(); @@ -168,19 +168,19 @@ public class TestRegionServerObserver { @Override public void preRollBackMerge(ObserverContext ctx, - HRegion regionA, HRegion regionB) throws IOException { + Region regionA, Region regionB) throws IOException { preRollBackMergeCalled = true; } @Override public void postRollBackMerge(ObserverContext ctx, - HRegion regionA, HRegion regionB) throws IOException { + Region regionA, Region regionB) throws IOException { postRollBackMergeCalled = true; } @Override - public void postMerge(ObserverContext c, HRegion regionA, - HRegion regionB, HRegion mergedRegion) throws IOException { + public void postMerge(ObserverContext c, Region regionA, + Region regionB, Region mergedRegion) throws IOException { postMergeCalled = true; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index 9d448b47cd8..82ea5d463fc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -67,7 +68,7 @@ import com.google.common.base.Throwables; @Category({FilterTests.class, SmallTests.class}) public class TestFilter { private final static Log LOG = LogFactory.getLog(TestFilter.class); - private HRegion region; + private Region region; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); // @@ -165,7 +166,7 @@ public class TestFilter { } // Flush - this.region.flushcache(); + this.region.flush(true); // Insert second half (reverse families) for(byte [] ROW : ROWS_ONE) { @@ -242,7 +243,7 @@ public class TestFilter { this.region.put(p); } // Flush - this.region.flushcache(); + this.region.flush(true); // Insert second half (reverse families) for (byte[] ROW : ROWS_THREE) { @@ -1451,7 +1452,7 @@ public class TestFilter { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("TestFilter")); htd.addFamily(new HColumnDescriptor(family)); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), + Region testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd); for(int i=0; i<5; i++) { @@ -1460,7 +1461,7 @@ public class TestFilter { p.add(family, qualifier, Bytes.toBytes(String.valueOf(111+i))); testRegion.put(p); } - testRegion.flushcache(); + testRegion.flush(true); // rows starting with "b" PrefixFilter pf = new PrefixFilter(new byte[] {'b'}) ; @@ -1486,8 +1487,8 @@ public class TestFilter { assertEquals(2, resultCount); scanner.close(); - WAL wal = testRegion.getWAL(); - testRegion.close(); + WAL wal = ((HRegion)testRegion).getWAL(); + ((HRegion)testRegion).close(); wal.close(); } @@ -1820,7 +1821,7 @@ public class TestFilter { p.setDurability(Durability.SKIP_WAL); p.add(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]); this.region.put(p); - this.region.flushcache(); + this.region.flush(true); // Set of KVs (page: 1; pageSize: 1) - the first set of 1 column per row KeyValue [] expectedKVs = { @@ -2011,7 +2012,7 @@ public class TestFilter { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testNestedFilterListWithSCVF")); htd.addFamily(new HColumnDescriptor(FAMILIES[0])); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), + Region testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd); for(int i=0; i<10; i++) { Put p = new Put(Bytes.toBytes("row" + i)); @@ -2019,7 +2020,7 @@ public class TestFilter { p.add(FAMILIES[0], columnStatus, Bytes.toBytes(i%2)); testRegion.put(p); } - testRegion.flushcache(); + testRegion.flush(true); // 1. got rows > "row4" Filter rowFilter = new RowFilter(CompareOp.GREATER,new BinaryComparator(Bytes.toBytes("row4"))); Scan s1 = new Scan(); @@ -2095,8 +2096,8 @@ public class TestFilter { results.clear(); } assertFalse(NextState.hasMoreValues(scanner.next(results))); - WAL wal = testRegion.getWAL(); - testRegion.close(); + WAL wal = ((HRegion)testRegion).getWAL(); + ((HRegion)testRegion).close(); wal.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java index c94054b9ba7..b88bbbfaf4e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java @@ -35,10 +35,11 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -61,7 +62,7 @@ public class TestInvocationRecordFilter { private static final String VALUE_PREFIX = "value"; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private HRegion region; + private Region region; @Before public void setUp() throws Exception { @@ -79,7 +80,7 @@ public class TestInvocationRecordFilter { Bytes.toBytes(VALUE_PREFIX + i)); } this.region.put(put); - this.region.flushcache(); + this.region.flush(true); } @Test @@ -151,8 +152,8 @@ public class TestInvocationRecordFilter { @After public void tearDown() throws Exception { - WAL wal = region.getWAL(); - region.close(); + WAL wal = ((HRegion)region).getWAL(); + ((HRegion)region).close(); wal.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java index 613d1ea92aa..504350c9ffc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -285,7 +286,7 @@ public class TestBlockReorder { int nbTest = 0; while (nbTest < 10) { - final List regions = targetRs.getOnlineRegions(h.getName()); + final List regions = targetRs.getOnlineRegions(h.getName()); final CountDownLatch latch = new CountDownLatch(regions.size()); // listen for successful log rolls final WALActionsListener listener = new WALActionsListener.Base() { @@ -294,8 +295,8 @@ public class TestBlockReorder { latch.countDown(); } }; - for (HRegion region : regions) { - region.getWAL().registerWALActionsListener(listener); + for (Region region : regions) { + ((HRegion)region).getWAL().registerWALActionsListener(listener); } htu.getHBaseAdmin().rollWALWriter(targetRs.getServerName()); @@ -308,8 +309,8 @@ public class TestBlockReorder { "tests fail, it's probably because we should still be waiting."); Thread.currentThread().interrupt(); } - for (HRegion region : regions) { - region.getWAL().unregisterWALActionsListener(listener); + for (Region region : regions) { + ((HRegion)region).getWAL().unregisterWALActionsListener(listener); } // We need a sleep as the namenode is informed asynchronously diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java index e0874579556..5ccb20652b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.LruBlockCache; import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -113,7 +113,7 @@ public class TestEncodedSeekers { setBlocksize(BLOCK_SIZE). setBloomFilterType(BloomType.NONE). setCompressTags(compressTags); - HRegion region = testUtil.createTestRegion(TABLE_NAME, hcd); + Region region = testUtil.createTestRegion(TABLE_NAME, hcd); //write the data, but leave some in the memstore doPuts(region); @@ -122,10 +122,9 @@ public class TestEncodedSeekers { doGets(region); //verify correctness again after compacting - region.compactStores(); + region.compact(false); doGets(region); - Map encodingCounts = cache.getEncodingCountsForTest(); // Ensure that compactions don't pollute the cache with unencoded blocks @@ -138,7 +137,7 @@ public class TestEncodedSeekers { } - private void doPuts(HRegion region) throws IOException{ + private void doPuts(Region region) throws IOException{ LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(MIN_VALUE_SIZE, MAX_VALUE_SIZE); for (int i = 0; i < NUM_ROWS; ++i) { byte[] key = LoadTestKVGenerator.md5PrefixedKey(i).getBytes(); @@ -162,13 +161,13 @@ public class TestEncodedSeekers { region.put(put); } if (i % NUM_ROWS_PER_FLUSH == 0) { - region.flushcache(); + region.flush(true); } } } - private void doGets(HRegion region) throws IOException{ + private void doGets(Region region) throws IOException{ for (int i = 0; i < NUM_ROWS; ++i) { final byte[] rowKey = LoadTestKVGenerator.md5PrefixedKey(i).getBytes(); for (int j = 0; j < NUM_COLS_PER_ROW; ++j) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java index 6a64119e41e..1eda567f923 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java @@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -66,7 +66,7 @@ public class TestPrefixTree { private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); - private HRegion region; + private Region region; @Before public void setUp() throws Exception { @@ -101,7 +101,7 @@ public class TestPrefixTree { put = new Put(row4_bytes); put.addColumn(fam, qual2, Bytes.toBytes("c2-value-3")); region.put(put); - region.flushcache(); + region.flush(true); String[] rows = new String[3]; rows[0] = row1; rows[1] = row2; @@ -182,7 +182,7 @@ public class TestPrefixTree { region.put(new Put(Bytes.toBytes("obj29")).addColumn(fam, qual1, Bytes.toBytes("whatever"))); region.put(new Put(Bytes.toBytes("obj2")).addColumn(fam, qual1, Bytes.toBytes("whatever"))); region.put(new Put(Bytes.toBytes("obj3")).addColumn(fam, qual1, Bytes.toBytes("whatever"))); - region.flushcache(); + region.flush(true); Scan scan = new Scan(Bytes.toBytes("obj29995")); RegionScanner scanner = region.getScanner(scan); List cells = new ArrayList(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java index 7ec7e081aae..00639cfc692 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -433,7 +434,7 @@ public class TestCacheOnWrite { final String cf = "myCF"; final byte[] cfBytes = Bytes.toBytes(cf); final int maxVersions = 3; - HRegion region = TEST_UTIL.createTestRegion(table, + Region region = TEST_UTIL.createTestRegion(table, new HColumnDescriptor(cf) .setCompressionType(compress) .setBloomFilterType(BLOOM_TYPE) @@ -467,18 +468,18 @@ public class TestCacheOnWrite { p.setDurability(Durability.ASYNC_WAL); region.put(p); } - region.flushcache(); + region.flush(true); } clearBlockCache(blockCache); assertEquals(0, blockCache.getBlockCount()); - region.compactStores(); + region.compact(false); LOG.debug("compactStores() returned"); for (CachedBlock block: blockCache) { assertNotEquals(BlockType.ENCODED_DATA, block.getBlockType()); assertNotEquals(BlockType.DATA, block.getBlockType()); } - region.close(); + ((HRegion)region).close(); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java index 2af3a6eaeef..762584267ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; import org.junit.Test; @@ -111,7 +111,7 @@ public class TestForceCacheImportantBlocks { setBloomFilterType(BLOOM_TYPE); hcd.setBlocksize(BLOCK_SIZE); hcd.setBlockCacheEnabled(cfCacheEnabled); - HRegion region = TEST_UTIL.createTestRegion(TABLE, hcd); + Region region = TEST_UTIL.createTestRegion(TABLE, hcd); BlockCache cache = region.getStore(hcd.getName()).getCacheConfig().getBlockCache(); CacheStats stats = cache.getStats(); writeTestData(region); @@ -128,7 +128,7 @@ public class TestForceCacheImportantBlocks { else assertTrue(stats.getMissCount() > missCount); } - private void writeTestData(HRegion region) throws IOException { + private void writeTestData(Region region) throws IOException { for (int i = 0; i < NUM_ROWS; ++i) { Put put = new Put(Bytes.toBytes("row" + i)); for (int j = 0; j < NUM_COLS_PER_ROW; ++j) { @@ -139,7 +139,7 @@ public class TestForceCacheImportantBlocks { } region.put(put); if ((i + 1) % ROWS_PER_HFILE == 0) { - region.flushcache(); + region.flush(true); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java index 4f29af37dad..6baadbb32bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java @@ -36,11 +36,11 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.Test; @@ -100,7 +100,7 @@ public class TestScannerSelectionUsingKeyRange { HTableDescriptor htd = new HTableDescriptor(TABLE); htd.addFamily(hcd); HRegionInfo info = new HRegionInfo(TABLE); - HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf, + Region region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf, htd); for (int iFile = 0; iFile < NUM_FILES; ++iFile) { @@ -112,7 +112,7 @@ public class TestScannerSelectionUsingKeyRange { } region.put(put); } - region.flushcache(); + region.flush(true); } Scan scan = new Scan(Bytes.toBytes("aaa"), Bytes.toBytes("aaz")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java index 17ed451b974..4e0743dbb65 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java @@ -35,10 +35,10 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -107,10 +107,8 @@ public class TestScannerSelectionUsingTTL { HTableDescriptor htd = new HTableDescriptor(TABLE); htd.addFamily(hcd); HRegionInfo info = new HRegionInfo(TABLE); - HRegion region = - HBaseTestingUtility.createRegionAndWAL(info, - TEST_UTIL.getDataTestDir(info.getEncodedName()), - conf, htd); + Region region = HBaseTestingUtility.createRegionAndWAL(info, + TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, htd); long ts = EnvironmentEdgeManager.currentTime(); long version = 0; //make sure each new set of Put's have a new ts @@ -128,7 +126,7 @@ public class TestScannerSelectionUsingTTL { } region.put(put); } - region.flushcache(); + region.flush(true); version++; } @@ -156,7 +154,7 @@ public class TestScannerSelectionUsingTTL { HStore store = (HStore)region.getStore(FAMILY_BYTES); store.compactRecentForTestingAssumingDefaultPolicy(totalNumFiles); } else { - region.compactStores(); + region.compact(false); } HBaseTestingUtility.closeRegionAndWAL(region); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java index 8bd67718ade..a64b7e05df1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java @@ -38,14 +38,11 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -54,7 +51,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.Tool; @@ -249,11 +246,11 @@ public class TestImportTSVWithOperationAttributes implements Configurable { @Override public void prePut(ObserverContext e, Put put, WALEdit edit, Durability durability) throws IOException { - HRegion region = e.getEnvironment().getRegion(); + Region region = e.getEnvironment().getRegion(); if (!region.getRegionInfo().isMetaTable() && !region.getRegionInfo().getTable().isSystemTable()) { if (put.getAttribute(TEST_ATR_KEY) != null) { - LOG.debug("allow any put to happen " + region.getRegionNameAsString()); + LOG.debug("allow any put to happen " + region.getRegionInfo().getRegionNameAsString()); } else { e.bypass(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java index a5cceb03f20..53bdf709daf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java @@ -41,12 +41,11 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; - import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -159,7 +158,7 @@ public class TestImportTSVWithTTLs implements Configurable { @Override public void prePut(ObserverContext e, Put put, WALEdit edit, Durability durability) throws IOException { - HRegion region = e.getEnvironment().getRegion(); + Region region = e.getEnvironment().getRegion(); if (!region.getRegionInfo().isMetaTable() && !region.getRegionInfo().getTable().isSystemTable()) { // The put carries the TTL attribute diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index abe2c651df0..cb92a7425af 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -99,6 +99,7 @@ import org.apache.hadoop.hbase.regionserver.FlushRequester; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HeapMemoryManager; import org.apache.hadoop.hbase.regionserver.Leases; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.ServerNonceManager; @@ -255,12 +256,12 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public void addToOnlineRegions(HRegion r) { + public void addToOnlineRegions(Region r) { // TODO Auto-generated method stub } @Override - public boolean removeFromOnlineRegions(HRegion r, ServerName destination) { + public boolean removeFromOnlineRegions(Region r, ServerName destination) { // TODO Auto-generated method stub return false; } @@ -334,8 +335,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public void postOpenDeployTasks(HRegion r) - throws KeeperException, IOException { + public void postOpenDeployTasks(Region r) throws KeeperException, IOException { // TODO Auto-generated method stub } @@ -531,7 +531,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public List getOnlineRegions(TableName tableName) throws IOException { + public List getOnlineRegions(TableName tableName) throws IOException { // TODO Auto-generated method stub return null; } @@ -583,7 +583,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public Map getRecoveringRegions() { + public Map getRecoveringRegions() { // TODO Auto-generated method stub return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java index f482dfa837a..abeab3febae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java @@ -34,17 +34,15 @@ import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; - import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; - import org.junit.experimental.categories.Category; @Category({MasterTests.class, MediumTests.class}) @@ -233,8 +231,8 @@ public class TestAssignmentListener { admin.majorCompact(TABLE_NAME); mergeable = 0; for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) { - for (HRegion region: regionThread.getRegionServer().getOnlineRegions(TABLE_NAME)) { - mergeable += region.isMergeable() ? 1 : 0; + for (Region region: regionThread.getRegionServer().getOnlineRegions(TABLE_NAME)) { + mergeable += ((HRegion)region).isMergeable() ? 1 : 0; } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index 0038d71befa..8a16c0df8d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -90,6 +90,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -1479,7 +1480,7 @@ public class TestDistributedLogSplitting { } LOG.debug("adding data to rs = " + rst.getName() + " region = "+ hri.getRegionNameAsString()); - HRegion region = hrs.getOnlineRegion(hri.getRegionName()); + Region region = hrs.getOnlineRegion(hri.getRegionName()); assertTrue(region != null); putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family); } @@ -1500,7 +1501,7 @@ public class TestDistributedLogSplitting { } LOG.debug("adding data to rs = " + mt.getName() + " region = "+ hri.getRegionNameAsString()); - HRegion region = hrs.getOnlineRegion(hri.getRegionName()); + Region region = hrs.getOnlineRegion(hri.getRegionName()); assertTrue(region != null); putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family); } @@ -1614,7 +1615,7 @@ public class TestDistributedLogSplitting { TEST_UTIL.waitUntilNoRegionsInTransition(60000); } - private void putData(HRegion region, byte[] startRow, int numRows, byte [] qf, + private void putData(Region region, byte[] startRow, int numRows, byte [] qf, byte [] ...families) throws IOException { for(int i = 0; i < numRows; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java index e4be517cb69..5e6bff809ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java @@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; @@ -77,10 +77,10 @@ public class TestGetLastFlushedSequenceId { table.flushCommits(); MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster(); List rsts = cluster.getRegionServerThreads(); - HRegion region = null; + Region region = null; for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { HRegionServer hrs = rsts.get(i).getRegionServer(); - for (HRegion r : hrs.getOnlineRegions(tableName)) { + for (Region r : hrs.getOnlineRegions(tableName)) { region = r; break; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java index f0c6bc2c6dc..3daa711c34f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -383,9 +384,9 @@ public class TestMasterFailover { // region server should expire (how it can be verified?) MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(), rs.getServerName(), State.PENDING_OPEN); - HRegion meta = rs.getFromOnlineRegions(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); + Region meta = rs.getFromOnlineRegions(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); rs.removeFromOnlineRegions(meta, null); - meta.close(); + ((HRegion)meta).close(); log("Aborting master"); activeMaster.abort("test-kill"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index 40342919b81..a00741f1eab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java @@ -55,8 +55,8 @@ import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper; import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer; import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan; import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan.Position; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -189,10 +189,10 @@ public class TestRegionPlacement { // kill a random non-meta server carrying at least one region killIndex = random.nextInt(SLAVES); serverToKill = TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getServerName(); - Collection regs = + Collection regs = TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getOnlineRegionsLocalContext(); isNamespaceServer = false; - for (HRegion r : regs) { + for (Region r : regs) { if (r.getRegionInfo().getTable().getNamespaceAsString() .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { isNamespaceServer = true; @@ -416,8 +416,7 @@ public class TestRegionPlacement { MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); for (int i = 0; i < SLAVES; i++) { HRegionServer rs = cluster.getRegionServer(i); - for (HRegion region: rs.getOnlineRegions( - TableName.valueOf("testRegionAssignment"))) { + for (Region region: rs.getOnlineRegions(TableName.valueOf("testRegionAssignment"))) { InetSocketAddress[] favoredSocketAddress = rs.getFavoredNodesForRegion( region.getRegionInfo().getEncodedName()); List favoredServerList = plan.getAssignmentMap().get(region.getRegionInfo()); @@ -446,7 +445,7 @@ public class TestRegionPlacement { assertNotNull(addrFromPlan); assertTrue("Region server " + rs.getServerName().getHostAndPort() + " has the " + positions[j] + - " for region " + region.getRegionNameAsString() + " is " + + " for region " + region.getRegionInfo().getRegionNameAsString() + " is " + addrFromRS + " which is inconsistent with the plan " + addrFromPlan, addrFromRS.equals(addrFromPlan)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java index 55b1e1f1d48..a19a97566e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java @@ -374,7 +374,7 @@ public class TestTableLockManager { try { HRegion region = TEST_UTIL.getSplittableRegion(tableName, -1); if (region != null) { - byte[] regionName = region.getRegionName(); + byte[] regionName = region.getRegionInfo().getRegionName(); admin.flushRegion(regionName); admin.compactRegion(regionName); admin.splitRegion(regionName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java index 1c0ae116556..c86e0ffbbc6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java @@ -72,6 +72,7 @@ import org.apache.hadoop.hbase.quotas.QuotaExceededException; import org.apache.hadoop.hbase.quotas.QuotaUtil; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -281,7 +282,7 @@ public class TestNamespaceAuditor { @Override public synchronized void preMerge(ObserverContext ctx, - HRegion regionA, HRegion regionB) throws IOException { + Region regionA, Region regionB) throws IOException { triggered = true; notifyAll(); if (shouldFailMerge) { @@ -458,7 +459,8 @@ public class TestNamespaceAuditor { observer = (CustomObserver) actualRegion.getCoprocessorHost().findCoprocessor( CustomObserver.class.getName()); assertNotNull(observer); - ADMIN.split(tableOne, getSplitKey(actualRegion.getStartKey(), actualRegion.getEndKey())); + ADMIN.split(tableOne, getSplitKey(actualRegion.getRegionInfo().getStartKey(), + actualRegion.getRegionInfo().getEndKey())); observer.postSplit.await(); // Make sure no regions have been added. List hris = ADMIN.getTableRegions(tableOne); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java index 36493cd6ee1..d5e022144d7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java @@ -74,7 +74,7 @@ public class NoOpScanPolicyObserver extends BaseRegionObserver { public KeyValueScanner preStoreScannerOpen(final ObserverContext c, Store store, final Scan scan, final NavigableSet targetCols, KeyValueScanner s) throws IOException { - HRegion r = c.getEnvironment().getRegion(); + Region r = c.getEnvironment().getRegion(); return scan.isReversed() ? new ReversedStoreScanner(store, store.getScanInfo(), scan, targetCols, r.getReadpoint(scan .getIsolationLevel())) : new StoreScanner(store, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index 9a2c23bca86..478e239e1ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -81,7 +81,7 @@ public class TestAtomicOperation { static final Log LOG = LogFactory.getLog(TestAtomicOperation.class); @Rule public TestName name = new TestName(); - HRegion region = null; + Region region = null; private HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); // Test names @@ -102,7 +102,7 @@ public class TestAtomicOperation { @After public void teardown() throws IOException { if (region != null) { - region.close(); + ((HRegion)region).close(); region = null; } } @@ -126,11 +126,11 @@ public class TestAtomicOperation { a.setReturnResults(false); a.add(fam1, qual1, Bytes.toBytes(v1)); a.add(fam1, qual2, Bytes.toBytes(v2)); - assertNull(region.append(a)); + assertNull(region.append(a, HConstants.NO_NONCE, HConstants.NO_NONCE)); a = new Append(row); a.add(fam1, qual1, Bytes.toBytes(v2)); a.add(fam1, qual2, Bytes.toBytes(v1)); - Result result = region.append(a); + Result result = region.append(a, HConstants.NO_NONCE, HConstants.NO_NONCE); assertEquals(0, Bytes.compareTo(Bytes.toBytes(v1+v2), result.getValue(fam1, qual1))); assertEquals(0, Bytes.compareTo(Bytes.toBytes(v2+v1), result.getValue(fam1, qual2))); } @@ -217,12 +217,12 @@ public class TestAtomicOperation { */ public static class Incrementer extends Thread { - private final HRegion region; + private final Region region; private final int numIncrements; private final int amount; - public Incrementer(HRegion region, + public Incrementer(Region region, int threadNumber, int amount, int numIncrements) { this.region = region; this.numIncrements = numIncrements; @@ -239,7 +239,7 @@ public class TestAtomicOperation { inc.addColumn(fam1, qual2, amount*2); inc.addColumn(fam2, qual3, amount*3); inc.setDurability(Durability.ASYNC_WAL); - region.increment(inc); + region.increment(inc, HConstants.NO_NONCE, HConstants.NO_NONCE); // verify: Make sure we only see completed increments Get g = new Get(row); @@ -277,7 +277,7 @@ public class TestAtomicOperation { a.add(fam1, qual2, val); a.add(fam2, qual3, val); a.setDurability(Durability.ASYNC_WAL); - region.append(a); + region.append(a, HConstants.NO_NONCE, HConstants.NO_NONCE); Get g = new Get(row); Result result = region.get(g); @@ -341,9 +341,9 @@ public class TestAtomicOperation { if (i%10==0) { synchronized(region) { LOG.debug("flushing"); - region.flushcache(); + region.flush(true); if (i%100==0) { - region.compactStores(); + region.compact(false); } } } @@ -434,9 +434,9 @@ public class TestAtomicOperation { if (i%10==0) { synchronized(region) { LOG.debug("flushing"); - region.flushcache(); + region.flush(true); if (i%100==0) { - region.compactStores(); + region.compact(false); } } } @@ -461,7 +461,7 @@ public class TestAtomicOperation { p.add(fam1, qual1, value2); mrm.add(p); } - region.mutateRowsWithLocks(mrm, rowsToLock); + region.mutateRowsWithLocks(mrm, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE); op ^= true; // check: should always see exactly one column Scan s = new Scan(row); @@ -500,13 +500,13 @@ public class TestAtomicOperation { } public static class AtomicOperation extends Thread { - protected final HRegion region; + protected final Region region; protected final int numOps; protected final AtomicLong timeStamps; protected final AtomicInteger failures; protected final Random r = new Random(); - public AtomicOperation(HRegion region, int numOps, AtomicLong timeStamps, + public AtomicOperation(Region region, int numOps, AtomicLong timeStamps, AtomicInteger failures) { this.region = region; this.numOps = numOps; @@ -541,14 +541,14 @@ public class TestAtomicOperation { conf.setClass(HConstants.REGION_IMPL, MockHRegion.class, HeapSize.class); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)) .addFamily(new HColumnDescriptor(family)); - final MockHRegion region = (MockHRegion) TEST_UTIL.createLocalHRegion(htd, null, null); - + final Region region = TEST_UTIL.createLocalHRegion(htd, null, null); + Put[] puts = new Put[1]; Put put = new Put(Bytes.toBytes("r1")); put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10")); puts[0] = put; - region.batchMutate(puts); + region.batchMutate(puts, HConstants.NO_NONCE, HConstants.NO_NONCE); MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf); ctx.addThread(new PutThread(ctx, region)); @@ -569,8 +569,8 @@ public class TestAtomicOperation { } private class PutThread extends TestThread { - private MockHRegion region; - PutThread(TestContext ctx, MockHRegion region) { + private Region region; + PutThread(TestContext ctx, Region region) { super(ctx); this.region = region; } @@ -581,13 +581,13 @@ public class TestAtomicOperation { put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50")); puts[0] = put; testStep = TestStep.PUT_STARTED; - region.batchMutate(puts); + region.batchMutate(puts, HConstants.NO_NONCE, HConstants.NO_NONCE); } } private class CheckAndPutThread extends TestThread { - private MockHRegion region; - CheckAndPutThread(TestContext ctx, MockHRegion region) { + private Region region; + CheckAndPutThread(TestContext ctx, Region region) { super(ctx); this.region = region; } @@ -622,10 +622,10 @@ public class TestAtomicOperation { return new WrappedRowLock(super.getRowLockInternal(row, waitForLock)); } - public class WrappedRowLock extends RowLock { + public class WrappedRowLock extends RowLockImpl { private WrappedRowLock(RowLock rowLock) { - super(rowLock.context); + setContext(((RowLockImpl)rowLock).getContext()); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java index cdb57640d50..4a3d35236d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java @@ -69,7 +69,7 @@ public class TestBlocksRead extends HBaseTestCase { return conf; } - HRegion region = null; + Region region = null; private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final String DIR = TEST_UTIL.getDataTestDir("TestBlocksRead").toString(); @@ -98,7 +98,7 @@ public class TestBlocksRead extends HBaseTestCase { * @throws IOException * @return created and initialized region. */ - private HRegion initHRegion(byte[] tableName, String callingMethod, + private Region initHRegion(byte[] tableName, String callingMethod, HBaseConfiguration conf, String family) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); HColumnDescriptor familyDesc; @@ -112,7 +112,7 @@ public class TestBlocksRead extends HBaseTestCase { HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); Path path = new Path(DIR + callingMethod); - HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); + Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); blockCache = new CacheConfig(conf).getBlockCache(); return r; } @@ -234,7 +234,7 @@ public class TestBlocksRead extends HBaseTestCase { putData(FAMILY, "row", "col5", 5); putData(FAMILY, "row", "col6", 6); putData(FAMILY, "row", "col7", 7); - region.flushcache(); + region.flush(true); // Expected block reads: 1 // The top block has the KV we are @@ -287,12 +287,12 @@ public class TestBlocksRead extends HBaseTestCase { // File 1 putData(FAMILY, "row", "col1", 1); putData(FAMILY, "row", "col2", 2); - region.flushcache(); + region.flush(true); // File 2 putData(FAMILY, "row", "col1", 3); putData(FAMILY, "row", "col2", 4); - region.flushcache(); + region.flush(true); // Expected blocks read: 1. // File 2's top block is also the KV we are @@ -312,7 +312,7 @@ public class TestBlocksRead extends HBaseTestCase { // File 3: Add another column putData(FAMILY, "row", "col3", 5); - region.flushcache(); + region.flush(true); // Expected blocks read: 1 // File 3's top block has the "col3" KV we are @@ -331,7 +331,7 @@ public class TestBlocksRead extends HBaseTestCase { // File 4: Delete the entire row. deleteFamily(FAMILY, "row", 6); - region.flushcache(); + region.flush(true); // For ROWCOL Bloom filter: Expected blocks read: 2. // For ROW Bloom filter: Expected blocks read: 3. @@ -347,14 +347,14 @@ public class TestBlocksRead extends HBaseTestCase { // File 5: Delete deleteFamily(FAMILY, "row", 10); - region.flushcache(); + region.flush(true); // File 6: some more puts, but with timestamps older than the // previous delete. putData(FAMILY, "row", "col1", 7); putData(FAMILY, "row", "col2", 8); putData(FAMILY, "row", "col3", 9); - region.flushcache(); + region.flush(true); // Baseline expected blocks read: 6. [HBASE-4532] kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 6, 7, 7); @@ -364,7 +364,7 @@ public class TestBlocksRead extends HBaseTestCase { putData(FAMILY, "row", "col1", 11); putData(FAMILY, "row", "col2", 12); putData(FAMILY, "row", "col3", 13); - region.flushcache(); + region.flush(true); // Expected blocks read: 8. [HBASE-4585, HBASE-13109] @@ -394,7 +394,7 @@ public class TestBlocksRead extends HBaseTestCase { try { putData(FAMILY, "row", "col1", 1); putData(FAMILY, "row", "col2", 2); - region.flushcache(); + region.flush(true); // Execute a scan with caching turned off // Expected blocks stored: 0 @@ -441,7 +441,7 @@ public class TestBlocksRead extends HBaseTestCase { putData(FAMILY, "row", "col" + i, i); } putData(FAMILY, "row", "col99", 201); - region.flushcache(); + region.flush(true); kvs = getData(FAMILY, "row", Arrays.asList("col0"), 2); assertEquals(0, kvs.length); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java index ff092f0e88c..446c64ccafc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java @@ -90,10 +90,9 @@ public class TestBlocksScanned extends HBaseTestCase { } private void _testBlocksScanned(HTableDescriptor table) throws Exception { - HRegion r = createNewHRegion(table, START_KEY, END_KEY, - TEST_UTIL.getConfiguration()); + Region r = createNewHRegion(table, START_KEY, END_KEY, TEST_UTIL.getConfiguration()); addContent(r, FAMILY, COL); - r.flushcache(); + r.flush(true); CacheStats stats = new CacheConfig(TEST_UTIL.getConfiguration()).getBlockCache().getStats(); long before = stats.getHitCount() + stats.getMissCount(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java index 5629050ebd5..aa57e2230d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java @@ -130,25 +130,26 @@ public class TestBulkLoad { }; context.checking(expection); testRegionWithFamiliesAndSpecifiedTableName(tableName, family1) - .bulkLoadHFiles(familyPaths, false); + .bulkLoadHFiles(familyPaths, false, null); } @Test public void bulkHLogShouldThrowNoErrorAndWriteMarkerWithBlankInput() throws IOException { - testRegionWithFamilies(family1).bulkLoadHFiles(new ArrayList>(), false); + testRegionWithFamilies(family1).bulkLoadHFiles(new ArrayList>(), + false, null); } @Test public void shouldBulkLoadSingleFamilyHLog() throws IOException { context.checking(callOnce); - testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1), false); + testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1), false, null); } @Test public void shouldBulkLoadManyFamilyHLog() throws IOException { context.checking(callOnce); testRegionWithFamilies(family1, family2).bulkLoadHFiles(withFamilyPathsFor(family1, family2), - false); + false, null); } @Test @@ -156,31 +157,32 @@ public class TestBulkLoad { context.checking(callOnce); TableName tableName = TableName.valueOf("test", "test"); testRegionWithFamiliesAndSpecifiedTableName(tableName, family1, family2) - .bulkLoadHFiles(withFamilyPathsFor(family1, family2), false); + .bulkLoadHFiles(withFamilyPathsFor(family1, family2), false, null); } @Test(expected = DoNotRetryIOException.class) public void shouldCrashIfBulkLoadFamiliesNotInTable() throws IOException { - testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1, family2), false); + testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1, family2), false, + null); } @Test(expected = DoNotRetryIOException.class) public void bulkHLogShouldThrowErrorWhenFamilySpecifiedAndHFileExistsButNotInTableDescriptor() throws IOException { - testRegionWithFamilies().bulkLoadHFiles(withFamilyPathsFor(family1), false); + testRegionWithFamilies().bulkLoadHFiles(withFamilyPathsFor(family1), false, null); } @Test(expected = DoNotRetryIOException.class) public void shouldThrowErrorIfBadFamilySpecifiedAsFamilyPath() throws IOException { testRegionWithFamilies() .bulkLoadHFiles(asList(withInvalidColumnFamilyButProperHFileLocation(family1)), - false); + false, null); } @Test(expected = FileNotFoundException.class) public void shouldThrowErrorIfHFileDoesNotExist() throws IOException { List> list = asList(withMissingHFileForFamily(family1)); - testRegionWithFamilies(family1).bulkLoadHFiles(list, false); + testRegionWithFamilies(family1).bulkLoadHFiles(list, false, null); } private Pair withMissingHFileForFamily(byte[] family) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java index 0ec122b0560..c09b32d8d17 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java @@ -74,7 +74,7 @@ public class TestColumnSeeking { htd.addFamily(hcd); HRegionInfo info = new HRegionInfo(table, null, null, false); // Set this so that the archiver writes to the temp dir as well. - HRegion region = TEST_UTIL.createLocalHRegion(info, htd); + Region region = TEST_UTIL.createLocalHRegion(info, htd); try { List rows = generateRandomWords(10, "row"); List allColumns = generateRandomWords(10, "column"); @@ -125,17 +125,17 @@ public class TestColumnSeeking { region.put(p); if (Math.random() < flushPercentage) { LOG.info("Flushing... "); - region.flushcache(); + region.flush(true); } if (Math.random() < minorPercentage) { LOG.info("Minor compacting... "); - region.compactStores(false); + region.compact(false); } if (Math.random() < majorPercentage) { LOG.info("Major compacting... "); - region.compactStores(true); + region.compact(true); } } } @@ -186,7 +186,7 @@ public class TestColumnSeeking { htd.addFamily(hcd); HRegionInfo info = new HRegionInfo(table, null, null, false); - HRegion region = TEST_UTIL.createLocalHRegion(info, htd); + Region region = TEST_UTIL.createLocalHRegion(info, htd); List rows = generateRandomWords(10, "row"); List allColumns = generateRandomWords(100, "column"); @@ -238,17 +238,17 @@ public class TestColumnSeeking { region.put(p); if (Math.random() < flushPercentage) { LOG.info("Flushing... "); - region.flushcache(); + region.flush(true); } if (Math.random() < minorPercentage) { LOG.info("Minor compacting... "); - region.compactStores(false); + region.compact(false); } if (Math.random() < majorPercentage) { LOG.info("Major compacting... "); - region.compactStores(true); + region.compact(true); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java index 64668adb5db..9d1136e0862 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -189,7 +189,7 @@ public class TestCompaction { delete.deleteFamily(famAndQf[0]); r.delete(delete); } - r.flushcache(); + r.flush(true); // Multiple versions allowed for an entry, so the delete isn't enough // Lower TTL and expire to ensure that all our entries have been wiped @@ -204,7 +204,7 @@ public class TestCompaction { } Thread.sleep(ttl); - r.compactStores(true); + r.compact(true); assertEquals(0, count()); } } @@ -318,7 +318,7 @@ public class TestCompaction { CountDownLatch latch = new CountDownLatch(numStores); // create some store files and setup requests for each store on which we want to do a // compaction - for (Store store : r.getStores().values()) { + for (Store store : r.getStores()) { createStoreFile(r, store.getColumnFamilyName()); createStoreFile(r, store.getColumnFamilyName()); createStoreFile(r, store.getColumnFamilyName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java index 3f5f9058be7..d9894a5eef5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java @@ -133,7 +133,7 @@ public class TestCompactionState { ht = TEST_UTIL.createTable(table, families); loadData(ht, families, 3000, flushes); HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0); - List regions = rs.getOnlineRegions(table); + List regions = rs.getOnlineRegions(table); int countBefore = countStoreFilesInFamilies(regions, families); int countBeforeSingleFamily = countStoreFilesInFamily(regions, family); assertTrue(countBefore > 0); // there should be some data files @@ -163,7 +163,7 @@ public class TestCompactionState { // Now, should have the right compaction state, // otherwise, the compaction should have already been done if (expectedState != state) { - for (HRegion region: regions) { + for (Region region: regions) { state = region.getCompactionState(); assertEquals(CompactionState.NONE, state); } @@ -201,13 +201,13 @@ public class TestCompactionState { } private static int countStoreFilesInFamily( - List regions, final byte[] family) { + List regions, final byte[] family) { return countStoreFilesInFamilies(regions, new byte[][]{family}); } - private static int countStoreFilesInFamilies(List regions, final byte[][] families) { + private static int countStoreFilesInFamilies(List regions, final byte[][] families) { int count = 0; - for (HRegion region: regions) { + for (Region region: regions) { count += region.getStoreFileList(families).size(); } return count; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index 200790b774f..e1e5b895f78 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -24,7 +24,6 @@ import java.lang.management.MemoryMXBean; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -924,10 +923,10 @@ public class TestDefaultMemStore extends TestCase { HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf); HRegion region = hbaseUtility.createTestRegion("foobar", new HColumnDescriptor("foo")); - Map stores = region.getStores(); + List stores = region.getStores(); assertTrue(stores.size() == 1); - Store s = stores.entrySet().iterator().next().getValue(); + Store s = stores.iterator().next(); edge.setCurrentTimeMillis(1234); s.add(KeyValueTestUtil.create("r", "f", "q", 100, "v")); edge.setCurrentTimeMillis(1234 + 100); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java index b791fdb716e..c4ae2458e39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.Predicate; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.crypto.Encryption; @@ -197,9 +196,9 @@ public class TestEncryptionKeyRotation { private static List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList(); - for (HRegion region: + for (Region region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(tableName)) { - for (Store store: region.getStores().values()) { + for (Store store: region.getStores()) { for (StoreFile storefile: store.getStorefiles()) { paths.add(storefile.getPath()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java index ebfc89c1a7d..0a6b2b57781 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.crypto.Encryption; @@ -54,9 +53,9 @@ public class TestEncryptionRandomKeying { private static List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList(); - for (HRegion region: + for (Region region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { - for (Store store: region.getStores().values()) { + for (Store store: region.getStores()) { for (StoreFile storefile: store.getStorefiles()) { paths.add(storefile.getPath()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 523aadc3e47..ca96f509ce5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -105,12 +105,12 @@ public class TestEndToEndSplitTransaction { // this will also cache the region byte[] regionName = conn.getRegionLocator(tableName).getRegionLocation(splitRow) .getRegionInfo().getRegionName(); - HRegion region = server.getRegion(regionName); + Region region = server.getRegion(regionName); SplitTransaction split = new SplitTransaction(region, splitRow); split.prepare(); // 1. phase I - PairOfSameType regions = split.createDaughters(server, server); + PairOfSameType regions = split.createDaughters(server, server); assertFalse(test(conn, tableName, firstRow, server)); assertFalse(test(conn, tableName, lastRow, server)); @@ -403,7 +403,7 @@ public class TestEndToEndSplitTransaction { admin.flushRegion(regionName); log("blocking until flush is complete: " + Bytes.toStringBinary(regionName)); Threads.sleepWithoutInterrupt(500); - while (rs.getOnlineRegion(regionName).getMemstoreSize().get() > 0) { + while (rs.getOnlineRegion(regionName).getMemstoreSize() > 0) { Threads.sleep(50); } } @@ -415,7 +415,7 @@ public class TestEndToEndSplitTransaction { log("blocking until compaction is complete: " + Bytes.toStringBinary(regionName)); Threads.sleepWithoutInterrupt(500); outer: for (;;) { - for (Store store : rs.getOnlineRegion(regionName).getStores().values()) { + for (Store store : rs.getOnlineRegion(regionName).getStores()) { if (store.getStorefilesCount() > 1) { Threads.sleep(50); continue outer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index 6c8c77afc61..416ee28137f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -75,7 +76,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { // Up flush size else we bind up when we use default catalog flush of 16k. fsTableDescriptors.get(TableName.META_TABLE_NAME).setMemStoreFlushSize(64 * 1024 * 1024); - HRegion mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO, + Region mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO, rootdir, this.conf, fsTableDescriptors.get(TableName.META_TABLE_NAME)); try { // Write rows for three tables 'A', 'B', and 'C'. @@ -107,7 +108,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { findRow(mr, 'C', 45, 44); findRow(mr, 'C', 46, 46); findRow(mr, 'C', 43, 42); - mr.flushcache(); + mr.flush(true); findRow(mr, 'C', 44, 44); findRow(mr, 'C', 45, 44); findRow(mr, 'C', 46, 46); @@ -132,7 +133,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { findRow(mr, 'C', 45, -1); findRow(mr, 'C', 46, -1); findRow(mr, 'C', 43, -1); - mr.flushcache(); + mr.flush(true); findRow(mr, 'C', 44, -1); findRow(mr, 'C', 45, -1); findRow(mr, 'C', 46, -1); @@ -150,7 +151,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { * @return Row found. * @throws IOException */ - private byte [] findRow(final HRegion mr, final char table, + private byte [] findRow(final Region mr, final char table, final int rowToFind, final int answer) throws IOException { TableName tableb = TableName.valueOf("" + table); @@ -160,7 +161,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { tableb, tofindBytes, HConstants.NINES, false); LOG.info("find=" + new String(metaKey)); - Result r = mr.getClosestRowBefore(metaKey); + Result r = mr.getClosestRowBefore(metaKey, HConstants.CATALOG_FAMILY); if (answer == -1) { assertNull(r); return null; @@ -187,7 +188,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { */ @Test public void testGetClosestRowBefore3() throws IOException{ - HRegion region = null; + Region region = null; byte [] c0 = COLUMNS[0]; byte [] c1 = COLUMNS[1]; try { @@ -232,7 +233,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { r = region.getClosestRowBefore(T31, c0); assertTrue(Bytes.equals(T10, r.getRow())); - region.flushcache(); + region.flush(true); // try finding "010" after flush r = region.getClosestRowBefore(T30, c0); @@ -250,7 +251,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { r = region.getClosestRowBefore(T31, c0); assertTrue(Bytes.equals(T10, r.getRow())); - region.flushcache(); + region.flush(true); r = region.getClosestRowBefore(T30, c0); assertTrue(Bytes.equals(T10, r.getRow())); @@ -268,7 +269,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { // Ask for a value off the end of the file. Should return t10. r = region.getClosestRowBefore(T31, c0); assertTrue(Bytes.equals(T10, r.getRow())); - region.flushcache(); + region.flush(true); r = region.getClosestRowBefore(T31, c0); assertTrue(Bytes.equals(T10, r.getRow())); @@ -284,11 +285,12 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { } finally { if (region != null) { try { - region.close(); + WAL wal = ((HRegion)region).getWAL(); + ((HRegion)region).close(); + wal.close(); } catch (Exception e) { e.printStackTrace(); } - region.getWAL().close(); } } } @@ -296,7 +298,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { /** For HBASE-694 */ @Test public void testGetClosestRowBefore2() throws IOException{ - HRegion region = null; + Region region = null; byte [] c0 = COLUMNS[0]; try { HTableDescriptor htd = createTableDescriptor(getName()); @@ -318,7 +320,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { Result r = region.getClosestRowBefore(T35, c0); assertTrue(Bytes.equals(T30, r.getRow())); - region.flushcache(); + region.flush(true); // try finding "035" r = region.getClosestRowBefore(T35, c0); @@ -332,7 +334,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { r = region.getClosestRowBefore(T35, c0); assertTrue(Bytes.equals(T30, r.getRow())); - region.flushcache(); + region.flush(true); // try finding "035" r = region.getClosestRowBefore(T35, c0); @@ -340,11 +342,12 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase { } finally { if (region != null) { try { - region.close(); + WAL wal = ((HRegion)region).getWAL(); + ((HRegion)region).close(); + wal.close(); } catch (Exception e) { e.printStackTrace(); } - region.getWAL().close(); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index ca5135d7e10..cb8d0befc95 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -128,8 +128,8 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.Stor import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor; import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; -import org.apache.hadoop.hbase.regionserver.HRegion.RowLock; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; +import org.apache.hadoop.hbase.regionserver.Region.RowLock; import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler; @@ -260,7 +260,7 @@ public class TestHRegion { region.put(put); // Close with something in memstore and something in the snapshot. Make sure all is cleared. region.close(); - assertEquals(0, region.getMemstoreSize().get()); + assertEquals(0, region.getMemstoreSize()); HBaseTestingUtility.closeRegionAndWAL(region); } @@ -371,17 +371,17 @@ public class TestHRegion { // Initialize region region = initHRegion(tableName, null, null, callingMethod, conf, false, Durability.SYNC_WAL, wal, COLUMN_FAMILY_BYTES); - long size = region.getMemstoreSize().get(); + long size = region.getMemstoreSize(); Assert.assertEquals(0, size); // Put one item into memstore. Measure the size of one item in memstore. Put p1 = new Put(row); p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[])null)); region.put(p1); - final long sizeOfOnePut = region.getMemstoreSize().get(); + final long sizeOfOnePut = region.getMemstoreSize(); // Fail a flush which means the current memstore will hang out as memstore 'snapshot'. try { LOG.info("Flushing"); - region.flushcache(); + region.flush(true); Assert.fail("Didn't bubble up IOE!"); } catch (DroppedSnapshotException dse) { // What we are expecting @@ -389,20 +389,20 @@ public class TestHRegion { // Make it so all writes succeed from here on out ffs.fault.set(false); // Check sizes. Should still be the one entry. - Assert.assertEquals(sizeOfOnePut, region.getMemstoreSize().get()); + Assert.assertEquals(sizeOfOnePut, region.getMemstoreSize()); // Now add two entries so that on this next flush that fails, we can see if we // subtract the right amount, the snapshot size only. Put p2 = new Put(row); p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual2, 2, (byte[])null)); p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null)); region.put(p2); - Assert.assertEquals(sizeOfOnePut * 3, region.getMemstoreSize().get()); + Assert.assertEquals(sizeOfOnePut * 3, region.getMemstoreSize()); // Do a successful flush. It will clear the snapshot only. Thats how flushes work. // If already a snapshot, we clear it else we move the memstore to be snapshot and flush // it - region.flushcache(); + region.flush(true); // Make sure our memory accounting is right. - Assert.assertEquals(sizeOfOnePut * 2, region.getMemstoreSize().get()); + Assert.assertEquals(sizeOfOnePut * 2, region.getMemstoreSize()); } finally { HBaseTestingUtility.closeRegionAndWAL(region); } @@ -435,7 +435,7 @@ public class TestHRegion { // Initialize region region = initHRegion(tableName, null, null, callingMethod, conf, false, Durability.SYNC_WAL, wal, COLUMN_FAMILY_BYTES); - long size = region.getMemstoreSize().get(); + long size = region.getMemstoreSize(); Assert.assertEquals(0, size); // Put one item into memstore. Measure the size of one item in memstore. Put p1 = new Put(row); @@ -475,7 +475,7 @@ public class TestHRegion { Put put = new Put(Bytes.toBytes("r1")); put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1")); region.put(put); - region.flushcache(); + region.flush(true); Scan scan = new Scan(); scan.setMaxVersions(3); @@ -484,7 +484,7 @@ public class TestHRegion { Delete delete = new Delete(Bytes.toBytes("r1")); region.delete(delete); - region.flushcache(); + region.flush(true); // open the second scanner RegionScanner scanner2 = region.getScanner(scan); @@ -494,7 +494,7 @@ public class TestHRegion { System.out.println("Smallest read point:" + region.getSmallestReadPoint()); // make a major compaction - region.compactStores(true); + region.compact(true); // open the third scanner RegionScanner scanner3 = region.getScanner(scan); @@ -526,7 +526,7 @@ public class TestHRegion { put = new Put(Bytes.toBytes("r2")); put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1")); region.put(put); - region.flushcache(); + region.flush(true); Scan scan = new Scan(); scan.setMaxVersions(3); @@ -535,7 +535,7 @@ public class TestHRegion { System.out.println("Smallest read point:" + region.getSmallestReadPoint()); - region.compactStores(true); + region.compact(true); scanner1.reseek(Bytes.toBytes("r2")); List results = new ArrayList(); @@ -578,7 +578,7 @@ public class TestHRegion { } MonitoredTask status = TaskMonitor.get().createStatus(method); Map maxSeqIdInStores = new TreeMap(Bytes.BYTES_COMPARATOR); - for (Store store : region.getStores().values()) { + for (Store store : region.getStores()) { maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId - 1); } long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status); @@ -632,7 +632,7 @@ public class TestHRegion { long recoverSeqId = 1030; MonitoredTask status = TaskMonitor.get().createStatus(method); Map maxSeqIdInStores = new TreeMap(Bytes.BYTES_COMPARATOR); - for (Store store : region.getStores().values()) { + for (Store store : region.getStores()) { maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1); } long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status); @@ -677,7 +677,7 @@ public class TestHRegion { dos.close(); Map maxSeqIdInStores = new TreeMap(Bytes.BYTES_COMPARATOR); - for (Store store : region.getStores().values()) { + for (Store store : region.getStores()) { maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId); } long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, null); @@ -699,9 +699,9 @@ public class TestHRegion { Path regiondir = region.getRegionFileSystem().getRegionDir(); FileSystem fs = region.getRegionFileSystem().getFileSystem(); byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes(); + byte[][] columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]); - assertEquals(0, region.getStoreFileList( - region.getStores().keySet().toArray(new byte[0][])).size()); + assertEquals(0, region.getStoreFileList(columns).size()); Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir); @@ -737,15 +737,14 @@ public class TestHRegion { long recoverSeqId = 1030; Map maxSeqIdInStores = new TreeMap(Bytes.BYTES_COMPARATOR); MonitoredTask status = TaskMonitor.get().createStatus(method); - for (Store store : region.getStores().values()) { + for (Store store : region.getStores()) { maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1); } long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status); assertEquals(maxSeqId, seqId); // assert that the files are flushed - assertEquals(1, region.getStoreFileList( - region.getStores().keySet().toArray(new byte[0][])).size()); + assertEquals(1, region.getStoreFileList(columns).size()); } finally { HBaseTestingUtility.closeRegionAndWAL(this.region); @@ -773,7 +772,7 @@ public class TestHRegion { Put put = new Put(Bytes.toBytes(i)); put.add(family, Bytes.toBytes(i), Bytes.toBytes(i)); region.put(put); - region.flushcache(); + region.flush(true); } // this will create a region with 3 files @@ -875,7 +874,7 @@ public class TestHRegion { Put put = new Put(Bytes.toBytes(i)); put.add(family, Bytes.toBytes(i), Bytes.toBytes(i)); region.put(put); - region.flushcache(); + region.flush(true); } // this will create a region with 3 files from flush @@ -1039,7 +1038,7 @@ public class TestHRegion { // start cache flush will throw exception try { - region.flushcache(); + region.flush(true); fail("This should have thrown exception"); } catch (DroppedSnapshotException unexpected) { // this should not be a dropped snapshot exception. Meaning that RS will not abort @@ -1052,7 +1051,7 @@ public class TestHRegion { isFlushWALMarker.set(FlushAction.COMMIT_FLUSH); try { - region.flushcache(); + region.flush(true); fail("This should have thrown exception"); } catch (DroppedSnapshotException expected) { // we expect this exception, since we were able to write the snapshot, but failed to @@ -1072,7 +1071,7 @@ public class TestHRegion { isFlushWALMarker.set(FlushAction.COMMIT_FLUSH, FlushAction.ABORT_FLUSH); try { - region.flushcache(); + region.flush(true); fail("This should have thrown exception"); } catch (DroppedSnapshotException expected) { // we expect this exception, since we were able to write the snapshot, but failed to @@ -2443,7 +2442,7 @@ public class TestHRegion { this.region = initHRegion(tableName, method, hc, families); try { LOG.info("" + HBaseTestCase.addContent(region, fam3)); - region.flushcache(); + region.flush(true); region.compactStores(); byte[] splitRow = region.checkSplit(); assertNotNull(splitRow); @@ -2490,7 +2489,7 @@ public class TestHRegion { * @throws IOException */ HRegion[] splitRegion(final HRegion parent, final byte[] midkey) throws IOException { - PairOfSameType result = null; + PairOfSameType result = null; SplitTransaction st = new SplitTransaction(parent, midkey); // If prepare does not return true, for some reason -- logged inside in // the prepare call -- we are not ready to split just now. Just return. @@ -2502,21 +2501,22 @@ public class TestHRegion { result = st.execute(null, null); } catch (IOException ioe) { try { - LOG.info("Running rollback of failed split of " + parent.getRegionNameAsString() + "; " - + ioe.getMessage()); + LOG.info("Running rollback of failed split of " + + parent.getRegionInfo().getRegionNameAsString() + "; " + ioe.getMessage()); st.rollback(null, null); - LOG.info("Successful rollback of failed split of " + parent.getRegionNameAsString()); + LOG.info("Successful rollback of failed split of " + + parent.getRegionInfo().getRegionNameAsString()); return null; } catch (RuntimeException e) { // If failed rollback, kill this server to avoid having a hole in table. - LOG.info("Failed rollback of failed split of " + parent.getRegionNameAsString() - + " -- aborting server", e); + LOG.info("Failed rollback of failed split of " + + parent.getRegionInfo().getRegionNameAsString() + " -- aborting server", e); } } finally { parent.clearSplit(); } - return new HRegion[] { result.getFirst(), result.getSecond() }; + return new HRegion[] { (HRegion)result.getFirst(), (HRegion)result.getSecond() }; } // //////////////////////////////////////////////////////////////////////////// @@ -2813,7 +2813,7 @@ public class TestHRegion { put.add(kv22); put.add(kv21); region.put(put); - region.flushcache(); + region.flush(true); // Expected List expected = new ArrayList(); @@ -2875,19 +2875,19 @@ public class TestHRegion { put.add(kv14); put.add(kv24); region.put(put); - region.flushcache(); + region.flush(true); put = new Put(row1); put.add(kv23); put.add(kv13); region.put(put); - region.flushcache(); + region.flush(true); put = new Put(row1); put.add(kv22); put.add(kv12); region.put(put); - region.flushcache(); + region.flush(true); put = new Put(row1); put.add(kv21); @@ -3018,7 +3018,7 @@ public class TestHRegion { put.add(kv22); put.add(kv21); region.put(put); - region.flushcache(); + region.flush(true); // Expected List expected = new ArrayList(); @@ -3126,19 +3126,19 @@ public class TestHRegion { put.add(kv14); put.add(kv24); region.put(put); - region.flushcache(); + region.flush(true); put = new Put(row1); put.add(kv23); put.add(kv13); region.put(put); - region.flushcache(); + region.flush(true); put = new Put(row1); put.add(kv22); put.add(kv12); region.put(put); - region.flushcache(); + region.flush(true); put = new Put(row1); put.add(kv21); @@ -3356,7 +3356,7 @@ public class TestHRegion { try { LOG.info("" + HBaseTestCase.addContent(region, fam3)); - region.flushcache(); + region.flush(true); region.compactStores(); byte[] splitRow = region.checkSplit(); assertNotNull(splitRow); @@ -3385,7 +3385,7 @@ public class TestHRegion { } HBaseTestCase.addContent(regions[i], fam2); HBaseTestCase.addContent(regions[i], fam1); - regions[i].flushcache(); + regions[i].flush(true); } byte[][] midkeys = new byte[regions.length][]; @@ -3403,7 +3403,8 @@ public class TestHRegion { if (midkeys[i] != null) { rs = splitRegion(regions[i], midkeys[i]); for (int j = 0; j < rs.length; j++) { - sortedMap.put(Bytes.toString(rs[j].getRegionName()), HRegion.openHRegion(rs[j], null)); + sortedMap.put(Bytes.toString(rs[j].getRegionInfo().getRegionName()), + HRegion.openHRegion(rs[j], null)); } } } @@ -3447,7 +3448,7 @@ public class TestHRegion { putData(startRow, numRows, qualifier, families); int splitRow = startRow + numRows; putData(splitRow, numRows, qualifier, families); - region.flushcache(); + region.flush(true); HRegion[] regions = null; try { @@ -3487,7 +3488,7 @@ public class TestHRegion { int splitRow = startRow + numRows; byte[] splitRowBytes = Bytes.toBytes("" + splitRow); putData(splitRow, numRows, qualifier, families); - region.flushcache(); + region.flush(true); HRegion[] regions = null; try { @@ -3578,7 +3579,7 @@ public class TestHRegion { if (i != 0 && i % compactInterval == 0) { // System.out.println("iteration = " + i); - region.compactStores(true); + region.compact(true); } if (i % 10 == 5L) { @@ -3641,7 +3642,7 @@ public class TestHRegion { } } try { - region.flushcache(); + region.flush(true); } catch (IOException e) { if (!done) { LOG.error("Error while flusing cache", e); @@ -3706,7 +3707,7 @@ public class TestHRegion { for (int i = 0; i < testCount; i++) { if (i != 0 && i % compactInterval == 0) { - region.compactStores(true); + region.compact(true); } if (i != 0 && i % flushInterval == 0) { @@ -3729,7 +3730,7 @@ public class TestHRegion { putThread.done(); - region.flushcache(); + region.flush(true); putThread.join(); putThread.checkNoError(); @@ -3876,13 +3877,13 @@ public class TestHRegion { @Override public void doAnAction() throws Exception { - if (region.flushcache().isCompactionNeeded()) { + if (region.flush(true).isCompactionNeeded()) { ++flushesSinceCompact; } // Compact regularly to avoid creating too many files and exceeding // the ulimit. if (flushesSinceCompact == maxFlushesSinceCompact) { - region.compactStores(false); + region.compact(false); flushesSinceCompact = 0; } } @@ -3932,7 +3933,7 @@ public class TestHRegion { if (putThread != null) putThread.done(); - region.flushcache(); + region.flush(true); if (putThread != null) { putThread.join(); @@ -3980,7 +3981,7 @@ public class TestHRegion { put.add(family, qual1, 1L, Bytes.toBytes(1L)); region.put(put); - region.flushcache(); + region.flush(true); Delete delete = new Delete(Bytes.toBytes(1L), 1L); region.delete(delete); @@ -4038,7 +4039,7 @@ public class TestHRegion { region.put(put); } } - region.flushcache(); + region.flush(true); } // before compaction HStore store = (HStore) region.getStore(fam1); @@ -4051,7 +4052,7 @@ public class TestHRegion { assertEquals(num_unique_rows, reader.getFilterEntries()); } - region.compactStores(true); + region.compact(true); // after compaction storeFiles = store.getStorefiles(); @@ -4092,7 +4093,7 @@ public class TestHRegion { region.put(put); // Flush - region.flushcache(); + region.flush(true); // Get rows Get get = new Get(row); @@ -4136,11 +4137,11 @@ public class TestHRegion { Put put = new Put(row); put.add(familyName, col, 1, Bytes.toBytes("SomeRandomValue")); region.put(put); - region.flushcache(); + region.flush(true); Delete del = new Delete(row); region.delete(del); - region.flushcache(); + region.flush(true); // Get remaining rows (should have none) Get get = new Get(row); @@ -4187,7 +4188,7 @@ public class TestHRegion { HRegion firstRegion = htu.getHBaseCluster().getRegions(TableName.valueOf(this.getName())) .get(0); - firstRegion.flushcache(); + firstRegion.flush(true); HDFSBlocksDistribution blocksDistribution1 = firstRegion.getHDFSBlocksDistribution(); // Given the default replication factor is 2 and we have 2 HFiles, @@ -4360,7 +4361,7 @@ public class TestHRegion { public void run() { while (!incrementDone.get()) { try { - region.flushcache(); + region.flush(true); } catch (Exception e) { e.printStackTrace(); } @@ -4447,7 +4448,7 @@ public class TestHRegion { public void run() { while (!appendDone.get()) { try { - region.flushcache(); + region.flush(true); } catch (Exception e) { e.printStackTrace(); } @@ -4521,7 +4522,7 @@ public class TestHRegion { assertEquals(1, kvs.size()); assertArrayEquals(Bytes.toBytes("value0"), CellUtil.cloneValue(kvs.get(0))); - region.flushcache(); + region.flush(true); get = new Get(row); get.addColumn(family, qualifier); get.setMaxVersions(); @@ -4542,7 +4543,7 @@ public class TestHRegion { assertEquals(1, kvs.size()); assertArrayEquals(Bytes.toBytes("value1"), CellUtil.cloneValue(kvs.get(0))); - region.flushcache(); + region.flush(true); get = new Get(row); get.addColumn(family, qualifier); get.setMaxVersions(); @@ -4689,7 +4690,7 @@ public class TestHRegion { putData(primaryRegion, 0, 1000, cq, families); // flush region - primaryRegion.flushcache(); + primaryRegion.flush(true); // open secondary region secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF); @@ -4739,7 +4740,7 @@ public class TestHRegion { putData(primaryRegion, 0, 1000, cq, families); // flush region - primaryRegion.flushcache(); + primaryRegion.flush(true); // open secondary region secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF); @@ -4800,7 +4801,7 @@ public class TestHRegion { putData(primaryRegion, 0, 1000, cq, families); // flush region - primaryRegion.flushcache(); + primaryRegion.flush(true); // open secondary region secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF); @@ -4925,7 +4926,7 @@ public class TestHRegion { this.region = initHRegion(tableName, method, family); // empty memstore, flush doesn't run - HRegion.FlushResult fr = region.flushcache(); + HRegion.FlushResult fr = region.flush(true); assertFalse(fr.isFlushSucceeded()); assertFalse(fr.isCompactionNeeded()); @@ -4933,7 +4934,7 @@ public class TestHRegion { for (int i = 0; i < 2; i++) { Put put = new Put(tableName).add(family, family, tableName); region.put(put); - fr = region.flushcache(); + fr = region.flush(true); assertTrue(fr.isFlushSucceeded()); assertFalse(fr.isCompactionNeeded()); } @@ -4942,7 +4943,7 @@ public class TestHRegion { for (int i = 0; i < 2; i++) { Put put = new Put(tableName).add(family, family, tableName); region.put(put); - fr = region.flushcache(); + fr = region.flush(true); assertTrue(fr.isFlushSucceeded()); assertTrue(fr.isCompactionNeeded()); } @@ -5263,7 +5264,7 @@ public class TestHRegion { put = new Put(rowE); put.add(kv5); region.put(put); - region.flushcache(); + region.flush(true); Scan scan = new Scan(rowD, rowA); scan.addColumn(families[0], col1); scan.setReversed(true); @@ -5343,7 +5344,7 @@ public class TestHRegion { put = new Put(rowE); put.add(kv5); region.put(put); - region.flushcache(); + region.flush(true); Scan scan = new Scan(rowD, rowA); scan.addColumn(families[0], col1); scan.setReversed(true); @@ -5446,7 +5447,7 @@ public class TestHRegion { put.add(kv4_5_4); put.add(kv4_5_5); region.put(put); - region.flushcache(); + region.flush(true); // hfiles(cf1/cf3) : "row1" (1 kvs) / "row2" (1 kv) / "row4" (2 kv) put = new Put(row4); put.add(kv4_5_1); @@ -5458,7 +5459,7 @@ public class TestHRegion { put = new Put(row2); put.add(kv2_4_4); region.put(put); - region.flushcache(); + region.flush(true); // hfiles(cf1/cf3) : "row2"(2 kv) / "row3"(1 kvs) / "row4" (1 kv) put = new Put(row4); put.add(kv4_5_2); @@ -5470,7 +5471,7 @@ public class TestHRegion { put = new Put(row3); put.add(kv3_2_2); region.put(put); - region.flushcache(); + region.flush(true); // memstore(cf1/cf2/cf3) : "row0" (1 kvs) / "row3" ( 1 kv) / "row5" (max) // ( 2 kv) put = new Put(row0); @@ -5569,17 +5570,17 @@ public class TestHRegion { Put put = new Put(row1); put.add(kv1); region.put(put); - region.flushcache(); + region.flush(true); // storefile2 put = new Put(row2); put.add(kv2); region.put(put); - region.flushcache(); + region.flush(true); // storefile3 put = new Put(row3); put.add(kv3); region.put(put); - region.flushcache(); + region.flush(true); // memstore put = new Put(row4); put.add(kv4); @@ -5633,7 +5634,7 @@ public class TestHRegion { int splitRow = startRow + numRows; putData(splitRow, numRows, qualifier, families); int endRow = splitRow + numRows; - region.flushcache(); + region.flush(true); HRegion [] regions = null; try { @@ -5758,7 +5759,7 @@ public class TestHRegion { // create a file in fam1 for the region before opening in OpenRegionHandler region.put(new Put(Bytes.toBytes("a")).add(fam1, fam1, fam1)); - region.flushcache(); + region.flush(true); HBaseTestingUtility.closeRegionAndWAL(region); ArgumentCaptor editCaptor = ArgumentCaptor.forClass(WALEdit.class); @@ -5844,7 +5845,7 @@ public class TestHRegion { // create a file in fam1 for the region before opening in OpenRegionHandler region.put(new Put(Bytes.toBytes("a")).add(fam1, fam1, fam1)); - region.flushcache(); + region.flush(true); HBaseTestingUtility.closeRegionAndWAL(region); ArgumentCaptor editCaptor = ArgumentCaptor.forClass(WALEdit.class); @@ -5854,7 +5855,7 @@ public class TestHRegion { when(rss.getWAL((HRegionInfo) any())).thenReturn(wal); // add the region to recovering regions - HashMap recoveringRegions = Maps.newHashMap(); + HashMap recoveringRegions = Maps.newHashMap(); recoveringRegions.put(region.getRegionInfo().getEncodedName(), null); when(rss.getRecoveringRegions()).thenReturn(recoveringRegions); @@ -6063,7 +6064,7 @@ public class TestHRegion { region.put(new Put(row).add(fam1, q4, now + 10000 - 1, HConstants.EMPTY_BYTE_ARRAY)); // Flush so we are sure store scanning gets this right - region.flushcache(); + region.flush(true); // A query at time T+0 should return all cells Result r = region.get(new Get(row)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java index b89ba005f45..1ced6278bfd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java @@ -68,7 +68,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor; -import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult; +import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl; import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult; import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -219,8 +219,8 @@ public class TestHRegionReplayEvents { verifyData(secondaryRegion, 0, 1000, cq, families); // flush region - FlushResult flush = secondaryRegion.flushcache(); - assertEquals(flush.result, FlushResult.Result.CANNOT_FLUSH); + FlushResultImpl flush = (FlushResultImpl)secondaryRegion.flush(true); + assertEquals(flush.result, FlushResultImpl.Result.CANNOT_FLUSH); verifyData(secondaryRegion, 0, 1000, cq, families); @@ -243,7 +243,7 @@ public class TestHRegionReplayEvents { LOG.info("-- Writing some data to primary from " + start + " to " + (start+100)); putData(primaryRegion, Durability.SYNC_WAL, start, 100, cq, families); LOG.info("-- Flushing primary, creating 3 files for 3 stores"); - primaryRegion.flushcache(); + primaryRegion.flush(true); // now replay the edits and the flush marker reader = createWALReaderForPrimary(); @@ -327,7 +327,7 @@ public class TestHRegionReplayEvents { verifyData(secondaryRegion, 0, lastReplayed, cq, families); Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); long storeMemstoreSize = store.getMemStoreSize(); - long regionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long regionMemstoreSize = secondaryRegion.getMemstoreSize(); long storeFlushableSize = store.getFlushableSize(); long storeSize = store.getSize(); long storeSizeUncompressed = store.getStoreSizeUncompressed(); @@ -349,14 +349,14 @@ public class TestHRegionReplayEvents { // assert that the flush files are picked expectedStoreFileCount++; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } long newFlushableSize = store.getFlushableSize(); assertTrue(storeFlushableSize > newFlushableSize); // assert that the region memstore is smaller now - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); assertTrue(regionMemstoreSize > newRegionMemstoreSize); // assert that the store sizes are bigger @@ -370,7 +370,7 @@ public class TestHRegionReplayEvents { secondaryRegion.replayWALCompactionMarker(compactionDesc, true, false, Long.MAX_VALUE); // assert that the compaction is applied - for (Store store : secondaryRegion.getStores().values()) { + for (Store store : secondaryRegion.getStores()) { if (store.getColumnFamilyName().equals("cf1")) { assertEquals(1, store.getStorefilesCount()); } else { @@ -388,7 +388,7 @@ public class TestHRegionReplayEvents { LOG.info("-- Verifying edits from primary. Ensuring that files are not deleted"); verifyData(primaryRegion, 0, lastReplayed, cq, families); - for (Store store : primaryRegion.getStores().values()) { + for (Store store : primaryRegion.getStores()) { if (store.getColumnFamilyName().equals("cf1")) { assertEquals(1, store.getStorefilesCount()); } else { @@ -426,7 +426,7 @@ public class TestHRegionReplayEvents { // first verify that everything is replayed and visible before flush event replay Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); long storeMemstoreSize = store.getMemStoreSize(); - long regionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long regionMemstoreSize = secondaryRegion.getMemstoreSize(); long storeFlushableSize = store.getFlushableSize(); if (flushDesc.getAction() == FlushAction.START_FLUSH) { @@ -466,7 +466,7 @@ public class TestHRegionReplayEvents { assertNotNull(secondaryRegion.getPrepareFlushResult()); assertEquals(secondaryRegion.getPrepareFlushResult().flushOpSeqId, startFlushDesc.getFlushSequenceNumber()); - assertTrue(secondaryRegion.getMemstoreSize().get() > 0); // memstore is not empty + assertTrue(secondaryRegion.getMemstoreSize() > 0); // memstore is not empty verifyData(secondaryRegion, 0, numRows, cq, families); // Test case 2: replay a flush start marker with a smaller seqId @@ -479,7 +479,7 @@ public class TestHRegionReplayEvents { assertNotNull(secondaryRegion.getPrepareFlushResult()); assertEquals(secondaryRegion.getPrepareFlushResult().flushOpSeqId, startFlushDesc.getFlushSequenceNumber()); - assertTrue(secondaryRegion.getMemstoreSize().get() > 0); // memstore is not empty + assertTrue(secondaryRegion.getMemstoreSize() > 0); // memstore is not empty verifyData(secondaryRegion, 0, numRows, cq, families); // Test case 3: replay a flush start marker with a larger seqId @@ -492,7 +492,7 @@ public class TestHRegionReplayEvents { assertNotNull(secondaryRegion.getPrepareFlushResult()); assertEquals(secondaryRegion.getPrepareFlushResult().flushOpSeqId, startFlushDesc.getFlushSequenceNumber()); - assertTrue(secondaryRegion.getMemstoreSize().get() > 0); // memstore is not empty + assertTrue(secondaryRegion.getMemstoreSize() > 0); // memstore is not empty verifyData(secondaryRegion, 0, numRows, cq, families); LOG.info("-- Verifying edits from secondary"); @@ -558,10 +558,10 @@ public class TestHRegionReplayEvents { // no store files in the region int expectedStoreFileCount = 0; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } - long regionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long regionMemstoreSize = secondaryRegion.getMemstoreSize(); // Test case 1: replay the a flush commit marker smaller than what we have prepared LOG.info("Testing replaying flush COMMIT " + commitFlushDesc + " on top of flush START" @@ -573,7 +573,7 @@ public class TestHRegionReplayEvents { // assert that the flush files are picked expectedStoreFileCount++; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); @@ -581,7 +581,7 @@ public class TestHRegionReplayEvents { assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped // assert that the region memstore is same as before - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); assertEquals(regionMemstoreSize, newRegionMemstoreSize); assertNotNull(secondaryRegion.getPrepareFlushResult()); // not dropped @@ -648,10 +648,10 @@ public class TestHRegionReplayEvents { // no store files in the region int expectedStoreFileCount = 0; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } - long regionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long regionMemstoreSize = secondaryRegion.getMemstoreSize(); // Test case 1: replay the a flush commit marker larger than what we have prepared LOG.info("Testing replaying flush COMMIT " + commitFlushDesc + " on top of flush START" @@ -663,7 +663,7 @@ public class TestHRegionReplayEvents { // assert that the flush files are picked expectedStoreFileCount++; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); @@ -671,7 +671,7 @@ public class TestHRegionReplayEvents { assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped // assert that the region memstore is smaller than before, but not empty - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); assertTrue(newRegionMemstoreSize > 0); assertTrue(regionMemstoreSize > newRegionMemstoreSize); @@ -749,17 +749,17 @@ public class TestHRegionReplayEvents { // no store files in the region int expectedStoreFileCount = 0; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } - long regionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long regionMemstoreSize = secondaryRegion.getMemstoreSize(); // Test case 1: replay a flush commit marker without start flush marker assertNull(secondaryRegion.getPrepareFlushResult()); assertTrue(commitFlushDesc.getFlushSequenceNumber() > 0); // ensure all files are visible in secondary - for (Store store : secondaryRegion.getStores().values()) { + for (Store store : secondaryRegion.getStores()) { assertTrue(store.getMaxSequenceId() <= secondaryRegion.getSequenceId().get()); } @@ -768,7 +768,7 @@ public class TestHRegionReplayEvents { // assert that the flush files are picked expectedStoreFileCount++; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); @@ -780,7 +780,7 @@ public class TestHRegionReplayEvents { } // assert that the region memstore is same as before (we could not drop) - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); if (droppableMemstore) { assertTrue(0 == newRegionMemstoreSize); } else { @@ -847,10 +847,10 @@ public class TestHRegionReplayEvents { // no store files in the region int expectedStoreFileCount = 0; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } - long regionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long regionMemstoreSize = secondaryRegion.getMemstoreSize(); assertTrue(regionMemstoreSize == 0); // now replay the region open event that should contain new file locations @@ -859,7 +859,7 @@ public class TestHRegionReplayEvents { // assert that the flush files are picked expectedStoreFileCount++; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); @@ -867,7 +867,7 @@ public class TestHRegionReplayEvents { assertTrue(newFlushableSize == 0); // assert that the region memstore is empty - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); assertTrue(newRegionMemstoreSize == 0); assertNull(secondaryRegion.getPrepareFlushResult()); //prepare snapshot should be dropped if any @@ -928,7 +928,7 @@ public class TestHRegionReplayEvents { // no store files in the region int expectedStoreFileCount = 0; - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } @@ -938,7 +938,7 @@ public class TestHRegionReplayEvents { // assert that the flush files are picked expectedStoreFileCount = 2; // two flushes happened - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); @@ -946,7 +946,7 @@ public class TestHRegionReplayEvents { assertTrue(newSnapshotSize == 0); // assert that the region memstore is empty - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize().get(); + long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); assertTrue(newRegionMemstoreSize == 0); assertNull(secondaryRegion.getPrepareFlushResult()); //prepare snapshot should be dropped if any @@ -1027,7 +1027,7 @@ public class TestHRegionReplayEvents { LOG.info("-- Writing some data to primary from " + start + " to " + (start+100)); putData(primaryRegion, Durability.SYNC_WAL, start, 100, cq, families); LOG.info("-- Flushing primary, creating 3 files for 3 stores"); - primaryRegion.flushcache(); + primaryRegion.flush(true); // now replay the flush marker reader = createWALReaderForPrimary(); @@ -1125,7 +1125,7 @@ public class TestHRegionReplayEvents { .setAction(FlushAction.START_FLUSH) .setEncodedRegionName( ByteString.copyFrom(primaryRegion.getRegionInfo().getEncodedNameAsBytes())) - .setRegionName(ByteString.copyFrom(primaryRegion.getRegionName())) + .setRegionName(ByteString.copyFrom(primaryRegion.getRegionInfo().getRegionName())) .build()); verify(walSecondary, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(), @@ -1170,15 +1170,15 @@ public class TestHRegionReplayEvents { @Test public void testWriteFlushRequestMarker() throws IOException { // primary region is empty at this point. Request a flush with writeFlushRequestWalMarker=false - FlushResult result = primaryRegion.flushcache(true, false); + FlushResultImpl result = (FlushResultImpl)((HRegion)primaryRegion).flushcache(true, false); assertNotNull(result); - assertEquals(result.result, FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY); + assertEquals(result.result, FlushResultImpl.Result.CANNOT_FLUSH_MEMSTORE_EMPTY); assertFalse(result.wroteFlushWalMarker); // request flush again, but this time with writeFlushRequestWalMarker = true - result = primaryRegion.flushcache(true, true); + result = (FlushResultImpl)((HRegion)primaryRegion).flushcache(true, true); assertNotNull(result); - assertEquals(result.result, FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY); + assertEquals(result.result, FlushResultImpl.Result.CANNOT_FLUSH_MEMSTORE_EMPTY); assertTrue(result.wroteFlushWalMarker); List flushes = Lists.newArrayList(); @@ -1242,7 +1242,7 @@ public class TestHRegionReplayEvents { // put some data in primary putData(primaryRegion, Durability.SYNC_WAL, 0, 100, cq, families); - primaryRegion.flushcache(); + primaryRegion.flush(true); reader = createWALReaderForPrimary(); while (true) { @@ -1276,7 +1276,7 @@ public class TestHRegionReplayEvents { // put some data in primary putData(primaryRegion, Durability.SYNC_WAL, 0, 100, cq, families); - primaryRegion.flushcache(); + primaryRegion.flush(true); reader = createWALReaderForPrimary(); while (true) { @@ -1380,7 +1380,7 @@ public class TestHRegionReplayEvents { LOG.info("-- Replaying edits in secondary"); // Test case 4: replay some edits, ensure that memstore is dropped. - assertTrue(secondaryRegion.getMemstoreSize().get() == 0); + assertTrue(secondaryRegion.getMemstoreSize() == 0); putDataWithFlushes(primaryRegion, 400, 400, 0); numRows = 400; @@ -1398,11 +1398,11 @@ public class TestHRegionReplayEvents { } } - assertTrue(secondaryRegion.getMemstoreSize().get() > 0); + assertTrue(secondaryRegion.getMemstoreSize() > 0); secondaryRegion.refreshStoreFiles(); - assertTrue(secondaryRegion.getMemstoreSize().get() == 0); + assertTrue(secondaryRegion.getMemstoreSize() == 0); LOG.info("-- Verifying edits from primary"); verifyData(primaryRegion, 0, numRows, cq, families); @@ -1466,7 +1466,7 @@ public class TestHRegionReplayEvents { randomValues))); expectedLoadFileCount++; } - primaryRegion.bulkLoadHFiles(familyPaths, false); + primaryRegion.bulkLoadHFiles(familyPaths, false, null); // now replay the edits and the bulk load marker reader = createWALReaderForPrimary(); @@ -1497,7 +1497,7 @@ public class TestHRegionReplayEvents { storeFileName.addAll(storeDesc.getStoreFileList()); } // assert that the bulk loaded files are picked - for (Store s : secondaryRegion.getStores().values()) { + for (Store s : secondaryRegion.getStores()) { for (StoreFile sf : s.getStorefiles()) { storeFileName.remove(sf.getPath().getName()); } @@ -1543,7 +1543,7 @@ public class TestHRegionReplayEvents { LOG.info("-- Writing some data to primary from " + start + " to " + (start+flushInterval)); putData(region, Durability.SYNC_WAL, start, flushInterval, cq, families); LOG.info("-- Flushing primary, creating 3 files for 3 stores"); - region.flushcache(); + region.flush(true); } LOG.info("-- Writing some more data to primary, not flushing"); putData(region, Durability.SYNC_WAL, start, numRowsAfterFlush, cq, families); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java index b96a6a5daf2..2965071331d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java @@ -415,12 +415,12 @@ public class TestHeapMemoryManager { } @Override - public void requestFlush(HRegion region, boolean forceFlushAllStores) { + public void requestFlush(Region region, boolean forceFlushAllStores) { this.listener.flushRequested(flushType, region); } @Override - public void requestDelayedFlush(HRegion region, long delay, boolean forceFlushAllStores) { + public void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java index fafd58f50b7..c480cd8f2af 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java @@ -99,7 +99,7 @@ public class TestKeepDeletes { // keep 3 versions, rows do not expire HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -131,15 +131,15 @@ public class TestKeepDeletes { checkResult(r, c0, c0, T2,T1); // flush - region.flushcache(); + region.flush(true); // yep, T2 still there, T1 gone r = region.get(g); checkResult(r, c0, c0, T2); // major compact - region.compactStores(true); - region.compactStores(true); + region.compact(true); + region.compact(true); // one delete marker left (the others did not // have older puts) @@ -170,9 +170,9 @@ public class TestKeepDeletes { r = region.get(g); assertTrue(r.isEmpty()); - region.flushcache(); - region.compactStores(true); - region.compactStores(true); + region.flush(true); + region.compact(true); + region.compact(true); // verify that the delete marker itself was collected region.put(p); @@ -196,7 +196,7 @@ public class TestKeepDeletes { // KEEP_DELETED_CELLS is NOT enabled HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.FALSE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -216,8 +216,8 @@ public class TestKeepDeletes { scan.next(kvs); assertEquals(2, kvs.size()); - region.flushcache(); - region.compactStores(true); + region.flush(true); + region.compact(true); // after compaction they are gone // (note that this a test with a Store without @@ -241,7 +241,7 @@ public class TestKeepDeletes { // KEEP_DELETED_CELLS is NOT enabled HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.FALSE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -268,10 +268,10 @@ public class TestKeepDeletes { assertTrue(kvs.isEmpty()); // flushing and minor compaction keep delete markers - region.flushcache(); - region.compactStores(); + region.flush(true); + region.compact(false); assertEquals(1, countDeleteMarkers(region)); - region.compactStores(true); + region.compact(true); // major compaction deleted it assertEquals(0, countDeleteMarkers(region)); @@ -285,7 +285,7 @@ public class TestKeepDeletes { public void testRawScanWithColumns() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); Scan s = new Scan(); s.setRaw(true); @@ -309,7 +309,7 @@ public class TestKeepDeletes { public void testRawScan() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -399,7 +399,7 @@ public class TestKeepDeletes { public void testDeleteMarkerExpirationEmptyStore() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -423,13 +423,13 @@ public class TestKeepDeletes { assertEquals(4, countDeleteMarkers(region)); // neither flush nor minor compaction removes any marker - region.flushcache(); + region.flush(true); assertEquals(4, countDeleteMarkers(region)); - region.compactStores(false); + region.compact(false); assertEquals(4, countDeleteMarkers(region)); // major compaction removes all, since there are no puts they affect - region.compactStores(true); + region.compact(true); assertEquals(0, countDeleteMarkers(region)); HBaseTestingUtility.closeRegionAndWAL(region); @@ -442,7 +442,7 @@ public class TestKeepDeletes { public void testDeleteMarkerExpiration() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -475,9 +475,9 @@ public class TestKeepDeletes { // 1 family marker, 1 column marker, 2 version markers assertEquals(4, countDeleteMarkers(region)); - region.flushcache(); + region.flush(true); assertEquals(4, countDeleteMarkers(region)); - region.compactStores(false); + region.compact(false); assertEquals(4, countDeleteMarkers(region)); // another put will push out the earlier put... @@ -485,14 +485,14 @@ public class TestKeepDeletes { p.add(c0, c0, T1); region.put(p); - region.flushcache(); + region.flush(true); // no markers are collected, since there is an affected put - region.compactStores(true); + region.compact(true); assertEquals(4, countDeleteMarkers(region)); // the last collections collected the earlier put // so after this collection all markers - region.compactStores(true); + region.compact(true); assertEquals(0, countDeleteMarkers(region)); HBaseTestingUtility.closeRegionAndWAL(region); @@ -505,7 +505,7 @@ public class TestKeepDeletes { public void testWithOldRow() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -538,9 +538,9 @@ public class TestKeepDeletes { // 1 family marker, 1 column marker, 2 version markers assertEquals(4, countDeleteMarkers(region)); - region.flushcache(); + region.flush(true); assertEquals(4, countDeleteMarkers(region)); - region.compactStores(false); + region.compact(false); assertEquals(4, countDeleteMarkers(region)); // another put will push out the earlier put... @@ -548,14 +548,14 @@ public class TestKeepDeletes { p.add(c0, c0, T1); region.put(p); - region.flushcache(); + region.flush(true); // no markers are collected, since there is an affected put - region.compactStores(true); + region.compact(true); assertEquals(4, countDeleteMarkers(region)); // all markers remain, since we have the older row // and we haven't pushed the inlined markers past MAX_VERSIONS - region.compactStores(true); + region.compact(true); assertEquals(4, countDeleteMarkers(region)); // another put will push out the earlier put... @@ -565,12 +565,12 @@ public class TestKeepDeletes { // this pushed out the column and version marker // but the family markers remains. THIS IS A PROBLEM! - region.compactStores(true); + region.compact(true); assertEquals(1, countDeleteMarkers(region)); // no amount of compacting is getting this of this one // KEEP_DELETED_CELLS=>TTL is an option to avoid this. - region.compactStores(true); + region.compact(true); assertEquals(1, countDeleteMarkers(region)); HBaseTestingUtility.closeRegionAndWAL(region); @@ -583,7 +583,7 @@ public class TestKeepDeletes { public void testRanges() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -665,7 +665,7 @@ public class TestKeepDeletes { public void testDeleteMarkerVersioning() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -691,9 +691,9 @@ public class TestKeepDeletes { d.deleteColumn(c0, c0, ts+3); region.delete(d); - region.flushcache(); - region.compactStores(true); - region.compactStores(true); + region.flush(true); + region.compact(true); + region.compact(true); assertEquals(3, countDeleteMarkers(region)); // add two more puts, since max version is 1 @@ -723,7 +723,7 @@ public class TestKeepDeletes { assertEquals(1, countDeleteMarkers(region)); // flush cache only sees what is in the memstore - region.flushcache(); + region.flush(true); // Here we have the three markers again, because the flush above // removed the 2nd put before the file is written. @@ -732,7 +732,7 @@ public class TestKeepDeletes { // delete, put, delete, delete assertEquals(3, countDeleteMarkers(region)); - region.compactStores(true); + region.compact(true); assertEquals(3, countDeleteMarkers(region)); // add one more put @@ -740,12 +740,12 @@ public class TestKeepDeletes { p.add(c0, c0, T4); region.put(p); - region.flushcache(); + region.flush(true); // one trailing delete marker remains (but only one) // because delete markers do not increase the version count assertEquals(1, countDeleteMarkers(region)); - region.compactStores(true); - region.compactStores(true); + region.compact(true); + region.compact(true); assertEquals(1, countDeleteMarkers(region)); HBaseTestingUtility.closeRegionAndWAL(region); @@ -757,7 +757,7 @@ public class TestKeepDeletes { public void testWithMixedCFs() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -809,7 +809,7 @@ public class TestKeepDeletes { public void testWithMinVersions() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.TRUE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past @@ -845,7 +845,7 @@ public class TestKeepDeletes { // 3 families, one column delete marker assertEquals(4, countDeleteMarkers(region)); - region.flushcache(); + region.flush(true); // no delete marker removes by the flush assertEquals(4, countDeleteMarkers(region)); @@ -854,7 +854,7 @@ public class TestKeepDeletes { p = new Put(T1, ts+1); p.add(c0, c0, T4); region.put(p); - region.flushcache(); + region.flush(true); assertEquals(4, countDeleteMarkers(region)); @@ -867,14 +867,14 @@ public class TestKeepDeletes { p.add(c0, c0, T5); region.put(p); - region.flushcache(); - region.compactStores(true); + region.flush(true); + region.compact(true); // the two family markers without puts are gone assertEquals(2, countDeleteMarkers(region)); // the last compactStores updated the earliestPutTs, // so after the next compaction the last family delete marker is also gone - region.compactStores(true); + region.compact(true); assertEquals(0, countDeleteMarkers(region)); HBaseTestingUtility.closeRegionAndWAL(region); @@ -888,7 +888,7 @@ public class TestKeepDeletes { public void testWithTTL() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 1, 1000, 1, KeepDeletedCells.TTL); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past @@ -911,21 +911,21 @@ public class TestKeepDeletes { // 3 families, one column delete marker assertEquals(3, countDeleteMarkers(region)); - region.flushcache(); + region.flush(true); // no delete marker removes by the flush assertEquals(3, countDeleteMarkers(region)); // but the Put is gone checkGet(region, T1, c0, c0, ts+1); - region.compactStores(true); + region.compact(true); // all delete marker gone assertEquals(0, countDeleteMarkers(region)); HBaseTestingUtility.closeRegionAndWAL(region); } - private void checkGet(HRegion region, byte[] row, byte[] fam, byte[] col, + private void checkGet(Region region, byte[] row, byte[] fam, byte[] col, long time, byte[]... vals) throws IOException { Get g = new Get(row); g.addColumn(fam, col); @@ -936,11 +936,11 @@ public class TestKeepDeletes { } - private int countDeleteMarkers(HRegion region) throws IOException { + private int countDeleteMarkers(Region region) throws IOException { Scan s = new Scan(); s.setRaw(true); // use max versions from the store(s) - s.setMaxVersions(region.getStores().values().iterator().next().getScanInfo().getMaxVersions()); + s.setMaxVersions(region.getStores().iterator().next().getScanInfo().getMaxVersions()); InternalScanner scan = region.getScanner(s); List kvs = new ArrayList(); int res = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java index 0ca66ee6832..12ab2ad7778 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java @@ -78,7 +78,7 @@ public class TestMajorCompaction { private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU(); protected Configuration conf = UTIL.getConfiguration(); - private HRegion r = null; + private Region r = null; private HTableDescriptor htd = null; private static final byte [] COLUMN_FAMILY = fam1; private final byte [] STARTROW = Bytes.toBytes(START_KEY); @@ -111,8 +111,8 @@ public class TestMajorCompaction { @After public void tearDown() throws Exception { - WAL wal = r.getWAL(); - this.r.close(); + WAL wal = ((HRegion)r).getWAL(); + ((HRegion)r).close(); wal.close(); } @@ -138,9 +138,9 @@ public class TestMajorCompaction { } while(true); s.close(); // Flush - r.flushcache(); + r.flush(true); // Major compact. - r.compactStores(true); + r.compact(true); s = r.getScanner(new Scan()); int counter = 0; do { @@ -174,24 +174,22 @@ public class TestMajorCompaction { public void majorCompactionWithDataBlockEncoding(boolean inCacheOnly) throws Exception { - Map replaceBlockCache = - new HashMap(); - for (Entry pair : r.getStores().entrySet()) { - HStore store = (HStore) pair.getValue(); + Map replaceBlockCache = + new HashMap(); + for (Store store : r.getStores()) { HFileDataBlockEncoder blockEncoder = store.getDataBlockEncoder(); replaceBlockCache.put(store, blockEncoder); final DataBlockEncoding inCache = DataBlockEncoding.PREFIX; final DataBlockEncoding onDisk = inCacheOnly ? DataBlockEncoding.NONE : inCache; - store.setDataBlockEncoderInTest(new HFileDataBlockEncoderImpl(onDisk)); + ((HStore)store).setDataBlockEncoderInTest(new HFileDataBlockEncoderImpl(onDisk)); } majorCompaction(); // restore settings - for (Entry entry : - replaceBlockCache.entrySet()) { - entry.getKey().setDataBlockEncoderInTest(entry.getValue()); + for (Entry entry : replaceBlockCache.entrySet()) { + ((HStore)entry.getKey()).setDataBlockEncoderInTest(entry.getValue()); } } @@ -211,16 +209,16 @@ public class TestMajorCompaction { assertEquals(compactionThreshold, result.size()); // see if CompactionProgress is in place but null - for (Store store : this.r.stores.values()) { + for (Store store : r.getStores()) { assertNull(store.getCompactionProgress()); } - r.flushcache(); - r.compactStores(true); + r.flush(true); + r.compact(true); // see if CompactionProgress has done its thing on at least one store int storeCount = 0; - for (Store store : this.r.stores.values()) { + for (Store store : r.getStores()) { CompactionProgress progress = store.getCompactionProgress(); if( progress != null ) { ++storeCount; @@ -259,20 +257,20 @@ public class TestMajorCompaction { result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100)); assertTrue("Second row should have been deleted", result.isEmpty()); - r.flushcache(); + r.flush(true); result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100)); assertTrue("Second row should have been deleted", result.isEmpty()); // Add a bit of data and flush. Start adding at 'bbb'. createSmallerStoreFile(this.r); - r.flushcache(); + r.flush(true); // Assert that the second row is still deleted. result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100)); assertTrue("Second row should still be deleted", result.isEmpty()); // Force major compaction. - r.compactStores(true); + r.compact(true); assertEquals(r.getStore(COLUMN_FAMILY_TEXT).getStorefiles().size(), 1); result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100)); @@ -286,7 +284,7 @@ public class TestMajorCompaction { // Multiple versions allowed for an entry, so the delete isn't enough // Lower TTL and expire to ensure that all our entries have been wiped final int ttl = 1000; - for (Store hstore : this.r.stores.values()) { + for (Store hstore : r.getStores()) { HStore store = ((HStore) hstore); ScanInfo old = store.getScanInfo(); ScanInfo si = new ScanInfo(old.getFamily(), @@ -296,7 +294,7 @@ public class TestMajorCompaction { } Thread.sleep(1000); - r.compactStores(true); + r.compact(true); int count = count(); assertEquals("Should not see anything after TTL has expired", 0, count); } @@ -314,11 +312,11 @@ public class TestMajorCompaction { try { createStoreFile(r); createStoreFile(r); - r.compactStores(true); + r.compact(true); // add one more file & verify that a regular compaction won't work createStoreFile(r); - r.compactStores(false); + r.compact(false); assertEquals(2, s.getStorefilesCount()); // ensure that major compaction time is deterministic @@ -338,7 +336,7 @@ public class TestMajorCompaction { Thread.sleep(mcTime); // trigger a compaction request and ensure that it's upgraded to major - r.compactStores(false); + r.compact(false); assertEquals(1, s.getStorefilesCount()); } finally { // reset the timed compaction settings @@ -346,7 +344,7 @@ public class TestMajorCompaction { conf.setFloat("hbase.hregion.majorcompaction.jitter", 0.20F); // run a major to reset the cache createStoreFile(r); - r.compactStores(true); + r.compact(true); assertEquals(1, s.getStorefilesCount()); } } @@ -354,7 +352,7 @@ public class TestMajorCompaction { private void verifyCounts(int countRow1, int countRow2) throws Exception { int count1 = 0; int count2 = 0; - for (StoreFile f: this.r.stores.get(COLUMN_FAMILY_TEXT).getStorefiles()) { + for (StoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) { HFileScanner scanner = f.getReader().getScanner(false, false); scanner.seekTo(); do { @@ -373,8 +371,7 @@ public class TestMajorCompaction { private int count() throws IOException { int count = 0; - for (StoreFile f: this.r.stores. - get(COLUMN_FAMILY_TEXT).getStorefiles()) { + for (StoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) { HFileScanner scanner = f.getReader().getScanner(false, false); if (!scanner.seekTo()) { continue; @@ -386,17 +383,17 @@ public class TestMajorCompaction { return count; } - private void createStoreFile(final HRegion region) throws IOException { + private void createStoreFile(final Region region) throws IOException { createStoreFile(region, Bytes.toString(COLUMN_FAMILY)); } - private void createStoreFile(final HRegion region, String family) throws IOException { + private void createStoreFile(final Region region, String family) throws IOException { HRegionIncommon loader = new HRegionIncommon(region); HBaseTestCase.addContent(loader, family); loader.flushcache(); } - private void createSmallerStoreFile(final HRegion region) throws IOException { + private void createSmallerStoreFile(final Region region) throws IOException { HRegionIncommon loader = new HRegionIncommon(region); HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY), ("" + "bbb").getBytes(), null); @@ -466,9 +463,9 @@ public class TestMajorCompaction { } while (true); s.close(); // Flush - r.flushcache(); + r.flush(true); // Major compact. - r.compactStores(true); + r.compact(true); scan = new Scan(); scan.setReversed(true); s = r.getScanner(scan); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java index 16f29dc3bdc..7f8a20b11fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java @@ -68,7 +68,7 @@ public class TestMinVersions { public void testGetClosestBefore() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 1, 1000, 1, KeepDeletedCells.FALSE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); try { // 2s in the past @@ -96,8 +96,8 @@ public class TestMinVersions { checkResult(r, c0, T4); // now flush/compact - region.flushcache(); - region.compactStores(true); + region.flush(true); + region.compact(true); r = region.getClosestRowBefore(T1, c0); checkResult(r, c0, T4); @@ -118,7 +118,7 @@ public class TestMinVersions { // keep 3 versions minimum HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.FALSE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); // 2s in the past long ts = EnvironmentEdgeManager.currentTime() - 2000; @@ -132,8 +132,8 @@ public class TestMinVersions { region.put(p); // now flush/compact - region.flushcache(); - region.compactStores(true); + region.flush(true); + region.compact(true); p = new Put(T1, ts); p.add(c0, c0, T3); @@ -173,7 +173,7 @@ public class TestMinVersions { public void testDelete() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.FALSE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); // 2s in the past long ts = EnvironmentEdgeManager.currentTime() - 2000; @@ -206,8 +206,8 @@ public class TestMinVersions { checkResult(r, c0, T3); // now flush/compact - region.flushcache(); - region.compactStores(true); + region.flush(true); + region.compact(true); // try again g = new Get(T1); @@ -232,7 +232,7 @@ public class TestMinVersions { public void testMemStore() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 2, 1000, 1, KeepDeletedCells.FALSE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); // 2s in the past long ts = EnvironmentEdgeManager.currentTime() - 2000; @@ -254,8 +254,8 @@ public class TestMinVersions { region.put(p); // now flush/compact - region.flushcache(); - region.compactStores(true); + region.flush(true); + region.compact(true); // now put the first version (backdated) p = new Put(T1, ts-3); @@ -308,7 +308,7 @@ public class TestMinVersions { // 1 version minimum, 1000 versions maximum, ttl = 1s HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 2, 1000, 1, KeepDeletedCells.FALSE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); try { // 2s in the past @@ -361,7 +361,7 @@ public class TestMinVersions { checkResult(r, c0, T4,T3); // now flush - region.flushcache(); + region.flush(true); // with HBASE-4241 a flush will eliminate the expired rows g = new Get(T1); @@ -370,7 +370,7 @@ public class TestMinVersions { assertTrue(r.isEmpty()); // major compaction - region.compactStores(true); + region.compact(true); // after compaction the 4th version is still available g = new Get(T1); @@ -400,7 +400,7 @@ public class TestMinVersions { public void testFilters() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 2, 1000, 1, KeepDeletedCells.FALSE); - HRegion region = hbu.createLocalHRegion(htd, null, null); + Region region = hbu.createLocalHRegion(htd, null, null); final byte [] c1 = COLUMNS[1]; // 2s in the past @@ -446,8 +446,8 @@ public class TestMinVersions { checkResult(r, c0, T2,T1); // now flush/compact - region.flushcache(); - region.compactStores(true); + region.flush(true); + region.compact(true); g = new Get(T1); g.addColumn(c1,c1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java index 7ac6eefd71e..b694fe2b351 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java @@ -57,7 +57,7 @@ public class TestMinorCompaction { private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU(); protected Configuration conf = UTIL.getConfiguration(); - private HRegion r = null; + private Region r = null; private HTableDescriptor htd = null; private int compactionThreshold; private byte[] firstRowBytes, secondRowBytes, thirdRowBytes; @@ -90,8 +90,8 @@ public class TestMinorCompaction { @After public void tearDown() throws Exception { - WAL wal = r.getWAL(); - this.r.close(); + WAL wal = ((HRegion)r).getWAL(); + ((HRegion)r).close(); wal.close(); } @@ -172,7 +172,7 @@ public class TestMinorCompaction { thirdRowBytes, i); HBaseTestCase.addContent(loader, Bytes.toString(fam2), Bytes.toString(col2), firstRowBytes, thirdRowBytes, i); - r.flushcache(); + r.flush(true); } Result result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100)); @@ -193,7 +193,7 @@ public class TestMinorCompaction { result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100)); assertEquals(compactionThreshold, result.size()); - r.flushcache(); + r.flush(true); // should not change anything. // Let us check again @@ -205,7 +205,7 @@ public class TestMinorCompaction { assertEquals(compactionThreshold, result.size()); // do a compaction - Store store2 = this.r.stores.get(fam2); + Store store2 = r.getStore(fam2); int numFiles1 = store2.getStorefiles().size(); assertTrue("Was expecting to see 4 store files", numFiles1 > compactionThreshold); // > 3 ((HStore)store2).compactRecentForTestingAssumingDefaultPolicy(compactionThreshold); // = 3 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java index 33454370be5..28d3ab9329f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java @@ -146,7 +146,7 @@ public class TestMultiColumnScanner { @Test public void testMultiColumnScanner() throws IOException { - HRegion region = TEST_UTIL.createTestRegion(TABLE_NAME, + Region region = TEST_UTIL.createTestRegion(TABLE_NAME, new HColumnDescriptor(FAMILY) .setCompressionType(comprAlgo) .setBloomFilterType(bloomType) @@ -221,7 +221,7 @@ public class TestMultiColumnScanner { region.delete(d); } } - region.flushcache(); + region.flush(true); } Collections.sort(kvs, KeyValue.COMPARATOR); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java index c131aa6bed9..82689e424ef 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java @@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.WAL; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -128,8 +129,8 @@ public class TestPerColumnFamilyFlush { conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200 * 1024); conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName()); conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND, 100 * 1024); - // Intialize the HRegion - HRegion region = initHRegion("testSelectiveFlushWhenEnabled", conf); + // Intialize the region + Region region = initHRegion("testSelectiveFlushWhenEnabled", conf); // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3 for (int i = 1; i <= 1200; i++) { region.put(createPut(1, i)); @@ -142,7 +143,7 @@ public class TestPerColumnFamilyFlush { } } - long totalMemstoreSize = region.getMemstoreSize().get(); + long totalMemstoreSize = region.getMemstoreSize(); // Find the smallest LSNs for edits wrt to each CF. long smallestSeqCF1 = region.getOldestSeqIdOfStore(FAMILY1); @@ -155,8 +156,8 @@ public class TestPerColumnFamilyFlush { long cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); // Get the overall smallest LSN in the region's memstores. - long smallestSeqInRegionCurrentMemstore = - region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + long smallestSeqInRegionCurrentMemstore = getWAL(region) + .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); // The overall smallest LSN in the region's memstores should be the same as // the LSN of the smallest edit in CF1 @@ -175,7 +176,7 @@ public class TestPerColumnFamilyFlush { + cf2MemstoreSize + cf3MemstoreSize); // Flush! - region.flushcache(false); + region.flush(false); // Will use these to check if anything changed. long oldCF2MemstoreSize = cf2MemstoreSize; @@ -185,9 +186,9 @@ public class TestPerColumnFamilyFlush { cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); - totalMemstoreSize = region.getMemstoreSize().get(); - smallestSeqInRegionCurrentMemstore = - region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + totalMemstoreSize = region.getMemstoreSize(); + smallestSeqInRegionCurrentMemstore = getWAL(region) + .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); // We should have cleared out only CF1, since we chose the flush thresholds // and number of puts accordingly. @@ -217,15 +218,15 @@ public class TestPerColumnFamilyFlush { oldCF3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); // Flush again - region.flushcache(false); + region.flush(false); // Recalculate everything cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); - totalMemstoreSize = region.getMemstoreSize().get(); - smallestSeqInRegionCurrentMemstore = - region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + totalMemstoreSize = region.getMemstoreSize(); + smallestSeqInRegionCurrentMemstore = getWAL(region) + .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); // CF1 and CF2, both should be absent. assertEquals(DefaultMemStore.DEEP_OVERHEAD, cf1MemstoreSize); @@ -240,7 +241,7 @@ public class TestPerColumnFamilyFlush { // In that case, we should flush all the CFs. // Clearing the existing memstores. - region.flushcache(true); + region.flush(true); // The memstore limit is 200*1024 and the column family flush threshold is // around 50*1024. We try to just hit the memstore limit with each CF's @@ -253,10 +254,11 @@ public class TestPerColumnFamilyFlush { region.put(createPut(5, i)); } - region.flushcache(false); + region.flush(false); + // Since we won't find any CF above the threshold, and hence no specific // store to flush, we should flush all the memstores. - assertEquals(0, region.getMemstoreSize().get()); + assertEquals(0, region.getMemstoreSize()); HBaseTestingUtility.closeRegionAndWAL(region); } @@ -280,7 +282,7 @@ public class TestPerColumnFamilyFlush { } } - long totalMemstoreSize = region.getMemstoreSize().get(); + long totalMemstoreSize = region.getMemstoreSize(); // Find the sizes of the memstores of each CF. long cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); @@ -298,12 +300,12 @@ public class TestPerColumnFamilyFlush { + cf2MemstoreSize + cf3MemstoreSize); // Flush! - region.flushcache(false); + region.flush(false); cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); - totalMemstoreSize = region.getMemstoreSize().get(); + totalMemstoreSize = region.getMemstoreSize(); long smallestSeqInRegionCurrentMemstore = region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); @@ -317,12 +319,12 @@ public class TestPerColumnFamilyFlush { } // Find the (first) region which has the specified name. - private static Pair getRegionWithName(TableName tableName) { + private static Pair getRegionWithName(TableName tableName) { MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster(); List rsts = cluster.getRegionServerThreads(); for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { HRegionServer hrs = rsts.get(i).getRegionServer(); - for (HRegion region : hrs.getOnlineRegions(tableName)) { + for (Region region : hrs.getOnlineRegions(tableName)) { return Pair.newPair(region, hrs); } } @@ -361,16 +363,16 @@ public class TestPerColumnFamilyFlush { table.flushCommits(); Thread.sleep(1000); - Pair desiredRegionAndServer = getRegionWithName(TABLENAME); - HRegion desiredRegion = desiredRegionAndServer.getFirst(); + Pair desiredRegionAndServer = getRegionWithName(TABLENAME); + Region desiredRegion = desiredRegionAndServer.getFirst(); assertTrue("Could not find a region which hosts the new region.", desiredRegion != null); // Flush the region selectively. - desiredRegion.flushcache(false); + desiredRegion.flush(false); long totalMemstoreSize; long cf1MemstoreSize, cf2MemstoreSize, cf3MemstoreSize; - totalMemstoreSize = desiredRegion.getMemstoreSize().get(); + totalMemstoreSize = desiredRegion.getMemstoreSize(); // Find the sizes of the memstores of each CF. cf1MemstoreSize = desiredRegion.getStore(FAMILY1).getMemStoreSize(); @@ -429,8 +431,12 @@ public class TestPerColumnFamilyFlush { doTestLogReplay(); } - private int getNumRolledLogFiles(HRegion region) { - return ((FSHLog) region.getWAL()).getNumRolledLogFiles(); + private WAL getWAL(Region region) { + return ((HRegion)region).getWAL(); + } + + private int getNumRolledLogFiles(Region region) { + return ((FSHLog)getWAL(region)).getNumRolledLogFiles(); } /** @@ -468,8 +474,8 @@ public class TestPerColumnFamilyFlush { try (Admin admin = TEST_UTIL.getConnection().getAdmin()) { admin.flush(TableName.NAMESPACE_TABLE_NAME); } - Pair desiredRegionAndServer = getRegionWithName(tableName); - final HRegion desiredRegion = desiredRegionAndServer.getFirst(); + Pair desiredRegionAndServer = getRegionWithName(tableName); + final Region desiredRegion = desiredRegionAndServer.getFirst(); assertTrue("Could not find a region which hosts the new region.", desiredRegion != null); LOG.info("Writing to region=" + desiredRegion); @@ -486,7 +492,7 @@ public class TestPerColumnFamilyFlush { table.flushCommits(); // Roll the WAL. The log file count is less than maxLogs so no flush is triggered. int currentNumRolledLogFiles = getNumRolledLogFiles(desiredRegion); - assertNull(desiredRegion.getWAL().rollWriter()); + assertNull(getWAL(desiredRegion).rollWriter()); while (getNumRolledLogFiles(desiredRegion) <= currentNumRolledLogFiles) { Thread.sleep(100); } @@ -505,12 +511,12 @@ public class TestPerColumnFamilyFlush { @Override public boolean evaluate() throws Exception { - return desiredRegion.getMemstoreSize().get() == 0; + return desiredRegion.getMemstoreSize() == 0; } @Override public String explainFailure() throws Exception { - long memstoreSize = desiredRegion.getMemstoreSize().get(); + long memstoreSize = desiredRegion.getMemstoreSize(); if (memstoreSize > 0) { return "Still have unflushed entries in memstore, memstore size is " + memstoreSize; } @@ -526,7 +532,7 @@ public class TestPerColumnFamilyFlush { assertEquals(DefaultMemStore.DEEP_OVERHEAD, desiredRegion.getStore(FAMILY3).getMemStoreSize()); // let WAL cleanOldLogs - assertNull(desiredRegion.getWAL().rollWriter(true)); + assertNull(getWAL(desiredRegion).rollWriter(true)); assertTrue(getNumRolledLogFiles(desiredRegion) < maxLogs); } finally { TEST_UTIL.shutdownMiniCluster(); @@ -534,7 +540,7 @@ public class TestPerColumnFamilyFlush { } private void doPut(Table table, long memstoreFlushSize) throws IOException, InterruptedException { - HRegion region = getRegionWithName(table.getName()).getFirst(); + Region region = getRegionWithName(table.getName()).getFirst(); // cf1 4B per row, cf2 40B per row and cf3 400B per row byte[] qf = Bytes.toBytes("qf"); Random rand = new Random(); @@ -552,7 +558,7 @@ public class TestPerColumnFamilyFlush { put.addColumn(FAMILY3, qf, value3); table.put(put); // slow down to let regionserver flush region. - while (region.getMemstoreSize().get() > memstoreFlushSize) { + while (region.getMemstoreSize() > memstoreFlushSize) { Thread.sleep(100); } } @@ -596,7 +602,7 @@ public class TestPerColumnFamilyFlush { table.close(); conn.close(); - HRegion region = getRegionWithName(TABLENAME).getFirst(); + Region region = getRegionWithName(TABLENAME).getFirst(); cf1StoreFileCount = region.getStore(FAMILY1).getStorefilesCount(); cf2StoreFileCount = region.getStore(FAMILY2).getStorefilesCount(); cf3StoreFileCount = region.getStore(FAMILY3).getStorefilesCount(); @@ -617,7 +623,7 @@ public class TestPerColumnFamilyFlush { table.close(); conn.close(); - HRegion region = getRegionWithName(TABLENAME).getFirst(); + Region region = getRegionWithName(TABLENAME).getFirst(); cf1StoreFileCount1 = region.getStore(FAMILY1).getStorefilesCount(); cf2StoreFileCount1 = region.getStore(FAMILY2).getStorefilesCount(); cf3StoreFileCount1 = region.getStore(FAMILY3).getStorefilesCount(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionFavoredNodes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionFavoredNodes.java index 8e7fe0488cf..ebce100d1f7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionFavoredNodes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionFavoredNodes.java @@ -108,11 +108,11 @@ public class TestRegionFavoredNodes { } // For each region, choose some datanodes as the favored nodes then assign - // them as favored nodes through the HRegion. + // them as favored nodes through the region. for (int i = 0; i < REGION_SERVERS; i++) { HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i); - List regions = server.getOnlineRegions(TABLE_NAME); - for (HRegion region : regions) { + List regions = server.getOnlineRegions(TABLE_NAME); + for (Region region : regions) { ListfavoredNodes = new ArrayList(3); String encodedRegionName = region.getRegionInfo().getEncodedName(); @@ -139,8 +139,8 @@ public class TestRegionFavoredNodes { // they are consistent with the favored nodes for that region. for (int i = 0; i < REGION_SERVERS; i++) { HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i); - List regions = server.getOnlineRegions(TABLE_NAME); - for (HRegion region : regions) { + List regions = server.getOnlineRegions(TABLE_NAME); + for (Region region : regions) { List files = region.getStoreFileList(new byte[][]{COLUMN_FAMILY}); for (String file : files) { FileStatus status = TEST_UTIL.getDFSCluster().getFileSystem(). diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java index bb47b78a119..4a2f9bae374 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java @@ -125,9 +125,9 @@ public class TestRegionMergeTransaction { false); RegionMergeTransaction spyMT = Mockito.spy(mt); doReturn(false).when(spyMT).hasMergeQualifierInMeta(null, - region_a.getRegionName()); + region_a.getRegionInfo().getRegionName()); doReturn(false).when(spyMT).hasMergeQualifierInMeta(null, - region_b.getRegionName()); + region_b.getRegionInfo().getRegionName()); assertTrue(spyMT.prepare(null)); return spyMT; } @@ -164,9 +164,9 @@ public class TestRegionMergeTransaction { true); RegionMergeTransaction spyMT = Mockito.spy(mt); doReturn(false).when(spyMT).hasMergeQualifierInMeta(null, - region_a.getRegionName()); + region_a.getRegionInfo().getRegionName()); doReturn(false).when(spyMT).hasMergeQualifierInMeta(null, - region_c.getRegionName()); + region_c.getRegionInfo().getRegionName()); assertTrue("Since focible is true, should merge two regions even if they are not adjacent", spyMT.prepare(null)); } @@ -206,9 +206,9 @@ public class TestRegionMergeTransaction { false); RegionMergeTransaction spyMT = Mockito.spy(mt); doReturn(true).when(spyMT).hasMergeQualifierInMeta(null, - region_a.getRegionName()); + region_a.getRegionInfo().getRegionName()); doReturn(true).when(spyMT).hasMergeQualifierInMeta(null, - region_b.getRegionName()); + region_b.getRegionInfo().getRegionName()); assertFalse(spyMT.prepare(null)); } @@ -239,10 +239,10 @@ public class TestRegionMergeTransaction { // to be under the merged region dirs. assertEquals(0, this.fs.listStatus(mt.getMergesDir()).length); // Check merged region have correct key span. - assertTrue(Bytes.equals(this.region_a.getStartKey(), - mergedRegion.getStartKey())); - assertTrue(Bytes.equals(this.region_b.getEndKey(), - mergedRegion.getEndKey())); + assertTrue(Bytes.equals(this.region_a.getRegionInfo().getStartKey(), + mergedRegion.getRegionInfo().getStartKey())); + assertTrue(Bytes.equals(this.region_b.getRegionInfo().getEndKey(), + mergedRegion.getRegionInfo().getEndKey())); // Count rows. merged region are already open try { int mergedRegionRowCount = countRows(mergedRegion); @@ -468,7 +468,7 @@ public class TestRegionMergeTransaction { } } if (flush) { - r.flushcache(); + r.flush(true); } } return rowCount; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java index 43c124ccf09..b18a0f48c1e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java @@ -183,7 +183,7 @@ public class TestRegionReplicaFailover { // read from it the same data from primary and secondaries boolean aborted = false; for (RegionServerThread rs : HTU.getMiniHBaseCluster().getRegionServerThreads()) { - for (HRegion r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) { + for (Region r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) { if (r.getRegionInfo().getReplicaId() == 0) { LOG.info("Aborting region server hosting primary region replica"); rs.getRegionServer().abort("for test"); @@ -244,7 +244,7 @@ public class TestRegionReplicaFailover { // read from it the same data boolean aborted = false; for (RegionServerThread rs : HTU.getMiniHBaseCluster().getRegionServerThreads()) { - for (HRegion r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) { + for (Region r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) { if (r.getRegionInfo().getReplicaId() == 1) { LOG.info("Aborting region server hosting secondary region replica"); rs.getRegionServer().abort("for test"); @@ -304,7 +304,7 @@ public class TestRegionReplicaFailover { try { boolean aborted = false; for (RegionServerThread rs : HTU.getMiniHBaseCluster().getRegionServerThreads()) { - for (HRegion r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) { + for (Region r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) { if (r.getRegionInfo().getReplicaId() == 1) { LOG.info("Aborting region server hosting secondary region replica"); rs.getRegionServer().abort("for test"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 538e47e60c1..85a8cd23dc2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -150,12 +150,13 @@ public class TestRegionReplicas { // assert that we can read back from primary Assert.assertEquals(1000, HTU.countRows(table)); // flush so that region replica can read - getRS().getRegionByEncodedName(hriPrimary.getEncodedName()).flushcache(); + Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + region.flush(true); openRegion(HTU, getRS(), hriSecondary); // first try directly against region - HRegion region = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); + region = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); assertGet(region, 42, true); assertGetRpc(hriSecondary, 42, true); @@ -173,7 +174,8 @@ public class TestRegionReplicas { // assert that we can read back from primary Assert.assertEquals(1000, HTU.countRows(table)); // flush so that region replica can read - getRS().getRegionByEncodedName(hriPrimary.getEncodedName()).flushcache(); + Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + region.flush(true); openRegion(HTU, getRS(), hriSecondary); @@ -190,7 +192,7 @@ public class TestRegionReplicas { } } - private void assertGet(HRegion region, int value, boolean expect) throws IOException { + private void assertGet(Region region, int value, boolean expect) throws IOException { byte[] row = Bytes.toBytes(String.valueOf(value)); Get get = new Get(row); Result result = region.get(get); @@ -242,14 +244,15 @@ public class TestRegionReplicas { Assert.assertEquals(1000, HTU.countRows(table)); // flush so that region replica can read LOG.info("Flushing primary region"); - getRS().getRegionByEncodedName(hriPrimary.getEncodedName()).flushcache(); + Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + region.flush(true); // ensure that chore is run LOG.info("Sleeping for " + (4 * refreshPeriod)); Threads.sleep(4 * refreshPeriod); LOG.info("Checking results from secondary region replica"); - HRegion secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); + Region secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); Assert.assertEquals(1, secondaryRegion.getStore(f).getStorefilesCount()); assertGet(secondaryRegion, 42, true); @@ -258,10 +261,12 @@ public class TestRegionReplicas { //load some data to primary HTU.loadNumericRows(table, f, 1000, 1100); - getRS().getRegionByEncodedName(hriPrimary.getEncodedName()).flushcache(); + region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + region.flush(true); HTU.loadNumericRows(table, f, 2000, 2100); - getRS().getRegionByEncodedName(hriPrimary.getEncodedName()).flushcache(); + region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + region.flush(true); // ensure that chore is run Threads.sleep(4 * refreshPeriod); @@ -428,20 +433,21 @@ public class TestRegionReplicas { LOG.info("Loading data to primary region"); for (int i = 0; i < 3; ++i) { HTU.loadNumericRows(table, f, i * 1000, (i + 1) * 1000); - getRS().getRegionByEncodedName(hriPrimary.getEncodedName()).flushcache(); + Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + region.flush(true); } - HRegion primaryRegion = getRS().getFromOnlineRegions(hriPrimary.getEncodedName()); + Region primaryRegion = getRS().getFromOnlineRegions(hriPrimary.getEncodedName()); Assert.assertEquals(3, primaryRegion.getStore(f).getStorefilesCount()); // Refresh store files on the secondary - HRegion secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); + Region secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); secondaryRegion.getStore(f).refreshStoreFiles(); Assert.assertEquals(3, secondaryRegion.getStore(f).getStorefilesCount()); // force compaction LOG.info("Force Major compaction on primary region " + hriPrimary); - primaryRegion.compactStores(true); + primaryRegion.compact(true); Assert.assertEquals(1, primaryRegion.getStore(f).getStorefilesCount()); // scan all the hfiles on the secondary. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java index 65aed5bae00..786a4e18dad 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java @@ -125,9 +125,9 @@ public class TestRegionServerNoMaster { /** Flush the given region in the mini cluster. Since no master, we cannot use HBaseAdmin.flush() */ public static void flushRegion(HBaseTestingUtility HTU, HRegionInfo regionInfo) throws IOException { for (RegionServerThread rst : HTU.getMiniHBaseCluster().getRegionServerThreads()) { - HRegion region = rst.getRegionServer().getRegionByEncodedName(regionInfo.getEncodedName()); + Region region = rst.getRegionServer().getRegionByEncodedName(regionInfo.getEncodedName()); if (region != null) { - region.flushcache(); + region.flush(true); return; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java index c58e9c6c08e..9c6ee1a7f94 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java @@ -55,7 +55,7 @@ public class TestRegionServerOnlineConfigChange { private static HTable t1 = null; private static HRegionServer rs1 = null; private static byte[] r1name = null; - private static HRegion r1 = null; + private static Region r1 = null; private final static String table1Str = "table1"; private final static String columnFamily1Str = "columnFamily1"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java index 924a196ef20..6a234bbf870 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java @@ -25,7 +25,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; @@ -47,7 +46,7 @@ public class TestRegionSplitPolicy { private Configuration conf; private HTableDescriptor htd; private HRegion mockRegion; - private TreeMap stores; + private List stores; private static final TableName TABLENAME = TableName.valueOf("t"); @Before @@ -58,8 +57,7 @@ public class TestRegionSplitPolicy { mockRegion = Mockito.mock(HRegion.class); Mockito.doReturn(htd).when(mockRegion).getTableDesc(); Mockito.doReturn(hri).when(mockRegion).getRegionInfo(); - - stores = new TreeMap(Bytes.BYTES_COMPARATOR); + stores = new ArrayList(); Mockito.doReturn(stores).when(mockRegion).getStores(); } @@ -71,7 +69,7 @@ public class TestRegionSplitPolicy { // Now make it so the mock region has a RegionServerService that will // return 'online regions'. RegionServerServices rss = Mockito.mock(RegionServerServices.class); - final List regions = new ArrayList(); + final List regions = new ArrayList(); Mockito.when(rss.getOnlineRegions(TABLENAME)).thenReturn(regions); Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss); // Set max size for this 'table'. @@ -95,7 +93,7 @@ public class TestRegionSplitPolicy { HStore mockStore = Mockito.mock(HStore.class); Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); - stores.put(new byte[]{1}, mockStore); + stores.add(mockStore); // It should split assertTrue(policy.shouldSplit()); @@ -158,7 +156,7 @@ public class TestRegionSplitPolicy { Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); Mockito.doReturn(Bytes.toBytes("abcd")).when(mockStore).getSplitPoint(); - stores.put(new byte[] { 1 }, mockStore); + stores.add(mockStore); KeyPrefixRegionSplitPolicy policy = (KeyPrefixRegionSplitPolicy) RegionSplitPolicy .create(myMockRegion, conf); @@ -195,7 +193,7 @@ public class TestRegionSplitPolicy { HStore mockStore = Mockito.mock(HStore.class); Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); - stores.put(new byte[]{1}, mockStore); + stores.add(mockStore); assertTrue(policy.shouldSplit()); @@ -235,7 +233,7 @@ public class TestRegionSplitPolicy { Mockito.doReturn(true).when(mockStore).canSplit(); Mockito.doReturn(Bytes.toBytes("store 1 split")) .when(mockStore).getSplitPoint(); - stores.put(new byte[]{1}, mockStore); + stores.add(mockStore); assertEquals("store 1 split", Bytes.toString(policy.getSplitPoint())); @@ -246,7 +244,7 @@ public class TestRegionSplitPolicy { Mockito.doReturn(true).when(mockStore2).canSplit(); Mockito.doReturn(Bytes.toBytes("store 2 split")) .when(mockStore2).getSplitPoint(); - stores.put(new byte[]{2}, mockStore2); + stores.add(mockStore2); assertEquals("store 2 split", Bytes.toString(policy.getSplitPoint())); @@ -267,7 +265,7 @@ public class TestRegionSplitPolicy { Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); Mockito.doReturn(Bytes.toBytes("ab,cd")).when(mockStore).getSplitPoint(); - stores.put(new byte[] { 1 }, mockStore); + stores.add(mockStore); DelimitedKeyPrefixRegionSplitPolicy policy = (DelimitedKeyPrefixRegionSplitPolicy) RegionSplitPolicy .create(myMockRegion, conf); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java index dd7ef29d2dc..3e022434206 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java @@ -69,7 +69,7 @@ public class TestResettingCounters { throw new IOException("Failed delete of " + path); } } - HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, path, conf, htd); + Region region = HBaseTestingUtility.createRegionAndWAL(hri, path, conf, htd); try { Increment odd = new Increment(rows[0]); odd.setDurability(Durability.SKIP_WAL); @@ -84,14 +84,14 @@ public class TestResettingCounters { } // increment odd qualifiers 5 times and flush - for (int i=0;i<5;i++) region.increment(odd); - region.flushcache(); + for (int i=0;i<5;i++) region.increment(odd, HConstants.NO_NONCE, HConstants.NO_NONCE); + region.flush(true); // increment even qualifiers 5 times - for (int i=0;i<5;i++) region.increment(even); + for (int i=0;i<5;i++) region.increment(even, HConstants.NO_NONCE, HConstants.NO_NONCE); // increment all qualifiers, should have value=6 for all - Result result = region.increment(all); + Result result = region.increment(all, HConstants.NO_NONCE, HConstants.NO_NONCE); assertEquals(numQualifiers, result.size()); Cell [] kvs = result.rawCells(); for (int i=0;i allColIds = new TreeSet(); - private HRegion region; + private Region region; private BloomType bloomType; private FileSystem fs; private Configuration conf; @@ -210,7 +210,7 @@ public class TestScanWithBloomError { p.add(kv); } region.put(p); - region.flushcache(); + region.flush(true); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java index 14231274953..ee72ce46268 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java @@ -100,7 +100,7 @@ public class TestScanner { private static final long START_CODE = Long.MAX_VALUE; - private HRegion r; + private Region r; private HRegionIncommon region; private byte[] firstRowBytes, secondRowBytes, thirdRowBytes; @@ -272,7 +272,7 @@ public class TestScanner { // Close and re-open - r.close(); + ((HRegion)r).close(); r = HRegion.openHRegion(r, null); region = new HRegionIncommon(r); @@ -310,7 +310,7 @@ public class TestScanner { // Close and reopen - r.close(); + ((HRegion)r).close(); r = HRegion.openHRegion(r,null); region = new HRegionIncommon(r); @@ -345,7 +345,7 @@ public class TestScanner { // Close and reopen - r.close(); + ((HRegion)r).close(); r = HRegion.openHRegion(r,null); region = new HRegionIncommon(r); @@ -526,17 +526,17 @@ public class TestScanner { /* delete column1 of firstRow */ dc.deleteColumns(fam1, col1); r.delete(dc); - r.flushcache(); + r.flush(true); HBaseTestCase.addContent(hri, Bytes.toString(fam1), Bytes.toString(col1), secondRowBytes, thirdRowBytes); HBaseTestCase.addContent(hri, Bytes.toString(fam2), Bytes.toString(col1), secondRowBytes, thirdRowBytes); - r.flushcache(); + r.flush(true); InternalScanner s = r.getScanner(new Scan()); // run a major compact, column1 of firstRow will be cleaned. - r.compactStores(true); + r.compact(true); List results = new ArrayList(); s.next(results); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java index ba6f7580cc2..dfced3b8fc3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java @@ -107,7 +107,7 @@ public class TestSeekOptimizations { private static final int[] MAX_VERSIONS_VALUES = new int[] { 1, 2 }; // Instance variables - private HRegion region; + private Region region; private Put put; private Delete del; private Random rand; @@ -434,7 +434,7 @@ public class TestSeekOptimizations { } } - region.flushcache(); + region.flush(true); } @After diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java index ec6fa7cd5fe..0ed76457b41 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java @@ -225,7 +225,7 @@ public class TestSplitTransaction { // Run the execute. Look at what it returns. Server mockServer = Mockito.mock(Server.class); when(mockServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration()); - PairOfSameType daughters = st.execute(mockServer, null); + PairOfSameType daughters = st.execute(mockServer, null); // Do some assertions about execution. assertTrue(this.fs.exists(this.parent.getRegionFileSystem().getSplitsDir())); // Assert the parent region is closed. @@ -235,13 +235,15 @@ public class TestSplitTransaction { // to be under the daughter region dirs. assertEquals(0, this.fs.listStatus(this.parent.getRegionFileSystem().getSplitsDir()).length); // Check daughters have correct key span. - assertTrue(Bytes.equals(this.parent.getStartKey(), daughters.getFirst().getStartKey())); - assertTrue(Bytes.equals(GOOD_SPLIT_ROW, daughters.getFirst().getEndKey())); - assertTrue(Bytes.equals(daughters.getSecond().getStartKey(), GOOD_SPLIT_ROW)); - assertTrue(Bytes.equals(this.parent.getEndKey(), daughters.getSecond().getEndKey())); + assertTrue(Bytes.equals(parent.getRegionInfo().getStartKey(), + daughters.getFirst().getRegionInfo().getStartKey())); + assertTrue(Bytes.equals(GOOD_SPLIT_ROW, daughters.getFirst().getRegionInfo().getEndKey())); + assertTrue(Bytes.equals(daughters.getSecond().getRegionInfo().getStartKey(), GOOD_SPLIT_ROW)); + assertTrue(Bytes.equals(parent.getRegionInfo().getEndKey(), + daughters.getSecond().getRegionInfo().getEndKey())); // Count rows. daughters are already open int daughtersRowCount = 0; - for (HRegion openRegion: daughters) { + for (Region openRegion: daughters) { try { int count = countRows(openRegion); assertTrue(count > 0 && count != rowcount); @@ -322,10 +324,10 @@ public class TestSplitTransaction { // Now retry the split but do not throw an exception this time. assertTrue(st.prepare()); - PairOfSameType daughters = st.execute(mockServer, null); + PairOfSameType daughters = st.execute(mockServer, null); // Count rows. daughters are already open int daughtersRowCount = 0; - for (HRegion openRegion: daughters) { + for (Region openRegion: daughters) { try { int count = countRows(openRegion); assertTrue(count > 0 && count != rowcount); @@ -351,7 +353,7 @@ public class TestSplitTransaction { private class MockedFailedDaughterCreation extends IOException {} private class MockedFailedDaughterOpen extends IOException {} - private int countRows(final HRegion r) throws IOException { + private int countRows(final Region r) throws IOException { int rowcount = 0; InternalScanner scanner = r.getScanner(new Scan()); try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index 9d14be63b59..826495ddae3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -201,7 +201,7 @@ public class TestSplitTransactionOnCluster { Coprocessor.PRIORITY_USER, region.getBaseConf()); // split async - this.admin.split(region.getRegionName(), new byte[] {42}); + this.admin.split(region.getRegionInfo().getRegionName(), new byte[] {42}); // we have to wait until the SPLITTING state is seen by the master FailingSplitRegionObserver observer = (FailingSplitRegionObserver) region @@ -240,7 +240,7 @@ public class TestSplitTransactionOnCluster { HRegion region = cluster.getRegions(tableName).get(0); Store store = region.getStore(cf); - int regionServerIndex = cluster.getServerWith(region.getRegionName()); + int regionServerIndex = cluster.getServerWith(region.getRegionInfo().getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); Table t = TESTING_UTIL.getConnection().getTable(tableName); @@ -636,7 +636,8 @@ public class TestSplitTransactionOnCluster { List regions = null; try { regions = cluster.getRegions(tableName); - int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName()); + int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionInfo() + .getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); insertData(tableName, admin, t); // Turn off balancer so it doesn't cut in and mess up our placements. @@ -691,7 +692,8 @@ public class TestSplitTransactionOnCluster { } while (oldRegions.size() != 2); for (HRegion h : oldRegions) LOG.debug("OLDREGION " + h.getRegionInfo()); try { - int regionServerIndex = cluster.getServerWith(oldRegions.get(0).getRegionName()); + int regionServerIndex = cluster.getServerWith(oldRegions.get(0).getRegionInfo() + .getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); insertData(tableName, admin, t); // Turn off balancer so it doesn't cut in and mess up our placements. @@ -702,7 +704,7 @@ public class TestSplitTransactionOnCluster { tableName); assertEquals("The specified table should be present.", true, tableExists); final HRegion region = findSplittableRegion(oldRegions); - regionServerIndex = cluster.getServerWith(region.getRegionName()); + regionServerIndex = cluster.getServerWith(region.getRegionInfo().getRegionName()); regionServer = cluster.getRegionServer(regionServerIndex); assertTrue("not able to find a splittable region", region != null); SplitTransaction st = new SplitTransaction(region, Bytes.toBytes("row2")); @@ -781,7 +783,8 @@ public class TestSplitTransactionOnCluster { List regions = cluster.getRegions(tableName); HRegionInfo hri = getAndCheckSingleTableRegion(regions); ensureTableRegionNotOnSameServerAsMeta(admin, hri); - int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName()); + int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionInfo() + .getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); // Turn off balancer so it doesn't cut in and mess up our placements. this.admin.setBalancerRunning(false, true); @@ -864,7 +867,8 @@ public class TestSplitTransactionOnCluster { List regions = cluster.getRegions(tableName); HRegionInfo hri = getAndCheckSingleTableRegion(regions); ensureTableRegionNotOnSameServerAsMeta(admin, hri); - int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName()); + int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionInfo() + .getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); // Turn off balancer so it doesn't cut in and mess up our placements. this.admin.setBalancerRunning(false, true); @@ -921,8 +925,8 @@ public class TestSplitTransactionOnCluster { if (firstTableRegions.size() == 0 || secondTableRegions.size() == 0) { fail("Each table should have at least one region."); } - ServerName serverName = - cluster.getServerHoldingRegion(firstTable, firstTableRegions.get(0).getRegionName()); + ServerName serverName = cluster.getServerHoldingRegion(firstTable, + firstTableRegions.get(0).getRegionInfo().getRegionName()); admin.move(secondTableRegions.get(0).getRegionInfo().getEncodedNameAsBytes(), Bytes.toBytes(serverName.getServerName())); Table table1 = null; @@ -969,9 +973,9 @@ public class TestSplitTransactionOnCluster { } admin.flush(desc.getTableName()); List regions = cluster.getRegions(desc.getTableName()); - int serverWith = cluster.getServerWith(regions.get(0).getRegionName()); + int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName()); HRegionServer regionServer = cluster.getRegionServer(serverWith); - cluster.getServerWith(regions.get(0).getRegionName()); + cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName()); SplitTransaction st = new SplitTransaction(regions.get(0), Bytes.toBytes("r3")); st.prepare(); st.stepsBeforePONR(regionServer, regionServer, false); @@ -1010,13 +1014,14 @@ public class TestSplitTransactionOnCluster { List regions = awaitTableRegions(tableName); assertTrue("Table not online", cluster.getRegions(tableName).size() != 0); - int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName()); + int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionInfo() + .getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); final HRegion region = findSplittableRegion(regions); assertTrue("not able to find a splittable region", region != null); SplitTransaction st = new MockedSplitTransaction(region, Bytes.toBytes("row2")) { @Override - public PairOfSameType stepsBeforePONR(final Server server, + public PairOfSameType stepsBeforePONR(final Server server, final RegionServerServices services, boolean testing) throws IOException { throw new SplittingNodeCreationFailedException (); } @@ -1054,7 +1059,7 @@ public class TestSplitTransactionOnCluster { p.add(Bytes.toBytes("f"), Bytes.toBytes("q"), Bytes.toBytes("value"+i)); region.put(p); } - region.flushcache(); + region.flush(true); Store store = region.getStore(Bytes.toBytes("f")); Collection storefiles = store.getStorefiles(); assertEquals(storefiles.size(), 1); @@ -1314,17 +1319,17 @@ public class TestSplitTransactionOnCluster { public static class MockedRegionObserver extends BaseRegionObserver { private SplitTransaction st = null; - private PairOfSameType daughterRegions = null; + private PairOfSameType daughterRegions = null; @Override public void preSplitBeforePONR(ObserverContext ctx, byte[] splitKey, List metaEntries) throws IOException { RegionCoprocessorEnvironment environment = ctx.getEnvironment(); HRegionServer rs = (HRegionServer) environment.getRegionServerServices(); - List onlineRegions = + List onlineRegions = rs.getOnlineRegions(TableName.valueOf("testSplitHooksBeforeAndAfterPONR_2")); - HRegion region = onlineRegions.get(0); - for (HRegion r : onlineRegions) { + Region region = onlineRegions.get(0); + for (Region r : onlineRegions) { if (r.getRegionInfo().containsRow(splitKey)) { region = r; break; @@ -1337,7 +1342,7 @@ public class TestSplitTransactionOnCluster { ctx.bypass(); return; } - region.forceSplit(splitKey); + ((HRegion)region).forceSplit(splitKey); daughterRegions = st.stepsBeforePONR(rs, rs, false); HRegionInfo copyOfParent = new HRegionInfo(region.getRegionInfo()); copyOfParent.setOffline(true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java index 2da1f7e7b7c..ed0ac25f16f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java @@ -94,13 +94,15 @@ public class TestStoreFileRefresherChore { } } - private HRegion initHRegion(HTableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId) throws IOException { + private Region initHRegion(HTableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId) + throws IOException { Configuration conf = TEST_UTIL.getConfiguration(); Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName()); HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false, 0, replicaId); - HRegionFileSystem fs = new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, info); + HRegionFileSystem fs = new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, + info); final Configuration walConf = new Configuration(conf); FSUtils.setRootDir(walConf, tableDir); final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId); @@ -111,7 +113,8 @@ public class TestStoreFileRefresherChore { return region; } - private void putData(HRegion region, int startRow, int numRows, byte[] qf, byte[]... families) throws IOException { + private void putData(Region region, int startRow, int numRows, byte[] qf, byte[]... families) + throws IOException { for (int i = startRow; i < startRow + numRows; i++) { Put put = new Put(Bytes.toBytes("" + i)); put.setDurability(Durability.SKIP_WAL); @@ -122,7 +125,7 @@ public class TestStoreFileRefresherChore { } } - private void verifyData(HRegion newReg, int startRow, int numRows, byte[] qf, byte[]... families) + private void verifyData(Region newReg, int startRow, int numRows, byte[] qf, byte[]... families) throws IOException { for (int i = startRow; i < startRow + numRows; i++) { byte[] row = Bytes.toBytes("" + i); @@ -160,13 +163,13 @@ public class TestStoreFileRefresherChore { byte[] qf = Bytes.toBytes("cq"); HRegionServer regionServer = mock(HRegionServer.class); - List regions = new ArrayList(); + List regions = new ArrayList(); when(regionServer.getOnlineRegionsLocalContext()).thenReturn(regions); when(regionServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration()); HTableDescriptor htd = getTableDesc(TableName.valueOf("testIsStale"), families); - HRegion primary = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 0); - HRegion replica1 = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 1); + Region primary = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 0); + Region replica1 = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 1); regions.add(primary); regions.add(replica1); @@ -174,7 +177,7 @@ public class TestStoreFileRefresherChore { // write some data to primary and flush putData(primary, 0, 100, qf, families); - primary.flushcache(); + primary.flush(true); verifyData(primary, 0, 100, qf, families); try { @@ -187,11 +190,11 @@ public class TestStoreFileRefresherChore { verifyData(replica1, 0, 100, qf, families); // simulate an fs failure where we cannot refresh the store files for the replica - ((FailingHRegionFileSystem)replica1.getRegionFileSystem()).fail = true; + ((FailingHRegionFileSystem)((HRegion)replica1).getRegionFileSystem()).fail = true; // write some more data to primary and flush putData(primary, 100, 100, qf, families); - primary.flushcache(); + primary.flush(true); verifyData(primary, 0, 200, qf, families); chore.chore(); // should not throw ex, but we cannot refresh the store files diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactionWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactionWithThroughputController.java index 8d0d5a845d1..b2f2898d360 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactionWithThroughputController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactionWithThroughputController.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreEngine; import org.apache.hadoop.hbase.regionserver.StripeStoreConfig; @@ -73,8 +74,8 @@ public class TestCompactionWithThroughputController { List rsts = cluster.getRegionServerThreads(); for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { HRegionServer hrs = rsts.get(i).getRegionServer(); - for (HRegion region : hrs.getOnlineRegions(tableName)) { - return region.getStores().values().iterator().next(); + for (Region region : hrs.getOnlineRegions(tableName)) { + return region.getStores().iterator().next(); } } return null; @@ -122,8 +123,6 @@ public class TestCompactionWithThroughputController { assertEquals(10, store.getStorefilesCount()); long startTime = System.currentTimeMillis(); TEST_UTIL.getHBaseAdmin().majorCompact(tableName); - Thread.sleep(5000); - assertEquals(10, store.getStorefilesCount()); while (store.getStorefilesCount() != 1) { Thread.sleep(20); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java index 2c25e3b1c9b..77071ce1343 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java @@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -451,7 +452,7 @@ public class TestFSHLog { System.currentTimeMillis(), clusterIds, -1, -1); wal.append(htd, info, logkey, edits, region.getSequenceId(), true, null); } - region.flushcache(); + region.flush(true); // FlushResult.flushSequenceId is not visible here so go get the current sequence id. long currentSequenceId = region.getSequenceId().get(); // Now release the appends diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 111acf3a27f..2c75f234beb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -53,8 +53,8 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.fs.HFileSystem; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; @@ -223,11 +223,8 @@ public class TestLogRolling { " log files"); // flush all regions - - List regions = - new ArrayList(server.getOnlineRegionsLocalContext()); - for (HRegion r: regions) { - r.flushcache(); + for (Region r: server.getOnlineRegionsLocalContext()) { + r.flush(true); } // Now roll the log @@ -530,9 +527,8 @@ public class TestLogRolling { assertTrue(loggedRows.contains("row1005")); // flush all regions - List regions = new ArrayList(server.getOnlineRegionsLocalContext()); - for (HRegion r: regions) { - r.flushcache(); + for (Region r: server.getOnlineRegionsLocalContext()) { + r.flush(true); } ResultScanner scanner = table.getScanner(new Scan()); @@ -574,7 +570,7 @@ public class TestLogRolling { server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); final WAL log = server.getWAL(null); - HRegion region = server.getOnlineRegions(table2.getName()).get(0); + Region region = server.getOnlineRegions(table2.getName()).get(0); Store s = region.getStore(HConstants.CATALOG_FAMILY); //have to flush namespace to ensure it doesn't affect wall tests @@ -595,7 +591,7 @@ public class TestLogRolling { assertEquals("Should have WAL; one table is not flushed", 1, DefaultWALProvider.getNumRolledLogFiles(log)); admin.flush(table2.getName()); - region.compactStores(); + region.compact(false); // Wait for compaction in case if flush triggered it before us. Assert.assertNotNull(s); for (int waitTime = 3000; s.getStorefilesCount() > 1 && waitTime > 0; waitTime -= 200) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index 03f48c69f62..afdcdc75761 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState; import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.Store; @@ -203,9 +204,9 @@ public class TestWALReplay { assertEquals(1, regions.size()); // move region to another regionserver - HRegion destRegion = regions.get(0); + Region destRegion = regions.get(0); int originServerNum = hbaseCluster - .getServerWith(destRegion.getRegionName()); + .getServerWith(destRegion.getRegionInfo().getRegionName()); assertTrue("Please start more than 1 regionserver", hbaseCluster .getRegionServerThreads().size() > 1); int destServerNum = 0; @@ -229,13 +230,13 @@ public class TestWALReplay { assertEquals(0, count); // flush region and make major compaction - destServer.getOnlineRegion(destRegion.getRegionName()).flushcache(); + Region region = destServer.getOnlineRegion(destRegion.getRegionInfo().getRegionName()); + region.flush(true); // wait to complete major compaction - for (Store store : destServer.getOnlineRegion(destRegion.getRegionName()) - .getStores().values()) { + for (Store store : region.getStores()) { store.triggerMajorCompaction(); } - destServer.getOnlineRegion(destRegion.getRegionName()).compactStores(); + region.compact(true); // move region to origin regionserver moveRegionAndWait(destRegion, originServer); @@ -251,7 +252,7 @@ public class TestWALReplay { resultScanner.close(); } - private void moveRegionAndWait(HRegion destRegion, HRegionServer destServer) + private void moveRegionAndWait(Region destRegion, HRegionServer destServer) throws InterruptedException, MasterNotRunningException, ZooKeeperConnectionException, IOException { HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster(); @@ -286,7 +287,7 @@ public class TestWALReplay { deleteDir(basedir); HTableDescriptor htd = createBasic3FamilyHTD(tableName); - HRegion region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); + Region region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region2); final byte [] rowName = tableName.getName(); @@ -347,10 +348,10 @@ public class TestWALReplay { final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString()); deleteDir(basedir); final HTableDescriptor htd = createBasic3FamilyHTD(tableName); - HRegion region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); + Region region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region2); WAL wal = createWAL(this.conf); - HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf); + Region region = HRegion.openHRegion(hri, htd, wal, this.conf); byte [] family = htd.getFamilies().iterator().next().getName(); Path f = new Path(basedir, "hfile"); @@ -358,7 +359,7 @@ public class TestWALReplay { Bytes.toBytes("z"), 10); List > hfs= new ArrayList>(1); hfs.add(Pair.newPair(family, f.toString())); - region.bulkLoadHFiles(hfs, true); + region.bulkLoadHFiles(hfs, true, null); // Add an edit so something in the WAL byte [] row = tableName.getName(); @@ -431,12 +432,12 @@ public class TestWALReplay { Bytes.toBytes(i + "50"), 10); hfs.add(Pair.newPair(family, f.toString())); } - region.bulkLoadHFiles(hfs, true); + region.bulkLoadHFiles(hfs, true, null); final int rowsInsertedCount = 31; assertEquals(rowsInsertedCount, getScannedCount(region.getScanner(new Scan()))); // major compact to turn all the bulk loaded files into one normal file - region.compactStores(true); + region.compact(true); assertEquals(rowsInsertedCount, getScannedCount(region.getScanner(new Scan()))); // Now 'crash' the region by stealing its wal @@ -498,7 +499,7 @@ public class TestWALReplay { addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region, "x"); if (first ) { // If first, so we have at least one family w/ different seqid to rest. - region.flushcache(); + region.flush(true); first = false; } } @@ -613,7 +614,7 @@ public class TestWALReplay { result.size()); // Let us flush the region - region.flushcache(); + region.flush(true); region.close(true); wal.shutdown(); @@ -707,7 +708,7 @@ public class TestWALReplay { // Let us flush the region CustomStoreFlusher.throwExceptionWhenFlushing.set(true); try { - region.flushcache(); + region.flush(true); fail("Injected exception hasn't been thrown"); } catch (Throwable t) { LOG.info("Expected simulated exception when flushing region," @@ -727,7 +728,7 @@ public class TestWALReplay { // call flush again CustomStoreFlusher.throwExceptionWhenFlushing.set(false); try { - region.flushcache(); + region.flush(true); } catch (IOException t) { LOG.info("Expected exception when flushing region because server is stopped," + t.getMessage()); @@ -883,7 +884,7 @@ public class TestWALReplay { // Let us flush the region // But this time completeflushcache is not yet done - region.flushcache(); + region.flush(true); for (HColumnDescriptor hcd : htd.getFamilies()) { addRegionEdits(rowName, hcd.getName(), 5, this.ee, region, "x"); } @@ -958,16 +959,16 @@ public class TestWALReplay { private HRegion r; @Override - public void requestFlush(HRegion region, boolean forceFlushAllStores) { + public void requestFlush(Region region, boolean force) { try { - r.flushcache(forceFlushAllStores); + r.flush(force); } catch (IOException e) { throw new RuntimeException("Exception flushing", e); } } @Override - public void requestDelayedFlush(HRegion region, long when, boolean forceFlushAllStores) { + public void requestDelayedFlush(Region region, long when, boolean forceFlushAllStores) { // TODO Auto-generated method stub } @@ -1006,7 +1007,7 @@ public class TestWALReplay { } static List addRegionEdits (final byte [] rowName, final byte [] family, - final int count, EnvironmentEdge ee, final HRegion r, + final int count, EnvironmentEdge ee, final Region r, final String qualifierPrefix) throws IOException { List puts = new ArrayList(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index b30820b385e..b670e659ddc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -225,7 +225,7 @@ public class TestMultiSlaveReplication { // request a roll admin.rollWALWriter(cluster.getServerHoldingRegion(region.getTableDesc().getTableName(), - region.getRegionName())); + region.getRegionInfo().getRegionName())); // wait try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java index 2226fcbbf4c..2231f0ea031 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java @@ -43,8 +43,8 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RpcRetryingCallerImpl; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -210,22 +210,22 @@ public class TestRegionReplicaReplicationEndpoint { private void verifyReplication(TableName tableName, int regionReplication, final int startRow, final int endRow, final boolean present) throws Exception { // find the regions - final HRegion[] regions = new HRegion[regionReplication]; + final Region[] regions = new Region[regionReplication]; for (int i=0; i < NB_SERVERS; i++) { HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(i); - List onlineRegions = rs.getOnlineRegions(tableName); - for (HRegion region : onlineRegions) { + List onlineRegions = rs.getOnlineRegions(tableName); + for (Region region : onlineRegions) { regions[region.getRegionInfo().getReplicaId()] = region; } } - for (HRegion region : regions) { + for (Region region : regions) { assertNotNull(region); } for (int i = 1; i < regionReplication; i++) { - final HRegion region = regions[i]; + final Region region = regions[i]; // wait until all the data is replicated to all secondary regions Waiter.waitFor(HTU.getConfiguration(), 90000, new Waiter.Predicate() { @Override @@ -234,8 +234,7 @@ public class TestRegionReplicaReplicationEndpoint { try { HTU.verifyNumericRows(region, HBaseTestingUtility.fam1, startRow, endRow, present); } catch(Throwable ex) { - LOG.warn("Verification from secondary region is not complete yet. Got:" + ex - + " " + ex.getMessage()); + LOG.warn("Verification from secondary region is not complete yet", ex); // still wait return false; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java index 9cbb0348243..930ffbac5a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WALKey; @@ -176,7 +177,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster { // replay the edits to the secondary using replay callable replicateUsingCallable(connection, entries); - HRegion region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); + Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); HTU.verifyNumericRows(region, f, 0, 1000); HTU.deleteNumericRows(table, f, 0, 1000); @@ -216,7 +217,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster { // replay the edits to the secondary using replay callable replicateUsingCallable(connection, entries); - HRegion region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); + Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); HTU.verifyNumericRows(region, f, 0, 1000); HTU.loadNumericRows(table, f, 1000, 2000); // load some more data to primary @@ -259,7 +260,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster { // replay the edits to the secondary using replay callable replicator.replicate(new ReplicateContext().setEntries(Lists.newArrayList(entries))); - HRegion region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); + Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); HTU.verifyNumericRows(region, f, 0, 1000); HTU.deleteNumericRows(table, f, 0, 1000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java index 6487ebe1167..89f10db2416 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; @@ -288,7 +288,7 @@ public class SecureTestUtil { private static List getAccessControllers(MiniHBaseCluster cluster) { List result = Lists.newArrayList(); for (RegionServerThread t: cluster.getLiveRegionServerThreads()) { - for (HRegion region: t.getRegionServer().getOnlineRegionsLocalContext()) { + for (Region region: t.getRegionServer().getOnlineRegionsLocalContext()) { Coprocessor cp = region.getCoprocessorHost() .findCoprocessor(AccessController.class.getName()); if (cp != null) { @@ -323,7 +323,7 @@ public class SecureTestUtil { for (Map.Entry e: mtimes.entrySet()) { if (!oldMTimes.containsKey(e.getKey())) { LOG.error("Snapshot of AccessController state does not include instance on region " + - e.getKey().getRegion().getRegionNameAsString()); + e.getKey().getRegion().getRegionInfo().getRegionNameAsString()); // Error out the predicate, we will try again return false; } @@ -331,8 +331,8 @@ public class SecureTestUtil { long now = e.getValue(); if (now <= old) { LOG.info("AccessController on region " + - e.getKey().getRegion().getRegionNameAsString() + " has not updated: mtime=" + - now); + e.getKey().getRegion().getRegionInfo().getRegionNameAsString() + + " has not updated: mtime=" + now); return false; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 8fab5f2186f..3e460dac32e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -96,6 +96,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermi import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.regionserver.ScanType; @@ -237,7 +238,7 @@ public class TestAccessController extends SecureTestUtil { admin.createTable(htd, new byte[][] { Bytes.toBytes("s") }); TEST_UTIL.waitUntilAllRegionsAssigned(TEST_TABLE.getTableName()); - HRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE.getTableName()).get(0); + Region region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE.getTableName()).get(0); RegionCoprocessorHost rcpHost = region.getCoprocessorHost(); RCP_ENV = rcpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf); @@ -2264,7 +2265,7 @@ public class TestAccessController extends SecureTestUtil { for (JVMClusterUtil.RegionServerThread thread: TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) { HRegionServer rs = thread.getRegionServer(); - for (HRegion region: rs.getOnlineRegions(TEST_TABLE.getTableName())) { + for (Region region: rs.getOnlineRegions(TEST_TABLE.getTableName())) { region.getCoprocessorHost().load(PingCoprocessor.class, Coprocessor.PRIORITY_USER, conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java index 3c71a8ccf9d..63fe4180f1e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java @@ -49,8 +49,8 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.OperationStatus; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.AccessControlLists; import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode; @@ -80,7 +80,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer private final ExpressionParser expressionParser = new ExpressionParser(); private final ExpressionExpander expressionExpander = new ExpressionExpander(); private Configuration conf; - private HRegion labelsRegion; + private Region labelsRegion; private List scanLabelGenerators; private List superUsers; private List superGroups; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java index 93c42ba8374..3671386bf9f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java @@ -57,8 +57,8 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResul import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; @@ -295,7 +295,7 @@ public abstract class TestVisibilityLabels { List regionServerThreads = TEST_UTIL.getHBaseCluster() .getRegionServerThreads(); for (RegionServerThread rsThread : regionServerThreads) { - List onlineRegions = rsThread.getRegionServer().getOnlineRegions( + List onlineRegions = rsThread.getRegionServer().getOnlineRegions( LABELS_TABLE_NAME); if (onlineRegions.size() > 0) { rsThread.getRegionServer().abort("Aborting "); @@ -329,7 +329,7 @@ public abstract class TestVisibilityLabels { for (RegionServerThread rsThread : regionServerThreads) { while (true) { if (!rsThread.getRegionServer().isAborted()) { - List onlineRegions = rsThread.getRegionServer().getOnlineRegions( + List onlineRegions = rsThread.getRegionServer().getOnlineRegions( LABELS_TABLE_NAME); if (onlineRegions.size() > 0) { break; @@ -386,7 +386,7 @@ public abstract class TestVisibilityLabels { } catch (InterruptedException e) { } } - HRegion labelsTableRegion = regionServer.getOnlineRegions(LABELS_TABLE_NAME).get(0); + Region labelsTableRegion = regionServer.getOnlineRegions(LABELS_TABLE_NAME).get(0); while (labelsTableRegion.isRecovering()) { try { Thread.sleep(10); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java index b663a2aedd6..d7b552e82c9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java @@ -62,9 +62,9 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSVisitor; @@ -630,8 +630,8 @@ public class SnapshotTestingUtils { final TableName tableName) throws IOException, InterruptedException { HRegionServer rs = util.getRSForFirstRegionInTable(tableName); - List onlineRegions = rs.getOnlineRegions(tableName); - for (HRegion region : onlineRegions) { + List onlineRegions = rs.getOnlineRegions(tableName); + for (Region region : onlineRegions) { region.waitForFlushesAndCompactions(); } // Wait up to 60 seconds for a table to be available. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index 618763c2fb0..8b6e84c1a33 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -1248,9 +1248,9 @@ public class TestHBaseFsck { } admin.flush(desc.getTableName()); List regions = cluster.getRegions(desc.getTableName()); - int serverWith = cluster.getServerWith(regions.get(0).getRegionName()); + int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName()); HRegionServer regionServer = cluster.getRegionServer(serverWith); - cluster.getServerWith(regions.get(0).getRegionName()); + cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName()); SplitTransaction st = new SplitTransaction(regions.get(0), Bytes.toBytes("r3")); st.prepare(); st.stepsBeforePONR(regionServer, regionServer, false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java index 7c289a1ee8a..5c08f1fe389 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.crypto.Encryption; @@ -43,7 +42,7 @@ import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.io.crypto.aes.AES; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.security.EncryptionUtil; @@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker; import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil; - import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -142,9 +140,9 @@ public class TestHBaseFsckEncryption { private List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList(); - for (HRegion region: + for (Region region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { - for (Store store: region.getStores().values()) { + for (Store store: region.getStores()) { for (StoreFile storefile: store.getStorefiles()) { paths.add(storefile.getPath()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java index 6a86580ca80..e31cb4e64da 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.Test; @@ -142,7 +143,7 @@ public class TestMergeTable { HRegionInfo hri = new HRegionInfo(desc.getTableName(), startKey, endKey); HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, rootdir, UTIL.getConfiguration(), desc); - LOG.info("Created region " + region.getRegionNameAsString()); + LOG.info("Created region " + region.getRegionInfo().getRegionNameAsString()); for(int i = firstRow; i < firstRow + nrows; i++) { Put put = new Put(Bytes.toBytes("row_" + String.format("%1$05d", i))); put.setDurability(Durability.SKIP_WAL); @@ -150,7 +151,7 @@ public class TestMergeTable { region.put(put); if (i % 10000 == 0) { LOG.info("Flushing write #" + i); - region.flushcache(); + region.flush(true); } } HBaseTestingUtility.closeRegionAndWAL(region); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFiltering.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFiltering.java index 66868a10a10..8833edac05a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFiltering.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFiltering.java @@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest; -import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; @@ -130,8 +130,8 @@ public class TestWALFiltering { private List getRegionsByServer(int rsId) throws IOException { List regionNames = Lists.newArrayList(); HRegionServer hrs = getRegionServer(rsId); - for (HRegion r : hrs.getOnlineRegions(TABLE_NAME)) { - regionNames.add(r.getRegionName()); + for (Region r : hrs.getOnlineRegions(TABLE_NAME)) { + regionNames.add(r.getRegionInfo().getRegionName()); } return regionNames; }