From 1f6d337ce54bf247a1d81d848bd3beae417d7b00 Mon Sep 17 00:00:00 2001 From: Phil Yang Date: Sun, 18 Sep 2016 14:41:00 +0800 Subject: [PATCH] HBASE-16505 Rewrite Region interface to pass deadline and support async operations --- .../hbase/ipc/DelegatingHBaseRpcController.java | 10 + .../hadoop/hbase/ipc/HBaseRpcController.java | 4 + .../hadoop/hbase/ipc/HBaseRpcControllerImpl.java | 13 + .../coprocessor/example/BulkDeleteEndpoint.java | 4 +- .../coordination/ZkSplitLogWorkerCoordination.java | 4 +- .../hbase/coprocessor/BaseRegionObserver.java | 6 +- .../coprocessor/BaseRegionServerObserver.java | 18 +- .../coprocessor/BaseRowProcessorEndpoint.java | 6 +- .../coprocessor/RegionCoprocessorEnvironment.java | 4 +- .../hadoop/hbase/coprocessor/RegionObserver.java | 12 +- .../hbase/coprocessor/RegionServerObserver.java | 14 +- .../org/apache/hadoop/hbase/ipc/CallRunner.java | 9 +- .../org/apache/hadoop/hbase/ipc/RpcServer.java | 7 +- .../hadoop/hbase/master/RegionStateStore.java | 4 +- .../procedure/flush/FlushTableSubprocedure.java | 12 +- .../RegionServerFlushTableProcedureManager.java | 6 +- .../hbase/quotas/RegionServerQuotaManager.java | 8 +- .../AnnotationReadingPriorityFunction.java | 2 +- .../hadoop/hbase/regionserver/BlockingRegion.java | 266 +++++++++ .../hbase/regionserver/CompactSplitThread.java | 32 +- .../regionserver/CompactedHFilesDischarger.java | 7 +- .../hbase/regionserver/CompactionRequestor.java | 14 +- .../hbase/regionserver/FlushRequestListener.java | 2 +- .../hadoop/hbase/regionserver/FlushRequester.java | 4 +- .../apache/hadoop/hbase/regionserver/HRegion.java | 302 +++++++++- .../hadoop/hbase/regionserver/HRegionServer.java | 92 +-- .../hbase/regionserver/HeapMemoryManager.java | 2 +- .../IncreasingToUpperBoundRegionSplitPolicy.java | 2 +- .../hadoop/hbase/regionserver/LogRoller.java | 2 +- .../hadoop/hbase/regionserver/MemStoreFlusher.java | 51 +- .../MetricsRegionServerWrapperImpl.java | 4 +- .../MetricsTableWrapperAggregateImpl.java | 2 +- .../hadoop/hbase/regionserver/OnlineRegions.java | 14 +- .../hadoop/hbase/regionserver/RSDumpServlet.java | 2 +- .../hadoop/hbase/regionserver/RSRpcServices.java | 70 +-- .../apache/hadoop/hbase/regionserver/Region.java | 620 ++++----------------- .../hadoop/hbase/regionserver/RegionBase.java | 487 ++++++++++++++++ .../hbase/regionserver/RegionCoprocessorHost.java | 16 +- .../hbase/regionserver/RegionMergeRequest.java | 2 +- .../hbase/regionserver/RegionMergeTransaction.java | 4 +- .../RegionMergeTransactionFactory.java | 4 +- .../regionserver/RegionMergeTransactionImpl.java | 10 +- .../hbase/regionserver/RegionOperationContext.java | 44 ++ .../hbase/regionserver/RegionServerServices.java | 10 +- .../hadoop/hbase/regionserver/RowProcessor.java | 2 +- .../hbase/regionserver/SecureBulkLoadManager.java | 10 +- .../hadoop/hbase/regionserver/SplitRequest.java | 2 +- .../hbase/regionserver/SplitTransaction.java | 4 +- .../regionserver/SplitTransactionFactory.java | 4 +- .../hbase/regionserver/SplitTransactionImpl.java | 29 +- .../regionserver/StorefileRefresherChore.java | 2 +- .../handler/FinishRegionRecoveringHandler.java | 4 +- .../regionserver/handler/OpenRegionHandler.java | 4 +- .../snapshot/FlushSnapshotSubprocedure.java | 12 +- .../snapshot/RegionServerSnapshotManager.java | 12 +- .../hbase/security/access/AccessControlLists.java | 6 +- .../hbase/security/access/AccessController.java | 35 +- .../DefaultVisibilityLabelServiceImpl.java | 6 +- .../security/visibility/VisibilityController.java | 8 +- .../hbase/security/visibility/VisibilityUtils.java | 4 +- .../hbase-webapps/regionserver/region.jsp | 4 +- .../org/apache/hadoop/hbase/HBaseTestCase.java | 6 +- .../apache/hadoop/hbase/HBaseTestingUtility.java | 16 +- .../org/apache/hadoop/hbase/MiniHBaseCluster.java | 18 +- .../hadoop/hbase/MockRegionServerServices.java | 18 +- .../hadoop/hbase/TestGlobalMemStoreSize.java | 8 +- .../org/apache/hadoop/hbase/TestIOFencing.java | 4 +- .../hadoop/hbase/backup/TestHFileArchiving.java | 10 +- .../example/TestZooKeeperTableArchiveClient.java | 12 +- .../org/apache/hadoop/hbase/client/TestAdmin2.java | 4 +- .../hbase/client/TestBlockEvictionFromClient.java | 28 +- .../hadoop/hbase/client/TestClientPushback.java | 6 +- .../hadoop/hbase/client/TestFromClientSide.java | 7 +- .../org/apache/hadoop/hbase/client/TestHCM.java | 6 +- .../apache/hadoop/hbase/client/TestMetaCache.java | 4 +- .../hbase/client/TestSplitOrMergeStatus.java | 4 +- .../ColumnAggregationEndpointNullResponse.java | 4 +- .../ColumnAggregationEndpointWithErrors.java | 4 +- .../hbase/coprocessor/SimpleRegionObserver.java | 6 +- .../hadoop/hbase/coprocessor/TestClassLoading.java | 18 +- .../coprocessor/TestCoprocessorInterface.java | 28 +- .../TestRegionObserverScannerOpenHook.java | 14 +- .../coprocessor/TestRegionServerObserver.java | 22 +- .../org/apache/hadoop/hbase/filter/TestFilter.java | 8 +- .../hbase/filter/TestInvocationRecordFilter.java | 4 +- .../apache/hadoop/hbase/fs/TestBlockReorder.java | 8 +- .../hbase/io/encoding/TestEncodedSeekers.java | 8 +- .../hadoop/hbase/io/encoding/TestPrefixTree.java | 4 +- .../io/encoding/TestSeekBeforeWithReverseScan.java | 4 +- .../hadoop/hbase/io/hfile/TestCacheOnWrite.java | 4 +- .../io/hfile/TestForceCacheImportantBlocks.java | 6 +- .../hfile/TestScannerSelectionUsingKeyRange.java | 5 +- .../io/hfile/TestScannerSelectionUsingTTL.java | 4 +- .../TestImportTSVWithOperationAttributes.java | 4 +- .../hbase/mapreduce/TestImportTSVWithTTLs.java | 4 +- .../hadoop/hbase/master/MockRegionServer.java | 14 +- .../hbase/master/TestAssignmentListener.java | 4 +- .../hbase/master/TestDistributedLogSplitting.java | 9 +- .../hbase/master/TestGetLastFlushedSequenceId.java | 6 +- .../hadoop/hbase/master/TestMasterFailover.java | 4 +- .../hadoop/hbase/master/TestRegionPlacement.java | 8 +- .../master/balancer/TestRegionLocationFinder.java | 8 +- .../TestSimpleRegionNormalizerOnCluster.java | 4 +- .../hbase/namespace/TestNamespaceAuditor.java | 4 +- .../hbase/regionserver/NoOpScanPolicyObserver.java | 2 +- .../hadoop/hbase/regionserver/RegionAsTable.java | 4 +- .../hbase/regionserver/TestAtomicOperation.java | 18 +- .../hadoop/hbase/regionserver/TestBlocksRead.java | 6 +- .../hbase/regionserver/TestBlocksScanned.java | 2 +- .../hbase/regionserver/TestColumnSeeking.java | 4 +- .../hbase/regionserver/TestCompactionState.java | 10 +- .../regionserver/TestEncryptionKeyRotation.java | 6 +- .../regionserver/TestEncryptionRandomKeying.java | 2 +- .../regionserver/TestEndToEndSplitTransaction.java | 4 +- .../regionserver/TestGetClosestAtOrBefore.java | 8 +- .../hadoop/hbase/regionserver/TestHRegion.java | 8 +- .../regionserver/TestHRegionReplayEvents.java | 4 +- .../hbase/regionserver/TestHeapMemoryManager.java | 4 +- .../hadoop/hbase/regionserver/TestKeepDeletes.java | 30 +- .../hbase/regionserver/TestMajorCompaction.java | 8 +- .../hadoop/hbase/regionserver/TestMinVersions.java | 12 +- .../hbase/regionserver/TestMinorCompaction.java | 2 +- .../hbase/regionserver/TestMultiColumnScanner.java | 2 +- .../regionserver/TestPerColumnFamilyFlush.java | 24 +- .../hbase/regionserver/TestRegionFavoredNodes.java | 8 +- .../regionserver/TestRegionReplicaFailover.java | 6 +- .../hbase/regionserver/TestRegionReplicas.java | 16 +- .../regionserver/TestRegionServerMetrics.java | 2 +- .../regionserver/TestRegionServerNoMaster.java | 2 +- .../TestRegionServerOnlineConfigChange.java | 2 +- .../hbase/regionserver/TestRegionSplitPolicy.java | 4 +- .../hbase/regionserver/TestResettingCounters.java | 2 +- .../hbase/regionserver/TestReversibleScanners.java | 4 +- .../hadoop/hbase/regionserver/TestRowTooBig.java | 4 +- .../hbase/regionserver/TestScanWithBloomError.java | 2 +- .../hbase/regionserver/TestSeekOptimizations.java | 2 +- .../hbase/regionserver/TestSplitTransaction.java | 12 +- .../TestSplitTransactionOnCluster.java | 11 +- .../hbase/regionserver/TestSplitWalDataLoss.java | 4 +- .../regionserver/TestStoreFileRefresherChore.java | 14 +- .../TestWalAndCompactingMemStoreFlush.java | 6 +- .../compactions/TestCompactedHFilesDischarger.java | 6 +- .../compactions/TestFIFOCompactionPolicy.java | 4 +- .../TestCompactionWithThroughputController.java | 7 +- .../TestFlushWithThroughputController.java | 6 +- .../regionserver/wal/AbstractTestLogRolling.java | 7 +- .../regionserver/wal/AbstractTestWALReplay.java | 24 +- .../hbase/regionserver/wal/TestLogRolling.java | 4 +- .../TestRegionReplicaReplicationEndpoint.java | 12 +- ...stRegionReplicaReplicationEndpointNoMaster.java | 8 +- .../hbase/security/access/SecureTestUtil.java | 4 +- .../security/access/TestAccessController.java | 6 +- .../security/access/TestAccessController3.java | 4 +- .../access/TestWithDisabledAuthorization.java | 8 +- .../ExpAsStringVisibilityLabelServiceImpl.java | 4 +- .../security/visibility/TestVisibilityLabels.java | 8 +- .../hbase/snapshot/SnapshotTestingUtils.java | 6 +- .../hadoop/hbase/util/TestHBaseFsckEncryption.java | 4 +- .../apache/hadoop/hbase/wal/TestWALFiltering.java | 4 +- 159 files changed, 1910 insertions(+), 1204 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BlockingRegion.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionBase.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOperationContext.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java index 9f9c636..e098499 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java @@ -114,6 +114,16 @@ public class DelegatingHBaseRpcController implements HBaseRpcController { } @Override + public void setDeadline(long deadline) { + delegate.setDeadline(deadline); + } + + @Override + public long getDeadline() { + return delegate.getDeadline(); + } + + @Override public void setFailed(IOException e) { delegate.setFailed(e); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java index 2c4b335..9d87845 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java @@ -67,6 +67,10 @@ public interface HBaseRpcController extends RpcController, CellScannable { boolean hasCallTimeout(); + void setDeadline(long deadline); + + long getDeadline(); + /** * Set failed with an exception to pass on. For use in async rpc clients * @param e exception to set with diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java index a976473..4222558 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java @@ -51,6 +51,8 @@ public class HBaseRpcControllerImpl implements HBaseRpcController { private IOException exception; + private long deadline = Long.MAX_VALUE; + /** * Priority to set on this request. Set it here in controller so available composing the request. * This is the ordained way of setting priorities going forward. We will be undoing the old @@ -117,6 +119,7 @@ public class HBaseRpcControllerImpl implements HBaseRpcController { cellScanner = null; exception = null; callTimeout = null; + deadline = Long.MAX_VALUE; // In the implementations of some callable with replicas, rpc calls are executed in a executor // and we could cancel the operation from outside which means there could be a race between // reset and startCancel. Although I think the race should be handled by the callable since the @@ -148,6 +151,16 @@ public class HBaseRpcControllerImpl implements HBaseRpcController { } @Override + public void setDeadline(long deadline) { + this.deadline = deadline; + } + + @Override + public long getDeadline() { + return deadline; + } + + @Override public synchronized String errorText() { if (!done || exception == null) { return null; 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 c9ab23c..e1fea35 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 @@ -47,7 +47,7 @@ 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.OperationStatus; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.util.Bytes; @@ -112,7 +112,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor RpcCallback done) { long totalRowsDeleted = 0L; long totalVersionsDeleted = 0L; - Region region = env.getRegion(); + BlockingRegion region = env.getRegion(); int rowBatchSize = request.getRowBatchSize(); Long timestamp = null; if (request.hasTimestamp()) { 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 e788860..af96d15 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; 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/BaseRegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java index 748268e..4116727 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 @@ -48,8 +48,8 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; 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.BlockingRegion; +import org.apache.hadoop.hbase.regionserver.RegionBase.Operation; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; @@ -156,7 +156,7 @@ public class BaseRegionObserver implements RegionObserver { } @Override - public void postSplit(ObserverContext e, Region l, Region r) + public void postSplit(ObserverContext e, BlockingRegion l, BlockingRegion 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 8952a14..64d9fcd 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 @@ -29,7 +29,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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; /** @@ -52,28 +52,28 @@ public class BaseRegionServerObserver implements RegionServerObserver { public void stop(CoprocessorEnvironment env) throws IOException { } @Override - public void preMerge(ObserverContext ctx, Region regionA, - Region regionB) throws IOException { } + public void preMerge(ObserverContext ctx, BlockingRegion regionA, + BlockingRegion regionB) throws IOException { } @Override - public void postMerge(ObserverContext c, Region regionA, - Region regionB, Region mergedRegion) throws IOException { } + public void postMerge(ObserverContext c, BlockingRegion regionA, + BlockingRegion regionB, BlockingRegion mergedRegion) throws IOException { } @Override public void preMergeCommit(ObserverContext ctx, - Region regionA, Region regionB, List metaEntries) throws IOException { } + BlockingRegion regionA, BlockingRegion regionB, List metaEntries) throws IOException { } @Override public void postMergeCommit(ObserverContext ctx, - Region regionA, Region regionB, Region mergedRegion) throws IOException { } + BlockingRegion regionA, BlockingRegion regionB, BlockingRegion mergedRegion) throws IOException { } @Override public void preRollBackMerge(ObserverContext ctx, - Region regionA, Region regionB) throws IOException { } + BlockingRegion regionA, BlockingRegion regionB) throws IOException { } @Override public void postRollBackMerge(ObserverContext ctx, - Region regionA, Region regionB) throws IOException { } + BlockingRegion regionA, BlockingRegion 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 ab5fc78..9dfdaac 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; 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 Region#processRowsWithLocks} and Coprocessor endpoints. + * using {@link BlockingRegion#processRowsWithLocks} and Coprocessor endpoints. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving @@ -68,7 +68,7 @@ extends RowProcessorService implements CoprocessorService, Coprocessor { ProcessResponse resultProto = null; try { RowProcessor processor = constructRowProcessorFromRequest(request); - Region region = env.getRegion(); + BlockingRegion 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 bdf88af..5eb9cc2 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; 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 */ - Region getRegion(); + BlockingRegion 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 4c94644..5fd1bce 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 @@ -45,12 +45,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.Region; -import org.apache.hadoop.hbase.regionserver.Region.Operation; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; 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.RegionScanner; +import org.apache.hadoop.hbase.regionserver.RegionBase.Operation; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; @@ -373,8 +373,8 @@ public interface RegionObserver extends Coprocessor { * @deprecated Use postCompleteSplit() instead */ @Deprecated - void postSplit(final ObserverContext c, final Region l, - final Region r) throws IOException; + void postSplit(final ObserverContext c, final BlockingRegion l, + final BlockingRegion r) throws IOException; /** * This will be called before PONR step as part of split transaction. Calling @@ -629,7 +629,7 @@ public interface RegionObserver extends Coprocessor { /** * This will be called for region operations where read lock is acquired in - * {@link Region#startRegionOperation()}. + * {@link BlockingRegion#startRegionOperation()}. * @param ctx * @param operation The operation is about to be taken on the region * @throws IOException @@ -638,7 +638,7 @@ public interface RegionObserver extends Coprocessor { Operation operation) throws IOException; /** - * Called after releasing read lock in {@link Region#closeRegionOperation()}. + * Called after releasing read lock in {@link BlockingRegion#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 b1b94ff..bc5f957 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; 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 Region regionA, final Region regionB) throws IOException; + final BlockingRegion regionA, final BlockingRegion 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 Region regionA, final Region regionB, final Region mergedRegion) throws IOException; + final BlockingRegion regionA, final BlockingRegion regionB, final BlockingRegion 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 Region regionA, final Region regionB, + final BlockingRegion regionA, final BlockingRegion regionB, @MetaMutationAnnotation List metaEntries) throws IOException; /** @@ -86,7 +86,7 @@ public interface RegionServerObserver extends Coprocessor { * @throws IOException */ void postMergeCommit(final ObserverContext ctx, - final Region regionA, final Region regionB, final Region mergedRegion) throws IOException; + final BlockingRegion regionA, final BlockingRegion regionB, final BlockingRegion 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 Region regionA, final Region regionB) throws IOException; + final BlockingRegion regionA, final BlockingRegion 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 Region regionA, final Region regionB) throws IOException; + final BlockingRegion regionA, final BlockingRegion 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/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java index b2b3c66..166b093 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java @@ -16,6 +16,9 @@ package org.apache.hadoop.hbase.ipc; * See the License for the specific language governing permissions and * limitations under the License. */ + +import com.google.protobuf.Message; + import java.net.InetSocketAddress; import java.nio.channels.ClosedChannelException; @@ -26,6 +29,7 @@ import org.apache.hadoop.hbase.CellScanner; 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.exceptions.TimeoutIOException; import org.apache.hadoop.hbase.ipc.RpcServer.Call; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.util.Pair; @@ -34,8 +38,6 @@ import org.apache.hadoop.util.StringUtils; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; -import com.google.protobuf.Message; - /** * The request processing logic, which is usually executed in thread pools provided by an * {@link RpcScheduler}. Call {@link #run()} to actually execute the contained @@ -122,6 +124,9 @@ public class CallRunner { // make the call resultPair = this.rpcServer.call(call.service, call.md, call.param, call.cellScanner, call.timestamp, this.status, call.startTime, call.timeout); + } catch (TimeoutIOException e) { + RpcServer.LOG.info("Can not complete this call in time: " + call); + return; } catch (Throwable e) { RpcServer.LOG.debug(Thread.currentThread().getName() + ": " + call.toShortString(), e); errorThrowable = e; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 37b60c9..e4767e1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -58,6 +58,7 @@ import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -82,6 +83,7 @@ import org.apache.hadoop.hbase.client.VersionInfoUtil; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.conf.ConfigurationObserver; import org.apache.hadoop.hbase.exceptions.RegionMovedException; +import org.apache.hadoop.hbase.exceptions.TimeoutIOException; import org.apache.hadoop.hbase.io.ByteBufferInputStream; import org.apache.hadoop.hbase.io.ByteBufferListOutputStream; import org.apache.hadoop.hbase.io.ByteBufferOutputStream; @@ -400,7 +402,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { return "callId: " + this.id + " service: " + serviceName + " methodName: " + ((this.md != null) ? this.md.getName() : "n/a") + " size: " + StringUtils.TraditionalBinaryPrefix.long2String(this.size, "", 1) + - " connection: " + connection.toString(); + " connection: " + connection.toString() + + " timeout: " + timeout; } String toTraceString() { @@ -2210,6 +2213,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { //get an instance of the method arg type HBaseRpcController controller = new HBaseRpcControllerImpl(cellScanner); controller.setCallTimeout(timeout); + controller.setDeadline(timeout > 0 ? receiveTime + timeout : Long.MAX_VALUE); Message result = service.callBlockingMethod(md, controller, param); long endTime = System.currentTimeMillis(); int processingTime = (int) (endTime - startTime); @@ -2260,6 +2264,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { if (e instanceof LinkageError) throw new DoNotRetryIOException(e); if (e instanceof IOException) throw (IOException)e; + if (e instanceof TimeoutException) throw new TimeoutIOException(e); LOG.error("Unexpected throwable object ", e); throw new IOException(e.getMessage(), e); } 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 2dbc087..921b806 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 @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -58,7 +58,7 @@ public class RegionStateStore { /** The delimiter for meta columns for replicaIds > 0 */ protected static final char META_REPLICA_ID_DELIMITER = '_'; - private volatile Region metaRegion; + private volatile BlockingRegion metaRegion; private volatile boolean initialized; private MultiHConnection multiHConnection; private final MasterServices 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 9898bb9..e1ef9ea 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,7 @@ 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; /** * This flush region implementation uses the distributed procedure framework to flush @@ -40,12 +40,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 +54,8 @@ public class FlushTableSubprocedure extends Subprocedure { } private static class RegionFlushTask implements Callable { - Region region; - RegionFlushTask(Region region) { + BlockingRegion region; + RegionFlushTask(BlockingRegion region) { this.region = region; } @@ -90,7 +90,7 @@ public class FlushTableSubprocedure extends Subprocedure { } // Add all hfiles already existing in region. - for (Region region : regions) { + for (BlockingRegion 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 1aa959c..c3a48a3 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 @@ -47,7 +47,7 @@ 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.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -139,7 +139,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) { @@ -174,7 +174,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 0a63c13..3a38d31 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.RpcServer; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; 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 Region region, + public OperationQuota checkQuota(final BlockingRegion 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 Region region, + public OperationQuota checkQuota(final BlockingRegion 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 Region region, + private OperationQuota checkQuota(final BlockingRegion region, final int numWrites, final int numReads, final int numScans) throws IOException, ThrottlingException { User user = RpcServer.getRequestUser(); 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 1eacc75..4c0b46a 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 @@ -221,7 +221,7 @@ public 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); - Region region = rpcServices.getRegion(regionSpecifier); + BlockingRegion region = rpcServices.getRegion(regionSpecifier); if (region.getRegionInfo().isSystemTable()) { if (LOG.isTraceEnabled()) { LOG.trace("High priority because region=" + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BlockingRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BlockingRegion.java new file mode 100644 index 0000000..3881dac --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BlockingRegion.java @@ -0,0 +1,266 @@ +/* + * 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 org.apache.hadoop.hbase.Cell; +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.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.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.filter.ByteArrayComparable; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay; + +/** + * Region in old blocking style, deprecated sice 2.0, use Region with context and CompletableFuture + * in new feature. + */ +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving +@Deprecated +public interface BlockingRegion extends RegionBase { + + /** + * + * Get a row lock for the specified row. All locks are reentrant. + * + * Before calling this function make sure that a region operation has already been + * started (the calling thread has already acquired the region-close-guard lock). + * @param row The row actions will be performed against + * @param readLock is the lock reader or writer. True indicates that a non-exlcusive + * lock is requested + */ + RowLock getRowLock(byte[] row, boolean readLock) throws IOException; + + + /** + * 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 value and if it does, + * it performs the mutation. If the passed value is null, the lack of column value + * (ie: non-existence) is used. See checkAndRowMutate to do many checkAndPuts at a time on a + * single row. + * @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 values and if it does, + * it performs the row mutations. If the passed value is null, the lack of column value + * (ie: non-existence) is used. Use to do many mutations on a single row. Use checkAndMutate + * to do one checkAndMutate at a time. + * @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 mutations were 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; + + /** + * Do a get for duplicate non-idempotent operation. + * @param get query parameters. + * @param withCoprocessor + * @param nonceGroup Nonce group. + * @param nonce Nonce. + * @return list of cells resulting from the operation + * @throws IOException + */ + List get(Get get, boolean withCoprocessor, long nonceGroup, long nonce) 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; + + /** + * Return an iterator that scans over the HRegion, returning the indicated columns and rows + * specified by the {@link Scan}. The scanner will also include the additional scanners passed + * along with the scanners for the specified Scan instance. Should be careful with the usage to + * pass additional scanners only within this Region + *

+ * This Iterator must be closed by the caller. + * + * @param scan configured {@link Scan} + * @param additionalScanners Any additional scanners to be used + * @return RegionScanner + * @throws IOException read exceptions + */ + RegionScanner getScanner(Scan scan, List additionalScanners) 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; + +} 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 c1f82b9..71d281a 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 @@ -221,8 +221,8 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi return queueLists.toString(); } - public synchronized void requestRegionsMerge(final Region a, - final Region b, final boolean forcible, long masterSystemTime, User user) { + public synchronized void requestRegionsMerge(final BlockingRegion a, + final BlockingRegion b, final boolean forcible, long masterSystemTime, User user) { try { mergePool.execute(new RegionMergeRequest(a, b, this.server, forcible, masterSystemTime,user)); if (LOG.isDebugEnabled()) { @@ -235,7 +235,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi } } - public synchronized boolean requestSplit(final Region r) { + public synchronized boolean requestSplit(final BlockingRegion r) { // don't split regions that are blocking if (shouldSplitRegion() && ((HRegion)r).getCompactPriority() >= Store.PRIORITY_USER) { byte[] midKey = ((HRegion)r).checkSplit(); @@ -247,14 +247,14 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi return false; } - public synchronized void requestSplit(final Region r, byte[] midKey) { + public synchronized void requestSplit(final BlockingRegion r, byte[] midKey) { requestSplit(r, midKey, null); } /* * The User parameter allows the split thread to assume the correct user identity */ - public synchronized void requestSplit(final Region r, byte[] midKey, User user) { + public synchronized void requestSplit(final BlockingRegion r, byte[] midKey, User user) { if (midKey == null) { LOG.debug("Region " + r.getRegionInfo().getRegionNameAsString() + " not splittable because midkey=null"); @@ -274,30 +274,30 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi } @Override - public synchronized List requestCompaction(final Region r, final String why) + public synchronized List requestCompaction(final BlockingRegion r, final String why) throws IOException { return requestCompaction(r, why, null); } @Override - public synchronized List requestCompaction(final Region r, final String why, + public synchronized List requestCompaction(final BlockingRegion r, final String why, List> requests) throws IOException { return requestCompaction(r, why, Store.NO_PRIORITY, requests, null); } @Override - public synchronized CompactionRequest requestCompaction(final Region r, final Store s, + public synchronized CompactionRequest requestCompaction(final BlockingRegion r, final Store s, final String why, CompactionRequest request) throws IOException { return requestCompaction(r, s, why, Store.NO_PRIORITY, request, null); } @Override - public synchronized List requestCompaction(final Region r, final String why, + public synchronized List requestCompaction(final BlockingRegion r, final String why, int p, List> requests, User user) throws IOException { return requestCompactionInternal(r, why, p, requests, true, user); } - private List requestCompactionInternal(final Region r, final String why, + private List requestCompactionInternal(final BlockingRegion r, final String why, int p, List> requests, boolean selectNow, User user) throws IOException { // not a special compaction request, so make our own list @@ -318,18 +318,18 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi return ret; } - public CompactionRequest requestCompaction(final Region r, final Store s, + public CompactionRequest requestCompaction(final BlockingRegion r, final Store s, final String why, int priority, CompactionRequest request, User user) throws IOException { return requestCompactionInternal(r, s, why, priority, request, true, user); } public synchronized void requestSystemCompaction( - final Region r, final String why) throws IOException { + final BlockingRegion r, final String why) throws IOException { requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false, null); } public void requestSystemCompaction( - final Region r, final Store s, final String why) throws IOException { + final BlockingRegion r, final Store s, final String why) throws IOException { requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false, null); } @@ -341,7 +341,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi * @param request custom compaction request. Can be null in which case a simple * compaction will be used. */ - private synchronized CompactionRequest requestCompactionInternal(final Region r, final Store s, + private synchronized CompactionRequest requestCompactionInternal(final BlockingRegion r, final Store s, final String why, int priority, CompactionRequest request, boolean selectNow, User user) throws IOException { if (this.server.isStopped() @@ -368,7 +368,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi return selectNow ? compaction.getRequest() : null; } - private CompactionContext selectCompaction(final Region r, final Store s, + private CompactionContext selectCompaction(final BlockingRegion r, final Store s, int priority, CompactionRequest request, User user) throws IOException { CompactionContext compaction = s.requestCompaction(priority, request, user); if (compaction == null) { @@ -465,7 +465,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi private ThreadPoolExecutor parent; private User user; - public CompactionRunner(Store store, Region region, + public CompactionRunner(Store store, BlockingRegion region, CompactionContext compaction, ThreadPoolExecutor parent, User user) { super(); this.store = store; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java index 18cf35d..f9a9376 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java @@ -26,9 +26,6 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.executor.EventType; -import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.regionserver.RegionServerServices; -import org.apache.hadoop.hbase.regionserver.Store; import com.google.common.annotations.VisibleForTesting; @@ -76,9 +73,9 @@ public class CompactedHFilesDischarger extends ScheduledChore { // Noop if rss is null. This will never happen in a normal condition except for cases // when the test case is not spinning up a cluster if (regionServerServices == null) return; - List onlineRegions = regionServerServices.getOnlineRegions(); + List onlineRegions = regionServerServices.getOnlineRegions(); if (onlineRegions != null) { - for (Region region : onlineRegions) { + for (BlockingRegion region : onlineRegions) { if (LOG.isTraceEnabled()) { LOG.trace( "Started the compacted hfiles cleaner for the region " + region.getRegionInfo()); 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 4921db6..6236b36 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 @@ -35,7 +35,7 @@ public interface CompactionRequestor { * compactions were started * @throws IOException */ - List requestCompaction(final Region r, final String why) + List requestCompaction(final BlockingRegion r, final String why) throws IOException; /** @@ -49,7 +49,7 @@ public interface CompactionRequestor { * @throws IOException */ List requestCompaction( - final Region r, final String why, List> requests + final BlockingRegion r, final String why, List> requests ) throws IOException; @@ -57,13 +57,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 Region} and {@link Store}. Custom + * @param request custom compaction request for the {@link BlockingRegion} 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 Region r, final Store s, final String why, CompactionRequest request + final BlockingRegion r, final Store s, final String why, CompactionRequest request ) throws IOException; /** @@ -79,7 +79,7 @@ public interface CompactionRequestor { * @throws IOException */ List requestCompaction( - final Region r, final String why, int pri, List> requests, + final BlockingRegion r, final String why, int pri, List> requests, User user ) throws IOException; @@ -88,13 +88,13 @@ 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 Region} for the + * @param request custom compaction request to run. {@link Store} and {@link BlockingRegion} for the * request must match the region and store specified here. * @param user * @return The created {@link CompactionRequest} or null if no compaction was started * @throws IOException */ CompactionRequest requestCompaction( - final Region r, final Store s, final String why, int pri, CompactionRequest request, User user + final BlockingRegion r, final Store s, final String why, int pri, CompactionRequest request, User user ) throws IOException; } 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 0e6bc4f..e860b32 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, Region region); + void flushRequested(FlushType type, BlockingRegion 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 c7e155a..2eb5904 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 @@ -33,7 +33,7 @@ public interface FlushRequester { * @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log * rolling. */ - void requestFlush(Region region, boolean forceFlushAllStores); + void requestFlush(BlockingRegion region, boolean forceFlushAllStores); /** * Tell the listener the cache needs to be flushed after a delay @@ -43,7 +43,7 @@ public interface FlushRequester { * @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log * rolling. */ - void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores); + void requestDelayedFlush(BlockingRegion 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 f97f6b2..65fcbcf 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 @@ -57,6 +57,7 @@ import java.util.Set; import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionService; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -118,6 +119,7 @@ import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.conf.ConfigurationManager; @@ -128,6 +130,7 @@ import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException; import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; import org.apache.hadoop.hbase.filter.ByteArrayComparable; +import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.FilterWrapper; import org.apache.hadoop.hbase.filter.IncompatibleFilterException; @@ -194,10 +197,9 @@ import org.apache.hadoop.util.StringUtils; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; - @SuppressWarnings("deprecation") @InterfaceAudience.Private -public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region { +public class HRegion implements HeapSize, PropagatingConfigurationObserver, BlockingRegion, Region { private static final Log LOG = LogFactory.getLog(HRegion.class); public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY = @@ -723,7 +725,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi 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.recovering = true; @@ -2791,6 +2793,298 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture getRowLock(RegionOperationContext context, + byte[] row, boolean readLock) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(getRowLock(row, readLock)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture append(RegionOperationContext context, Append append, + long nonceGroup, long nonce) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(append(append, nonceGroup, nonce)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture batchMutate(RegionOperationContext context, + Mutation[] mutations, long nonceGroup, long nonce) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(batchMutate(mutations, nonceGroup, nonce)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture batchReplay(RegionOperationContext context, + MutationReplay[] mutations, long replaySeqId) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(batchReplay(mutations, replaySeqId)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture checkAndMutate(RegionOperationContext context, byte[] row, + byte[] family, byte[] qualifier, CompareOp compareOp, + ByteArrayComparable comparator, Mutation mutation, boolean writeToWAL) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(checkAndMutate(row, family, qualifier, compareOp, comparator, mutation, + writeToWAL)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture checkAndRowMutate(RegionOperationContext context, byte[] row, + byte[] family, byte[] qualifier, CompareOp compareOp, + ByteArrayComparable comparator, RowMutations mutations, boolean writeToWAL) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(checkAndRowMutate(row, family, qualifier, compareOp, comparator, mutations, + writeToWAL)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture delete(RegionOperationContext context, Delete delete) { + CompletableFuture future = new CompletableFuture<>(); + try { + delete(delete); + future.complete(null); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture get(RegionOperationContext context, Get get) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(get(get)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture> get(RegionOperationContext context, Get get, + boolean withCoprocessor) { + CompletableFuture> future = new CompletableFuture<>(); + try { + future.complete(get(get, withCoprocessor)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture> get(RegionOperationContext context, Get get, + boolean withCoprocessor, long nonceGroup, long nonce) { + CompletableFuture> future = new CompletableFuture<>(); + try { + future.complete(get(get, withCoprocessor, nonceGroup, nonce)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture getScanner(RegionOperationContext context, Scan scan) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(getScanner(scan)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture getScanner(RegionOperationContext context, Scan scan, + List additionalScanners) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(getScanner(scan, additionalScanners)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture increment(RegionOperationContext context, Increment increment, + long nonceGroup, long nonce) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(increment(increment, nonceGroup, nonce)); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture mutateRow(RegionOperationContext context, RowMutations mutations) { + CompletableFuture future = new CompletableFuture<>(); + try { + mutateRow(mutations); + future.complete(null); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture mutateRowsWithLocks(RegionOperationContext context, + Collection mutations, Collection rowsToLock, long nonceGroup, long nonce) { + CompletableFuture future = new CompletableFuture<>(); + try { + mutateRowsWithLocks(mutations, rowsToLock, nonceGroup, nonce); + future.complete(null); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture processRowsWithLocks(RegionOperationContext context, + RowProcessor processor) { + CompletableFuture future = new CompletableFuture<>(); + try { + processRowsWithLocks(processor); + future.complete(null); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture processRowsWithLocks(RegionOperationContext context, + RowProcessor processor, long nonceGroup, long nonce) { + CompletableFuture future = new CompletableFuture<>(); + try { + processRowsWithLocks(processor, nonceGroup, nonce); + future.complete(null); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture processRowsWithLocks(RegionOperationContext context, + RowProcessor processor, long timeout, long nonceGroup, long nonce) { + CompletableFuture future = new CompletableFuture<>(); + try { + processRowsWithLocks(processor, timeout, nonceGroup, nonce); + future.complete(null); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + /** + * This method is still synchronous now. + */ + @Override + public CompletableFuture put(RegionOperationContext context, Put put) { + CompletableFuture future = new CompletableFuture<>(); + try { + put(put); + future.complete(null); + } catch (Throwable e) { + future.completeExceptionally(e); + } + return future; + } + + + /** * Struct-like class that tracks the progress of a batch operation, * accumulating status codes and tracking the index at which processing * is proceeding. @@ -6526,7 +6820,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return r.openHRegion(reporter); } - public static Region openHRegion(final Region other, final CancelableProgressable reporter) + public static BlockingRegion openHRegion(final BlockingRegion other, final CancelableProgressable reporter) throws IOException { return openHRegion((HRegion)other, reporter); } 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 bcd0c3f..001432f 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 @@ -271,7 +271,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 @@ -289,8 +289,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; @@ -1171,7 +1171,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; @@ -1185,7 +1185,7 @@ public class HRegionServer extends HasThread implements */ private long getWriteRequestCount() { long writeCount = 0; - for (Map.Entry e: this.onlineRegions.entrySet()) { + for (Map.Entry e: this.onlineRegions.entrySet()) { writeCount += e.getValue().getWriteRequestsCount(); } return writeCount; @@ -1232,7 +1232,7 @@ public class HRegionServer extends HasThread implements // improved; Additionally the load balancer will be able to take advantage of a more complete // history. MetricsRegionServerWrapper regionServerWrapper = metricsRegionServer.getRegionServerWrapper(); - Collection regions = getOnlineRegionsLocalContext(); + Collection regions = getOnlineRegionsLocalContext(); MemoryUsage memory = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage(); ClusterStatusProtos.ServerLoad.Builder serverLoad = @@ -1248,7 +1248,7 @@ public class HRegionServer extends HasThread implements } RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder(); RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder(); - for (Region region : regions) { + for (BlockingRegion region : regions) { if (region.getCoprocessorHost() != null) { Set regionCoprocessors = region.getCoprocessorHost().getCoprocessors(); Iterator iterator = regionCoprocessors.iterator(); @@ -1290,7 +1290,7 @@ public class HRegionServer extends HasThread implements String getOnlineRegionsAsPrintableString() { StringBuilder sb = new StringBuilder(); - for (Region r: this.onlineRegions.values()) { + for (BlockingRegion r: this.onlineRegions.values()) { if (sb.length() > 0) sb.append(", "); sb.append(r.getRegionInfo().getEncodedName()); } @@ -1326,7 +1326,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())) { @@ -1502,7 +1502,7 @@ public class HRegionServer extends HasThread implements * * @throws IOException */ - private RegionLoad createRegionLoad(final Region r, RegionLoad.Builder regionLoadBldr, + private RegionLoad createRegionLoad(final BlockingRegion r, RegionLoad.Builder regionLoadBldr, RegionSpecifier.Builder regionSpecifier) throws IOException { byte[] name = r.getRegionInfo().getRegionName(); int stores = 0; @@ -1570,7 +1570,7 @@ public class HRegionServer extends HasThread implements * @return An instance of RegionLoad. */ public RegionLoad createRegionLoad(final String encodedRegionName) throws IOException { - Region r = onlineRegions.get(encodedRegionName); + BlockingRegion r = onlineRegions.get(encodedRegionName); return r != null ? createRegionLoad(r, null, null) : null; } @@ -1599,7 +1599,7 @@ public class HRegionServer extends HasThread implements @Override protected void chore() { - for (Region r : this.instance.onlineRegions.values()) { + for (BlockingRegion r : this.instance.onlineRegions.values()) { if (r == null) continue; for (Store s : r.getStores()) { @@ -1643,7 +1643,7 @@ public class HRegionServer extends HasThread implements @Override protected void chore() { final StringBuffer whyFlush = new StringBuffer(); - for (Region r : this.server.onlineRegions.values()) { + for (BlockingRegion r : this.server.onlineRegions.values()) { if (r == null) continue; if (((HRegion)r).shouldFlush(whyFlush)) { FlushRequester requester = server.getFlushRequester(); @@ -1980,14 +1980,14 @@ public class HRegionServer extends HasThread implements } @Override - public void postOpenDeployTasks(final Region r) throws KeeperException, IOException { + public void postOpenDeployTasks(final BlockingRegion r) throws KeeperException, IOException { postOpenDeployTasks(new PostOpenDeployContext(r, -1)); } @Override public void postOpenDeployTasks(final PostOpenDeployContext context) throws KeeperException, IOException { - Region r = context.getRegion(); + BlockingRegion r = context.getRegion(); long masterSystemTime = context.getMasterSystemTime(); Preconditions.checkArgument(r instanceof HRegion, "r must be an HRegion"); rpcServices.checkOpen(); @@ -2448,10 +2448,10 @@ public class HRegionServer extends HasThread implements * @param abort Whether we're running an abort. */ void closeMetaTableRegions(final boolean abort) { - Region meta = null; + BlockingRegion 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(); @@ -2473,8 +2473,8 @@ public class HRegionServer extends HasThread implements void closeUserRegions(final boolean abort) { this.lock.writeLock().lock(); try { - for (Map.Entry e: this.onlineRegions.entrySet()) { - Region r = e.getValue(); + for (Map.Entry e: this.onlineRegions.entrySet()) { + BlockingRegion r = e.getValue(); if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) { // Don't update zk with this close transition; pass false. closeRegionIgnoreErrors(r.getRegionInfo(), abort); @@ -2504,7 +2504,7 @@ public class HRegionServer extends HasThread implements } @Override - public Map getRecoveringRegions() { + public Map getRecoveringRegions() { return this.recoveringRegions; } @@ -2535,13 +2535,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(Region region) { + public void addToOnlineRegions(BlockingRegion region) { this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region); configurationManager.registerObserver(region); } @@ -2551,9 +2551,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) { @@ -2561,7 +2561,7 @@ public class HRegionServer extends HasThread implements } }); // Copy over all regions. Regions are sorted by size with biggest first. - for (Region region : this.onlineRegions.values()) { + for (BlockingRegion region : this.onlineRegions.values()) { sortedRegions.put(region.getMemstoreSize(), region); } return sortedRegions; @@ -2588,7 +2588,7 @@ public class HRegionServer extends HasThread implements */ protected HRegionInfo[] getMostLoadedRegions() { ArrayList regions = new ArrayList(); - for (Region r : onlineRegions.values()) { + for (BlockingRegion r : onlineRegions.values()) { if (!r.isAvailable()) { continue; } @@ -2783,10 +2783,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 (Region region: this.onlineRegions.values()) { + for (BlockingRegion region: this.onlineRegions.values()) { HRegionInfo regionInfo = region.getRegionInfo(); if(regionInfo.getTable().equals(tableName)) { tableRegions.add(region); @@ -2797,8 +2797,8 @@ public class HRegionServer extends HasThread implements } @Override - public List getOnlineRegions() { - List allRegions = new ArrayList(); + public List getOnlineRegions() { + List allRegions = new ArrayList(); synchronized (this.onlineRegions) { // Return a clone copy of the onlineRegions allRegions.addAll(onlineRegions.values()); @@ -2814,7 +2814,7 @@ public class HRegionServer extends HasThread implements public Set getOnlineTables() { Set tables = new HashSet(); synchronized (this.onlineRegions) { - for (Region region: this.onlineRegions.values()) { + for (BlockingRegion region: this.onlineRegions.values()) { tables.add(region.getTableDesc().getTableName()); } } @@ -2831,8 +2831,8 @@ public class HRegionServer extends HasThread implements "skipping."); LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception); } - Collection regions = getOnlineRegionsLocalContext(); - for (Region region: regions) { + Collection regions = getOnlineRegionsLocalContext(); + for (BlockingRegion region: regions) { coprocessors.addAll(region.getCoprocessorHost().getCoprocessors()); try { coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors()); @@ -2882,7 +2882,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. - Region actualRegion = this.getFromOnlineRegions(encodedName); + BlockingRegion actualRegion = this.getFromOnlineRegions(encodedName); // Can be null if we're calling close on a region that's not online if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) { try { @@ -2944,7 +2944,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 Region getOnlineRegion(final byte[] regionName) { + public BlockingRegion getOnlineRegion(final byte[] regionName) { String encodedRegionName = HRegionInfo.encodeRegionName(regionName); return this.onlineRegions.get(encodedRegionName); } @@ -2954,14 +2954,14 @@ public class HRegionServer extends HasThread implements } @Override - public Region getFromOnlineRegions(final String encodedRegionName) { + public BlockingRegion getFromOnlineRegions(final String encodedRegionName) { return this.onlineRegions.get(encodedRegionName); } @Override - public boolean removeFromOnlineRegions(final Region r, ServerName destination) { - Region toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName()); + public boolean removeFromOnlineRegions(final BlockingRegion r, ServerName destination) { + BlockingRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName()); if (destination != null) { long closeSeqNum = r.getMaxFlushedSeqId(); if (closeSeqNum == HConstants.NO_SEQNUM) { @@ -2983,20 +2983,20 @@ public class HRegionServer extends HasThread implements * @return {@link HRegion} for regionName * @throws NotServingRegionException */ - protected Region getRegion(final byte[] regionName) + protected BlockingRegion getRegion(final byte[] regionName) throws NotServingRegionException { String encodedRegionName = HRegionInfo.encodeRegionName(regionName); return getRegionByEncodedName(regionName, encodedRegionName); } - public Region getRegionByEncodedName(String encodedRegionName) + public BlockingRegion getRegionByEncodedName(String encodedRegionName) throws NotServingRegionException { return getRegionByEncodedName(null, encodedRegionName); } - protected Region getRegionByEncodedName(byte[] regionName, String encodedRegionName) + protected BlockingRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName) throws NotServingRegionException { - Region region = this.onlineRegions.get(encodedRegionName); + BlockingRegion region = this.onlineRegions.get(encodedRegionName); if (region == null) { MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName); if (moveInfo != null) { @@ -3252,7 +3252,7 @@ public class HRegionServer extends HasThread implements * @throws KeeperException * @throws IOException */ - private void updateRecoveringRegionLastFlushedSequenceId(Region r) throws KeeperException, + private void updateRecoveringRegionLastFlushedSequenceId(BlockingRegion r) throws KeeperException, IOException { if (!r.isRecovering()) { // return immdiately for non-recovering regions @@ -3418,7 +3418,7 @@ public class HRegionServer extends HasThread implements @Override public double getCompactionPressure() { double max = 0; - for (Region region : onlineRegions.values()) { + for (BlockingRegion region : onlineRegions.values()) { for (Store store : region.getStores()) { double normCount = store.getCompactionPressure(); if (normCount > max) { 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 c360a60..daafad6 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 @@ -372,7 +372,7 @@ public class HeapMemoryManager { } @Override - public void flushRequested(FlushType type, Region region) { + public void flushRequested(FlushType type, BlockingRegion 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 7144cb8..3c9d571 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 @@ -107,7 +107,7 @@ public class IncreasingToUpperBoundRegionSplitPolicy extends ConstantSizeRegionS TableName tablename = 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 fa217ee..ad9289a 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 @@ -178,7 +178,7 @@ public class LogRoller extends HasThread { */ private void scheduleFlush(final byte [] encodedRegionName) { boolean scheduled = false; - Region r = this.services.getFromOnlineRegions(Bytes.toString(encodedRegionName)); + BlockingRegion 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 09b3738..0d9ecd8 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,7 +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.regionserver.RegionBase.FlushResult; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Counter; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -80,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; @@ -140,8 +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); @@ -150,12 +150,13 @@ class MemStoreFlusher implements FlushRequester { while (!flushedOne) { // Find the biggest region that doesn't have too many storefiles // (might be null!) - Region bestFlushableRegion = getBiggestMemstoreRegion(regionsBySize, excludedRegions, true); + BlockingRegion + bestFlushableRegion = getBiggestMemstoreRegion(regionsBySize, excludedRegions, true); // Find the biggest region, total, even if it might have too many flushes. - Region bestAnyRegion = getBiggestMemstoreRegion( + BlockingRegion bestAnyRegion = getBiggestMemstoreRegion( regionsBySize, excludedRegions, false); // Find the biggest region that is a secondary region - Region bestRegionReplica = getBiggestMemstoreOfRegionReplica(regionsBySize, + BlockingRegion bestRegionReplica = getBiggestMemstoreOfRegionReplica(regionsBySize, excludedRegions); if (bestAnyRegion == null && bestRegionReplica == null) { @@ -163,7 +164,7 @@ class MemStoreFlusher implements FlushRequester { return false; } - Region regionToFlush; + BlockingRegion regionToFlush; if (bestFlushableRegion != null && bestAnyRegion.getMemstoreSize() > 2 * bestFlushableRegion.getMemstoreSize()) { // Even if it's not supposed to be flushed, pick a region if it's more than twice @@ -288,12 +289,12 @@ class MemStoreFlusher implements FlushRequester { } } - private Region getBiggestMemstoreRegion( - SortedMap regionsBySize, - Set excludedRegions, + private BlockingRegion getBiggestMemstoreRegion( + SortedMap regionsBySize, + Set excludedRegions, boolean checkStoreFileCount) { synchronized (regionsInQueue) { - for (Region region : regionsBySize.values()) { + for (BlockingRegion region : regionsBySize.values()) { if (excludedRegions.contains(region)) { continue; } @@ -312,10 +313,10 @@ class MemStoreFlusher implements FlushRequester { return null; } - private Region getBiggestMemstoreOfRegionReplica(SortedMap regionsBySize, - Set excludedRegions) { + private BlockingRegion getBiggestMemstoreOfRegionReplica(SortedMap regionsBySize, + Set excludedRegions) { synchronized (regionsInQueue) { - for (Region region : regionsBySize.values()) { + for (BlockingRegion region : regionsBySize.values()) { if (excludedRegions.contains(region)) { continue; } @@ -330,7 +331,7 @@ class MemStoreFlusher implements FlushRequester { return null; } - private boolean refreshStoreFilesAndReclaimMemory(Region region) { + private boolean refreshStoreFilesAndReclaimMemory(BlockingRegion region) { try { return region.refreshStoreFiles(); } catch (IOException e) { @@ -356,7 +357,7 @@ class MemStoreFlusher implements FlushRequester { } @Override - public void requestFlush(Region r, boolean forceFlushAllStores) { + public void requestFlush(BlockingRegion 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 +370,7 @@ class MemStoreFlusher implements FlushRequester { } @Override - public void requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) { + public void requestDelayedFlush(BlockingRegion r, long delay, boolean forceFlushAllStores) { synchronized (regionsInQueue) { if (!regionsInQueue.containsKey(r)) { // This entry has some delay @@ -435,7 +436,7 @@ class MemStoreFlusher implements FlushRequester { * not flushed. */ private boolean flushRegion(final FlushRegionEntry fqe) { - Region region = fqe.region; + BlockingRegion region = fqe.region; if (!region.getRegionInfo().isMetaRegion() && isTooManyStoreFiles(region)) { if (fqe.isMaximumWait(this.blockingWaitTime)) { @@ -484,7 +485,7 @@ class MemStoreFlusher implements FlushRequester { * false, there will be accompanying log messages explaining why the region was * not flushed. */ - private boolean flushRegion(final Region region, final boolean emergencyFlush, + private boolean flushRegion(final BlockingRegion region, final boolean emergencyFlush, boolean forceFlushAllStores) { synchronized (this.regionsInQueue) { FlushRegionEntry fqe = this.regionsInQueue.remove(region); @@ -534,7 +535,7 @@ class MemStoreFlusher implements FlushRequester { return true; } - private void notifyFlushRequest(Region region, boolean emergencyFlush) { + private void notifyFlushRequest(BlockingRegion region, boolean emergencyFlush) { FlushType type = FlushType.NORMAL; if (emergencyFlush) { type = isAboveHighWaterMark() ? FlushType.ABOVE_HIGHER_MARK : FlushType.ABOVE_LOWER_MARK; @@ -550,7 +551,7 @@ class MemStoreFlusher implements FlushRequester { } } - private boolean isTooManyStoreFiles(Region region) { + private boolean isTooManyStoreFiles(BlockingRegion region) { // When compaction is disabled, the region is flushable if (!region.getTableDesc().isCompactionEnabled()) { @@ -712,7 +713,7 @@ class MemStoreFlusher implements FlushRequester { * a while. */ static class FlushRegionEntry implements FlushQueueEntry { - private final Region region; + private final BlockingRegion region; private final long createTime; private long whenToExpire; @@ -720,7 +721,7 @@ class MemStoreFlusher implements FlushRequester { private boolean forceFlushAllStores; - FlushRegionEntry(final Region r, boolean forceFlushAllStores) { + FlushRegionEntry(final BlockingRegion r, boolean forceFlushAllStores) { this.region = r; this.createTime = EnvironmentEdgeManager.currentTime(); this.whenToExpire = this.createTime; 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 d65365e..f2d1e16 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 @@ -206,7 +206,7 @@ class MetricsRegionServerWrapperImpl @Override public long getNumOnlineRegions() { - Collection onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext(); + Collection onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext(); if (onlineRegionsLocalContext == null) { return 0; } @@ -672,7 +672,7 @@ class MetricsRegionServerWrapperImpl long tempMobScanCellsSize = 0; long tempBlockedRequestsCount = 0; int regionCount = 0; - for (Region r : regionServer.getOnlineRegionsLocalContext()) { + for (BlockingRegion r : regionServer.getOnlineRegionsLocalContext()) { tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL(); tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL(); tempReadRequestsCount += r.getReadRequestsCount(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java index c5f0f7b..c8387e5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java @@ -61,7 +61,7 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr public void run() { Map localMetricsTableMap = new HashMap<>(); - for (Region r : regionServer.getOnlineRegionsLocalContext()) { + for (BlockingRegion r : regionServer.getOnlineRegionsLocalContext()) { TableName tbl= r.getTableDesc().getTableName(); MetricsTableValues metricsTable = localMetricsTableMap.get(tbl); if (metricsTable == null) { 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 310108c..a4084f0 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 @@ -30,7 +30,7 @@ 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 Region} instance. + * encoded name and the value is an {@link BlockingRegion} instance. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving @@ -39,7 +39,7 @@ public interface OnlineRegions extends Server { * Add to online regions. * @param r */ - void addToOnlineRegions(final Region r); + void addToOnlineRegions(final BlockingRegion r); /** * This method removes Region corresponding to hri from the Map of onlineRegions. @@ -48,17 +48,17 @@ public interface OnlineRegions extends Server { * @param destination Destination, if any, null otherwise. * @return True if we removed a region from online list. */ - boolean removeFromOnlineRegions(final Region r, ServerName destination); + boolean removeFromOnlineRegions(final BlockingRegion r, ServerName destination); /** - * Return {@link Region} instance. + * Return {@link BlockingRegion} instance. * Only works if caller is in same context, in same JVM. Region is not * serializable. * @param encodedRegionName * @return Region for the passed encoded encodedRegionName or * null if named region is not member of the online regions. */ - Region getFromOnlineRegions(String encodedRegionName); + BlockingRegion getFromOnlineRegions(String encodedRegionName); /** * Get all online regions of a table in this RS. @@ -66,11 +66,11 @@ public interface OnlineRegions extends Server { * @return List of Region * @throws java.io.IOException */ - List getOnlineRegions(TableName tableName) throws IOException; + List getOnlineRegions(TableName tableName) throws IOException; /** * Get all online regions in this RS. * @return List of online Region */ - List getOnlineRegions(); + List getOnlineRegions(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java index 0c69bd9..522f642 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java @@ -105,7 +105,7 @@ public class RSDumpServlet extends StateDumpServlet { public static void dumpRowLock(HRegionServer hrs, PrintWriter out) { StringBuilder sb = new StringBuilder(); - for (Region region : hrs.getOnlineRegions()) { + for (BlockingRegion region : hrs.getOnlineRegions()) { HRegion hRegion = (HRegion)region; if (hRegion.getLockedRows().size() > 0) { for (HRegion.RowLockContext rowLockContext : hRegion.getLockedRows().values()) { 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 e857c8d..f62591e 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 @@ -169,7 +169,7 @@ import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager; import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; import org.apache.hadoop.hbase.regionserver.Leases.Lease; import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException; -import org.apache.hadoop.hbase.regionserver.Region.Operation; +import org.apache.hadoop.hbase.regionserver.RegionBase.Operation; import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope; import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler; @@ -338,11 +338,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler, private static class RegionScannerHolder { private AtomicLong nextCallSeq = new AtomicLong(0); private RegionScanner s; - private Region r; + private BlockingRegion r; final RpcCallback closeCallBack; final RpcCallback shippedCallback; - public RegionScannerHolder(RegionScanner s, Region r, RpcCallback closeCallBack, + public RegionScannerHolder(RegionScanner s, BlockingRegion r, RpcCallback closeCallBack, RpcCallback shippedCallback) { this.s = s; this.r = r; @@ -381,7 +381,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, RegionScanner s = rsh.s; LOG.info("Scanner " + this.scannerName + " lease expired on region " + s.getRegionInfo().getRegionNameAsString()); - Region region = null; + BlockingRegion region = null; try { region = regionServer.getRegion(s.getRegionInfo().getRegionName()); if (region != null && region.getCoprocessorHost() != null) { @@ -503,7 +503,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @param cellScanner if non-null, the mutation data -- the Cell content. * @throws IOException */ - private void mutateRows(final Region region, + private void mutateRows(final BlockingRegion region, final List actions, final CellScanner cellScanner, RegionActionResult.Builder builder) throws IOException { if (!region.getRegionInfo().isMetaTable()) { @@ -554,7 +554,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @param compareOp * @param comparator @throws IOException */ - private boolean checkAndRowMutate(final Region region, final List actions, + private boolean checkAndRowMutate(final BlockingRegion region, final List actions, final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier, CompareOp compareOp, ByteArrayComparable comparator, RegionActionResult.Builder builder) throws IOException { @@ -605,7 +605,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * bypassed as indicated by RegionObserver, null otherwise * @throws IOException */ - private Result append(final Region region, final OperationQuota quota, + private Result append(final BlockingRegion region, final OperationQuota quota, final MutationProto mutation, final CellScanner cellScanner, long nonceGroup) throws IOException { long before = EnvironmentEdgeManager.currentTime(); @@ -653,7 +653,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @return the Result * @throws IOException */ - private Result increment(final Region region, final OperationQuota quota, + private Result increment(final BlockingRegion region, final OperationQuota quota, final MutationProto mutation, final CellScanner cells, long nonceGroup) throws IOException { long before = EnvironmentEdgeManager.currentTime(); @@ -706,7 +706,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @param context the current RpcCallContext * @return Return the cellScanner passed */ - private List doNonAtomicRegionMutation(final Region region, + private List doNonAtomicRegionMutation(final BlockingRegion region, final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner, final RegionActionResult.Builder builder, List cellsToReturn, long nonceGroup, final RegionScannersCloseCallBack closeCallBack, RpcCallContext context) { @@ -861,7 +861,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @param region * @param mutations */ - private void doBatchOp(final RegionActionResult.Builder builder, final Region region, + private void doBatchOp(final RegionActionResult.Builder builder, final BlockingRegion region, final OperationQuota quota, final List mutations, final CellScanner cells) { Mutation[] mArray = new Mutation[mutations.size()]; @@ -940,7 +940,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * exceptionMessage if any * @throws IOException */ - private OperationStatus [] doReplayBatchOp(final Region region, + private OperationStatus [] doReplayBatchOp(final BlockingRegion region, final List mutations, long replaySeqId) throws IOException { long before = EnvironmentEdgeManager.currentTime(); boolean batchContainsPuts = false, batchContainsDelete = false; @@ -1178,7 +1178,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return lastBlock; } - RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r) + RegionScannerHolder addScanner(String scannerName, RegionScanner s, BlockingRegion r) throws LeaseStillHeldException { Lease lease = regionServer.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod, new ScannerListener(scannerName)); @@ -1204,7 +1204,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * but failed to find the region */ @VisibleForTesting - public Region getRegion( + public BlockingRegion getRegion( final RegionSpecifier regionSpecifier) throws IOException { ByteString value = regionSpecifier.getValue(); RegionSpecifierType type = regionSpecifier.getType(); @@ -1370,7 +1370,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); region.startRegionOperation(Operation.COMPACT_REGION); LOG.info("Compacting " + region.getRegionInfo().getRegionNameAsString()); boolean major = false; @@ -1429,7 +1429,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); LOG.info("Flushing " + region.getRegionInfo().getRegionNameAsString()); boolean shouldFlush = true; if (request.hasIfOlderThanTs()) { @@ -1471,9 +1471,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 (Region region: onlineRegions.values()) { + for (BlockingRegion region: onlineRegions.values()) { list.add(region.getRegionInfo()); } Collections.sort(list); @@ -1490,7 +1490,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); HRegionInfo info = region.getRegionInfo(); GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder(); builder.setRegionInfo(HRegionInfo.convert(info)); @@ -1531,7 +1531,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, final GetStoreFileRequest request) throws ServiceException { try { checkOpen(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); requestCount.increment(); Set columnFamilies; if (request.getFamilyCount() == 0) { @@ -1568,8 +1568,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region regionA = getRegion(request.getRegionA()); - Region regionB = getRegion(request.getRegionB()); + BlockingRegion regionA = getRegion(request.getRegionA()); + BlockingRegion regionB = getRegion(request.getRegionB()); boolean forcible = request.getForcible(); long masterSystemTime = request.hasMasterSystemTime() ? request.getMasterSystemTime() : -1; regionA.startRegionOperation(Operation.MERGE_REGION); @@ -1677,7 +1677,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { String encodedName = region.getEncodedName(); byte[] encodedNameBytes = region.getEncodedNameAsBytes(); - final Region onlineRegion = regionServer.getFromOnlineRegions(encodedName); + final BlockingRegion 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:" @@ -1797,7 +1797,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, checkOpen(); String encodedName = region.getEncodedName(); byte[] encodedNameBytes = region.getEncodedNameAsBytes(); - final Region onlineRegion = regionServer.getFromOnlineRegions(encodedName); + final BlockingRegion onlineRegion = regionServer.getFromOnlineRegions(encodedName); if (onlineRegion != null) { LOG.info("Region already online. Skipping warming up " + region); @@ -1848,7 +1848,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return ReplicateWALEntryResponse.newBuilder().build(); } ByteString regionName = entries.get(0).getKey().getEncodedRegionName(); - Region region = regionServer.getRegionByEncodedName(regionName.toStringUtf8()); + BlockingRegion region = regionServer.getRegionByEncodedName(regionName.toStringUtf8()); RegionCoprocessorHost coprocessorHost = ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo()) ? region.getCoprocessorHost() @@ -1924,7 +1924,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } - WAL getWAL(Region region) { + WAL getWAL(BlockingRegion region) { return ((HRegion)region).getWAL(); } @@ -1995,7 +1995,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion 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. " @@ -2061,7 +2061,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); boolean bypass = false; boolean loaded = false; @@ -2101,7 +2101,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); String bulkToken = regionServer.secureBulkLoadManager.prepareBulkLoad(region, request); PrepareBulkLoadResponse.Builder builder = PrepareBulkLoadResponse.newBuilder(); @@ -2119,7 +2119,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); regionServer.secureBulkLoadManager.cleanupBulkLoad(region, request); CleanupBulkLoadResponse response = CleanupBulkLoadResponse.newBuilder().build(); @@ -2135,7 +2135,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); Message result = execServiceOnRegion(region, request.getCall()); CoprocessorServiceResponse.Builder builder = CoprocessorServiceResponse.newBuilder(); @@ -2150,7 +2150,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } - private Message execServiceOnRegion(Region region, + private Message execServiceOnRegion(BlockingRegion region, final ClientProtos.CoprocessorServiceCall serviceCall) throws IOException { // ignore the passed in controller (from the serialized call) ServerRpcController execController = new ServerRpcController(); @@ -2173,7 +2173,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, checkOpen(); requestCount.increment(); rpcGetRequestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); GetResponse.Builder builder = GetResponse.newBuilder(); ClientProtos.Get get = request.getGet(); @@ -2319,7 +2319,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, for (RegionAction regionAction : request.getRegionActionList()) { this.requestCount.add(regionAction.getActionCount()); OperationQuota quota; - Region region; + BlockingRegion region; regionActionResultBuilder.clear(); RegionSpecifier regionSpecifier = regionAction.getRegion(); try { @@ -2447,7 +2447,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, checkOpen(); requestCount.increment(); rpcMutateRequestCount.increment(); - Region region = getRegion(request.getRegion()); + BlockingRegion region = getRegion(request.getRegion()); MutateResponse.Builder builder = MutateResponse.newBuilder(); MutationProto mutation = request.getMutation(); if (!region.getRegionInfo().isMetaTable()) { @@ -2594,7 +2594,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, rpcScanRequestCount.increment(); int ttl = 0; - Region region = null; + BlockingRegion region = null; RegionScanner scanner = null; RegionScannerHolder rsh = null; boolean moreResults = true; 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 index efd68b8..fa5fed0 100644 --- 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 @@ -17,48 +17,34 @@ */ package org.apache.hadoop.hbase.regionserver; -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.Message; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; import java.io.IOException; import java.util.Collection; import java.util.List; -import java.util.Map; +import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -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 org.apache.hadoop.hbase.filter.CompareFilter; +import org.apache.hadoop.hbase.wal.WALSplitter; /** * 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 @@ -67,343 +53,152 @@ import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay; * 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. + * + * Methods in this interface are suggested to be implemented in asynchronous mode. But at start + * the implementation may still synchronous. See HBASE-16583 for details. BlockingRegion's methods + * will always be synchronous but has been deprecated now. */ @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; no edit in memory will have - * a sequence id that is less that what is returned here. - */ - long getMaxFlushedSeqId(); - - /** @return the oldest flushed sequence id for the given family; can be beyond - * {@link #getMaxFlushedSeqId()} in case where we've flushed a subset of a regions column - * families - * @deprecated Since version 1.2.0. Exposes too much about our internals; shutting it down. - * Do not use. - */ - @VisibleForTesting - @Deprecated - 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; pass null for default*/ - long getReadPoint(IsolationLevel isolationLevel); - - /** - * @return readpoint considering given IsolationLevel - * @deprecated Since 1.2.0. Use {@link #getReadPoint(IsolationLevel)} instead. - */ - @Deprecated - 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 filtered read requests count for this region */ - long getFilteredReadRequestsCount(); - - /** @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 store services for this region, to access services required by store level needs */ - RegionServicesForStores getRegionServicesForStores(); - - /** @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} and elsewhere to provide - * context for various checks. - */ - 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; +public interface Region extends RegionBase { /** - * If the given list of row locks is not null, releases all locks. + * Get a row lock for the specified row. All locks are reentrant. + *

+ * Before calling this function make sure that a region operation has already been + * started (the calling thread has already acquired the region-close-guard lock). + * + * @param row The row actions will be performed against + * @param readLock is the lock reader or writer. True indicates that a non-exlcusive + * lock is requested */ - void releaseRowLocks(List rowLocks); - - /////////////////////////////////////////////////////////////////////////// - // Region operations + CompletableFuture getRowLock(RegionOperationContext context, byte[] row, + boolean readLock); /** * 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; + CompletableFuture append(RegionOperationContext context, Append append, long nonceGroup, + long nonce); /** * 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 mutations the list of mutations * @param nonceGroup * @param nonce * @return an array of OperationStatus which internally contains the - * OperationStatusCode and the exceptionMessage if any. + * OperationStatusCode and the exceptionMessage if any. * @throws IOException */ - OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce) - throws IOException; + CompletableFuture batchMutate(RegionOperationContext context, + Mutation[] mutations, + long nonceGroup, long nonce); /** * Replay a batch of mutations. - * @param mutations mutations to replay. + * + * @param mutations mutations to replay. * @param replaySeqId * @return an array of OperationStatus which internally contains the - * OperationStatusCode and the exceptionMessage if any. + * OperationStatusCode and the exceptionMessage if any. * @throws IOException */ - OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId) throws IOException; + CompletableFuture batchReplay(RegionOperationContext context, + WALSplitter.MutationReplay[] mutations, + long replaySeqId); /** * Atomically checks if a row/family/qualifier value matches the expected value and if it does, * it performs the mutation. If the passed value is null, the lack of column value * (ie: non-existence) is used. See checkAndRowMutate to do many checkAndPuts at a time on a * single row. - * @param row to check - * @param family column family to check - * @param qualifier column qualifier to check - * @param compareOp the comparison operator + * + * @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; + CompletableFuture checkAndMutate(RegionOperationContext context, byte[] row, + byte[] family, + byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, + Mutation mutation, + boolean writeToWAL); /** * Atomically checks if a row/family/qualifier value matches the expected values and if it does, * it performs the row mutations. If the passed value is null, the lack of column value * (ie: non-existence) is used. Use to do many mutations on a single row. Use checkAndMutate * to do one checkAndMutate at a time. - * @param row to check - * @param family column family to check - * @param qualifier column qualifier to check - * @param compareOp the comparison operator + * + * @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 mutations were applied, false otherwise * @throws IOException */ - boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier, CompareOp compareOp, - ByteArrayComparable comparator, RowMutations mutations, boolean writeToWAL) - throws IOException; + CompletableFuture checkAndRowMutate(RegionOperationContext context, byte[] row, + byte[] family, + byte[] qualifier, CompareFilter.CompareOp compareOp, ByteArrayComparable comparator, + RowMutations mutations, boolean writeToWAL); /** * Deletes the specified cells/row. + * * @param delete * @throws IOException */ - void delete(Delete delete) throws IOException; + CompletableFuture delete(RegionOperationContext context, Delete delete); /** * Do a get based on the get parameter. + * * @param get query parameters * @return result of the operation */ - Result get(Get get) throws IOException; + CompletableFuture get(RegionOperationContext context, Get get); /** * Do a get based on the get parameter. - * @param get query parameters + * + * @param get query parameters * @param withCoprocessor invoke coprocessor or not. We don't want to - * always invoke cp. + * always invoke cp. * @return list of cells resulting from the operation */ - List get(Get get, boolean withCoprocessor) throws IOException; + CompletableFuture> get(RegionOperationContext context, Get get, + boolean withCoprocessor); /** * Do a get for duplicate non-idempotent operation. - * @param get query parameters. + * + * @param get query parameters. * @param withCoprocessor - * @param nonceGroup Nonce group. - * @param nonce Nonce. + * @param nonceGroup Nonce group. + * @param nonce Nonce. * @return list of cells resulting from the operation * @throws IOException */ - List get(Get get, boolean withCoprocessor, long nonceGroup, long nonce) throws IOException; + CompletableFuture> get(RegionOperationContext context, Get get, + boolean withCoprocessor, + long nonceGroup, long nonce); /** * Return an iterator that scans over the HRegion, returning the indicated @@ -415,7 +210,7 @@ public interface Region extends ConfigurationObserver { * @return RegionScanner * @throws IOException read exceptions */ - RegionScanner getScanner(Scan scan) throws IOException; + CompletableFuture getScanner(RegionOperationContext context, Scan scan); /** * Return an iterator that scans over the HRegion, returning the indicated columns and rows @@ -425,25 +220,26 @@ public interface Region extends ConfigurationObserver { *

* This Iterator must be closed by the caller. * - * @param scan configured {@link Scan} + * @param scan configured {@link Scan} * @param additionalScanners Any additional scanners to be used * @return RegionScanner * @throws IOException read exceptions */ - RegionScanner getScanner(Scan scan, List additionalScanners) throws IOException; - - /** The comparator to be used with the region */ - CellComparator getCellCompartor(); + CompletableFuture getScanner(RegionOperationContext context, Scan scan, + List additionalScanners); /** * 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; + CompletableFuture increment(RegionOperationContext context, Increment increment, + long nonceGroup, + long nonce); /** * Performs multiple mutations atomically on a single row. Currently @@ -452,269 +248,65 @@ public interface Region extends ConfigurationObserver { * @param mutations object that specifies the set of mutations to perform atomically * @throws IOException */ - void mutateRow(RowMutations mutations) throws IOException; + CompletableFuture mutateRow(RegionOperationContext context, RowMutations mutations); /** * 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 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. + * @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; + CompletableFuture mutateRowsWithLocks(RegionOperationContext context, + Collection mutations, + Collection rowsToLock, long nonceGroup, long nonce); /** * 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; + CompletableFuture processRowsWithLocks(RegionOperationContext context, + RowProcessor processor); /** * Performs atomic multiple reads and writes on a given row. * - * @param processor The object defines the reads and writes to a 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") + * @param nonce Optional nonce of the operation (unique random id to ensure "more + * idempotence") */ - void processRowsWithLocks(RowProcessor processor, long nonceGroup, long nonce) - throws IOException; + CompletableFuture processRowsWithLocks(RegionOperationContext context, + RowProcessor processor, + long nonceGroup, long 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 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") + * @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; + CompletableFuture processRowsWithLocks(RegionOperationContext context, + RowProcessor processor, + long timeout, long nonceGroup, long nonce); /** * 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<byte[] column family, String hfilePath> - * @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 org.apache.hadoop.hbase.HConstants#LATEST_TIMESTAMP} - * 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 - * 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()} + * @param put * @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(); + CompletableFuture put(RegionOperationContext context, Put put); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionBase.java new file mode 100644 index 0000000..e46093c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionBase.java @@ -0,0 +1,487 @@ +/* + * 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 com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.Message; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; + +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.CellComparator; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; +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.Delete; +import org.apache.hadoop.hbase.client.IsolationLevel; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.conf.ConfigurationObserver; +import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.util.Pair; + +/** + * Base interface for Region and BlockingRegion, holding common methods. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface RegionBase 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; no edit in memory will have + * a sequence id that is less that what is returned here. + */ + long getMaxFlushedSeqId(); + + /** @return the oldest flushed sequence id for the given family; can be beyond + * {@link #getMaxFlushedSeqId()} in case where we've flushed a subset of a regions column + * families + * @deprecated Since version 1.2.0. Exposes too much about our internals; shutting it down. + * Do not use. + */ + @VisibleForTesting + @Deprecated + 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; pass null for default*/ + long getReadPoint(IsolationLevel isolationLevel); + + /** + * @return readpoint considering given IsolationLevel + * @deprecated Since 1.2.0. Use {@link #getReadPoint(IsolationLevel)} instead. + */ + @Deprecated + 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 filtered read requests count for this region */ + long getFilteredReadRequestsCount(); + + /** @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 store services for this region, to access services required by store level needs */ + RegionServicesForStores getRegionServicesForStores(); + + /** @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 BlockingRegion#startRegionOperation} and elsewhere to provide + * context for various checks. + */ + 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(); + } + + /** The comparator to be used with the region */ + CellComparator getCellCompartor(); + + /** + * If the given list of row locks is not null, releases all locks. + */ + void releaseRowLocks(List rowLocks); + + + /////////////////////////////////////////////////////////////////////////// + // 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 + * 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. + */ + AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState(); + + /** Wait for all current flushes and compactions of the region to complete */ + void waitForFlushesAndCompactions(); + + + + /** + * 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<byte[] column family, String hfilePath> + * @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 BlockingRegion#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 BlockingRegion#registerService(com.google.protobuf.Service) + */ + Message execService(RpcController controller, ClientProtos.CoprocessorServiceCall call) throws IOException; + + /** + * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to + * be available for handling + * {@link BlockingRegion#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 org.apache.hadoop.hbase.HConstants#LATEST_TIMESTAMP} + * provided current timestamp. + * @param values + * @param now + */ + void updateCellTimestamps(final Iterable> values, final byte[] now) + throws IOException; + +} 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 567664e..0181920 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 @@ -75,7 +75,7 @@ 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.ipc.RpcServer; -import org.apache.hadoop.hbase.regionserver.Region.Operation; +import org.apache.hadoop.hbase.regionserver.RegionBase.Operation; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; @@ -88,7 +88,7 @@ import org.apache.hadoop.hbase.util.Pair; /** * Implements the coprocessor environment and runtime support for coprocessors - * loaded within a {@link Region}. + * loaded within a {@link BlockingRegion}. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving @@ -110,7 +110,7 @@ public class RegionCoprocessorHost static class RegionEnvironment extends CoprocessorHost.Environment implements RegionCoprocessorEnvironment { - private Region region; + private BlockingRegion region; private RegionServerServices rsServices; ConcurrentMap sharedData; private final boolean useLegacyPre; @@ -122,7 +122,7 @@ public class RegionCoprocessorHost * @param priority chaining priority */ public RegionEnvironment(final Coprocessor impl, final int priority, - final int seq, final Configuration conf, final Region region, + final int seq, final Configuration conf, final BlockingRegion region, final RegionServerServices services, final ConcurrentMap sharedData) { super(impl, priority, seq, conf); this.region = region; @@ -140,7 +140,7 @@ public class RegionCoprocessorHost /** @return the region */ @Override - public Region getRegion() { + public BlockingRegion getRegion() { return region; } @@ -200,7 +200,7 @@ public class RegionCoprocessorHost /** The region server services */ RegionServerServices rsServices; /** The region */ - Region region; + BlockingRegion region; /** * Constructor @@ -208,7 +208,7 @@ public class RegionCoprocessorHost * @param rsServices interface to available region server functionality * @param conf the configuration */ - public RegionCoprocessorHost(final Region region, + public RegionCoprocessorHost(final BlockingRegion region, final RegionServerServices rsServices, final Configuration conf) { super(rsServices); this.conf = conf; @@ -738,7 +738,7 @@ public class RegionCoprocessorHost * @param r the new right-hand daughter region * @throws IOException */ - public void postSplit(final Region l, final Region r, final User user) throws IOException { + public void postSplit(final BlockingRegion l, final BlockingRegion r, final User user) throws IOException { execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) { @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 ce69ad3..3fe458a 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 @@ -46,7 +46,7 @@ class RegionMergeRequest implements Runnable { private final long masterSystemTime; private final User user; - RegionMergeRequest(Region a, Region b, HRegionServer hrs, boolean forcible, + RegionMergeRequest(BlockingRegion a, BlockingRegion b, HRegionServer hrs, boolean forcible, long masterSystemTime, User user) { Preconditions.checkNotNull(hrs); this.region_a = (HRegion)a; 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 4a3f52f..718b33e 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 @@ -174,7 +174,7 @@ public interface RegionMergeTransaction { * @deprecated use #execute(Server, RegionServerServices, User) */ @Deprecated - Region execute(Server server, RegionServerServices services) throws IOException; + BlockingRegion execute(Server server, RegionServerServices services) throws IOException; /** * Run the transaction. @@ -187,7 +187,7 @@ public interface RegionMergeTransaction { * @throws IOException * @see #rollback(Server, RegionServerServices, User) */ - Region execute(Server server, RegionServerServices services, User user) throws IOException; + BlockingRegion execute(Server server, RegionServerServices services, User user) throws IOException; /** * Roll back a failed transaction diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionFactory.java index c844d54..b91dc4d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionFactory.java @@ -58,7 +58,7 @@ public class RegionMergeTransactionFactory implements Configurable { * @param forcible if false, we will only merge adjacent regions * @return transaction instance */ - public RegionMergeTransactionImpl create(final Region a, final Region b, + public RegionMergeTransactionImpl create(final BlockingRegion a, final BlockingRegion b, final boolean forcible) { // The implementation class must extend RegionMergeTransactionImpl, not only // implement the RegionMergeTransaction interface like you might expect, @@ -69,7 +69,7 @@ public class RegionMergeTransactionFactory implements Configurable { return ReflectionUtils.instantiateWithCustomCtor( conf.getClass(MERGE_TRANSACTION_IMPL_KEY, RegionMergeTransactionImpl.class, RegionMergeTransactionImpl.class).getName(), - new Class[] { Region.class, Region.class, boolean.class }, + new Class[] { BlockingRegion.class, BlockingRegion.class, boolean.class }, new Object[] { a, b, forcible }); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java index 9e7f97b..bca9906 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransactionImpl.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import java.io.InterruptedIOException; -import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; import java.util.ListIterator; @@ -125,7 +123,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction { * @param b region b to merge * @param forcible if false, we will only merge adjacent regions */ - public RegionMergeTransactionImpl(final Region a, final Region b, + public RegionMergeTransactionImpl(final BlockingRegion a, final BlockingRegion b, final boolean forcible) { this(a, b, forcible, EnvironmentEdgeManager.currentTime()); } @@ -136,7 +134,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction { * @param forcible if false, we will only merge adjacent regions * @param masterSystemTime the time at the master side */ - public RegionMergeTransactionImpl(final Region a, final Region b, + public RegionMergeTransactionImpl(final BlockingRegion a, final BlockingRegion b, final boolean forcible, long masterSystemTime) { if (a.getRegionInfo().compareTo(b.getRegionInfo()) <= 0) { this.region_a = (HRegion)a; @@ -228,7 +226,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction { } @Override - public Region execute(final Server server, final RegionServerServices services) + public BlockingRegion execute(final Server server, final RegionServerServices services) throws IOException { if (User.isHBaseSecurityEnabled(region_a.getBaseConf())) { LOG.warn("Should use execute(Server, RegionServerServices, User)"); @@ -237,7 +235,7 @@ public class RegionMergeTransactionImpl implements RegionMergeTransaction { } @Override - public Region execute(final Server server, final RegionServerServices services, User user) + public BlockingRegion execute(final Server server, final RegionServerServices services, User user) throws IOException { this.server = server; this.rsServices = services; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOperationContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOperationContext.java new file mode 100644 index 0000000..a3dfef1 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOperationContext.java @@ -0,0 +1,44 @@ +/* + * 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 org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * In each operation of Region, we pass a context object with information of the request. + * We can pass deadline of this request to Region implementation to drop timeout request and + * not waste time on timed out requests. + * We can add listeners to watch the event of completion/failure of this operation, which helps us + * make operations of Region synchronous, see HBASE-16583 for details. + * The context is RPC-free, don't add RPC related code. + */ +@InterfaceAudience.Private +public class RegionOperationContext { + + private long deadline = Long.MAX_VALUE; + + public long getDeadline() { + return deadline; + } + + public void setDeadline(long deadline) { + this.deadline = deadline; + } + + +} 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 bfd0431..a1ff4f8 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 @@ -95,15 +95,15 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi * Context for postOpenDeployTasks(). */ class PostOpenDeployContext { - private final Region region; + private final BlockingRegion region; private final long masterSystemTime; @InterfaceAudience.Private - public PostOpenDeployContext(Region region, long masterSystemTime) { + public PostOpenDeployContext(BlockingRegion region, long masterSystemTime) { this.region = region; this.masterSystemTime = masterSystemTime; } - public Region getRegion() { + public BlockingRegion getRegion() { return region; } public long getMasterSystemTime() { @@ -131,7 +131,7 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi * @deprecated use {@link #postOpenDeployTasks(PostOpenDeployContext)} */ @Deprecated - void postOpenDeployTasks(final Region r) throws KeeperException, IOException; + void postOpenDeployTasks(final BlockingRegion r) throws KeeperException, IOException; class RegionStateTransitionContext { private final TransitionCode code; @@ -209,7 +209,7 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi /** * @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/RowProcessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java index 34901b7..598a60f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RowProcessor.java @@ -35,7 +35,7 @@ import com.google.protobuf.Message; * Defines the procedure to atomically perform multiple scans and mutations * on a HRegion. * - * This is invoked by {@link Region#processRowsWithLocks(RowProcessor)}. + * This is invoked by {@link BlockingRegion#processRowsWithLocks(RowProcessor)}. * This class performs scans and generates mutations and WAL edits. * The locks and MVCC will be handled by HRegion. * diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java index 9f53ac5..830770e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CleanupBulkLoadRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.PrepareBulkLoadRequest; -import org.apache.hadoop.hbase.regionserver.Region.BulkLoadListener; +import org.apache.hadoop.hbase.regionserver.RegionBase.BulkLoadListener; import org.apache.hadoop.hbase.security.SecureBulkLoadUtil; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; @@ -145,7 +145,7 @@ public class SecureBulkLoadManager { public void stop() throws IOException { } - public String prepareBulkLoad(final Region region, final PrepareBulkLoadRequest request) + public String prepareBulkLoad(final BlockingRegion region, final PrepareBulkLoadRequest request) throws IOException { List bulkLoadObservers = getBulkLoadObservers(region); @@ -167,7 +167,7 @@ public class SecureBulkLoadManager { return bulkToken; } - public void cleanupBulkLoad(final Region region, final CleanupBulkLoadRequest request) + public void cleanupBulkLoad(final BlockingRegion region, final CleanupBulkLoadRequest request) throws IOException { List bulkLoadObservers = getBulkLoadObservers(region); @@ -185,7 +185,7 @@ public class SecureBulkLoadManager { fs.delete(new Path(request.getBulkToken()), true); } - public boolean secureBulkLoadHFiles(final Region region, + public boolean secureBulkLoadHFiles(final BlockingRegion region, final BulkLoadHFileRequest request) throws IOException { final List> familyPaths = new ArrayList>(request.getFamilyPathCount()); for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) { @@ -261,7 +261,7 @@ public class SecureBulkLoadManager { return loaded; } - private List getBulkLoadObservers(Region region) { + private List getBulkLoadObservers(BlockingRegion region) { List coprocessorList = region.getCoprocessorHost().findCoprocessors(BulkLoadObserver.class); 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 91a5f37..7a025a5 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 @@ -45,7 +45,7 @@ class SplitRequest implements Runnable { private final User user; private TableLock tableLock; - SplitRequest(Region region, byte[] midKey, HRegionServer hrs, User user) { + SplitRequest(BlockingRegion region, byte[] midKey, HRegionServer hrs, User user) { Preconditions.checkNotNull(hrs); this.parent = (HRegion)region; this.midKey = midKey; 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 d92bf07..2403a46 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 @@ -195,7 +195,7 @@ public interface SplitTransaction { * @deprecated use #execute(Server, RegionServerServices, User); as of 1.0.2, remove in 3.0 */ @Deprecated - PairOfSameType execute(Server server, RegionServerServices services) throws IOException; + PairOfSameType execute(Server server, RegionServerServices services) throws IOException; /** * Run the transaction. @@ -208,7 +208,7 @@ public interface SplitTransaction { * @throws IOException * @see #rollback(Server, RegionServerServices) */ - PairOfSameType execute(Server server, RegionServerServices services, User user) + PairOfSameType execute(Server server, RegionServerServices services, User user) throws IOException; /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransactionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransactionFactory.java index 7df8233..d3b75cc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransactionFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransactionFactory.java @@ -57,7 +57,7 @@ public class SplitTransactionFactory implements Configurable { * @param splitrow the split point in the keyspace * @return transaction instance */ - public SplitTransaction create(final Region r, final byte [] splitrow) { + public SplitTransaction create(final BlockingRegion r, final byte [] splitrow) { return ReflectionUtils.instantiateWithCustomCtor( // The implementation class must extend SplitTransactionImpl, not only // implement the SplitTransaction interface like you might expect, @@ -67,7 +67,7 @@ public class SplitTransactionFactory implements Configurable { // If not compatible we will throw a runtime exception from here. conf.getClass(SPLIT_TRANSACTION_IMPL_KEY, SplitTransactionImpl.class, SplitTransactionImpl.class).getName(), - new Class[] { Region.class, byte[].class }, + new Class[] { BlockingRegion.class, byte[].class }, new Object[] { r, splitrow }); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransactionImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransactionImpl.java index 21c117f..d3039a3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransactionImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransactionImpl.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.io.InterruptedIOException; -import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.List; import java.util.ListIterator; @@ -130,7 +129,7 @@ public class SplitTransactionImpl implements SplitTransaction { * @param r Region to split * @param splitrow Row to split around */ - public SplitTransactionImpl(final Region r, final byte [] splitrow) { + public SplitTransactionImpl(final BlockingRegion r, final byte [] splitrow) { this.parent = (HRegion)r; this.splitrow = splitrow; this.journal.add(new JournalEntryImpl(SplitTransactionPhase.STARTED)); @@ -214,7 +213,7 @@ public class SplitTransactionImpl implements SplitTransaction { * @return Regions created */ @VisibleForTesting - PairOfSameType createDaughters(final Server server, + PairOfSameType createDaughters(final Server server, final RegionServerServices services, User user) throws IOException { LOG.info("Starting split of region " + this.parent); if ((server != null && server.isStopped()) || @@ -242,7 +241,7 @@ public class SplitTransactionImpl implements SplitTransaction { server.getConfiguration().getLong("hbase.regionserver.fileSplitTimeout", this.fileSplitTimeout); - PairOfSameType daughterRegions = stepsBeforePONR(server, services, testing); + PairOfSameType daughterRegions = stepsBeforePONR(server, services, testing); final List metaEntries = new ArrayList(); boolean ret = false; @@ -306,7 +305,7 @@ public class SplitTransactionImpl implements SplitTransaction { } @VisibleForTesting - 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)) { @@ -379,7 +378,7 @@ public class SplitTransactionImpl implements SplitTransaction { assertReferenceFileCount(expectedReferences.getSecond(), new Path(this.parent.getRegionFileSystem().getTableDir(), this.hri_b.getEncodedName())); - return new PairOfSameType(a, b); + return new PairOfSameType(a, b); } @VisibleForTesting @@ -402,8 +401,8 @@ public class SplitTransactionImpl implements SplitTransaction { * Call {@link #rollback(Server, RegionServerServices)} */ @VisibleForTesting - void openDaughters(final Server server, final RegionServerServices services, Region a, - Region b) throws IOException { + void openDaughters(final Server server, final RegionServerServices services, BlockingRegion a, + BlockingRegion b) throws IOException { boolean stopped = server != null && server.isStopped(); boolean stopping = services != null && services.isStopping(); // TODO: Is this check needed here? @@ -453,7 +452,7 @@ public class SplitTransactionImpl implements SplitTransaction { } @Override - public PairOfSameType execute(final Server server, + public PairOfSameType execute(final Server server, final RegionServerServices services) throws IOException { if (User.isHBaseSecurityEnabled(parent.getBaseConf())) { @@ -463,11 +462,11 @@ public class SplitTransactionImpl implements SplitTransaction { } @Override - public PairOfSameType execute(final Server server, final RegionServerServices services, + public PairOfSameType execute(final Server server, final RegionServerServices services, User user) throws IOException { this.server = server; this.rsServices = services; - PairOfSameType regions = createDaughters(server, services, user); + PairOfSameType regions = createDaughters(server, services, user); stepsAfterPONR(server, services, regions, user); transition(SplitTransactionPhase.COMPLETED); return regions; @@ -475,7 +474,7 @@ public class SplitTransactionImpl implements SplitTransaction { @VisibleForTesting void stepsAfterPONR(final Server server, - final RegionServerServices services, final PairOfSameType regions, User user) + final RegionServerServices services, final PairOfSameType regions, User user) throws IOException { if (this.parent.getCoprocessorHost() != null) { parent.getCoprocessorHost().preSplitAfterPONR(user); @@ -499,10 +498,10 @@ public class SplitTransactionImpl implements SplitTransaction { */ private class DaughterOpener extends HasThread { private final Server server; - private final Region r; + private final BlockingRegion r; private Throwable t = null; - DaughterOpener(final Server s, final Region r) { + DaughterOpener(final Server s, final BlockingRegion r) { super((s == null? "null-services": s.getServerName()) + "-daughterOpener=" + r.getRegionInfo().getEncodedName()); setDaemon(true); @@ -536,7 +535,7 @@ public class SplitTransactionImpl implements SplitTransaction { * @throws KeeperException */ @VisibleForTesting - void openDaughterRegion(final Server server, final Region daughter) + void openDaughterRegion(final Server server, final BlockingRegion daughter) throws IOException, KeeperException { HRegionInfo hri = daughter.getRegionInfo(); LoggingProgressable reporter = server == null ? null 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 a2a0dcc..955978f 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,7 +84,7 @@ public class StorefileRefresherChore extends ScheduledChore { @Override protected void chore() { - for (Region r : regionServer.getOnlineRegionsLocalContext()) { + for (BlockingRegion r : regionServer.getOnlineRegionsLocalContext()) { if (!r.isReadOnly()) { // skip checking for this region if it can accept writes continue; 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 19838d3..596d40c 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,7 +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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerServices; public class FinishRegionRecoveringHandler extends EventHandler { @@ -46,7 +46,7 @@ public class FinishRegionRecoveringHandler extends EventHandler { @Override public void process() throws IOException { - Region region = this.rss.getRecoveringRegions().remove(regionName); + BlockingRegion region = this.rss.getRecoveringRegions().remove(regionName); if (region != null) { ((HRegion)region).setRecovering(false); LOG.info(path + " deleted; " + regionName + " recovered."); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java index 87f6161..c4bd560 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerServices.PostOpenDeployContext; @@ -220,7 +220,7 @@ public class OpenRegionHandler extends EventHandler { /** * Thread to run region post open tasks. Call {@link #getException()} after the thread finishes - * to check for exceptions running {@link RegionServerServices#postOpenDeployTasks(Region)}. + * to check for exceptions running {@link RegionServerServices#postOpenDeployTasks(BlockingRegion)}. */ static class PostOpenDeployTasksThread extends Thread { private Throwable exception = null; 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 39735af..cb9f7bf 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,7 +30,7 @@ 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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager.SnapshotSubprocedurePool; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; @@ -46,14 +46,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; @@ -69,8 +69,8 @@ public class FlushSnapshotSubprocedure extends Subprocedure { * Callable for adding files to snapshot manifest working dir. Ready for multithreading. */ private class RegionSnapshotTask implements Callable { - Region region; - RegionSnapshotTask(Region region) { + BlockingRegion region; + RegionSnapshotTask(BlockingRegion region) { this.region = region; } @@ -127,7 +127,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure { } // Add all hfiles already existing in region. - for (Region region : regions) { + for (BlockingRegion 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 537329a..f6771f1 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 @@ -52,7 +52,7 @@ 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.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -162,7 +162,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) { @@ -222,12 +222,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()) { - Region r = iterator.next(); + BlockingRegion 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 75b3c5e..d341257 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 @@ -64,7 +64,7 @@ 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.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -326,7 +326,7 @@ public class AccessControlLists { * Returns {@code true} if the given region is part of the {@code _acl_} * metadata table. */ - static boolean isAclRegion(Region region) { + static boolean isAclRegion(BlockingRegion region) { return ACL_TABLE_NAME.equals(region.getTableDesc().getTableName()); } @@ -345,7 +345,7 @@ public class AccessControlLists { * @return a map of the permissions for this table. * @throws IOException */ - static Map> loadAll(Region aclRegion) + static Map> loadAll(BlockingRegion 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 ff27b41..8c22f97 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 @@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.ProcedureInfo; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; @@ -100,7 +99,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.ScannerContext; @@ -225,7 +224,7 @@ public class AccessController extends BaseMasterAndRegionObserver (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS); } - public Region getRegion() { + public BlockingRegion getRegion() { return regionEnv != null ? regionEnv.getRegion() : null; } @@ -234,7 +233,7 @@ public class AccessController extends BaseMasterAndRegionObserver } void initialize(RegionCoprocessorEnvironment e) throws IOException { - final Region region = e.getRegion(); + final BlockingRegion region = e.getRegion(); Configuration conf = e.getConfiguration(); Map> tables = AccessControlLists.loadAll(region); @@ -1437,7 +1436,7 @@ public class AccessController extends BaseMasterAndRegionObserver public void preOpen(ObserverContext c) throws IOException { RegionCoprocessorEnvironment env = c.getEnvironment(); - final Region region = env.getRegion(); + final BlockingRegion region = env.getRegion(); if (region == null) { LOG.error("NULL region from RegionCoprocessorEnvironment in preOpen()"); } else { @@ -1453,7 +1452,7 @@ public class AccessController extends BaseMasterAndRegionObserver @Override public void postOpen(ObserverContext c) { RegionCoprocessorEnvironment env = c.getEnvironment(); - final Region region = env.getRegion(); + final BlockingRegion region = env.getRegion(); if (region == null) { LOG.error("NULL region from RegionCoprocessorEnvironment in postOpen()"); return; @@ -1538,7 +1537,7 @@ public class AccessController extends BaseMasterAndRegionObserver throw new RuntimeException("Unhandled operation " + opType); } AuthResult authResult = permissionGranted(opType, user, env, families, Action.READ); - Region region = getRegion(env); + BlockingRegion region = getRegion(env); TableName table = getTableName(region); Map cfVsMaxVersions = Maps.newHashMap(); for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) { @@ -2452,19 +2451,19 @@ public class AccessController extends BaseMasterAndRegionObserver return AccessControlProtos.AccessControlService.newReflectiveService(this); } - private Region getRegion(RegionCoprocessorEnvironment e) { + private BlockingRegion getRegion(RegionCoprocessorEnvironment e) { return e.getRegion(); } private TableName getTableName(RegionCoprocessorEnvironment e) { - Region region = e.getRegion(); + BlockingRegion region = e.getRegion(); if (region != null) { return getTableName(region); } return null; } - private TableName getTableName(Region region) { + private TableName getTableName(BlockingRegion region) { HRegionInfo regionInfo = region.getRegionInfo(); if (regionInfo != null) { return regionInfo.getTable(); @@ -2573,31 +2572,31 @@ public class AccessController extends BaseMasterAndRegionObserver } @Override - public void preMerge(ObserverContext ctx, Region regionA, - Region regionB) throws IOException { + public void preMerge(ObserverContext ctx, BlockingRegion regionA, + BlockingRegion regionB) throws IOException { requirePermission(getActiveUser(ctx), "mergeRegions", regionA.getTableDesc().getTableName(), null, null, Action.ADMIN); } @Override - public void postMerge(ObserverContext c, Region regionA, - Region regionB, Region mergedRegion) throws IOException { } + public void postMerge(ObserverContext c, BlockingRegion regionA, + BlockingRegion regionB, BlockingRegion mergedRegion) throws IOException { } @Override public void preMergeCommit(ObserverContext ctx, - Region regionA, Region regionB, List metaEntries) throws IOException { } + BlockingRegion regionA, BlockingRegion regionB, List metaEntries) throws IOException { } @Override public void postMergeCommit(ObserverContext ctx, - Region regionA, Region regionB, Region mergedRegion) throws IOException { } + BlockingRegion regionA, BlockingRegion regionB, BlockingRegion mergedRegion) throws IOException { } @Override public void preRollBackMerge(ObserverContext ctx, - Region regionA, Region regionB) throws IOException { } + BlockingRegion regionA, BlockingRegion regionB) throws IOException { } @Override public void postRollBackMerge(ObserverContext ctx, - Region regionA, Region regionB) throws IOException { } + BlockingRegion regionA, BlockingRegion regionB) throws IOException { } @Override public void preRollWALWriterRequest(ObserverContext ctx) 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 f1aec09..ac19cff 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 @@ -60,7 +60,7 @@ 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.OperationStatus; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.User; @@ -80,7 +80,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService private AtomicInteger ordinalCounter = new AtomicInteger(-1); private Configuration conf; - private Region labelsRegion; + private BlockingRegion labelsRegion; private VisibilityLabelsCache labelsCache; private List scanLabelGenerators; @@ -193,7 +193,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService return new Pair, Map>>(labels, userAuths); } - protected void addSystemLabel(Region region, Map labels, + protected void addSystemLabel(BlockingRegion region, Map labels, Map> userAuths) throws IOException { if (!labels.containsKey(SYSTEM_LABEL)) { Put p = new Put(Bytes.toBytes(SYSTEM_LABEL_ORDINAL)); 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 82fed73..fbe71ee 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 @@ -94,7 +94,7 @@ import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; 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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; @@ -556,7 +556,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements if (!authorizationEnabled) { return s; } - Region region = e.getEnvironment().getRegion(); + BlockingRegion region = e.getEnvironment().getRegion(); Authorizations authorizations = null; try { authorizations = scan.getAuthorizations(); @@ -594,7 +594,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements if (!authorizationEnabled) { return delTracker; } - Region region = ctx.getEnvironment().getRegion(); + BlockingRegion region = ctx.getEnvironment().getRegion(); TableName table = region.getRegionInfo().getTable(); if (table.isSystemTable()) { return delTracker; @@ -662,7 +662,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements if (!authorizationEnabled) { return; } - Region region = e.getEnvironment().getRegion(); + BlockingRegion 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 1db506d..e495098 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 @@ -52,7 +52,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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode; @@ -264,7 +264,7 @@ public class VisibilityUtils { return false; } - public static Filter createVisibilityLabelFilter(Region region, Authorizations authorizations) + public static Filter createVisibilityLabelFilter(BlockingRegion region, Authorizations authorizations) throws IOException { Map cfVsMaxVersions = new HashMap(); for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) { diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp index 874ac43..29d29f3 100644 --- a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp @@ -27,8 +27,8 @@ import="org.apache.hadoop.hbase.HColumnDescriptor" import="org.apache.hadoop.hbase.HBaseConfiguration" import="org.apache.hadoop.hbase.HRegionInfo" + import="org.apache.hadoop.hbase.regionserver.BlockingRegion" 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.regionserver.StoreFile"%> <% @@ -36,7 +36,7 @@ HRegionServer rs = (HRegionServer) getServletContext().getAttribute(HRegionServer.REGIONSERVER); Configuration conf = rs.getConfiguration(); - Region region = rs.getFromOnlineRegions(regionName); + BlockingRegion region = rs.getFromOnlineRegions(regionName); String displayName = HRegionInfo.getRegionNameAsStringForDisplay(region.getRegionInfo(), rs.getConfiguration()); %> 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 153f36b..d654fc9 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 @@ -32,7 +32,7 @@ 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.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionAsTable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; @@ -225,7 +225,7 @@ public abstract class HBaseTestCase extends TestCase { * @throws IOException * @return count of what we added. */ - public static long addContent(final Region r, final byte [] columnFamily, final byte[] column) + public static long addContent(final BlockingRegion r, final byte [] columnFamily, final byte[] column) throws IOException { byte [] startKey = r.getRegionInfo().getStartKey(); byte [] endKey = r.getRegionInfo().getEndKey(); @@ -237,7 +237,7 @@ public abstract class HBaseTestCase extends TestCase { startKeyBytes, endKey, -1); } - public static long addContent(final Region r, final byte [] columnFamily) throws IOException { + public static long addContent(final BlockingRegion r, final byte [] columnFamily) throws IOException { return addContent(r, columnFamily, null); } 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 c164091..8776347 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 @@ -96,7 +96,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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; @@ -357,7 +357,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Close both the region {@code r} and it's underlying WAL. For use in tests. */ - public static void closeRegionAndWAL(final Region r) throws IOException { + public static void closeRegionAndWAL(final BlockingRegion r) throws IOException { closeRegionAndWAL((HRegion)r); } @@ -2058,7 +2058,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return loadRegion(r, f, false); } - public int loadRegion(final Region r, final byte[] f) throws IOException { + public int loadRegion(final BlockingRegion r, final byte[] f) throws IOException { return loadRegion((HRegion)r, f); } @@ -2136,7 +2136,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } } - public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow) + public void verifyNumericRows(BlockingRegion region, final byte[] f, int startRow, int endRow) throws IOException { verifyNumericRows((HRegion)region, f, startRow, endRow); } @@ -2146,7 +2146,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { verifyNumericRows(region, f, startRow, endRow, true); } - public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow, + public void verifyNumericRows(BlockingRegion region, final byte[] f, int startRow, int endRow, final boolean present) throws IOException { verifyNumericRows((HRegion)region, f, startRow, endRow, present); } @@ -3789,8 +3789,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { if (server.equals(rs.getServerName())) { continue; } - Collection hrs = rs.getOnlineRegionsLocalContext(); - for (Region r: hrs) { + Collection hrs = rs.getOnlineRegionsLocalContext(); + for (BlockingRegion r: hrs) { assertTrue("Region should not be double assigned", r.getRegionInfo().getRegionId() != hri.getRegionId()); } @@ -4026,7 +4026,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return supportedAlgos.toArray(new Algorithm[supportedAlgos.size()]); } - public Result getClosestRowBefore(Region r, byte[] row, byte[] family) throws IOException { + public Result getClosestRowBefore(BlockingRegion r, byte[] row, byte[] family) throws IOException { Scan scan = new Scan(row); scan.setSmall(true); scan.setCaching(1); 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 706d8e7..85fe671 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,7 +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.regionserver.BlockingRegion; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.util.JVMClusterUtil; @@ -588,7 +588,7 @@ public class MiniHBaseCluster extends HBaseCluster { public void flushcache() throws IOException { for (JVMClusterUtil.RegionServerThread t: this.hbaseCluster.getRegionServers()) { - for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) { + for(BlockingRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { r.flush(true); } } @@ -601,7 +601,7 @@ public class MiniHBaseCluster extends HBaseCluster { public void flushcache(TableName tableName) throws IOException { for (JVMClusterUtil.RegionServerThread t: this.hbaseCluster.getRegionServers()) { - for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) { + for(BlockingRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { if(r.getTableDesc().getTableName().equals(tableName)) { r.flush(true); } @@ -616,7 +616,7 @@ public class MiniHBaseCluster extends HBaseCluster { public void compact(boolean major) throws IOException { for (JVMClusterUtil.RegionServerThread t: this.hbaseCluster.getRegionServers()) { - for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) { + for(BlockingRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { r.compact(major); } } @@ -629,7 +629,7 @@ public class MiniHBaseCluster extends HBaseCluster { public void compact(TableName tableName, boolean major) throws IOException { for (JVMClusterUtil.RegionServerThread t: this.hbaseCluster.getRegionServers()) { - for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) { + for(BlockingRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { if(r.getTableDesc().getTableName().equals(tableName)) { r.compact(major); } @@ -668,7 +668,7 @@ public class MiniHBaseCluster extends HBaseCluster { List ret = new ArrayList(); for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) { HRegionServer hrs = rst.getRegionServer(); - for (Region region : hrs.getOnlineRegionsLocalContext()) { + for (BlockingRegion region : hrs.getOnlineRegionsLocalContext()) { if (region.getTableDesc().getTableName().equals(tableName)) { ret.add((HRegion)region); } @@ -697,7 +697,7 @@ public class MiniHBaseCluster extends HBaseCluster { for (JVMClusterUtil.RegionServerThread rst: getRegionServerThreads()) { HRegionServer hrs = rst.getRegionServer(); if (!hrs.isStopped()) { - Region region = hrs.getOnlineRegion(regionName); + BlockingRegion region = hrs.getOnlineRegion(regionName); if (region != null) { index = count; break; @@ -716,7 +716,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(); - Region region = master.getOnlineRegion(regionName); + BlockingRegion region = master.getOnlineRegion(regionName); if (region != null) { return master.getServerName(); } @@ -766,7 +766,7 @@ public class MiniHBaseCluster extends HBaseCluster { ArrayList ret = new ArrayList(); for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) { HRegionServer hrs = rst.getRegionServer(); - for (Region region : hrs.getOnlineRegions(tableName)) { + for (BlockingRegion region : hrs.getOnlineRegions(tableName)) { if (region.getTableDesc().getTableName().equals(tableName)) { 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 e330093..c23a8db 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 @@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.regionserver.FlushRequester; import org.apache.hadoop.hbase.regionserver.HeapMemoryManager; import org.apache.hadoop.hbase.regionserver.Leases; import org.apache.hadoop.hbase.regionserver.MetricsRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager; @@ -63,7 +63,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; @@ -94,17 +94,17 @@ public class MockRegionServerServices implements RegionServerServices { } @Override - public boolean removeFromOnlineRegions(Region r, ServerName destination) { + public boolean removeFromOnlineRegions(BlockingRegion r, ServerName destination) { return this.regions.remove(r.getRegionInfo().getEncodedName()) != null; } @Override - public Region getFromOnlineRegions(String encodedRegionName) { + public BlockingRegion 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; } @@ -114,17 +114,17 @@ public class MockRegionServerServices implements RegionServerServices { } @Override - public List getOnlineRegions() { + public List getOnlineRegions() { return null; } @Override - public void addToOnlineRegions(Region r) { + public void addToOnlineRegions(BlockingRegion r) { this.regions.put(r.getRegionInfo().getEncodedName(), r); } @Override - public void postOpenDeployTasks(Region r) throws KeeperException, IOException { + public void postOpenDeployTasks(BlockingRegion r) throws KeeperException, IOException { addToOnlineRegions(r); } @@ -277,7 +277,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 8ac89da..e556e59 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 @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; @@ -104,7 +104,7 @@ public class TestGlobalMemStoreSize { for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) { - Region r = server.getFromOnlineRegions(regionInfo.getEncodedName()); + BlockingRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName()); flush(r, server); } LOG.info("Post flush on " + server.getServerName()); @@ -120,7 +120,7 @@ public class TestGlobalMemStoreSize { // our test was running.... for (HRegionInfo regionInfo : ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) { - Region r = server.getFromOnlineRegions(regionInfo.getEncodedName()); + BlockingRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName()); long l = r.getMemstoreSize(); if (l > 0) { // Only meta could have edits at this stage. Give it another flush @@ -145,7 +145,7 @@ public class TestGlobalMemStoreSize { * @param server * @throws IOException */ - private void flush(final Region r, final HRegionServer server) + private void flush(final BlockingRegion r, final HRegionServer server) throws IOException { LOG.info("Flush " + r.toString() + " on " + server.getServerName() + ", " + r.flush(true) + ", size=" + 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 0217b41..1b5e42f 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 @@ -37,7 +37,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.MemStore; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; @@ -318,7 +318,7 @@ public class TestIOFencing { Waiter.waitFor(c, 60000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { - Region newRegion = newServer.getOnlineRegion(REGION_NAME); + BlockingRegion 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 e30d719..042fedb 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 @@ -43,7 +43,7 @@ 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.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; 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()); - Region region = servingRegions.get(0); + BlockingRegion 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); @@ -311,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()); - Region region = servingRegions.get(0); + BlockingRegion region = servingRegions.get(0); // get the parent RS and monitor HRegionServer hrs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); @@ -322,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); 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 64139ee..0f2354a 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 @@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -178,7 +178,7 @@ public class TestZooKeeperTableArchiveClient { // create the region HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM); HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); - List regions = new ArrayList(); + List regions = new ArrayList(); regions.add(region); when(rss.getOnlineRegions()).thenReturn(regions); final CompactedHFilesDischarger compactionCleaner = @@ -231,7 +231,7 @@ public class TestZooKeeperTableArchiveClient { // create the region HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM); HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); - List regions = new ArrayList(); + List regions = new ArrayList(); regions.add(region); when(rss.getOnlineRegions()).thenReturn(regions); final CompactedHFilesDischarger compactionCleaner = @@ -241,7 +241,7 @@ public class TestZooKeeperTableArchiveClient { // create the another table that we don't archive hcd = new HColumnDescriptor(TEST_FAM); HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd); - regions = new ArrayList(); + regions = new ArrayList(); regions.add(otherRegion); when(rss.getOnlineRegions()).thenReturn(regions); final CompactedHFilesDischarger compactionCleaner1 = new CompactedHFilesDischarger(100, stop, @@ -400,7 +400,7 @@ public class TestZooKeeperTableArchiveClient { return allFiles; } - private void loadFlushAndCompact(Region region, byte[] family) throws IOException { + private void loadFlushAndCompact(BlockingRegion region, byte[] family) throws IOException { // create two hfiles in the region createHFileInRegion(region, family); createHFileInRegion(region, family); @@ -421,7 +421,7 @@ public class TestZooKeeperTableArchiveClient { * @param columnFamily family for which to add data * @throws IOException */ - private void createHFileInRegion(Region region, byte[] columnFamily) throws IOException { + private void createHFileInRegion(BlockingRegion region, byte[] columnFamily) throws IOException { // put one row in the region Put p = new Put(Bytes.toBytes("row")); p.addColumn(columnFamily, Bytes.toBytes("Qual"), Bytes.toBytes("v1")); 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 126eaa9..98ddffd 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 @@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -543,7 +543,7 @@ public class TestAdmin2 { + AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files"); // flush all regions - for (Region r : regionServer.getOnlineRegionsLocalContext()) { + for (BlockingRegion r : regionServer.getOnlineRegionsLocalContext()) { r.flush(true); } admin.rollWALWriter(regionServer.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java index ba75d6e..8469fe9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java @@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.io.hfile.CachedBlock; import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; @@ -175,7 +175,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); @@ -266,7 +266,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); @@ -325,7 +325,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); @@ -387,7 +387,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); BlockCache cache = setCacheProperties(region); Put put = new Put(ROW); @@ -480,7 +480,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); BlockCache cache = setCacheProperties(region); @@ -564,7 +564,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); @@ -621,7 +621,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); @@ -704,7 +704,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); BlockCache cache = setCacheProperties(region); @@ -774,7 +774,7 @@ public class TestBlockEvictionFromClient { } } - private BlockCache setCacheProperties(Region region) { + private BlockCache setCacheProperties(BlockingRegion region) { Iterator strItr = region.getStores().iterator(); BlockCache cache = null; while (strItr.hasNext()) { @@ -804,7 +804,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); @@ -870,7 +870,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); @@ -989,7 +989,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); @@ -1119,7 +1119,7 @@ public class TestBlockEvictionFromClient { // get the block cache and region RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions( regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); 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 f4fd603..b05aa58 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 @@ -34,7 +34,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.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -94,7 +94,7 @@ public class TestClientPushback { Table table = conn.getTable(tableName); HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0); - Region region = rs.getOnlineRegions(tableName).get(0); + BlockingRegion region = rs.getOnlineRegions(tableName).get(0); LOG.debug("Writing some data to "+tableName); // write some data @@ -174,7 +174,7 @@ public class TestClientPushback { ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf); Table table = conn.getTable(tableName); HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0); - Region region = rs.getOnlineRegions(tableName).get(0); + BlockingRegion region = rs.getOnlineRegions(tableName).get(0); RowMutations mutations = new RowMutations(Bytes.toBytes("row")); Put p = new Put(Bytes.toBytes("row")); 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 f465625..249ec32 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 @@ -86,14 +86,13 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest; 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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -4211,7 +4210,7 @@ public class TestFromClientSide { // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow // in Store.rowAtOrBeforeFromStoreFile String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName); Put put1 = new Put(firstRow); Put put2 = new Put(secondRow); @@ -4894,7 +4893,7 @@ public class TestFromClientSide { // get the block cache and region String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); - Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName) + BlockingRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName) .getFromOnlineRegions(regionName); Store store = region.getStores().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); 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 786f570..097b8b3 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 @@ -63,7 +63,7 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.ipc.ServerTooBusyException; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -1198,9 +1198,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 (Region region : regions) { + for (BlockingRegion 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/client/TestMetaCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java index a2ce0d2..1ef3dea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.quotas.ThrottlingException; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RSRpcServices; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -249,7 +249,7 @@ public class TestMetaCache { return super.scan(controller, request); } - public Region getRegion( + public BlockingRegion getRegion( final HBaseProtos.RegionSpecifier regionSpecifier) throws IOException { return super.getRegion(regionSpecifier); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java index f6b6361..ffa10f4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -161,7 +161,7 @@ public class TestSplitOrMergeStatus { admin.majorCompact(t); mergeable = 0; for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) { - for (Region region: regionThread.getRegionServer().getOnlineRegions(t)) { + for (BlockingRegion region: regionThread.getRegionServer().getOnlineRegions(t)) { mergeable += ((HRegion)region).isMergeable() ? 1 : 0; } } 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 49733d4..c0505ff 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 @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationNullResponseSumResponse; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.util.Bytes; import com.google.protobuf.RpcCallback; @@ -87,7 +87,7 @@ implements Coprocessor, CoprocessorService { int sumResult = 0; InternalScanner scanner = null; try { - Region region = this.env.getRegion(); + BlockingRegion region = this.env.getRegion(); // for the last region in the table, return null to test null handling if (Bytes.equals(region.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW)) { done.run(null); 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 aa8ef62..d52c9af 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 @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationWithErrorsSumResponse; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.util.Bytes; import com.google.protobuf.RpcCallback; @@ -88,7 +88,7 @@ implements Coprocessor, CoprocessorService { int sumResult = 0; InternalScanner scanner = null; try { - Region region = this.env.getRegion(); + BlockingRegion region = this.env.getRegion(); // throw an exception for requests to the last region in the table, to test error handling if (Bytes.equals(region.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW)) { throw new DoNotRetryIOException("An expected exception"); 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 dfd41a8..3cbd44a 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 @@ -58,8 +58,8 @@ 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.BlockingRegion; +import org.apache.hadoop.hbase.regionserver.RegionBase.Operation; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; @@ -230,7 +230,7 @@ public class SimpleRegionObserver extends BaseRegionObserver { } @Override - public void postSplit(ObserverContext c, Region l, Region r) { + public void postSplit(ObserverContext c, BlockingRegion l, BlockingRegion 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 f5d2a20..d7069f9 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 @@ -23,7 +23,7 @@ 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.TestServerCustomProtocol; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -166,10 +166,10 @@ 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 (Region region: + for (BlockingRegion region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { if (region.getRegionInfo().getRegionNameAsString().startsWith(tableName.getNameAsString())) { foundTableRegion = true; @@ -209,7 +209,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(), @@ -238,7 +238,7 @@ public class TestClassLoading { // verify that the coprocessor was loaded boolean found = false; MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster(); - for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { + for (BlockingRegion region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { if (region.getRegionInfo().getRegionNameAsString().startsWith(cpName3)) { found = (region.getCoprocessorHost().findCoprocessor(cpName3) != null); } @@ -263,7 +263,7 @@ public class TestClassLoading { // verify that the coprocessor was loaded correctly boolean found = false; MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster(); - for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { + for (BlockingRegion region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { if (region.getRegionInfo().getRegionNameAsString().startsWith(cpName4)) { Coprocessor cp = region.getCoprocessorHost().findCoprocessor(cpName4); if (cp != null) { @@ -334,7 +334,7 @@ public class TestClassLoading { found6_k4 = false; MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster(); - for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { + for (BlockingRegion region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { if (region.getRegionInfo().getRegionNameAsString().startsWith(tableName.getNameAsString())) { found_1 = found_1 || (region.getCoprocessorHost().findCoprocessor(cpName1) != null); @@ -422,7 +422,7 @@ public class TestClassLoading { boolean found1 = false, found2 = false, found2_k1 = false, found2_k2 = false, found2_k3 = false; MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster(); - for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { + for (BlockingRegion region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) { if (region.getRegionInfo().getRegionNameAsString().startsWith(tableName.getNameAsString())) { CoprocessorEnvironment env; env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName1); 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 b2ef1bd..c81957e 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,7 +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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; @@ -228,7 +228,7 @@ public class TestCoprocessorInterface { preSplitWithSplitRowCalled = true; } @Override - public void postSplit(ObserverContext e, Region l, Region r) { + public void postSplit(ObserverContext e, BlockingRegion l, BlockingRegion r) { postSplitCalled = true; } @@ -294,7 +294,7 @@ public class TestCoprocessorInterface { byte [][] families = { fam1, fam2, fam3 }; Configuration hc = initSplit(); - Region region = initHRegion(tableName, name.getMethodName(), hc, + BlockingRegion region = initHRegion(tableName, name.getMethodName(), hc, new Class[]{}, families); for (int i = 0; i < 3; i++) { @@ -306,7 +306,7 @@ public class TestCoprocessorInterface { byte [] splitRow = ((HRegion)region).checkSplit(); assertNotNull(splitRow); - Region [] regions = split(region, splitRow); + BlockingRegion[] regions = split(region, splitRow); for (int i = 0; i < regions.length; i++) { regions[i] = reopenRegion(regions[i], CoprocessorImpl.class, CoprocessorII.class); } @@ -372,7 +372,7 @@ public class TestCoprocessorInterface { byte [][] families = { fam1, fam2, fam3 }; Configuration hc = initSplit(); - Region region = initHRegion(tableName, name.getMethodName(), hc, + BlockingRegion region = initHRegion(tableName, name.getMethodName(), hc, new Class[]{CoprocessorImpl.class}, families); for (int i = 0; i < 3; i++) { HBaseTestCase.addContent(region, fam3); @@ -384,7 +384,7 @@ public class TestCoprocessorInterface { byte [] splitRow = ((HRegion)region).checkSplit(); assertNotNull(splitRow); - Region [] regions = split(region, splitRow); + BlockingRegion[] regions = split(region, splitRow); for (int i = 0; i < regions.length; i++) { regions[i] = reopenRegion(regions[i], CoprocessorImpl.class); } @@ -419,10 +419,10 @@ public class TestCoprocessorInterface { } } - Region reopenRegion(final Region closedRegion, Class ... implClasses) + BlockingRegion reopenRegion(final BlockingRegion closedRegion, Class ... implClasses) throws IOException { //HRegionInfo info = new HRegionInfo(tableName, null, null, false); - Region r = HRegion.openHRegion(closedRegion, null); + BlockingRegion r = HRegion.openHRegion(closedRegion, null); // this following piece is a hack. currently a coprocessorHost // is secretly loaded at OpenRegionHandler. we don't really @@ -446,7 +446,7 @@ public class TestCoprocessorInterface { return r; } - Region initHRegion (TableName tableName, String callingMethod, + BlockingRegion initHRegion (TableName tableName, String callingMethod, Configuration conf, Class [] implClasses, byte [][] families) throws IOException { HTableDescriptor htd = new HTableDescriptor(tableName); @@ -455,7 +455,7 @@ public class TestCoprocessorInterface { } HRegionInfo info = new HRegionInfo(tableName, null, null, false); Path path = new Path(DIR + callingMethod); - Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); + BlockingRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); // this following piece is a hack. RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf); @@ -493,8 +493,8 @@ public class TestCoprocessorInterface { return TEST_UTIL.getConfiguration(); } - private Region [] split(final Region r, final byte [] splitRow) throws IOException { - Region[] regions = new Region[2]; + private BlockingRegion[] split(final BlockingRegion r, final byte [] splitRow) throws IOException { + BlockingRegion[] regions = new BlockingRegion[2]; SplitTransaction st = new SplitTransactionFactory(TEST_UTIL.getConfiguration()) .create(r, splitRow); @@ -507,8 +507,8 @@ public class TestCoprocessorInterface { try { Server mockServer = Mockito.mock(Server.class); when(mockServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration()); - PairOfSameType daughters = st.execute(mockServer, null); - for (Region each_daughter: daughters) { + PairOfSameType daughters = st.execute(mockServer, null); + for (BlockingRegion each_daughter: daughters) { regions[i] = each_daughter; i++; } 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 02bb0d3..7871714 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 @@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.IsolationLevel; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; @@ -50,10 +49,9 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.FilterBase; 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.KeyValueScanner; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.ScanType; @@ -144,7 +142,7 @@ public class TestRegionObserverScannerOpenHook { } } - Region initHRegion(byte[] tableName, String callingMethod, Configuration conf, + BlockingRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf, byte[]... families) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); for (byte[] family : families) { @@ -171,7 +169,7 @@ public class TestRegionObserverScannerOpenHook { byte[][] FAMILIES = new byte[][] { A }; Configuration conf = HBaseConfiguration.create(); - Region region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES); + BlockingRegion 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); @@ -196,7 +194,7 @@ public class TestRegionObserverScannerOpenHook { byte[][] FAMILIES = new byte[][] { A }; Configuration conf = HBaseConfiguration.create(); - Region region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES); + BlockingRegion 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); @@ -281,9 +279,9 @@ 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()); - Region region = regions.get(0); + BlockingRegion region = regions.get(0); admin.flushRegion(region.getRegionInfo().getRegionName()); CountDownLatch latch = ((CompactionCompletionNotifyingRegion)region) .getCompactionStateChangeLatch(); 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 ecf9da1..e2ead4b 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 @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.client.Admin; 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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionMergeTransactionFactory; import org.apache.hadoop.hbase.regionserver.RegionMergeTransactionImpl; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; @@ -91,7 +91,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.mergeRegionsAsync( regions.get(0).getRegionInfo().getEncodedNameAsBytes(), regions.get(1).getRegionInfo().getEncodedNameAsBytes(), @@ -133,18 +133,18 @@ public class TestRegionServerObserver { } @Override - public void preMerge(ObserverContext ctx, Region regionA, - Region regionB) throws IOException { + public void preMerge(ObserverContext ctx, BlockingRegion regionA, + BlockingRegion regionB) throws IOException { preMergeCalled = true; } @Override public void preMergeCommit(ObserverContext ctx, - Region regionA, Region regionB, List metaEntries) throws IOException { + BlockingRegion regionA, BlockingRegion 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 = (RegionMergeTransactionImpl) new RegionMergeTransactionFactory(rs.getConfiguration()) .create(onlineRegions.get(0), onlineRegions.get(1), true); @@ -163,7 +163,7 @@ public class TestRegionServerObserver { @Override public void postMergeCommit(ObserverContext ctx, - Region regionA, Region regionB, Region mr) throws IOException { + BlockingRegion regionA, BlockingRegion regionB, BlockingRegion mr) throws IOException { preMergeAfterPONRCalled = true; RegionServerCoprocessorEnvironment environment = ctx.getEnvironment(); HRegionServer rs = (HRegionServer) environment.getRegionServerServices(); @@ -172,19 +172,19 @@ public class TestRegionServerObserver { @Override public void preRollBackMerge(ObserverContext ctx, - Region regionA, Region regionB) throws IOException { + BlockingRegion regionA, BlockingRegion regionB) throws IOException { preRollBackMergeCalled = true; } @Override public void postRollBackMerge(ObserverContext ctx, - Region regionA, Region regionB) throws IOException { + BlockingRegion regionA, BlockingRegion regionB) throws IOException { postRollBackMergeCalled = true; } @Override - public void postMerge(ObserverContext c, Region regionA, - Region regionB, Region mergedRegion) throws IOException { + public void postMerge(ObserverContext c, BlockingRegion regionA, + BlockingRegion regionB, BlockingRegion 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 4b8da96..0d1e48a 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,7 +48,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -68,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 Region region; + private BlockingRegion region; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); // @@ -1493,7 +1493,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); - Region testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), + BlockingRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd); for(int i=0; i<5; i++) { @@ -2054,7 +2054,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); - Region testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), + BlockingRegion 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)); 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 8291e52..cfb25a2 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 @@ -34,7 +34,7 @@ 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.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -61,7 +61,7 @@ public class TestInvocationRecordFilter { private static final String VALUE_PREFIX = "value"; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private Region region; + private BlockingRegion region; @Before public void setUp() throws Exception { 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 3180c50..6d6e43f 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 @@ -46,7 +46,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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -285,7 +285,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,7 +294,7 @@ public class TestBlockReorder { latch.countDown(); } }; - for (Region region : regions) { + for (BlockingRegion region : regions) { ((HRegion)region).getWAL().registerWALActionsListener(listener); } @@ -308,7 +308,7 @@ public class TestBlockReorder { "tests fail, it's probably because we should still be waiting."); Thread.currentThread().interrupt(); } - for (Region region : regions) { + for (BlockingRegion region : regions) { ((HRegion)region).getWAL().unregisterWALActionsListener(listener); } 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 0869df6..f4daf83 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 @@ -39,7 +39,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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -114,7 +114,7 @@ public class TestEncodedSeekers { setBlocksize(BLOCK_SIZE). setBloomFilterType(BloomType.NONE). setCompressTags(compressTags); - Region region = testUtil.createTestRegion(TABLE_NAME, hcd); + BlockingRegion region = testUtil.createTestRegion(TABLE_NAME, hcd); //write the data, but leave some in the memstore doPuts(region); @@ -138,7 +138,7 @@ public class TestEncodedSeekers { } - private void doPuts(Region region) throws IOException{ + private void doPuts(BlockingRegion 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(); @@ -168,7 +168,7 @@ public class TestEncodedSeekers { } - private void doGets(Region region) throws IOException{ + private void doGets(BlockingRegion 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 e31a73b..2135254 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,7 +37,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.client.Scan; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -65,7 +65,7 @@ public class TestPrefixTree { private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); - private Region region; + private BlockingRegion region; @Before public void setUp() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java index 2826694..d638c2e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java @@ -34,7 +34,7 @@ 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.filter.FirstKeyOnlyFilter; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -48,7 +48,7 @@ import org.junit.experimental.categories.Category; public class TestSeekBeforeWithReverseScan { private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); - private Region region; + private BlockingRegion region; private byte[] cfName = Bytes.toBytes("a"); private byte[] cqName = Bytes.toBytes("b"); 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 8f9c4f7..f105893 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 @@ -54,7 +54,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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -411,7 +411,7 @@ public class TestCacheOnWrite { final String cf = "myCF"; final byte[] cfBytes = Bytes.toBytes(cf); final int maxVersions = 3; - Region region = TEST_UTIL.createTestRegion(table, + BlockingRegion region = TEST_UTIL.createTestRegion(table, new HColumnDescriptor(cf) .setCompressionType(compress) .setBloomFilterType(BLOOM_TYPE) 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 68dc625..8fd08dc 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; import org.junit.Test; @@ -109,7 +109,7 @@ public class TestForceCacheImportantBlocks { setBloomFilterType(BLOOM_TYPE); hcd.setBlocksize(BLOCK_SIZE); hcd.setBlockCacheEnabled(cfCacheEnabled); - Region region = TEST_UTIL.createTestRegion(TABLE, hcd); + BlockingRegion region = TEST_UTIL.createTestRegion(TABLE, hcd); BlockCache cache = region.getStore(hcd.getName()).getCacheConfig().getBlockCache(); CacheStats stats = cache.getStats(); writeTestData(region); @@ -126,7 +126,7 @@ public class TestForceCacheImportantBlocks { else assertTrue(stats.getMissCount() > missCount); } - private void writeTestData(Region region) throws IOException { + private void writeTestData(BlockingRegion 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) { 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 9c6bb38..c66c0b7 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 @@ -37,7 +37,7 @@ 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.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -99,7 +99,8 @@ public class TestScannerSelectionUsingKeyRange { HTableDescriptor htd = new HTableDescriptor(TABLE); htd.addFamily(hcd); HRegionInfo info = new HRegionInfo(TABLE); - Region region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf, + BlockingRegion + region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf, htd); for (int iFile = 0; iFile < NUM_FILES; ++iFile) { 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 08b259d..8e1c81a 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 @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; 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.BlockingRegion; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -106,7 +106,7 @@ public class TestScannerSelectionUsingTTL { HTableDescriptor htd = new HTableDescriptor(TABLE); htd.addFamily(hcd); HRegionInfo info = new HRegionInfo(TABLE); - Region region = HBaseTestingUtility.createRegionAndWAL(info, + BlockingRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, htd); long ts = EnvironmentEdgeManager.currentTime(); 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 3d71924..fa4a090 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 @@ -52,7 +52,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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.Tool; @@ -248,7 +248,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable { @Override public void prePut(ObserverContext e, Put put, WALEdit edit, Durability durability) throws IOException { - Region region = e.getEnvironment().getRegion(); + BlockingRegion region = e.getEnvironment().getRegion(); if (!region.getRegionInfo().isMetaTable() && !region.getRegionInfo().getTable().isSystemTable()) { if (put.getAttribute(TEST_ATR_KEY) != null) { 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 3fc129f..8a3ea61 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,7 +41,7 @@ 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.Tool; @@ -156,7 +156,7 @@ public class TestImportTSVWithTTLs implements Configurable { @Override public void prePut(ObserverContext e, Put put, WALEdit edit, Durability durability) throws IOException { - Region region = e.getEnvironment().getRegion(); + BlockingRegion 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 2e47eb7..259cf39 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 @@ -105,7 +105,7 @@ 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.MetricsRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager; @@ -264,12 +264,12 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public void addToOnlineRegions(Region r) { + public void addToOnlineRegions(BlockingRegion r) { // TODO Auto-generated method stub } @Override - public boolean removeFromOnlineRegions(Region r, ServerName destination) { + public boolean removeFromOnlineRegions(BlockingRegion r, ServerName destination) { // TODO Auto-generated method stub return false; } @@ -344,7 +344,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public void postOpenDeployTasks(Region r) throws KeeperException, IOException { + public void postOpenDeployTasks(BlockingRegion r) throws KeeperException, IOException { // TODO Auto-generated method stub } @@ -470,7 +470,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public List getOnlineRegions() { + public List getOnlineRegions() { return null; } @@ -551,7 +551,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; } @@ -607,7 +607,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 6ad2c5d..bdf8967 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 @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.client.Admin; 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.regionserver.BlockingRegion; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.junit.AfterClass; @@ -231,7 +231,7 @@ public class TestAssignmentListener { admin.majorCompact(TABLE_NAME); mergeable = 0; for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) { - for (Region region: regionThread.getRegionServer().getOnlineRegions(TABLE_NAME)) { + for (BlockingRegion 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 002438a..28cfd02 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 @@ -89,7 +89,7 @@ import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch; 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.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -101,7 +101,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; -import org.apache.hadoop.hbase.wal.FSHLogProvider; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALSplitter; @@ -1526,7 +1525,7 @@ public class TestDistributedLogSplitting { } LOG.debug("adding data to rs = " + rst.getName() + " region = "+ hri.getRegionNameAsString()); - Region region = hrs.getOnlineRegion(hri.getRegionName()); + BlockingRegion region = hrs.getOnlineRegion(hri.getRegionName()); assertTrue(region != null); putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family); } @@ -1547,7 +1546,7 @@ public class TestDistributedLogSplitting { } LOG.debug("adding data to rs = " + mt.getName() + " region = "+ hri.getRegionNameAsString()); - Region region = hrs.getOnlineRegion(hri.getRegionName()); + BlockingRegion region = hrs.getOnlineRegion(hri.getRegionName()); assertTrue(region != null); putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family); } @@ -1662,7 +1661,7 @@ public class TestDistributedLogSplitting { TEST_UTIL.waitUntilNoRegionsInTransition(60000); } - private void putData(Region region, byte[] startRow, int numRows, byte [] qf, + private void putData(BlockingRegion 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 a9fa039..d8f80c1 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 @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; 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 { .addColumn(family, Bytes.toBytes("q"), Bytes.toBytes("v"))); MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster(); List rsts = cluster.getRegionServerThreads(); - Region region = null; + BlockingRegion region = null; for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { HRegionServer hrs = rsts.get(i).getRegionServer(); - for (Region r : hrs.getOnlineRegions(tableName)) { + for (BlockingRegion 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 a20327d..5c5a378 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,7 +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.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -384,7 +384,7 @@ public class TestMasterFailover { // region server should expire (how it can be verified?) MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(), rs.getServerName(), State.PENDING_OPEN); - Region meta = rs.getFromOnlineRegions(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); + BlockingRegion meta = rs.getFromOnlineRegions(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); rs.removeFromOnlineRegions(meta, null); ((HRegion)meta).close(); 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 d7f0a32..fa1f787 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 @@ -57,7 +57,7 @@ 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.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -191,10 +191,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 (Region r : regs) { + for (BlockingRegion r : regs) { if (r.getRegionInfo().getTable().getNamespaceAsString() .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { isNamespaceServer = true; @@ -418,7 +418,7 @@ public class TestRegionPlacement { MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); for (int i = 0; i < SLAVES; i++) { HRegionServer rs = cluster.getRegionServer(i); - for (Region region: rs.getOnlineRegions(TableName.valueOf("testRegionAssignment"))) { + for (BlockingRegion region: rs.getOnlineRegions(TableName.valueOf("testRegionAssignment"))) { InetSocketAddress[] favoredSocketAddress = rs.getFavoredNodesForRegion( region.getRegionInfo().getEncodedName()); List favoredServerList = plan.getAssignmentMap().get(region.getRegionInfo()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java index 039cac1..a88024c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -60,7 +60,7 @@ public class TestRegionLocationFinder { for (int i = 0; i < ServerNum; i++) { HRegionServer server = cluster.getRegionServer(i); - for (Region region : server.getOnlineRegions(tableName)) { + for (BlockingRegion region : server.getOnlineRegions(tableName)) { region.flush(true); } } @@ -81,7 +81,7 @@ public class TestRegionLocationFinder { public void testInternalGetTopBlockLocation() throws Exception { for (int i = 0; i < ServerNum; i++) { HRegionServer server = cluster.getRegionServer(i); - for (Region region : server.getOnlineRegions(tableName)) { + for (BlockingRegion region : server.getOnlineRegions(tableName)) { // get region's hdfs block distribution by region and RegionLocationFinder, // they should have same result HDFSBlocksDistribution blocksDistribution1 = region.getHDFSBlocksDistribution(); @@ -120,7 +120,7 @@ public class TestRegionLocationFinder { public void testGetTopBlockLocations() throws Exception { for (int i = 0; i < ServerNum; i++) { HRegionServer server = cluster.getRegionServer(i); - for (Region region : server.getOnlineRegions(tableName)) { + for (BlockingRegion region : server.getOnlineRegions(tableName)) { List servers = finder.getTopBlockLocations(finder .getBlockDistribution(region.getRegionInfo())); // test table may have empty region diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java index f8bc6ab..9c76c24 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType; import org.apache.hadoop.hbase.namespace.TestNamespaceAuditor; import org.apache.hadoop.hbase.quotas.QuotaUtil; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -236,7 +236,7 @@ public class TestSimpleRegionNormalizerOnCluster { admin.deleteTable(TABLENAME); } - private void generateTestData(Region region, int numRows) throws IOException { + private void generateTestData(BlockingRegion region, int numRows) throws IOException { // generating 1Mb values LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(1024 * 1024, 1024 * 1024); for (int i = 0; i < numRows; ++i) { 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 8c9db88..a7fed94 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 @@ -75,7 +75,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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; @@ -301,7 +301,7 @@ public class TestNamespaceAuditor { @Override public synchronized void preMerge(ObserverContext ctx, - Region regionA, Region regionB) throws IOException { + BlockingRegion regionA, BlockingRegion regionB) throws IOException { triggered = true; notifyAll(); if (shouldFailMerge) { 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 025a28d..36164f2 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 { - Region r = c.getEnvironment().getRegion(); + BlockingRegion 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/RegionAsTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java index d2e78b7..01c2765 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java @@ -61,12 +61,12 @@ import com.google.protobuf.ServiceException; * over the network. */ public class RegionAsTable implements Table { - private final Region region; + private final BlockingRegion region; /** * @param region Region to decorate with Table API. */ - public RegionAsTable(final Region region) { + public RegionAsTable(final BlockingRegion region) { this.region = region; } 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 11bd280..c609636 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 @@ -83,7 +83,7 @@ public class TestAtomicOperation { private static final Log LOG = LogFactory.getLog(TestAtomicOperation.class); @Rule public TestName name = new TestName(); - Region region = null; + BlockingRegion region = null; private HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); // Test names @@ -263,12 +263,12 @@ public class TestAtomicOperation { */ public static class Incrementer extends Thread { - private final Region region; + private final BlockingRegion region; private final int numIncrements; private final int amount; - public Incrementer(Region region, int threadNumber, int amount, int numIncrements) { + public Incrementer(BlockingRegion region, int threadNumber, int amount, int numIncrements) { super("Incrementer." + threadNumber); this.region = region; this.numIncrements = numIncrements; @@ -553,13 +553,13 @@ public class TestAtomicOperation { } public static class AtomicOperation extends Thread { - protected final Region region; + protected final BlockingRegion region; protected final int numOps; protected final AtomicLong timeStamps; protected final AtomicInteger failures; protected final Random r = new Random(); - public AtomicOperation(Region region, int numOps, AtomicLong timeStamps, + public AtomicOperation(BlockingRegion region, int numOps, AtomicLong timeStamps, AtomicInteger failures) { this.region = region; this.numOps = numOps; @@ -620,8 +620,8 @@ public class TestAtomicOperation { } private class PutThread extends TestThread { - private Region region; - PutThread(TestContext ctx, Region region) { + private BlockingRegion region; + PutThread(TestContext ctx, BlockingRegion region) { super(ctx); this.region = region; } @@ -637,8 +637,8 @@ public class TestAtomicOperation { } private class CheckAndPutThread extends TestThread { - private Region region; - CheckAndPutThread(TestContext ctx, Region region) { + private BlockingRegion region; + CheckAndPutThread(TestContext ctx, BlockingRegion region) { super(ctx); this.region = region; } 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 a574d25..d8f19f7 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 @@ -63,7 +63,7 @@ public class TestBlocksRead { BloomType.ROW, BloomType.NONE }; private static BlockCache blockCache; - Region region = null; + BlockingRegion region = null; private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private final String DIR = TEST_UTIL.getDataTestDir("TestBlocksRead").toString(); private Configuration conf = TEST_UTIL.getConfiguration(); @@ -88,7 +88,7 @@ public class TestBlocksRead { * @throws IOException * @return created and initialized region. */ - private Region initHRegion(byte[] tableName, String callingMethod, + private BlockingRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf, String family) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); HColumnDescriptor familyDesc; @@ -102,7 +102,7 @@ public class TestBlocksRead { HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); Path path = new Path(DIR + callingMethod); - Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); + BlockingRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); blockCache = new CacheConfig(conf).getBlockCache(); return r; } 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 b2ba97c..3cd3c3a 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 @@ -89,7 +89,7 @@ public class TestBlocksScanned extends HBaseTestCase { } private void _testBlocksScanned(HTableDescriptor table) throws Exception { - Region r = createNewHRegion(table, START_KEY, END_KEY, TEST_UTIL.getConfiguration()); + BlockingRegion r = createNewHRegion(table, START_KEY, END_KEY, TEST_UTIL.getConfiguration()); addContent(r, FAMILY, COL); r.flush(true); 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 eca7703..67789ea 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 @@ -76,7 +76,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. - Region region = TEST_UTIL.createLocalHRegion(info, htd); + BlockingRegion region = TEST_UTIL.createLocalHRegion(info, htd); try { List rows = generateRandomWords(10, "row"); List allColumns = generateRandomWords(10, "column"); @@ -188,7 +188,7 @@ public class TestColumnSeeking { htd.addFamily(hcd); HRegionInfo info = new HRegionInfo(table, null, null, false); - Region region = TEST_UTIL.createLocalHRegion(info, htd); + BlockingRegion region = TEST_UTIL.createLocalHRegion(info, htd); List rows = generateRandomWords(10, "row"); List allColumns = generateRandomWords(100, "column"); 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 d6302b3..ae8d2c1 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 (Region region: regions) { + for (BlockingRegion region: regions) { state = CompactionState.valueOf(region.getCompactionState().toString()); 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 (Region region: regions) { + for (BlockingRegion region: regions) { count += region.getStoreFileList(families).size(); } return count; 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 cee64e0..710cb49 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 @@ -204,7 +204,7 @@ public class TestEncryptionKeyRotation { private static void waitForCompaction(TableName tableName) throws IOException, InterruptedException { boolean compacted = false; - for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName) + for (BlockingRegion region : TEST_UTIL.getRSForFirstRegionInTable(tableName) .getOnlineRegions(tableName)) { for (Store store : region.getStores()) { compacted = false; @@ -230,7 +230,7 @@ public class TestEncryptionKeyRotation { private static List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList(); - for (Region region: + for (BlockingRegion region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(tableName)) { for (Store store: region.getStores()) { for (StoreFile storefile: store.getStorefiles()) { @@ -243,7 +243,7 @@ public class TestEncryptionKeyRotation { private static List findCompactedStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList(); - for (Region region: + for (BlockingRegion region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(tableName)) { for (Store store : region.getStores()) { Collection compactedfiles = 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 ad7cf2f..3abd510 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 @@ -53,7 +53,7 @@ public class TestEncryptionRandomKeying { private static List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList(); - for (Region region: + for (BlockingRegion region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { for (Store store: region.getStores()) { for (StoreFile storefile: store.getStorefiles()) { 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 f756ebe..8c39170 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(); - Region region = server.getRegion(regionName); + BlockingRegion region = server.getRegion(regionName); SplitTransactionImpl split = new SplitTransactionImpl((HRegion) region, splitRow); split.prepare(); // 1. phase I - PairOfSameType regions = split.createDaughters(server, server, null); + PairOfSameType regions = split.createDaughters(server, server, null); assertFalse(test(conn, tableName, firstRow, server)); assertFalse(test(conn, tableName, lastRow, server)); 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 7ee3f0b..48695e9 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 @@ -81,7 +81,7 @@ public class TestGetClosestAtOrBefore { // Up flush size else we bind up when we use default catalog flush of 16k. UTIL.getMetaTableDescriptor().setMemStoreFlushSize(64 * 1024 * 1024); - Region mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO, + BlockingRegion mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO, rootdir, this.conf, UTIL.getMetaTableDescriptor()); try { // Write rows for three tables 'A', 'B', and 'C'. @@ -156,7 +156,7 @@ public class TestGetClosestAtOrBefore { * @return Row found. * @throws IOException */ - private byte [] findRow(final Region mr, final char table, + private byte [] findRow(final BlockingRegion mr, final char table, final int rowToFind, final int answer) throws IOException { TableName tableb = TableName.valueOf("" + table); @@ -193,7 +193,7 @@ public class TestGetClosestAtOrBefore { */ @Test public void testGetClosestRowBefore3() throws IOException{ - Region region = null; + BlockingRegion region = null; byte [] c0 = UTIL.COLUMNS[0]; byte [] c1 = UTIL.COLUMNS[1]; try { @@ -304,7 +304,7 @@ public class TestGetClosestAtOrBefore { /** For HBASE-694 */ @Test public void testGetClosestRowBefore2() throws IOException{ - Region region = null; + BlockingRegion region = null; byte [] c0 = UTIL.COLUMNS[0]; try { TableName tn = TableName.valueOf(testName.getMethodName()); 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 2042f52..acc5b9c 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 @@ -90,7 +90,6 @@ 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.client.Table; -import org.apache.hadoop.hbase.client.TestMobSnapshotCloneIndependence; import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; @@ -103,7 +102,6 @@ import org.apache.hadoop.hbase.filter.PrefixFilter; import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter; import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.master.procedure.TestMasterFailoverWithProcedures; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; @@ -115,7 +113,7 @@ 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.Region.RowLock; +import org.apache.hadoop.hbase.regionserver.RegionBase.RowLock; import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem; import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler; import org.apache.hadoop.hbase.regionserver.wal.FSHLog; @@ -2673,7 +2671,7 @@ public class TestHRegion { * @throws IOException */ HRegion[] splitRegion(final HRegion parent, final byte[] midkey) throws IOException { - PairOfSameType result = null; + PairOfSameType result = null; SplitTransactionImpl st = new SplitTransactionImpl(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. @@ -6153,7 +6151,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); 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 a3804dd..b439e4d 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 @@ -179,7 +179,7 @@ public class TestHRegionReplayEvents { when(rss.getExecutorService()).thenReturn(es); primaryRegion = HRegion.createHRegion(primaryHri, rootDir, CONF, htd, walPrimary); primaryRegion.close(); - List regions = new ArrayList(); + List regions = new ArrayList(); regions.add(primaryRegion); when(rss.getOnlineRegions()).thenReturn(regions); @@ -1389,7 +1389,7 @@ public class TestHRegionReplayEvents { // Test case 3: compact primary files primaryRegion.compactStores(); - List regions = new ArrayList(); + List regions = new ArrayList(); regions.add(primaryRegion); when(rss.getOnlineRegions()).thenReturn(regions); CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, rss, false); 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 5f42a03..26f64ca 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 @@ -638,12 +638,12 @@ public class TestHeapMemoryManager { } @Override - public void requestFlush(Region region, boolean forceFlushAllStores) { + public void requestFlush(BlockingRegion region, boolean forceFlushAllStores) { this.listener.flushRequested(flushType, region); } @Override - public void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores) { + public void requestDelayedFlush(BlockingRegion 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 3e32772..0c5f92f 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); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -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); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -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); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -294,7 +294,7 @@ public class TestKeepDeletes { public void testRawScanWithColumns() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.TRUE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); Scan s = new Scan(); s.setRaw(true); @@ -318,7 +318,7 @@ public class TestKeepDeletes { public void testRawScan() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.TRUE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -408,7 +408,7 @@ public class TestKeepDeletes { public void testDeleteMarkerExpirationEmptyStore() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -451,7 +451,7 @@ public class TestKeepDeletes { public void testDeleteMarkerExpiration() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -514,7 +514,7 @@ public class TestKeepDeletes { public void testWithOldRow() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -592,7 +592,7 @@ public class TestKeepDeletes { public void testRanges() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HConstants.FOREVER, KeepDeletedCells.TRUE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -674,7 +674,7 @@ public class TestKeepDeletes { public void testDeleteMarkerVersioning() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); @@ -766,7 +766,7 @@ public class TestKeepDeletes { public void testWithMixedCFs() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HConstants.FOREVER, KeepDeletedCells.TRUE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); @@ -818,7 +818,7 @@ public class TestKeepDeletes { public void testWithMinVersions() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.TRUE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past @@ -897,7 +897,7 @@ public class TestKeepDeletes { public void testWithTTL() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 1, 1000, 1, KeepDeletedCells.TTL); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past @@ -934,7 +934,7 @@ public class TestKeepDeletes { HBaseTestingUtility.closeRegionAndWAL(region); } - private void checkGet(Region region, byte[] row, byte[] fam, byte[] col, + private void checkGet(BlockingRegion region, byte[] row, byte[] fam, byte[] col, long time, byte[]... vals) throws IOException { Get g = new Get(row); g.addColumn(fam, col); @@ -945,7 +945,7 @@ public class TestKeepDeletes { } - private int countDeleteMarkers(Region region) throws IOException { + private int countDeleteMarkers(BlockingRegion region) throws IOException { Scan s = new Scan(); s.setRaw(true); // use max versions from the store(s) 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 3ef89ad..39bf473 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 @@ -77,7 +77,7 @@ public class TestMajorCompaction { private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU(); protected Configuration conf = UTIL.getConfiguration(); - private Region r = null; + private BlockingRegion r = null; private HTableDescriptor htd = null; private static final byte [] COLUMN_FAMILY = fam1; private final byte [] STARTROW = Bytes.toBytes(START_KEY); @@ -382,17 +382,17 @@ public class TestMajorCompaction { return count; } - private void createStoreFile(final Region region) throws IOException { + private void createStoreFile(final BlockingRegion region) throws IOException { createStoreFile(region, Bytes.toString(COLUMN_FAMILY)); } - private void createStoreFile(final Region region, String family) throws IOException { + private void createStoreFile(final BlockingRegion region, String family) throws IOException { Table loader = new RegionAsTable(region); HBaseTestCase.addContent(loader, family); region.flush(true); } - private void createSmallerStoreFile(final Region region) throws IOException { + private void createSmallerStoreFile(final BlockingRegion region) throws IOException { Table loader = new RegionAsTable(region); HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY), ("" + "bbb").getBytes(), null); 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 661583e..b056231 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); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); try { // 2s in the past @@ -118,7 +118,7 @@ public class TestMinVersions { // keep 3 versions minimum HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.FALSE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); // 2s in the past long ts = EnvironmentEdgeManager.currentTime() - 2000; @@ -173,7 +173,7 @@ public class TestMinVersions { public void testDelete() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.FALSE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); // 2s in the past long ts = EnvironmentEdgeManager.currentTime() - 2000; @@ -232,7 +232,7 @@ public class TestMinVersions { public void testMemStore() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 2, 1000, 1, KeepDeletedCells.FALSE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); // 2s in the past long ts = EnvironmentEdgeManager.currentTime() - 2000; @@ -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); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); try { // 2s in the past @@ -400,7 +400,7 @@ public class TestMinVersions { public void testFilters() throws Exception { HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 2, 1000, 1, KeepDeletedCells.FALSE); - Region region = hbu.createLocalHRegion(htd, null, null); + BlockingRegion region = hbu.createLocalHRegion(htd, null, null); final byte [] c1 = COLUMNS[1]; // 2s in the past 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 1bd20c6..38c7fd5 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 Region r = null; + private BlockingRegion r = null; private HTableDescriptor htd = null; private int compactionThreshold; private byte[] firstRowBytes, secondRowBytes, thirdRowBytes; 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 ff6f09b..985ddac 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 @@ -145,7 +145,7 @@ public class TestMultiColumnScanner { @Test public void testMultiColumnScanner() throws IOException { - Region region = TEST_UTIL.createTestRegion(TABLE_NAME, + BlockingRegion region = TEST_UTIL.createTestRegion(TABLE_NAME, new HColumnDescriptor(FAMILY) .setCompressionType(comprAlgo) .setBloomFilterType(bloomType) 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 6bfaa59..46ccc8c 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 @@ -130,7 +130,7 @@ public class TestPerColumnFamilyFlush { conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 100 * 1024); // Intialize the region - Region region = initHRegion("testSelectiveFlushWhenEnabled", conf); + BlockingRegion 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)); @@ -323,12 +323,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 (Region region : hrs.getOnlineRegions(tableName)) { + for (BlockingRegion region : hrs.getOnlineRegions(tableName)) { return Pair.newPair(region, hrs); } } @@ -366,8 +366,8 @@ public class TestPerColumnFamilyFlush { } Thread.sleep(1000); - Pair desiredRegionAndServer = getRegionWithName(TABLENAME); - Region desiredRegion = desiredRegionAndServer.getFirst(); + Pair desiredRegionAndServer = getRegionWithName(TABLENAME); + BlockingRegion desiredRegion = desiredRegionAndServer.getFirst(); assertTrue("Could not find a region which hosts the new region.", desiredRegion != null); // Flush the region selectively. @@ -435,11 +435,11 @@ public class TestPerColumnFamilyFlush { doTestLogReplay(); } - private WAL getWAL(Region region) { + private WAL getWAL(BlockingRegion region) { return ((HRegion)region).getWAL(); } - private int getNumRolledLogFiles(Region region) { + private int getNumRolledLogFiles(BlockingRegion region) { return AbstractFSWALProvider.getNumRolledLogFiles(getWAL(region)); } @@ -477,8 +477,8 @@ public class TestPerColumnFamilyFlush { try (Admin admin = TEST_UTIL.getConnection().getAdmin()) { admin.flush(TableName.NAMESPACE_TABLE_NAME); } - Pair desiredRegionAndServer = getRegionWithName(tableName); - final Region desiredRegion = desiredRegionAndServer.getFirst(); + Pair desiredRegionAndServer = getRegionWithName(tableName); + final BlockingRegion desiredRegion = desiredRegionAndServer.getFirst(); assertTrue("Could not find a region which hosts the new region.", desiredRegion != null); LOG.info("Writing to region=" + desiredRegion); @@ -541,7 +541,7 @@ public class TestPerColumnFamilyFlush { } private void doPut(Table table, long memstoreFlushSize) throws IOException, InterruptedException { - Region region = getRegionWithName(table.getName()).getFirst(); + BlockingRegion 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(); @@ -602,7 +602,7 @@ public class TestPerColumnFamilyFlush { table.close(); conn.close(); - Region region = getRegionWithName(TABLENAME).getFirst(); + BlockingRegion region = getRegionWithName(TABLENAME).getFirst(); cf1StoreFileCount = region.getStore(FAMILY1).getStorefilesCount(); cf2StoreFileCount = region.getStore(FAMILY2).getStorefilesCount(); cf3StoreFileCount = region.getStore(FAMILY3).getStorefilesCount(); @@ -625,7 +625,7 @@ public class TestPerColumnFamilyFlush { table.close(); conn.close(); - Region region = getRegionWithName(TABLENAME).getFirst(); + BlockingRegion 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 449a150..e43fbd4 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 @@ -114,8 +114,8 @@ public class TestRegionFavoredNodes { // 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 (Region region : regions) { + List regions = server.getOnlineRegions(TABLE_NAME); + for (BlockingRegion region : regions) { ListfavoredNodes = new ArrayList(3); String encodedRegionName = region.getRegionInfo().getEncodedName(); @@ -142,8 +142,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 (Region region : regions) { + List regions = server.getOnlineRegions(TABLE_NAME); + for (BlockingRegion 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/TestRegionReplicaFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicaFailover.java index 2496e2b..3d72b27 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 @@ -185,7 +185,7 @@ public class TestRegionReplicaFailover { // read from it the same data from primary and secondaries boolean aborted = false; for (RegionServerThread rs : HTU.getMiniHBaseCluster().getRegionServerThreads()) { - for (Region r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) { + for (BlockingRegion 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"); @@ -247,7 +247,7 @@ public class TestRegionReplicaFailover { // read from it the same data boolean aborted = false; for (RegionServerThread rs : HTU.getMiniHBaseCluster().getRegionServerThreads()) { - for (Region r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) { + for (BlockingRegion 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"); @@ -308,7 +308,7 @@ public class TestRegionReplicaFailover { try { boolean aborted = false; for (RegionServerThread rs : HTU.getMiniHBaseCluster().getRegionServerThreads()) { - for (Region r : rs.getRegionServer().getOnlineRegions(htd.getTableName())) { + for (BlockingRegion 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 99f5801..bf2bb6b 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 @@ -154,7 +154,7 @@ public class TestRegionReplicas { // assert that we can read back from primary Assert.assertEquals(1000, HTU.countRows(table)); // flush so that region replica can read - Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + BlockingRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); region.flush(true); openRegion(HTU, getRS(), hriSecondary); @@ -178,7 +178,7 @@ public class TestRegionReplicas { // assert that we can read back from primary Assert.assertEquals(1000, HTU.countRows(table)); // flush so that region replica can read - Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + BlockingRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); region.flush(true); openRegion(HTU, getRS(), hriSecondary); @@ -196,7 +196,7 @@ public class TestRegionReplicas { } } - private void assertGet(Region region, int value, boolean expect) throws IOException { + private void assertGet(BlockingRegion region, int value, boolean expect) throws IOException { byte[] row = Bytes.toBytes(String.valueOf(value)); Get get = new Get(row); Result result = region.get(get); @@ -248,7 +248,7 @@ public class TestRegionReplicas { Assert.assertEquals(1000, HTU.countRows(table)); // flush so that region replica can read LOG.info("Flushing primary region"); - Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + BlockingRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); region.flush(true); HRegion primaryRegion = (HRegion) region; @@ -257,7 +257,7 @@ public class TestRegionReplicas { Threads.sleep(4 * refreshPeriod); LOG.info("Checking results from secondary region replica"); - Region secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); + BlockingRegion secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); Assert.assertEquals(1, secondaryRegion.getStore(f).getStorefilesCount()); assertGet(secondaryRegion, 42, true); @@ -439,15 +439,15 @@ 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); - Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); + BlockingRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); region.flush(true); } - Region primaryRegion = getRS().getFromOnlineRegions(hriPrimary.getEncodedName()); + BlockingRegion primaryRegion = getRS().getFromOnlineRegions(hriPrimary.getEncodedName()); Assert.assertEquals(3, primaryRegion.getStore(f).getStorefilesCount()); // Refresh store files on the secondary - Region secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); + BlockingRegion secondaryRegion = getRS().getFromOnlineRegions(hriSecondary.getEncodedName()); secondaryRegion.getStore(f).refreshStoreFiles(); Assert.assertEquals(3, secondaryRegion.getStore(f).getStorefilesCount()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java index 89a82a7..41f1fdb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java @@ -412,7 +412,7 @@ public class TestRegionServerMetrics { byte[] val = Bytes.toBytes("mobdata"); try { Table table = TEST_UTIL.createTable(htd, new byte[0][0], conf); - Region region = rs.getOnlineRegions(tableName).get(0); + BlockingRegion region = rs.getOnlineRegions(tableName).get(0); for (int insertCount = 0; insertCount < numHfiles; insertCount++) { Put p = new Put(Bytes.toBytes(insertCount)); p.addColumn(cf, qualifier, val); 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 0ee75a8..1ad694c 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 @@ -130,7 +130,7 @@ 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()) { - Region region = rst.getRegionServer().getRegionByEncodedName(regionInfo.getEncodedName()); + BlockingRegion region = rst.getRegionServer().getRegionByEncodedName(regionInfo.getEncodedName()); if (region != null) { 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 8d0b1d3..6d4c2ff 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 @@ -57,7 +57,7 @@ public class TestRegionServerOnlineConfigChange { private static Table t1 = null; private static HRegionServer rs1 = null; private static byte[] r1name = null; - private static Region r1 = null; + private static BlockingRegion 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 2cae887..15d42de 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 @@ -98,7 +98,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'. @@ -157,7 +157,7 @@ public class TestRegionSplitPolicy { conf.setFloat("hbase.busy.policy.blockedRequests", 0.1f); 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); Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(0L); 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 3e02243..687e117 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); } } - Region region = HBaseTestingUtility.createRegionAndWAL(hri, path, conf, htd); + BlockingRegion region = HBaseTestingUtility.createRegionAndWAL(hri, path, conf, htd); try { Increment odd = new Increment(rows[0]); odd.setDurability(Durability.SKIP_WAL); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java index 0ec859c..cf9a4a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java @@ -313,7 +313,7 @@ public class TestReversibleScanners { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testtable")) .addFamily(new HColumnDescriptor(FAMILYNAME)) .addFamily(new HColumnDescriptor(FAMILYNAME2)); - Region region = TEST_UTIL.createLocalHRegion(htd, null, null); + BlockingRegion region = TEST_UTIL.createLocalHRegion(htd, null, null); loadDataToRegion(region, FAMILYNAME2); // verify row count with forward scan @@ -616,7 +616,7 @@ public class TestReversibleScanners { return nextReadableNum; } - private static void loadDataToRegion(Region region, byte[] additionalFamily) + private static void loadDataToRegion(BlockingRegion region, byte[] additionalFamily) throws IOException { for (int i = 0; i < ROWSIZE; i++) { Put put = new Put(ROWS[i]); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowTooBig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowTooBig.java index 4d3a1c3..4989e4f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowTooBig.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowTooBig.java @@ -85,7 +85,7 @@ public class TestRowTooBig { final HRegionInfo hri = new HRegionInfo(htd.getTableName(), HConstants.EMPTY_END_ROW, HConstants.EMPTY_END_ROW); - Region region = + BlockingRegion region = HBaseTestingUtility.createRegionAndWAL(hri, rootRegionDir, HTU.getConfiguration(), htd); try { // Add 5 cells to memstore @@ -132,7 +132,7 @@ public class TestRowTooBig { final HRegionInfo hri = new HRegionInfo(htd.getTableName(), HConstants.EMPTY_END_ROW, HConstants.EMPTY_END_ROW); - Region region = + BlockingRegion region = HBaseTestingUtility.createRegionAndWAL(hri, rootRegionDir, HTU.getConfiguration(), htd); try { // Add to memstore diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java index 027193f..bf06069 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java @@ -75,7 +75,7 @@ public class TestScanWithBloomError { private static final String QUALIFIER_PREFIX = "qual"; private static final byte[] ROW_BYTES = Bytes.toBytes(ROW); private static NavigableSet allColIds = new TreeSet(); - private Region region; + private BlockingRegion region; private BloomType bloomType; private FileSystem fs; private Configuration conf; 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 b31be9d..0431d6f 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 Region region; + private BlockingRegion region; private Put put; private Delete del; private Random rand; 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 2ade27a..162306a 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 @@ -196,7 +196,7 @@ public class TestSplitTransaction { st.prepare(); 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); verify(listener).transition(st, SplitTransaction.SplitTransactionPhase.STARTED, SplitTransaction.SplitTransactionPhase.PREPARED); verify(listener, times(15)).transition(any(SplitTransaction.class), @@ -243,7 +243,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. @@ -261,7 +261,7 @@ public class TestSplitTransaction { daughters.getSecond().getRegionInfo().getEndKey())); // Count rows. daughters are already open int daughtersRowCount = 0; - for (Region openRegion: daughters) { + for (BlockingRegion openRegion: daughters) { try { int count = countRows(openRegion); assertTrue(count > 0 && count != rowcount); @@ -342,10 +342,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 (Region openRegion: daughters) { + for (BlockingRegion openRegion: daughters) { try { int count = countRows(openRegion); assertTrue(count > 0 && count != rowcount); @@ -371,7 +371,7 @@ public class TestSplitTransaction { private class MockedFailedDaughterCreation extends IOException {} private class MockedFailedDaughterOpen extends IOException {} - private int countRows(final Region r) throws IOException { + private int countRows(final BlockingRegion 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 7fbcfea..ef0d796 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 @@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; -import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -996,7 +995,7 @@ public class TestSplitTransactionOnCluster { assertTrue("not able to find a splittable region", region != null); SplitTransactionImpl 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 (); } @@ -1301,17 +1300,17 @@ public class TestSplitTransactionOnCluster { public static class MockedRegionObserver extends BaseRegionObserver { private SplitTransactionImpl 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")); - Region region = onlineRegions.get(0); - for (Region r : onlineRegions) { + BlockingRegion region = onlineRegions.get(0); + for (BlockingRegion r : onlineRegions) { if (r.getRegionInfo().containsRow(splitKey)) { region = r; break; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java index 1c1a98b..c095b67 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java @@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult; -import org.apache.hadoop.hbase.regionserver.Region.FlushResult; +import org.apache.hadoop.hbase.regionserver.RegionBase.FlushResult; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -120,7 +120,7 @@ public class TestSplitWalDataLoss { Matchers.> any()); // Find region key; don't pick up key for hbase:meta by mistake. String key = null; - for (Map.Entry entry: rs.onlineRegions.entrySet()) { + for (Map.Entry entry: rs.onlineRegions.entrySet()) { if (entry.getValue().getRegionInfo().getTable().equals(this.tableName)) { key = entry.getKey(); break; 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 74b3df9..7a300d9 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,7 +94,7 @@ public class TestStoreFileRefresherChore { } } - private Region initHRegion(HTableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId) + private BlockingRegion initHRegion(HTableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId) throws IOException { Configuration conf = TEST_UTIL.getConfiguration(); Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName()); @@ -115,7 +115,7 @@ public class TestStoreFileRefresherChore { return region; } - private void putData(Region region, int startRow, int numRows, byte[] qf, byte[]... families) + private void putData(BlockingRegion 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)); @@ -127,7 +127,7 @@ public class TestStoreFileRefresherChore { } } - private void verifyData(Region newReg, int startRow, int numRows, byte[] qf, byte[]... families) + private void verifyData(BlockingRegion newReg, int startRow, int numRows, byte[] qf, byte[]... families) throws IOException { for (int i = startRow; i < startRow + numRows; i++) { byte[] row = Bytes.toBytes("" + i); @@ -165,13 +165,15 @@ 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); - 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); + BlockingRegion + primary = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 0); + BlockingRegion + replica1 = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 1); regions.add(primary); regions.add(replica1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java index 74826b0..5375c84 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java @@ -132,7 +132,7 @@ public class TestWalAndCompactingMemStoreFlush { conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.25); // Intialize the region - Region region = initHRegion("testSelectiveFlushWhenEnabled", conf); + BlockingRegion region = initHRegion("testSelectiveFlushWhenEnabled", conf); // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3 for (int i = 1; i <= 1200; i++) { @@ -385,7 +385,7 @@ public class TestWalAndCompactingMemStoreFlush { conf.setInt("hbase.hregion.compacting.memstore.type",1); // Intialize the region - Region region = initHRegion("testSelectiveFlushWhenEnabled", conf); + BlockingRegion region = initHRegion("testSelectiveFlushWhenEnabled", conf); // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3 for (int i = 1; i <= 1200; i++) { @@ -744,7 +744,7 @@ public class TestWalAndCompactingMemStoreFlush { HBaseTestingUtility.closeRegionAndWAL(region); } - private WAL getWAL(Region region) { + private WAL getWAL(BlockingRegion region) { return ((HRegion)region).getWAL(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java index c23e794..d11777e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java @@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.Store; @@ -59,7 +59,7 @@ import org.junit.experimental.categories.Category; @Category({ MediumTests.class, RegionServerTests.class }) public class TestCompactedHFilesDischarger { private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); - private Region region; + private BlockingRegion region; private final static byte[] fam = Bytes.toBytes("cf_1"); private final static byte[] qual1 = Bytes.toBytes("qf_1"); private final static byte[] val = Bytes.toBytes("val"); @@ -77,7 +77,7 @@ public class TestCompactedHFilesDischarger { Path path = testUtil.getDataTestDir(getClass().getSimpleName()); region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(), htd); rss = mock(RegionServerServices.class); - List regions = new ArrayList(); + List regions = new ArrayList(); regions.add(region); when(rss.getOnlineRegions()).thenReturn(regions); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java index f3cfc39..95b2e24 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; 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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -70,7 +70,7 @@ public class TestFIFOCompactionPolicy { List rsts = cluster.getRegionServerThreads(); for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { HRegionServer hrs = rsts.get(i).getRegionServer(); - for (Region region : hrs.getOnlineRegions(tableName)) { + for (BlockingRegion region : hrs.getOnlineRegions(tableName)) { return region.getStores().iterator().next(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java index 41975eb..d700d93 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java @@ -40,15 +40,12 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; 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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreEngine; import org.apache.hadoop.hbase.regionserver.StripeStoreConfig; import org.apache.hadoop.hbase.regionserver.StripeStoreEngine; import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; -import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory; -import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; -import org.apache.hadoop.hbase.regionserver.throttle.PressureAwareCompactionThroughputController; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -76,7 +73,7 @@ public class TestCompactionWithThroughputController { List rsts = cluster.getRegionServerThreads(); for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { HRegionServer hrs = rsts.get(i).getRegionServer(); - for (Region region : hrs.getOnlineRegions(tableName)) { + for (BlockingRegion region : hrs.getOnlineRegions(tableName)) { return region.getStores().iterator().next(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java index ae6b036..8a19998 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreEngine; import org.apache.hadoop.hbase.regionserver.StripeStoreEngine; @@ -62,7 +62,7 @@ public class TestFlushWithThroughputController { List rsts = cluster.getRegionServerThreads(); for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { HRegionServer hrs = rsts.get(i).getRegionServer(); - for (Region region : hrs.getOnlineRegions(tableName)) { + for (BlockingRegion region : hrs.getOnlineRegions(tableName)) { return region.getStores().iterator().next(); } } @@ -177,7 +177,7 @@ public class TestFlushWithThroughputController { HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName); PressureAwareFlushThroughputController throughputController = (PressureAwareFlushThroughputController) regionServer.getFlushThroughputController(); - for (Region region : regionServer.getOnlineRegions()) { + for (BlockingRegion region : regionServer.getOnlineRegions()) { region.flush(true); } assertEquals(0.0, regionServer.getFlushPressure(), EPSILON); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java index 046071f..310d51a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java @@ -41,12 +41,11 @@ 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.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; -import org.apache.hadoop.hbase.wal.FSHLogProvider; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -189,7 +188,7 @@ public abstract class AbstractTestLogRolling { " log files"); // flush all regions - for (Region r: server.getOnlineRegionsLocalContext()) { + for (BlockingRegion r: server.getOnlineRegionsLocalContext()) { r.flush(true); } @@ -241,7 +240,7 @@ public abstract class AbstractTestLogRolling { table = createTestTable(getName()); server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); - Region region = server.getOnlineRegions(table.getName()).get(0); + BlockingRegion region = server.getOnlineRegions(table.getName()).get(0); final WAL log = server.getWAL(region.getRegionInfo()); Store s = region.getStore(HConstants.CATALOG_FAMILY); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java index faa539e..eff6e36 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java @@ -82,7 +82,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.Store; @@ -95,8 +95,6 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; -import org.apache.hadoop.hbase.wal.AsyncFSWALProvider; -import org.apache.hadoop.hbase.wal.FSHLogProvider; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALKey; @@ -214,7 +212,7 @@ public abstract class AbstractTestWALReplay { assertEquals(1, regions.size()); // move region to another regionserver - Region destRegion = regions.get(0); + BlockingRegion destRegion = regions.get(0); int originServerNum = hbaseCluster .getServerWith(destRegion.getRegionInfo().getRegionName()); assertTrue("Please start more than 1 regionserver", hbaseCluster @@ -240,7 +238,7 @@ public abstract class AbstractTestWALReplay { assertEquals(0, count); // flush region and make major compaction - Region region = destServer.getOnlineRegion(destRegion.getRegionInfo().getRegionName()); + BlockingRegion region = destServer.getOnlineRegion(destRegion.getRegionInfo().getRegionName()); region.flush(true); // wait to complete major compaction for (Store store : region.getStores()) { @@ -262,7 +260,7 @@ public abstract class AbstractTestWALReplay { resultScanner.close(); } - private void moveRegionAndWait(Region destRegion, HRegionServer destServer) + private void moveRegionAndWait(BlockingRegion destRegion, HRegionServer destServer) throws InterruptedException, MasterNotRunningException, ZooKeeperConnectionException, IOException { HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster(); @@ -299,7 +297,8 @@ public abstract class AbstractTestWALReplay { deleteDir(basedir); HTableDescriptor htd = createBasic3FamilyHTD(tableName); - Region region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); + BlockingRegion + region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region2); final byte [] rowName = tableName.getName(); @@ -365,10 +364,11 @@ public abstract class AbstractTestWALReplay { final Path basedir = new Path(this.hbaseRootDir, tableName.getNameAsString()); deleteDir(basedir); final HTableDescriptor htd = createBasic3FamilyHTD(tableName); - Region region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); + BlockingRegion + region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd); HBaseTestingUtility.closeRegionAndWAL(region2); WAL wal = createWAL(this.conf, hbaseRootDir, logName); - Region region = HRegion.openHRegion(hri, htd, wal, this.conf); + BlockingRegion region = HRegion.openHRegion(hri, htd, wal, this.conf); byte [] family = htd.getFamilies().iterator().next().getName(); Path f = new Path(basedir, "hfile"); @@ -1138,7 +1138,7 @@ public abstract class AbstractTestWALReplay { private HRegion r; @Override - public void requestFlush(Region region, boolean force) { + public void requestFlush(BlockingRegion region, boolean force) { try { r.flush(force); } catch (IOException e) { @@ -1147,7 +1147,7 @@ public abstract class AbstractTestWALReplay { } @Override - public void requestDelayedFlush(Region region, long when, boolean forceFlushAllStores) { + public void requestDelayedFlush(BlockingRegion region, long when, boolean forceFlushAllStores) { // TODO Auto-generated method stub } @@ -1204,7 +1204,7 @@ public abstract class AbstractTestWALReplay { } static List addRegionEdits(final byte[] rowName, final byte[] family, final int count, - EnvironmentEdge ee, final Region r, final String qualifierPrefix) throws IOException { + EnvironmentEdge ee, final BlockingRegion r, final String qualifierPrefix) throws IOException { List puts = new ArrayList(); for (int j = 0; j < count; j++) { byte[] qualifier = Bytes.toBytes(qualifierPrefix + Integer.toString(j)); 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 e4d4c5b..5537779 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 @@ -45,7 +45,7 @@ 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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -338,7 +338,7 @@ public class TestLogRolling extends AbstractTestLogRolling { assertTrue(loggedRows.contains("row1005")); // flush all regions - for (Region r : server.getOnlineRegionsLocalContext()) { + for (BlockingRegion r : server.getOnlineRegionsLocalContext()) { try { r.flush(true); } catch (Exception e) { 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 5147339..4d1d814 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 @@ -44,7 +44,7 @@ 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.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -211,22 +211,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 Region[] regions = new Region[regionReplication]; + final BlockingRegion[] regions = new BlockingRegion[regionReplication]; for (int i=0; i < NB_SERVERS; i++) { HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(i); - List onlineRegions = rs.getOnlineRegions(tableName); - for (Region region : onlineRegions) { + List onlineRegions = rs.getOnlineRegions(tableName); + for (BlockingRegion region : onlineRegions) { regions[region.getRegionInfo().getReplicaId()] = region; } } - for (Region region : regions) { + for (BlockingRegion region : regions) { assertNotNull(region); } for (int i = 1; i < regionReplication; i++) { - final Region region = regions[i]; + final BlockingRegion region = regions[i]; // wait until all the data is replicated to all secondary regions Waiter.waitFor(HTU.getConfiguration(), 90000, new Waiter.Predicate() { @Override 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 6f5ad56..ba9824d 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,7 +49,7 @@ import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; @@ -176,7 +176,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster { // replay the edits to the secondary using replay callable replicateUsingCallable(connection, entries); - Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); + BlockingRegion region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); HTU.verifyNumericRows(region, f, 0, 1000); HTU.deleteNumericRows(table, f, 0, 1000); @@ -216,7 +216,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster { // replay the edits to the secondary using replay callable replicateUsingCallable(connection, entries); - Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); + BlockingRegion region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); HTU.verifyNumericRows(region, f, 0, 1000); HTU.loadNumericRows(table, f, 1000, 2000); // load some more data to primary @@ -261,7 +261,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster { replicator.replicate(new ReplicateContext().setEntries(Lists.newArrayList(entries)) .setWalGroupId(fakeWalGroupId)); - Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName()); + BlockingRegion 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 274fe37..eb05e86 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 @@ -58,7 +58,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.Permission.Action; @@ -303,7 +303,7 @@ public class SecureTestUtil { private static List getAccessControllers(MiniHBaseCluster cluster) { List result = Lists.newArrayList(); for (RegionServerThread t: cluster.getLiveRegionServerThreads()) { - for (Region region: t.getRegionServer().getOnlineRegionsLocalContext()) { + for (BlockingRegion region: t.getRegionServer().getOnlineRegionsLocalContext()) { Coprocessor cp = region.getCoprocessorHost() .findCoprocessor(AccessController.class.getName()); if (cp != null) { 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 ddfdb7a..eac6aa1 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 @@ -109,7 +109,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState; 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.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.regionserver.ScanType; @@ -269,7 +269,7 @@ public class TestAccessController extends SecureTestUtil { htd.setOwner(USER_OWNER); createTable(TEST_UTIL, htd, new byte[][] { Bytes.toBytes("s") }); - Region region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0); + BlockingRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0); RegionCoprocessorHost rcpHost = region.getCoprocessorHost(); RCP_ENV = rcpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf); @@ -2506,7 +2506,7 @@ public class TestAccessController extends SecureTestUtil { for (JVMClusterUtil.RegionServerThread thread: TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) { HRegionServer rs = thread.getRegionServer(); - for (Region region: rs.getOnlineRegions(TEST_TABLE)) { + for (BlockingRegion region: rs.getOnlineRegions(TEST_TABLE)) { region.getCoprocessorHost().load(PingCoprocessor.class, Coprocessor.PRIORITY_USER, conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java index d757736..51bb5d3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.security.User; @@ -208,7 +208,7 @@ public class TestAccessController3 extends SecureTestUtil { htd.setOwner(USER_OWNER); createTable(TEST_UTIL, htd, new byte[][] { Bytes.toBytes("s") }); - Region region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0); + BlockingRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0); RegionCoprocessorHost rcpHost = region.getCoprocessorHost(); RCP_ENV = rcpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java index 4c6df38..e6d9b92 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java @@ -60,7 +60,7 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; @@ -192,7 +192,7 @@ public class TestWithDisabledAuthorization extends SecureTestUtil { admin.createTable(htd, new byte[][] { Bytes.toBytes("s") }); TEST_UTIL.waitUntilAllRegionsAssigned(TEST_TABLE.getTableName()); - Region region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE.getTableName()).get(0); + BlockingRegion 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, TEST_UTIL.getConfiguration()); @@ -829,9 +829,9 @@ public class TestWithDisabledAuthorization extends SecureTestUtil { @Override public Object run() throws Exception { HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName()); - Region region_a = mock(Region.class); + BlockingRegion region_a = mock(BlockingRegion.class); when(region_a.getTableDesc()).thenReturn(htd); - Region region_b = mock(Region.class); + BlockingRegion region_b = mock(BlockingRegion.class); when(region_b.getTableDesc()).thenReturn(htd); ACCESS_CONTROLLER.preMerge(ObserverContext.createAndPrepare(RSCP_ENV, null), region_a, region_b); 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 e601af7..e89a283 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 @@ -53,7 +53,7 @@ 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.OperationStatus; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode; @@ -82,7 +82,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer private final ExpressionParser expressionParser = new ExpressionParser(); private final ExpressionExpander expressionExpander = new ExpressionExpander(); private Configuration conf; - private Region labelsRegion; + private BlockingRegion labelsRegion; private List scanLabelGenerators; @Override 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 ab2bacc..0e5322f 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 @@ -60,7 +60,7 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.Visibil 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.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.security.User; @@ -307,7 +307,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 "); @@ -341,7 +341,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; @@ -398,7 +398,7 @@ public abstract class TestVisibilityLabels { } catch (InterruptedException e) { } } - Region labelsTableRegion = regionServer.getOnlineRegions(LABELS_TABLE_NAME).get(0); + BlockingRegion 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 dfd00b3..0f5e147 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 @@ -67,7 +67,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneReq import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; 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.regionserver.BlockingRegion; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSVisitor; @@ -742,8 +742,8 @@ public final class SnapshotTestingUtils { final TableName tableName) throws IOException, InterruptedException { HRegionServer rs = util.getRSForFirstRegionInTable(tableName); - List onlineRegions = rs.getOnlineRegions(tableName); - for (Region region : onlineRegions) { + List onlineRegions = rs.getOnlineRegions(tableName); + for (BlockingRegion 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/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java index 0ed8711..23d3504 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 @@ -42,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.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.security.EncryptionUtil; @@ -140,7 +140,7 @@ public class TestHBaseFsckEncryption { private List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList(); - for (Region region: + for (BlockingRegion region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { for (Store store: region.getStores()) { for (StoreFile storefile: store.getStorefiles()) { 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 bde3e49..d035e18 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 @@ -38,7 +38,7 @@ 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.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.BlockingRegion; import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.Before; @@ -130,7 +130,7 @@ public class TestWALFiltering { private List getRegionsByServer(int rsId) throws IOException { List regionNames = Lists.newArrayList(); HRegionServer hrs = getRegionServer(rsId); - for (Region r : hrs.getOnlineRegions(TABLE_NAME)) { + for (BlockingRegion r : hrs.getOnlineRegions(TABLE_NAME)) { regionNames.add(r.getRegionInfo().getRegionName()); } return regionNames; -- 2.7.4 (Apple Git-66)