diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index e8496a740e6..35bca3d9a10 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -2863,6 +2863,10 @@ public final class ProtobufUtil { return CompactionState.valueOf(state.toString()); } + public static GetRegionInfoResponse.CompactionState createCompactionState(CompactionState state) { + return GetRegionInfoResponse.CompactionState.valueOf(state.toString()); + } + public static Optional toOptionalTimestamp(MajorCompactionTimestampResponse resp) { long timestamp = resp.getCompactionTimestamp(); return timestamp == 0 ? Optional.empty() : Optional.of(timestamp); diff --git a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java index 9477eb1cdba..8b579bf4859 100644 --- a/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java +++ b/hbase-endpoint/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Re import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager; @@ -141,7 +142,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService implements Reg try { SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager(); BulkLoadHFileRequest bulkLoadHFileRequest = ConvertSecureBulkLoadHFilesRequest(request); - map = secureBulkLoadManager.secureBulkLoadHFiles(this.env.getRegion(), + map = secureBulkLoadManager.secureBulkLoadHFiles((HRegion) this.env.getRegion(), convert(bulkLoadHFileRequest)); loaded = map != null && !map.isEmpty(); } catch (IOException e) { 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 cb5373da1b1..233ea18ead1 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 @@ -150,8 +150,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements RegionCopro for (List deleteRow : deleteRows) { deleteArr[i++] = createDeleteMutation(deleteRow, deleteType, timestamp); } - OperationStatus[] opStatus = region.batchMutate(deleteArr, HConstants.NO_NONCE, - HConstants.NO_NONCE); + OperationStatus[] opStatus = region.batchMutate(deleteArr); for (i = 0; i < opStatus.length; i++) { if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) { break; 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 79c82f66247..230a42f3f00 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.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; -import org.apache.hadoop.hbase.regionserver.Region; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor; @@ -446,7 +446,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/procedure/flush/FlushTableSubprocedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/FlushTableSubprocedure.java index 145bc14ec8e..1b4c561c237 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.HRegion; /** * 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) { + HRegion region; + RegionFlushTask(HRegion 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 (HRegion 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 49192e18e6b..ea34714a1ec 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 @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager; import org.apache.hadoop.hbase.procedure.Subprocedure; import org.apache.hadoop.hbase.procedure.SubprocedureFactory; import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; @@ -139,7 +140,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,8 +175,8 @@ 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 { - return rss.getRegions(TableName.valueOf(table)); + private List getRegionsToFlush(String table) throws IOException { + return (List) rss.getRegions(TableName.valueOf(table)); } public class FlushTableSubprocedureBuilder implements SubprocedureFactory { 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 72f80e4e76c..8a0dee67e83 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 @@ -86,13 +86,13 @@ 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.getRegions(); + List onlineRegions = (List) regionServerServices.getRegions(); if (onlineRegions == null) return; - for (Region region : onlineRegions) { + for (HRegion region : onlineRegions) { if (LOG.isTraceEnabled()) { LOG.trace("Started compacted hfiles cleaner on " + region.getRegionInfo()); } - for (HStore store : ((HRegion) region).getStores()) { + for (HStore store : region.getStores()) { try { if (useExecutor && regionServerServices != null) { CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler( 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 e9f5d769f58..931a7374012 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(HRegion 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(HRegion 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 2d35fb96a72..559ac958da6 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 @@ -102,6 +102,7 @@ import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.CompactionState; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; @@ -179,10 +180,10 @@ import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps; import org.apache.hadoop.hbase.shaded.com.google.common.io.Closeables; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.Service; import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat; import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad; @@ -197,6 +198,21 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDe import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; +/** + * Regions store data for a certain region of a table. It stores all columns + * for each row. A given table consists of one or more Regions. + * + *

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

Locking at the Region level serves only one purpose: preventing the + * region from being closed (and consequently split) while other operations + * are ongoing. Each row level operation obtains both a row lock and a region + * read lock for the duration of the operation. While a scanner is being + * constructed, getScanner holds a read lock. If the scanner is successfully + * constructed, it holds a read lock until it is closed. A close takes out a + * write lock and consequently will block for ongoing operations and will block + * new operations from starting while the close is in progress. + */ @SuppressWarnings("deprecation") @InterfaceAudience.Private public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region { @@ -782,7 +798,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; @@ -1121,7 +1137,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi this.updatesLock.writeLock().unlock(); } - @Override public HDFSBlocksDistribution getHDFSBlocksDistribution() { HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); stores.values().stream().filter(s -> s.getStorefiles() != null) @@ -1238,7 +1253,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return readRequestsCount.sum(); } - @Override + /** + * Update the read request count for this region + * @param i increment + */ public void updateReadRequestsCount(long i) { readRequestsCount.add(i); } @@ -1253,7 +1271,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return writeRequestsCount.sum(); } - @Override + /** + * Update the write request count for this region + * @param i increment + */ public void updateWriteRequestsCount(long i) { writeRequestsCount.add(i); } @@ -1263,7 +1284,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return memstoreDataSize.get(); } - @Override + /** @return store services for this region, to access services required by store level needs */ public RegionServicesForStores getRegionServicesForStores() { return regionServicesForStores; } @@ -1293,7 +1314,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return checkAndMutateChecksFailed.sum(); } - @Override + // TODO Needs to check whether we should expose our metrics system to CPs. If CPs themselves doing + // the op and bypassing the core, this might be needed? Should be stop supporting the bypass + // feature? public MetricsRegion getMetrics() { return metricsRegion; } @@ -1433,12 +1456,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return mvcc.getReadPoint(); } - @Override - public long getReadpoint(IsolationLevel isolationLevel) { - return getReadPoint(isolationLevel); - } - - @Override public boolean isLoadingCfsOnDemandDefault() { return this.isLoadingCfsOnDemandDefault; } @@ -1719,7 +1736,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return stores.values().stream().mapToLong(s -> s.getMemStoreSize().getHeapSize()).sum(); } - @Override + /** Wait for all current flushes and compactions of the region to complete */ + // TODO HBASE-18906. Check the usage (if any) in Phoenix and expose this or give alternate way for + // Phoenix needs. public void waitForFlushesAndCompactions() { synchronized (writestate) { if (this.writestate.readOnly) { @@ -1748,7 +1767,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @Override + /** Wait for all current flushes of the region to complete + */ + // TODO HBASE-18906. Check the usage (if any) in Phoenix and expose this or give alternate way for + // Phoenix needs. public void waitForFlushes() { synchronized (writestate) { if (this.writestate.readOnly) { @@ -1941,7 +1963,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi stores.values().forEach(HStore::triggerMajorCompaction); } - @Override + /** + * 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 + */ public void compact(boolean majorCompaction) throws IOException { if (majorCompaction) { triggerMajorCompaction(); @@ -2157,11 +2191,51 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @Override + /** + * 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. + */ + // TODO HBASE-18905. We might have to expose a requestFlush API for CPs public FlushResult flush(boolean force) throws IOException { return flushcache(force, false); } + public static 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. * @@ -2805,7 +2879,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce); } - @Override + /** + * Prepare a delete for a row mutation processor + * @param delete The passed delete is modified by this method. WARNING! + * @throws IOException + */ public void prepareDelete(Delete delete) throws IOException { // Check to see if this is a deleteRow insert if(delete.getFamilyCellMap().isEmpty()){ @@ -2854,7 +2932,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi doBatchMutate(delete); } - @Override + /** + * Set up correct timestamps in the KVs in Delete object. + *

Caller should have the row and region locks. + * @param mutation + * @param familyMap + * @param byteNow + * @throws IOException + */ public void prepareDeleteTimestamps(Mutation mutation, Map> familyMap, byte[] byteNow) throws IOException { for (Map.Entry> e : familyMap.entrySet()) { @@ -3043,7 +3128,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @Override public OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce) throws IOException { // As it stands, this is used for 3 things @@ -3053,11 +3137,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return batchMutate(new MutationBatch(mutations, nonceGroup, nonce)); } + @Override public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException { return batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE); } - @Override public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId) throws IOException { if (!RegionReplicaUtil.isDefaultReplica(getRegionInfo()) @@ -3841,7 +3925,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @Override + /** + * Replace any cell timestamps set to {@link org.apache.hadoop.hbase.HConstants#LATEST_TIMESTAMP} + * provided current timestamp. + * @param cellItr + * @param now + */ public void updateCellTimestamps(final Iterable> cellItr, final byte[] now) throws IOException { for (List cells: cellItr) { @@ -3991,14 +4080,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi store.add(cell, memstoreSize); } - @Override + /** + * Check the collection of families for validity. + * @param families + * @throws NoSuchColumnFamilyException + */ public void checkFamilies(Collection families) throws NoSuchColumnFamilyException { for (byte[] family : families) { checkFamily(family); } } - @Override + /** + * Check the collection of families for valid timestamps + * @param familyMap + * @param now current timestamp + * @throws FailedSanityCheckException + */ public void checkTimestamps(final Map> familyMap, long now) throws FailedSanityCheckException { if (timestampSlop == HConstants.LATEST_TIMESTAMP) { @@ -5423,8 +5521,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - @Override - public void releaseRowLocks(List rowLocks) { + private void releaseRowLocks(List rowLocks) { if (rowLocks != null) { for (int i = 0; i < rowLocks.size(); i++) { rowLocks.get(i).release(); @@ -5560,13 +5657,67 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return multipleFamilies; } - @Override + /** + * 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 Map from family to List of store file paths if successful, null if failed recoverably + * @throws IOException if failed unrecoverably. + */ public Map> bulkLoadHFiles(Collection> familyPaths, boolean assignSeqId, BulkLoadListener bulkLoadListener) throws IOException { return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false); } - @Override + /** + * Listener class to enable callers of + * bulkLoadHFile() to perform any necessary + * pre/post processing of a given bulkload call + */ + public static 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, boolean copyFile) + 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 assignSeqId + * @param bulkLoadListener Internal hooks enabling massaging/preparation of a + * file about to be bulk loaded + * @param copyFile always copy hfiles if true + * @return Map from family to List of store file paths if successful, null if failed recoverably + * @throws IOException if failed unrecoverably. + */ public Map> bulkLoadHFiles(Collection> familyPaths, boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile) throws IOException { long seqId = -1; @@ -6875,7 +7026,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE); } - @Override public List get(Get get, boolean withCoprocessor, long nonceGroup, long nonce) throws IOException { List results = new ArrayList<>(); @@ -7167,22 +7317,21 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } + @Override public Result append(Append append) throws IOException { return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE); } - @Override public Result append(Append mutation, long nonceGroup, long nonce) throws IOException { return doDelta(Operation.APPEND, mutation, nonceGroup, nonce, mutation.isReturnResults()); } + @Override public Result increment(Increment increment) throws IOException { return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE); } - @Override - public Result increment(Increment mutation, long nonceGroup, long nonce) - throws IOException { + public Result increment(Increment mutation, long nonceGroup, long nonce) throws IOException { return doDelta(Operation.INCREMENT, mutation, nonceGroup, nonce, mutation.isReturnResults()); } @@ -7574,7 +7723,21 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return DEEP_OVERHEAD + stores.values().stream().mapToLong(HStore::heapSize).sum(); } - @Override + /** + * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to + * be available for handling 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 + */ public boolean registerService(com.google.protobuf.Service instance) { /* * No stacking of instances is allowed for a single service name @@ -7597,10 +7760,22 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return true; } - @Override + /** + * Executes a single protocol buffer coprocessor endpoint {@link Service} method using + * the registered protocol handlers. {@link Service} implementations must be registered via the + * {@link #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 #registerService(com.google.protobuf.Service) + */ public com.google.protobuf.Message execService(com.google.protobuf.RpcController controller, - CoprocessorServiceCall call) - throws IOException { + CoprocessorServiceCall call) throws IOException { String serviceName = call.getServiceName(); com.google.protobuf.Service service = coprocessorServiceHandlers.get(serviceName); if (service == null) { @@ -7971,7 +8146,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } }; - @Override + /** @return the latest sequence number that was read from storage when this region was opened */ public long getOpenSeqNum() { return this.openSeqNum; } @@ -7981,7 +8156,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return this.maxSeqIdInStores; } - @Override public long getOldestSeqIdOfStore(byte[] familyName) { return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName); } 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 16895bf96ce..821b41fce05 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 @@ -290,7 +290,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 @@ -308,8 +308,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; @@ -1235,7 +1235,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; @@ -1249,7 +1249,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; @@ -1369,7 +1369,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(); long usedMemory = -1L; long maxMemory = -1L; final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage(); @@ -1391,7 +1391,7 @@ public class HRegionServer extends HasThread implements } RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder(); RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder(); - for (Region region : regions) { + for (HRegion region : regions) { if (region.getCoprocessorHost() != null) { Set regionCoprocessors = region.getCoprocessorHost().getCoprocessors(); Iterator iterator = regionCoprocessors.iterator(); @@ -1469,7 +1469,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()) { RegionInfo hri = e.getValue().getRegionInfo(); if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes()) && !closedRegions.contains(hri.getEncodedName())) { @@ -1672,7 +1672,7 @@ public class HRegionServer extends HasThread implements * * @throws IOException */ - RegionLoad createRegionLoad(final Region r, RegionLoad.Builder regionLoadBldr, + RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr, RegionSpecifier.Builder regionSpecifier) throws IOException { byte[] name = r.getRegionInfo().getRegionName(); int stores = 0; @@ -1740,7 +1740,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); + HRegion r = onlineRegions.get(encodedRegionName); return r != null ? createRegionLoad(r, null, null) : null; } @@ -1821,9 +1821,9 @@ public class HRegionServer extends HasThread implements @Override protected void chore() { final StringBuffer whyFlush = new StringBuffer(); - for (Region r : this.server.onlineRegions.values()) { + for (HRegion r : this.server.onlineRegions.values()) { if (r == null) continue; - if (((HRegion)r).shouldFlush(whyFlush)) { + if (r.shouldFlush(whyFlush)) { FlushRequester requester = server.getFlushRequester(); if (requester != null) { long randomDelay = RandomUtils.nextInt(0, RANGE_OF_DELAY) + MIN_DELAY_TIME; @@ -2156,11 +2156,6 @@ public class HRegionServer extends HasThread implements } } - @Override - public void postOpenDeployTasks(final Region r) throws KeeperException, IOException { - postOpenDeployTasks(new PostOpenDeployContext(r, -1)); - } - @Override public void postOpenDeployTasks(final PostOpenDeployContext context) throws KeeperException, IOException { @@ -2197,18 +2192,6 @@ public class HRegionServer extends HasThread implements LOG.debug("Finished post open deploy task for " + r.getRegionInfo().getRegionNameAsString()); } - @Override - public boolean reportRegionStateTransition(TransitionCode code, RegionInfo... hris) { - return reportRegionStateTransition(code, HConstants.NO_SEQNUM, hris); - } - - @Override - public boolean reportRegionStateTransition( - TransitionCode code, long openSeqNum, RegionInfo... hris) { - return reportRegionStateTransition( - new RegionStateTransitionContext(code, HConstants.NO_SEQNUM, -1, hris)); - } - @Override public boolean reportRegionStateTransition(final RegionStateTransitionContext context) { TransitionCode code = context.getCode(); @@ -2664,10 +2647,10 @@ public class HRegionServer extends HasThread implements * @param abort Whether we're running an abort. */ void closeMetaTableRegions(final boolean abort) { - Region meta = null; + HRegion meta = null; this.lock.writeLock().lock(); try { - for (Map.Entry e: onlineRegions.entrySet()) { + for (Map.Entry e: onlineRegions.entrySet()) { RegionInfo hri = e.getValue().getRegionInfo(); if (hri.isMetaRegion()) { meta = e.getValue(); @@ -2689,8 +2672,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()) { + HRegion r = e.getValue(); if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) { // Don't update zk with this close transition; pass false. closeRegionIgnoreErrors(r.getRegionInfo(), abort); @@ -2720,7 +2703,7 @@ public class HRegionServer extends HasThread implements } @Override - public Map getRecoveringRegions() { + public Map getRecoveringRegions() { return this.recoveringRegions; } @@ -2751,13 +2734,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 addRegion(Region region) { + public void addRegion(HRegion region) { this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region); configurationManager.registerObserver(region); } @@ -2767,9 +2750,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) { @@ -2777,7 +2760,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 (HRegion region : this.onlineRegions.values()) { sortedRegions.put(region.getMemStoreSize(), region); } return sortedRegions; @@ -3003,10 +2986,10 @@ public class HRegionServer extends HasThread implements * @return Online regions from tableName */ @Override - public List getRegions(TableName tableName) { - List tableRegions = new ArrayList<>(); + public List getRegions(TableName tableName) { + List tableRegions = new ArrayList<>(); synchronized (this.onlineRegions) { - for (Region region: this.onlineRegions.values()) { + for (HRegion region: this.onlineRegions.values()) { RegionInfo regionInfo = region.getRegionInfo(); if(regionInfo.getTable().equals(tableName)) { tableRegions.add(region); @@ -3017,8 +3000,8 @@ public class HRegionServer extends HasThread implements } @Override - public List getRegions() { - List allRegions = new ArrayList<>(); + public List getRegions() { + List allRegions = new ArrayList<>(); synchronized (this.onlineRegions) { // Return a clone copy of the onlineRegions allRegions.addAll(onlineRegions.values()); @@ -3051,8 +3034,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 (HRegion region: regions) { coprocessors.addAll(region.getCoprocessorHost().getCoprocessors()); try { coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors()); @@ -3170,7 +3153,7 @@ public class HRegionServer extends HasThread implements protected boolean closeAndOfflineRegionForSplitOrMerge( final List regionEncodedName) throws IOException { for (int i = 0; i < regionEncodedName.size(); ++i) { - Region regionToClose = this.getRegion(regionEncodedName.get(i)); + HRegion regionToClose = this.getRegion(regionEncodedName.get(i)); if (regionToClose != null) { Map> hstoreFiles = null; Exception exceptionToThrow = null; @@ -3232,14 +3215,14 @@ public class HRegionServer extends HasThread implements } @Override - public Region getRegion(final String encodedRegionName) { + public HRegion getRegion(final String encodedRegionName) { return this.onlineRegions.get(encodedRegionName); } @Override - public boolean removeRegion(final Region r, ServerName destination) { - Region toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName()); + public boolean removeRegion(final HRegion r, ServerName destination) { + HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName()); if (destination != null) { long closeSeqNum = r.getMaxFlushedSeqId(); if (closeSeqNum == HConstants.NO_SEQNUM) { @@ -3261,20 +3244,20 @@ public class HRegionServer extends HasThread implements * @return {@link HRegion} for regionName * @throws NotServingRegionException */ - protected Region getRegion(final byte[] regionName) + protected HRegion getRegion(final byte[] regionName) throws NotServingRegionException { String encodedRegionName = RegionInfo.encodeRegionName(regionName); return getRegionByEncodedName(regionName, encodedRegionName); } - public Region getRegionByEncodedName(String encodedRegionName) + public HRegion getRegionByEncodedName(String encodedRegionName) throws NotServingRegionException { return getRegionByEncodedName(null, encodedRegionName); } - protected Region getRegionByEncodedName(byte[] regionName, String encodedRegionName) + protected HRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName) throws NotServingRegionException { - Region region = this.onlineRegions.get(encodedRegionName); + HRegion region = this.onlineRegions.get(encodedRegionName); if (region == null) { MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName); if (moveInfo != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableOnlineRegions.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableOnlineRegions.java index 3b846653ce4..ffd1fa7992c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableOnlineRegions.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ImmutableOnlineRegions.java @@ -50,11 +50,11 @@ public interface ImmutableOnlineRegions { * @return List of Region * @throws java.io.IOException */ - List getRegions(TableName tableName) throws IOException; + List getRegions(TableName tableName) throws IOException; /** * Get all online regions in this RS. * @return List of online Region */ - List getRegions(); + List getRegions(); } 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 82a5b32babb..3164e1c83f0 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 @@ -110,7 +110,7 @@ public class IncreasingToUpperBoundRegionSplitPolicy extends ConstantSizeRegionS TableName tablename = region.getTableDescriptor().getTableName(); int tableRegionsCount = 0; try { - List hri = rss.getRegions(tablename); + List hri = rss.getRegions(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 30f9a517a11..20b6c5f2013 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 @@ -204,7 +204,7 @@ public class LogRoller extends HasThread implements Closeable { */ private void scheduleFlush(final byte [] encodedRegionName) { boolean scheduled = false; - Region r = this.services.getRegion(Bytes.toString(encodedRegionName)); + HRegion r = (HRegion) this.services.getRegion(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 f37f8f6cbb4..82390bd7fc9 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 @@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.DroppedSnapshotException; import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.RegionReplicaUtil; -import org.apache.hadoop.hbase.regionserver.Region.FlushResult; +import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult; import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -126,8 +126,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); @@ -136,12 +136,12 @@ 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); + HRegion bestFlushableRegion = getBiggestMemStoreRegion(regionsBySize, excludedRegions, true); // Find the biggest region, total, even if it might have too many flushes. - Region bestAnyRegion = getBiggestMemStoreRegion( + HRegion bestAnyRegion = getBiggestMemStoreRegion( regionsBySize, excludedRegions, false); // Find the biggest region that is a secondary region - Region bestRegionReplica = getBiggestMemStoreOfRegionReplica(regionsBySize, + HRegion bestRegionReplica = getBiggestMemStoreOfRegionReplica(regionsBySize, excludedRegions); if (bestAnyRegion == null && bestRegionReplica == null) { @@ -149,7 +149,7 @@ class MemStoreFlusher implements FlushRequester { return false; } - Region regionToFlush; + HRegion 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 @@ -283,18 +283,17 @@ class MemStoreFlusher implements FlushRequester { } } - private Region getBiggestMemStoreRegion( - SortedMap regionsBySize, - Set excludedRegions, + private HRegion getBiggestMemStoreRegion( + SortedMap regionsBySize, + Set excludedRegions, boolean checkStoreFileCount) { synchronized (regionsInQueue) { - for (Region region : regionsBySize.values()) { + for (HRegion region : regionsBySize.values()) { if (excludedRegions.contains(region)) { continue; } - if (((HRegion)region).writestate.flushing || - !((HRegion)region).writestate.writesEnabled) { + if (region.writestate.flushing || !region.writestate.writesEnabled) { continue; } @@ -307,10 +306,10 @@ class MemStoreFlusher implements FlushRequester { return null; } - private Region getBiggestMemStoreOfRegionReplica(SortedMap regionsBySize, - Set excludedRegions) { + private HRegion getBiggestMemStoreOfRegionReplica(SortedMap regionsBySize, + Set excludedRegions) { synchronized (regionsInQueue) { - for (Region region : regionsBySize.values()) { + for (HRegion region : regionsBySize.values()) { if (excludedRegions.contains(region)) { continue; } @@ -349,8 +348,8 @@ class MemStoreFlusher implements FlushRequester { } @Override - public void requestFlush(Region r, boolean forceFlushAllStores) { - ((HRegion)r).incrementFlushesQueuedCount(); + public void requestFlush(HRegion r, boolean forceFlushAllStores) { + r.incrementFlushesQueuedCount(); synchronized (regionsInQueue) { if (!regionsInQueue.containsKey(r)) { // This entry has no delay so it will be added at the top of the flush @@ -363,8 +362,8 @@ class MemStoreFlusher implements FlushRequester { } @Override - public void requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) { - ((HRegion)r).incrementFlushesQueuedCount(); + public void requestDelayedFlush(HRegion r, long delay, boolean forceFlushAllStores) { + r.incrementFlushesQueuedCount(); synchronized (regionsInQueue) { if (!regionsInQueue.containsKey(r)) { // This entry has some delay @@ -430,7 +429,7 @@ class MemStoreFlusher implements FlushRequester { * not flushed. */ private boolean flushRegion(final FlushRegionEntry fqe) { - Region region = fqe.region; + HRegion region = fqe.region; if (!region.getRegionInfo().isMetaRegion() && isTooManyStoreFiles(region)) { if (fqe.isMaximumWait(this.blockingWaitTime)) { @@ -479,7 +478,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 HRegion region, final boolean emergencyFlush, boolean forceFlushAllStores) { synchronized (this.regionsInQueue) { FlushRegionEntry fqe = this.regionsInQueue.remove(region); @@ -497,12 +496,11 @@ class MemStoreFlusher implements FlushRequester { FlushResult flushResult = region.flush(forceFlushAllStores); boolean shouldCompact = flushResult.isCompactionNeeded(); // We just want to check the size - boolean shouldSplit = ((HRegion)region).checkSplit() != null; + boolean shouldSplit = region.checkSplit() != null; if (shouldSplit) { this.server.compactSplitThread.requestSplit(region); } else if (shouldCompact) { - server.compactSplitThread.requestSystemCompaction((HRegion) region, - Thread.currentThread().getName()); + server.compactSplitThread.requestSystemCompaction(region, Thread.currentThread().getName()); } } catch (DroppedSnapshotException ex) { // Cache flush can fail in a few places. If it fails in a critical @@ -728,7 +726,7 @@ class MemStoreFlusher implements FlushRequester { * a while. */ static class FlushRegionEntry implements FlushQueueEntry { - private final Region region; + private final HRegion region; private final long createTime; private long whenToExpire; @@ -736,7 +734,7 @@ class MemStoreFlusher implements FlushRequester { private boolean forceFlushAllStores; - FlushRegionEntry(final Region r, boolean forceFlushAllStores) { + FlushRegionEntry(final HRegion 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 a7feb946830..515b1eae0ce 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 @@ -214,7 +214,7 @@ class MetricsRegionServerWrapperImpl @Override public long getNumOnlineRegions() { - Collection onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext(); + Collection onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext(); if (onlineRegionsLocalContext == null) { return 0; } @@ -754,7 +754,7 @@ class MetricsRegionServerWrapperImpl long tempMobScanCellsSize = 0; long tempBlockedRequestsCount = 0; int regionCount = 0; - for (Region r : regionServer.getOnlineRegionsLocalContext()) { + for (HRegion 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/OnlineRegions.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java index d389bdf461d..c9f1c47c42e 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 @@ -32,7 +32,7 @@ public interface OnlineRegions extends ImmutableOnlineRegions { * Add to online regions. * @param r */ - void addRegion(final Region r); + void addRegion(final HRegion r); /** * This method removes Region corresponding to hri from the Map of onlineRegions. @@ -41,5 +41,5 @@ public interface OnlineRegions extends ImmutableOnlineRegions { * @param destination Destination, if any, null otherwise. * @return True if we removed a region from online list. */ - boolean removeRegion(final Region r, ServerName destination); + boolean removeRegion(final HRegion r, ServerName destination); } 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 70b34751785..440b318371d 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 @@ -388,13 +388,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler, private final AtomicLong nextCallSeq = new AtomicLong(0); private final String scannerName; private final RegionScanner s; - private final Region r; + private final HRegion r; private final RpcCallback closeCallBack; private final RpcCallback shippedCallback; private byte[] rowOfLastPartialResult; private boolean needCursor; - public RegionScannerHolder(String scannerName, RegionScanner s, Region r, + public RegionScannerHolder(String scannerName, RegionScanner s, HRegion r, RpcCallback closeCallBack, RpcCallback shippedCallback, boolean needCursor) { this.scannerName = scannerName; this.s = s; @@ -432,7 +432,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; + HRegion region = null; try { region = regionServer.getRegion(s.getRegionInfo().getRegionName()); if (region != null && region.getCoprocessorHost() != null) { @@ -547,7 +547,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 HRegion region, final List actions, final CellScanner cellScanner, RegionActionResult.Builder builder) throws IOException { if (!region.getRegionInfo().isMetaTable()) { @@ -600,7 +600,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 HRegion region, final List actions, final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier, CompareOperator op, ByteArrayComparable comparator, RegionActionResult.Builder builder, ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException { @@ -656,7 +656,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 HRegion region, final OperationQuota quota, final MutationProto mutation, final CellScanner cellScanner, long nonceGroup, ActivePolicyEnforcement spaceQuota) throws IOException { @@ -707,7 +707,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @return the Result * @throws IOException */ - private Result increment(final Region region, final OperationQuota quota, + private Result increment(final HRegion region, final OperationQuota quota, final MutationProto mutation, final CellScanner cells, long nonceGroup, ActivePolicyEnforcement spaceQuota) throws IOException { @@ -763,7 +763,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 HRegion region, final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner, final RegionActionResult.Builder builder, List cellsToReturn, long nonceGroup, final RegionScannersCloseCallBack closeCallBack, RpcCallContext context, @@ -926,11 +926,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return cellsToReturn; } - private void checkCellSizeLimit(final Region region, final Mutation m) throws IOException { - if (!(region instanceof HRegion)) { - return; - } - HRegion r = (HRegion)region; + private void checkCellSizeLimit(final HRegion r, final Mutation m) throws IOException { if (r.maxCellSize > 0) { CellScanner cells = m.cellScanner(); while (cells.advance()) { @@ -953,7 +949,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 HRegion region, final OperationQuota quota, final List mutations, final CellScanner cells, ActivePolicyEnforcement spaceQuotaEnforcement) { Mutation[] mArray = new Mutation[mutations.size()]; @@ -1050,7 +1046,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * exceptionMessage if any * @throws IOException */ - private OperationStatus [] doReplayBatchOp(final Region region, + private OperationStatus [] doReplayBatchOp(final HRegion region, final List mutations, long replaySeqId) throws IOException { long before = EnvironmentEdgeManager.currentTime(); boolean batchContainsPuts = false, batchContainsDelete = false; @@ -1335,7 +1331,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return lastBlock; } - private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r, + private RegionScannerHolder addScanner(String scannerName, RegionScanner s, HRegion r, boolean needCursor) throws LeaseStillHeldException { Lease lease = regionServer.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod, new ScannerListener(scannerName)); @@ -1363,7 +1359,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * but failed to find the region */ @VisibleForTesting - public Region getRegion( + public HRegion getRegion( final RegionSpecifier regionSpecifier) throws IOException { ByteString value = regionSpecifier.getValue(); RegionSpecifierType type = regionSpecifier.getType(); @@ -1652,9 +1648,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 (HRegion region: onlineRegions.values()) { list.add(region.getRegionInfo()); } Collections.sort(list, RegionInfo.COMPARATOR); @@ -1671,7 +1667,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + HRegion region = getRegion(request.getRegion()); RegionInfo info = region.getRegionInfo(); byte[] bestSplitRow = null; if (request.hasBestSplitRow() && request.getBestSplitRow()) { @@ -1690,7 +1686,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder(); builder.setRegionInfo(ProtobufUtil.toRegionInfo(info)); if (request.hasCompactionState() && request.getCompactionState()) { - builder.setCompactionState(region.getCompactionState()); + builder.setCompactionState(ProtobufUtil.createCompactionState(region.getCompactionState())); } builder.setSplittable(region.isSplittable()); builder.setMergeable(region.isMergeable()); @@ -1709,7 +1705,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, public GetRegionLoadResponse getRegionLoad(RpcController controller, GetRegionLoadRequest request) throws ServiceException { - List regions; + List regions; if (request.hasTableName()) { TableName tableName = ProtobufUtil.toTableName(request.getTableName()); regions = regionServer.getRegions(tableName); @@ -1721,7 +1717,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder(); try { - for (Region region : regions) { + for (HRegion region : regions) { rLoads.add(regionServer.createRegionLoad(region, regionLoadBuilder, regionSpecifier)); } } catch (IOException e) { @@ -1797,7 +1793,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, final GetStoreFileRequest request) throws ServiceException { try { checkOpen(); - Region region = getRegion(request.getRegion()); + HRegion region = getRegion(request.getRegion()); requestCount.increment(); Set columnFamilies; if (request.getFamilyCount() == 0) { @@ -1902,7 +1898,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { String encodedName = region.getEncodedName(); byte[] encodedNameBytes = region.getEncodedNameAsBytes(); - final Region onlineRegion = regionServer.getRegion(encodedName); + final HRegion onlineRegion = regionServer.getRegion(encodedName); if (onlineRegion != null) { // The region is already online. This should not happen any more. String error = "Received OPEN for the region:" @@ -2026,7 +2022,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, checkOpen(); String encodedName = region.getEncodedName(); byte[] encodedNameBytes = region.getEncodedNameAsBytes(); - final Region onlineRegion = regionServer.getRegion(encodedName); + final HRegion onlineRegion = regionServer.getRegion(encodedName); if (onlineRegion != null) { LOG.info("Region already online. Skipping warming up " + region); @@ -2077,7 +2073,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return ReplicateWALEntryResponse.newBuilder().build(); } ByteString regionName = entries.get(0).getKey().getEncodedRegionName(); - Region region = regionServer.getRegionByEncodedName(regionName.toStringUtf8()); + HRegion region = regionServer.getRegionByEncodedName(regionName.toStringUtf8()); RegionCoprocessorHost coprocessorHost = ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo()) ? region.getCoprocessorHost() @@ -2133,7 +2129,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } //sync wal at the end because ASYNC_WAL is used above - WAL wal = getWAL(region); + WAL wal = region.getWAL(); if (wal != null) { wal.sync(); } @@ -2155,10 +2151,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } - WAL getWAL(Region region) { - return ((HRegion)region).getWAL(); - } - /** * Replicate WAL entries on the region server. * @@ -2258,7 +2250,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + HRegion region = getRegion(request.getRegion()); boolean bypass = false; boolean loaded = false; Map> map = null; @@ -2328,7 +2320,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + HRegion region = getRegion(request.getRegion()); String bulkToken = regionServer.secureBulkLoadManager.prepareBulkLoad(region, request); PrepareBulkLoadResponse.Builder builder = PrepareBulkLoadResponse.newBuilder(); @@ -2346,7 +2338,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + HRegion region = getRegion(request.getRegion()); regionServer.secureBulkLoadManager.cleanupBulkLoad(region, request); CleanupBulkLoadResponse response = CleanupBulkLoadResponse.newBuilder().build(); @@ -2362,7 +2354,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, try { checkOpen(); requestCount.increment(); - Region region = getRegion(request.getRegion()); + HRegion region = getRegion(request.getRegion()); com.google.protobuf.Message result = execServiceOnRegion(region, request.getCall()); CoprocessorServiceResponse.Builder builder = CoprocessorServiceResponse.newBuilder(); builder.setRegion(RequestConverter.buildRegionSpecifier( @@ -2377,7 +2369,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } - private com.google.protobuf.Message execServiceOnRegion(Region region, + private com.google.protobuf.Message execServiceOnRegion(HRegion region, final ClientProtos.CoprocessorServiceCall serviceCall) throws IOException { // ignore the passed in controller (from the serialized call) ServerRpcController execController = new ServerRpcController(); @@ -2401,7 +2393,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, requestCount.increment(); requestRowActionCount.increment(); rpcGetRequestCount.increment(); - Region region = getRegion(request.getRegion()); + HRegion region = getRegion(request.getRegion()); GetResponse.Builder builder = GetResponse.newBuilder(); ClientProtos.Get get = request.getGet(); @@ -2567,7 +2559,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, for (RegionAction regionAction : request.getRegionActionList()) { this.requestRowActionCount.add(regionAction.getActionCount()); OperationQuota quota; - Region region; + HRegion region; regionActionResultBuilder.clear(); RegionSpecifier regionSpecifier = regionAction.getRegion(); try { @@ -2702,7 +2694,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, requestCount.increment(); requestRowActionCount.increment(); rpcMutateRequestCount.increment(); - Region region = getRegion(request.getRegion()); + HRegion region = getRegion(request.getRegion()); MutateResponse.Builder builder = MutateResponse.newBuilder(); MutationProto mutation = request.getMutation(); if (!region.getRegionInfo().isMetaTable()) { @@ -2892,7 +2884,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, private RegionScannerHolder newRegionScanner(ScanRequest request, ScanResponse.Builder builder) throws IOException { - Region region = getRegion(request.getRegion()); + HRegion region = getRegion(request.getRegion()); ClientProtos.Scan protoScan = request.getScan(); boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand(); Scan scan = ProtobufUtil.toScan(protoScan); @@ -2992,7 +2984,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, long maxQuotaResultSize, int maxResults, int limitOfRows, List results, ScanResponse.Builder builder, MutableObject lastBlock, RpcCallContext context) throws IOException { - Region region = rsh.r; + HRegion region = rsh.r; RegionScanner scanner = rsh.s; long maxResultSize; if (scanner.getMaxResultSize() > 0) { @@ -3226,7 +3218,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } throw new ServiceException(e); } - Region region = rsh.r; + HRegion region = rsh.r; String scannerName = rsh.scannerName; Leases.Lease lease; try { @@ -3407,7 +3399,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } - private void closeScanner(Region region, RegionScanner scanner, String scannerName, + private void closeScanner(HRegion region, RegionScanner scanner, String scannerName, RpcCallContext context) throws IOException { if (region.getCoprocessorHost() != null) { if (region.getCoprocessorHost().preScannerClose(scanner)) { 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 e3ba2fa8da4..7d72b8846ca 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 @@ -22,13 +22,12 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.CompactionState; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Increment; @@ -41,34 +40,17 @@ import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.conf.ConfigurationObserver; -import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.filter.ByteArrayComparable; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.hbase.shaded.com.google.protobuf.Service; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall; - /** - * Regions store data for a certain region of a table. It stores all columns - * for each row. A given table consists of one or more Regions. - * - *

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

Locking at the Region level serves only one purpose: preventing the - * region from being closed (and consequently split) while other operations - * are ongoing. Each row level operation obtains both a row lock and a region - * read lock for the duration of the operation. While a scanner is being - * constructed, getScanner holds a read lock. If the scanner is successfully - * constructed, it holds a read lock until it is closed. A close takes out a - * write lock and consequently will block for ongoing operations and will block - * new operations from starting while the close is in progress. + * Region is a subset of HRegion with operations required for the {@link RegionCoprocessor + * Coprocessors}. The operations include ability to do mutations, requesting compaction, getting + * different counters/sizes, locking rows and getting access to {@linkplain Store}s. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving @@ -131,24 +113,11 @@ public interface Region extends ConfigurationObserver { */ 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 majorCompactionOnly Only consider HFile that are the result of major compaction @@ -162,19 +131,9 @@ public interface Region extends ConfigurationObserver { */ 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 @@ -188,24 +147,12 @@ public interface Region extends ConfigurationObserver { /** @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. It just accounts data size of cells added to * the memstores of this Region. Means size in bytes for key, value and tags within Cells. @@ -213,9 +160,6 @@ public interface Region extends ConfigurationObserver { */ 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(); @@ -231,12 +175,6 @@ public interface Region extends ConfigurationObserver { /** @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 @@ -259,6 +197,8 @@ public interface Region extends ConfigurationObserver { * the operation has completed, whether it succeeded or failed. * @throws IOException */ + // TODO Exposing this and closeRegionOperation() as we have getRowLock() exposed. + // Remove if we get rid of exposing getRowLock(). void startRegionOperation() throws IOException; /** @@ -308,7 +248,9 @@ public interface Region extends ConfigurationObserver { * * 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). - * + *

+ * The obtained locks should be released after use by {@link RowLock#release()} + *

* NOTE: the boolean passed here has changed. It used to be a boolean that * stated whether or not to wait on the lock. Now it is whether it an exclusive * lock is requested. @@ -319,50 +261,33 @@ public interface Region extends ConfigurationObserver { * @see #startRegionOperation() * @see #startRegionOperation(Operation) */ + // TODO this needs to be exposed as we have RowProcessor now. If RowProcessor is removed, we can + // remove this too.. RowLock getRowLock(byte[] row, boolean readLock) throws IOException; - /** - * If the given list of row locks is not null, releases all locks. - */ - void releaseRowLocks(List rowLocks); - /////////////////////////////////////////////////////////////////////////// // Region operations /** * Perform one or more append operations on a row. * @param append - * @param nonceGroup - * @param nonce * @return result of the operation * @throws IOException */ - Result append(Append append, long nonceGroup, long nonce) throws IOException; + Result append(Append append) 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) + OperationStatus[] batchMutate(Mutation[] mutations) 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 @@ -423,17 +348,6 @@ public interface Region extends ConfigurationObserver { */ 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}. @@ -467,12 +381,10 @@ public interface Region extends ConfigurationObserver { /** * 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; + Result increment(Increment increment) throws IOException; /** * Performs multiple mutations atomically on a single row. Currently @@ -496,6 +408,8 @@ public interface Region extends ConfigurationObserver { * rowsToLock is sorted in order to avoid deadlocks. * @throws IOException */ + // TODO Should not be exposing with params nonceGroup, nonce. Change when doing the jira for + // Changing processRowsWithLocks and RowProcessor void mutateRowsWithLocks(Collection mutations, Collection rowsToLock, long nonceGroup, long nonce) throws IOException; @@ -513,6 +427,8 @@ public interface Region extends ConfigurationObserver { * @param nonceGroup Optional nonce group of the operation (client Id) * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") */ + // TODO Should not be exposing with params nonceGroup, nonce. Change when doing the jira for + // Changing processRowsWithLocks and RowProcessor void processRowsWithLocks(RowProcessor processor, long nonceGroup, long nonce) throws IOException; @@ -525,6 +441,8 @@ public interface Region extends ConfigurationObserver { * @param nonceGroup Optional nonce group of the operation (client Id) * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") */ + // TODO Should not be exposing with params nonceGroup, nonce. Change when doing the jira for + // Changing processRowsWithLocks and RowProcessor void processRowsWithLocks(RowProcessor processor, long timeout, long nonceGroup, long nonce) throws IOException; @@ -535,216 +453,14 @@ public interface Region extends ConfigurationObserver { */ 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, boolean copyFile) - 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 Map from family to List of store file paths if successful, null if failed recoverably - * @throws IOException if failed unrecoverably. - */ - Map> bulkLoadHFiles(Collection> familyPaths, - boolean assignSeqId, BulkLoadListener bulkLoadListener) 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 assignSeqId - * @param bulkLoadListener Internal hooks enabling massaging/preparation of a - * file about to be bulk loaded - * @param copyFile always copy hfiles if true - * @return Map from family to List of store file paths if successful, null if failed recoverably - * @throws IOException if failed unrecoverably. - */ - Map> bulkLoadHFiles(Collection> familyPaths, - boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile) throws IOException; - - /////////////////////////////////////////////////////////////////////////// - // Coprocessors - /** @return the coprocessor host */ + // TODO To be removed by HBASE-18954 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) - */ - com.google.protobuf.Message execService(com.google.protobuf.RpcController controller, - CoprocessorServiceCall call) - throws IOException; - - /** - * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to - * be available for handling 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(com.google.protobuf.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. *

@@ -770,11 +486,4 @@ public interface Region extends ConfigurationObserver { */ void requestCompaction(byte[] family, String why, int priority, CompactionLifeCycleTracker tracker, User user) throws IOException; - - /** Wait for all current flushes and compactions of the region to complete */ - void waitForFlushesAndCompactions(); - - /** Wait for all current flushes of the region to complete - */ - void waitForFlushes(); } 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 035c8d135a0..c204b32b1ad 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 @@ -201,7 +201,7 @@ public class RegionCoprocessorHost /** The region server services */ RegionServerServices rsServices; /** The region */ - Region region; + HRegion region; /** * Constructor @@ -209,7 +209,7 @@ public class RegionCoprocessorHost * @param rsServices interface to available region server functionality * @param conf the configuration */ - public RegionCoprocessorHost(final Region region, + public RegionCoprocessorHost(final HRegion region, final RegionServerServices rsServices, final Configuration conf) { super(rsServices); this.conf = conf; 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 1779e5ccdb8..f9e93a18cb0 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 @@ -22,9 +22,11 @@ import java.security.PrivilegedAction; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext; import org.apache.hadoop.hbase.security.User; import org.apache.yetus.audience.InterfaceAudience; @@ -91,7 +93,8 @@ class RegionMergeRequest implements Runnable { // The parent region will be unassigned and the two new regions will be assigned. // hri_a and hri_b objects may not reflect the regions that will be created, those objectes // are created just to pass the information to the reportRegionStateTransition(). - if (!server.reportRegionStateTransition(TransitionCode.READY_TO_MERGE, merged, region_a, region_b)) { + if (!server.reportRegionStateTransition(new RegionStateTransitionContext( + TransitionCode.READY_TO_MERGE, HConstants.NO_SEQNUM, -1, merged, region_a, region_b))) { LOG.error("Unable to ask master to merge: " + region_a + ", " + region_b); } } 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 b21d55af134..d04f3824f21 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 @@ -85,15 +85,15 @@ public interface RegionServerServices * Context for postOpenDeployTasks(). */ class PostOpenDeployContext { - private final Region region; + private final HRegion region; private final long masterSystemTime; @InterfaceAudience.Private - public PostOpenDeployContext(Region region, long masterSystemTime) { + public PostOpenDeployContext(HRegion region, long masterSystemTime) { this.region = region; this.masterSystemTime = masterSystemTime; } - public Region getRegion() { + public HRegion getRegion() { return region; } public long getMasterSystemTime() { @@ -111,18 +111,6 @@ public interface RegionServerServices */ void postOpenDeployTasks(final PostOpenDeployContext context) throws KeeperException, IOException; - /** - * Tasks to perform after region open to complete deploy of region on - * regionserver - * - * @param r Region to open. - * @throws KeeperException - * @throws IOException - * @deprecated use {@link #postOpenDeployTasks(PostOpenDeployContext)} - */ - @Deprecated - void postOpenDeployTasks(final Region r) throws KeeperException, IOException; - class RegionStateTransitionContext { private final TransitionCode code; private final long openSeqNum; @@ -156,20 +144,6 @@ public interface RegionServerServices */ boolean reportRegionStateTransition(final RegionStateTransitionContext context); - /** - * Notify master that a handler requests to change a region state - * @deprecated use {@link #reportRegionStateTransition(RegionStateTransitionContext)} - */ - @Deprecated - boolean reportRegionStateTransition(TransitionCode code, long openSeqNum, RegionInfo... hris); - - /** - * Notify master that a handler requests to change a region state - * @deprecated use {@link #reportRegionStateTransition(RegionStateTransitionContext)} - */ - @Deprecated - boolean reportRegionStateTransition(TransitionCode code, RegionInfo... hris); - /** * Returns a reference to the region server's RPC server */ @@ -194,7 +168,7 @@ public interface RegionServerServices /** * @return set of recovering regions on the hosting region server */ - Map getRecoveringRegions(); + Map getRecoveringRegions(); /** * Only required for "old" log replay; if it's removed, remove this. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java index 296e98c6b8a..c358f6cc081 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 @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.ipc.RpcServer; -import org.apache.hadoop.hbase.regionserver.Region.BulkLoadListener; +import org.apache.hadoop.hbase.regionserver.HRegion.BulkLoadListener; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.token.FsDelegationToken; @@ -157,7 +157,7 @@ public class SecureBulkLoadManager { LOG.info("Cleaned up " + path + " successfully."); } - public Map> secureBulkLoadHFiles(final Region region, + public Map> secureBulkLoadHFiles(final HRegion region, final BulkLoadHFileRequest request) throws IOException { final List> familyPaths = new ArrayList<>(request.getFamilyPathCount()); for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) { 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 0227f5b44b1..4b1ae31233c 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 @@ -22,9 +22,11 @@ import java.security.PrivilegedAction; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; @@ -85,7 +87,8 @@ class SplitRequest implements Runnable { // The parent region will be unassigned and the two new regions will be assigned. // hri_a and hri_b objects may not reflect the regions that will be created, those objects // are created just to pass the information to the reportRegionStateTransition(). - if (!server.reportRegionStateTransition(TransitionCode.READY_TO_SPLIT, parent, hri_a, hri_b)) { + if (!server.reportRegionStateTransition(new RegionStateTransitionContext( + TransitionCode.READY_TO_SPLIT, HConstants.NO_SEQNUM, -1, parent, hri_a, hri_b))) { LOG.error("Unable to ask master to split " + parent.getRegionNameAsString()); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java index 8c5ba9fff35..f48ee9260e2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java @@ -22,6 +22,7 @@ import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -29,6 +30,7 @@ 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.RegionServerServices; +import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; @@ -117,7 +119,8 @@ public class CloseRegionHandler extends EventHandler { } this.rsServices.removeRegion(region, destination); - rsServices.reportRegionStateTransition(TransitionCode.CLOSED, regionInfo); + rsServices.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED, + HConstants.NO_SEQNUM, -1, regionInfo)); // Done! Region is closed on this RS LOG.debug("Closed " + region.getRegionInfo().getRegionNameAsString()); 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 147317c3b86..e664cd52a9f 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 @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerServices.PostOpenDeployContext; +import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext; import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.yetus.audience.InterfaceAudience; @@ -160,7 +162,8 @@ public class OpenRegionHandler extends EventHandler { cleanupFailedOpen(region); } } finally { - rsServices.reportRegionStateTransition(TransitionCode.FAILED_OPEN, regionInfo); + rsServices.reportRegionStateTransition(new RegionStateTransitionContext( + TransitionCode.FAILED_OPEN, HConstants.NO_SEQNUM, -1, regionInfo)); } } 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 21963d8618f..13ab8c86e49 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 @@ -31,8 +31,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.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.regionserver.Region.FlushResult; +import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult; import org.apache.hadoop.hbase.regionserver.Region.Operation; import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager.SnapshotSubprocedurePool; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; @@ -50,7 +49,7 @@ 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; @@ -60,7 +59,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure { 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; @@ -76,12 +75,12 @@ public class FlushSnapshotSubprocedure extends Subprocedure { * Callable for adding files to snapshot manifest working dir. Ready for multithreading. */ public static class RegionSnapshotTask implements Callable { - private Region region; + private HRegion region; private boolean skipFlush; private ForeignExceptionDispatcher monitor; private SnapshotDescription snapshotDesc; - public RegionSnapshotTask(Region region, SnapshotDescription snapshotDesc, + public RegionSnapshotTask(HRegion region, SnapshotDescription snapshotDesc, boolean skipFlush, ForeignExceptionDispatcher monitor) { this.region = region; this.skipFlush = skipFlush; @@ -111,7 +110,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure { } else { LOG.debug("Flush Snapshotting region " + region.toString() + " started..."); boolean succeeded = false; - long readPt = region.getReadpoint(IsolationLevel.READ_COMMITTED); + long readPt = region.getReadPoint(IsolationLevel.READ_COMMITTED); for (int i = 0; i < MAX_RETRIES; i++) { FlushResult res = region.flush(true); if (res.getResult() == FlushResult.Result.CANNOT_FLUSH) { @@ -132,7 +131,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure { throw new IOException("Unable to complete flush after " + MAX_RETRIES + " attempts"); } } - ((HRegion)region).addRegionToSnapshot(snapshotDesc, monitor); + region.addRegionToSnapshot(snapshotDesc, monitor); if (skipFlush) { LOG.debug("... SkipFlush Snapshotting region " + region.toString() + " completed."); } else { @@ -162,7 +161,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure { } // Add all hfiles already existing in region. - for (Region region : regions) { + for (HRegion region : regions) { // submit one task per region for parallelize by region. taskManager.submitTask(new RegionSnapshotTask(region, snapshot, snapshotSkipFlush, monitor)); 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 775d63f9030..a4b43879975 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 @@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager; import org.apache.hadoop.hbase.procedure.Subprocedure; import org.apache.hadoop.hbase.procedure.SubprocedureFactory; import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; @@ -162,7 +163,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 +223,13 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager { * the given snapshot. * @throws IOException */ - private List getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException { - List onlineRegions = rss.getRegions(TableName.valueOf(snapshot.getTable())); - Iterator iterator = onlineRegions.iterator(); + private List getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException { + List onlineRegions = (List) rss + .getRegions(TableName.valueOf(snapshot.getTable())); + Iterator iterator = onlineRegions.iterator(); // remove the non-default regions while (iterator.hasNext()) { - Region r = iterator.next(); + HRegion r = iterator.next(); if (!RegionReplicaUtil.isDefaultReplica(r.getRegionInfo())) { iterator.remove(); } 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 26620c12561..8a5265da716 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 @@ -307,7 +307,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService private boolean mutateLabelsRegion(List mutations, OperationStatus[] finalOpStatus) throws IOException { OperationStatus[] opStatus = this.labelsRegion.batchMutate(mutations - .toArray(new Mutation[mutations.size()]), HConstants.NO_NONCE, HConstants.NO_NONCE); + .toArray(new Mutation[mutations.size()])); int i = 0; boolean updateZk = false; for (OperationStatus status : opStatus) { 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 59ad6de4e97..30ba66c1c4d 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 @@ -3947,8 +3947,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 (HRegion r: hrs) { assertTrue("Region should not be double assigned", r.getRegionInfo().getRegionId() != hri.getRegionId()); } 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 59a66ec2a60..fe4119a9907 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 @@ -615,7 +615,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(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { r.flush(true); } } @@ -628,7 +628,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(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { if(r.getTableDescriptor().getTableName().equals(tableName)) { r.flush(true); } @@ -643,7 +643,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(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { r.compact(major); } } @@ -656,7 +656,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(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) { if(r.getTableDescriptor().getTableName().equals(tableName)) { r.compact(major); } 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 e453be2854d..4d2a8cc3f8c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager; import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager; import org.apache.hadoop.hbase.regionserver.FlushRequester; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HeapMemoryManager; import org.apache.hadoop.hbase.regionserver.Leases; import org.apache.hadoop.hbase.regionserver.MetricsRegionServer; @@ -56,7 +57,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; import com.google.protobuf.Service; @@ -96,7 +96,7 @@ public class MockRegionServerServices implements RegionServerServices { } @Override - public boolean removeRegion(Region r, ServerName destination) { + public boolean removeRegion(HRegion r, ServerName destination) { return this.regions.remove(r.getRegionInfo().getEncodedName()) != null; } @@ -121,15 +121,10 @@ public class MockRegionServerServices implements RegionServerServices { } @Override - public void addRegion(Region r) { + public void addRegion(HRegion r) { this.regions.put(r.getRegionInfo().getEncodedName(), r); } - @Override - public void postOpenDeployTasks(Region r) throws KeeperException, IOException { - addRegion(r); - } - @Override public void postOpenDeployTasks(PostOpenDeployContext context) throws KeeperException, IOException { @@ -269,7 +264,7 @@ public class MockRegionServerServices implements RegionServerServices { } @Override - public Map getRecoveringRegions() { + public Map getRecoveringRegions() { // TODO Auto-generated method stub return null; } @@ -280,18 +275,6 @@ public class MockRegionServerServices implements RegionServerServices { return null; } - @Override - public boolean reportRegionStateTransition(TransitionCode code, long openSeqNum, - RegionInfo... hris) { - return false; - } - - @Override - public boolean reportRegionStateTransition(TransitionCode code, - RegionInfo... hris) { - return false; - } - @Override public boolean reportRegionStateTransition(RegionStateTransitionContext context) { return false; 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 202ea4b8ad6..34e8c3c00d8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java @@ -31,8 +31,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionLocator; 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.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; @@ -109,7 +109,7 @@ public class TestGlobalMemStoreSize { for (RegionInfo regionInfo : ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) { - Region r = server.getRegion(regionInfo.getEncodedName()); + HRegion r = server.getRegion(regionInfo.getEncodedName()); flush(r, server); } LOG.info("Post flush on " + server.getServerName()); @@ -125,7 +125,7 @@ public class TestGlobalMemStoreSize { // our test was running.... for (RegionInfo regionInfo : ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) { - Region r = server.getRegion(regionInfo.getEncodedName()); + HRegion r = server.getRegion(regionInfo.getEncodedName()); long l = r.getMemStoreSize(); if (l > 0) { // Only meta could have edits at this stage. Give it another flush @@ -150,7 +150,7 @@ public class TestGlobalMemStoreSize { * @param server * @throws IOException */ - private void flush(final Region r, final HRegionServer server) + private void flush(final HRegion 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/backup/TestHFileArchiving.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java index 327fc89a3f7..c95f7b33e65 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 @@ -230,7 +230,7 @@ public class TestHFileArchiving { List servingRegions = UTIL.getHBaseCluster().getRegions(tableName); // make sure we only have 1 region serving this table assertEquals(1, servingRegions.size()); - Region region = servingRegions.get(0); + HRegion region = servingRegions.get(0); // get the parent RS and monitor HRegionServer hrs = UTIL.getRSForFirstRegionInTable(tableName); @@ -241,7 +241,7 @@ public class TestHFileArchiving { UTIL.loadRegion(region, TEST_FAM); // get the hfiles in the region - List regions = hrs.getRegions(tableName); + List regions = hrs.getRegions(tableName); assertEquals("More that 1 region for test table.", 1, regions.size()); region = regions.get(0); @@ -309,7 +309,7 @@ public class TestHFileArchiving { List servingRegions = UTIL.getHBaseCluster().getRegions(tableName); // make sure we only have 1 region serving this table assertEquals(1, servingRegions.size()); - Region region = servingRegions.get(0); + HRegion region = servingRegions.get(0); // get the parent RS and monitor HRegionServer hrs = UTIL.getRSForFirstRegionInTable(tableName); @@ -320,7 +320,7 @@ public class TestHFileArchiving { UTIL.loadRegion(region, TEST_FAM); // get the hfiles in the region - List regions = hrs.getRegions(tableName); + List regions = hrs.getRegions(tableName); 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 3e8d42e5434..20cb5133bf1 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 @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import java.io.IOException; import java.util.ArrayList; @@ -48,7 +47,6 @@ 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.HStore; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -179,9 +177,9 @@ public class TestZooKeeperTableArchiveClient { // create the region ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM); HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); - List regions = new ArrayList<>(); + List regions = new ArrayList<>(); regions.add(region); - when(rss.getRegions()).thenReturn(regions); + Mockito.doReturn(regions).when(rss).getRegions(); final CompactedHFilesDischarger compactionCleaner = new CompactedHFilesDischarger(100, stop, rss, false); loadFlushAndCompact(region, TEST_FAM); @@ -232,9 +230,9 @@ public class TestZooKeeperTableArchiveClient { // create the region ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM); HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); - List regions = new ArrayList<>(); + List regions = new ArrayList<>(); regions.add(region); - when(rss.getRegions()).thenReturn(regions); + Mockito.doReturn(regions).when(rss).getRegions(); final CompactedHFilesDischarger compactionCleaner = new CompactedHFilesDischarger(100, stop, rss, false); loadFlushAndCompact(region, TEST_FAM); @@ -244,7 +242,7 @@ public class TestZooKeeperTableArchiveClient { HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd); regions = new ArrayList<>(); regions.add(otherRegion); - when(rss.getRegions()).thenReturn(regions); + Mockito.doReturn(regions).when(rss).getRegions(); final CompactedHFilesDischarger compactionCleaner1 = new CompactedHFilesDischarger(100, stop, rss, false); loadFlushAndCompact(otherRegion, TEST_FAM); @@ -422,7 +420,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(HRegion 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 30cd8bf6832..4ece5c0ad52 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 @@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -472,7 +473,7 @@ public class TestAdmin2 { + AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files"); // flush all regions - for (Region r : regionServer.getOnlineRegionsLocalContext()) { + for (HRegion r : regionServer.getOnlineRegionsLocalContext()) { r.flush(true); } admin.rollWALWriter(regionServer.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java index 6307210dc62..c173a7f1bef 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java @@ -41,8 +41,8 @@ import org.apache.hadoop.hbase.RegionLoad; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -140,7 +140,7 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase { + AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files"); // flush all regions - for (Region r : regionServer.getOnlineRegionsLocalContext()) { + for (HRegion r : regionServer.getOnlineRegionsLocalContext()) { r.flush(true); } admin.rollWALWriter(regionServer.getServerName()).join(); 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 400e109e6b1..d887e7bd56c 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 @@ -110,7 +110,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreScanner; @@ -4535,7 +4534,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 = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); + HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); Put put1 = new Put(firstRow); Put put2 = new Put(secondRow); Put put3 = new Put(thirdRow); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java index ca0a5ea06e5..c3772dbe15d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java @@ -963,7 +963,7 @@ public class TestFromClientSide3 { private static Region find(final TableName tableName) throws IOException, InterruptedException { HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName); - List regions = rs.getRegions(tableName); + List regions = rs.getRegions(tableName); assertEquals(1, regions.size()); return regions.get(0); } 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 1a674575229..e2bdaf42dcf 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 @@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; @@ -1347,7 +1348,7 @@ public class TestHCM { assertTrue(!destServerName.equals(metaServerName)); //find another row in the cur server that is less than ROW_X - List regions = curServer.getRegions(TABLE_NAME3); + List regions = curServer.getRegions(TABLE_NAME3); byte[] otherRow = null; for (Region region : regions) { if (!region.getRegionInfo().getEncodedName().equals(toMove.getRegionInfo().getEncodedName()) 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 3b128453959..e7aa60f437a 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,6 @@ import org.apache.hadoop.hbase.shaded.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.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -248,11 +247,6 @@ public class TestMetaCache { exceptions.throwOnScan(this, request); return super.scan(controller, request); } - - public Region getRegion( - final HBaseProtos.RegionSpecifier regionSpecifier) throws IOException { - return super.getRegion(regionSpecifier); - } } public static abstract class ExceptionInjector { 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 ecb3f244555..9fa9aa8d70a 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 @@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.regionserver.ChunkCreator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; @@ -289,7 +288,7 @@ public class TestCoprocessorInterface { byte [][] families = { fam1, fam2, fam3 }; Configuration hc = initConfig(); - Region region = initHRegion(tableName, name.getMethodName(), hc, new Class[]{}, families); + HRegion region = initHRegion(tableName, name.getMethodName(), hc, new Class[]{}, families); for (int i = 0; i < 3; i++) { HBaseTestCase.addContent(region, fam3); @@ -351,7 +350,7 @@ public class TestCoprocessorInterface { byte [][] families = { fam1, fam2, fam3 }; Configuration hc = initConfig(); - Region region = initHRegion(tableName, name.getMethodName(), hc, + HRegion region = initHRegion(tableName, name.getMethodName(), hc, new Class[]{CoprocessorImpl.class}, families); for (int i = 0; i < 3; i++) { HBaseTestCase.addContent(region, fam3); @@ -378,10 +377,10 @@ public class TestCoprocessorInterface { assertTrue(((CoprocessorImpl)c).wasCompacted()); } - Region reopenRegion(final Region closedRegion, Class ... implClasses) + HRegion reopenRegion(final HRegion closedRegion, Class ... implClasses) throws IOException { //RegionInfo info = new RegionInfo(tableName, null, null, false); - Region r = HRegion.openHRegion(closedRegion, null); + HRegion r = HRegion.openHRegion(closedRegion, null); // this following piece is a hack. currently a coprocessorHost // is secretly loaded at OpenRegionHandler. we don't really @@ -389,7 +388,7 @@ public class TestCoprocessorInterface { // and set it to region. Configuration conf = TEST_UTIL.getConfiguration(); RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf); - ((HRegion)r).setCoprocessorHost(host); + r.setCoprocessorHost(host); for (Class implClass : implClasses) { host.load((Class) implClass, Coprocessor.PRIORITY_USER, conf); @@ -405,7 +404,7 @@ public class TestCoprocessorInterface { return r; } - Region initHRegion (TableName tableName, String callingMethod, + HRegion initHRegion (TableName tableName, String callingMethod, Configuration conf, Class [] implClasses, byte [][] families) throws IOException { HTableDescriptor htd = new HTableDescriptor(tableName); @@ -419,11 +418,11 @@ public class TestCoprocessorInterface { .setSplit(false) .build(); Path path = new Path(DIR + callingMethod); - Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); + HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd); // this following piece is a hack. RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf); - ((HRegion)r).setCoprocessorHost(host); + r.setCoprocessorHost(host); for (Class implClass : implClasses) { host.load((Class) implClass, Coprocessor.PRIORITY_USER, conf); 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 0a95f4161d7..6099381db0a 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 @@ -186,7 +186,7 @@ public class TestRegionObserverScannerOpenHook { } } - Region initHRegion(byte[] tableName, String callingMethod, Configuration conf, + HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf, byte[]... families) throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName)); for (byte[] family : families) { @@ -239,7 +239,7 @@ public class TestRegionObserverScannerOpenHook { byte[][] FAMILIES = new byte[][] { A }; Configuration conf = HBaseConfiguration.create(); - Region region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES); + HRegion 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); @@ -323,7 +323,7 @@ public class TestRegionObserverScannerOpenHook { table.put(put); HRegionServer rs = UTIL.getRSForFirstRegionInTable(desc.getTableName()); - List regions = rs.getRegions(desc.getTableName()); + List regions = rs.getRegions(desc.getTableName()); assertEquals("More than 1 region serving test table with 1 row", 1, regions.size()); Region region = regions.get(0); admin.flushRegion(region.getRegionInfo().getRegionName()); 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 2624d952893..c68921eba5c 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 @@ -71,7 +71,7 @@ import org.junit.rules.TestName; @Category({FilterTests.class, SmallTests.class}) public class TestFilter { private final static Log LOG = LogFactory.getLog(TestFilter.class); - private Region region; + private HRegion region; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @Rule @@ -1499,7 +1499,7 @@ public class TestFilter { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); htd.addFamily(new HColumnDescriptor(family)); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - Region testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), + HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd); for(int i=0; i<5; i++) { @@ -2060,7 +2060,7 @@ public class TestFilter { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); htd.addFamily(new HColumnDescriptor(FAMILIES[0])); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); - Region testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), + HRegion 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 159769ee985..b6bc2f132d4 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,6 @@ 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.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -61,7 +60,7 @@ public class TestInvocationRecordFilter { private static final String VALUE_PREFIX = "value"; private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private Region region; + private HRegion 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 706c4630111..a8c45dd1a94 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 @@ -292,7 +292,7 @@ public class TestBlockReorder { int nbTest = 0; while (nbTest < 10) { - final List regions = targetRs.getRegions(h.getName()); + final List regions = targetRs.getRegions(h.getName()); final CountDownLatch latch = new CountDownLatch(regions.size()); // listen for successful log rolls final WALActionsListener listener = new WALActionsListener.Base() { @@ -301,8 +301,8 @@ public class TestBlockReorder { latch.countDown(); } }; - for (Region region : regions) { - ((HRegion)region).getWAL().registerWALActionsListener(listener); + for (HRegion region : regions) { + region.getWAL().registerWALActionsListener(listener); } htu.getAdmin().rollWALWriter(targetRs.getServerName()); 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 efdf76584a1..5c1baca361d 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,6 +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.HRegion; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -114,7 +115,7 @@ public class TestEncodedSeekers { setBlocksize(BLOCK_SIZE). setBloomFilterType(BloomType.NONE). setCompressTags(compressTags); - Region region = testUtil.createTestRegion(TABLE_NAME, hcd); + HRegion region = testUtil.createTestRegion(TABLE_NAME, hcd); //write the data, but leave some in the memstore doPuts(region); @@ -138,7 +139,7 @@ public class TestEncodedSeekers { } - private void doPuts(Region region) throws IOException{ + private void doPuts(HRegion 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(); 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 6b1389924b6..273f82d8047 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,6 +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.HRegion; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.testclassification.IOTests; @@ -65,7 +66,7 @@ public class TestPrefixTree { private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); - private Region region; + private HRegion 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 3bf189df405..e0d2a9be8b0 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.HRegion; 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 HRegion 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 950beec05d7..a86a5512e6b 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 @@ -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, + HRegion 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 e94859a17b2..9983e1d6169 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 @@ -127,7 +127,7 @@ public class TestForceCacheImportantBlocks { else assertTrue(stats.getMissCount() > missCount); } - private void writeTestData(Region region) throws IOException { + private void writeTestData(HRegion 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 c834fca549a..b1ae85513f2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java @@ -36,6 +36,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.regionserver.BloomType; +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.testclassification.IOTests; @@ -99,7 +100,7 @@ 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, + HRegion 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 4af48ce92d0..459deeb5e4f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java @@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Region; 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, + HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, htd); long ts = EnvironmentEdgeManager.currentTime(); @@ -150,7 +150,7 @@ public class TestScannerSelectionUsingTTL { // Exercise both compaction codepaths. if (explicitCompaction) { - HStore store = (HStore)region.getStore(FAMILY_BYTES); + HStore store = region.getStore(FAMILY_BYTES); store.compactRecentForTestingAssumingDefaultPolicy(totalNumFiles); } else { region.compact(false); 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 f77706727c0..ca5a83ba758 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 @@ -123,7 +123,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanReques import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; /** * A mock RegionServer implementation. @@ -264,12 +263,12 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public void addRegion(Region r) { + public void addRegion(HRegion r) { // TODO Auto-generated method stub } @Override - public boolean removeRegion(Region r, ServerName destination) { + public boolean removeRegion(HRegion r, ServerName destination) { // TODO Auto-generated method stub return false; } @@ -332,11 +331,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { return null; } - @Override - public void postOpenDeployTasks(Region r) throws KeeperException, IOException { - // TODO Auto-generated method stub - } - @Override public void postOpenDeployTasks(PostOpenDeployContext context) throws KeeperException, IOException { @@ -592,7 +586,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { } @Override - public Map getRecoveringRegions() { + public Map getRecoveringRegions() { // TODO Auto-generated method stub return null; } @@ -608,17 +602,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { return null; } - @Override - public boolean reportRegionStateTransition(TransitionCode code, RegionInfo... hris) { - return false; - } - - @Override - public boolean reportRegionStateTransition(TransitionCode code, long openSeqNum, - RegionInfo... hris) { - return false; - } - @Override public boolean reportRegionStateTransition(RegionStateTransitionContext context) { return false; 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 c1eaef7d9c8..7220ff29ed1 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 @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.master.RegionState.State; 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.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -225,9 +224,9 @@ public class TestMasterFailover { // region server should expire (how it can be verified?) MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(), rs.getServerName(), State.OPENING); - Region meta = rs.getRegion(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); + HRegion meta = rs.getRegion(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); rs.removeRegion(meta, null); - ((HRegion)meta).close(); + meta.close(); log("Aborting master"); activeMaster.abort("test-kill"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index c0464a4c53f..5dc55f2b17b 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,6 +57,7 @@ import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper; import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer; import org.apache.hadoop.hbase.favored.FavoredNodesPlan; import org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -191,10 +192,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 (HRegion r : regs) { if (r.getRegionInfo().getTable().getNamespaceAsString() .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { isNamespaceServer = true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java index 2f22eebfda4..d6f559f17dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.master.LoadBalancer; import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -544,7 +545,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase { private void compactTable(TableName tableName) throws IOException { for(JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) { - for(Region region : t.getRegionServer().getRegions(tableName)) { + for(HRegion region : t.getRegionServer().getRegions(tableName)) { region.compact(true); } } 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 010f57a9402..0b49b5ef868 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 @@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; 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.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -62,7 +62,7 @@ public class TestRegionLocationFinder { for (int i = 0; i < ServerNum; i++) { HRegionServer server = cluster.getRegionServer(i); - for (Region region : server.getRegions(tableName)) { + for (HRegion region : server.getRegions(tableName)) { region.flush(true); } } @@ -83,7 +83,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.getRegions(tableName)) { + for (HRegion region : server.getRegions(tableName)) { // get region's hdfs block distribution by region and RegionLocationFinder, // they should have same result HDFSBlocksDistribution blocksDistribution1 = region.getHDFSBlocksDistribution(); @@ -122,7 +122,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.getRegions(tableName)) { + for (HRegion region : server.getRegions(tableName)) { List servers = finder.getTopBlockLocations(region .getRegionInfo()); // test table may have empty region @@ -147,12 +147,12 @@ public class TestRegionLocationFinder { finder.getCache().invalidateAll(); for (int i = 0; i < ServerNum; i++) { HRegionServer server = cluster.getRegionServer(i); - List regions = server.getRegions(tableName); + List regions = server.getRegions(tableName); if (regions.size() <= 0) { continue; } List regionInfos = new ArrayList<>(regions.size()); - for (Region region : regions) { + for (HRegion region : regions) { regionInfos.add(region.getRegionInfo()); } finder.refreshAndWait(regionInfos); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java index 61ac2fa115a..7b75e5bbc0a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionsOnMasterOptions.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.LoadBalancer; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.JVMClusterUtil; @@ -150,7 +151,7 @@ public class TestRegionsOnMasterOptions { try { Table t = TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY, REGIONS); LOG.info("Server: " + cluster.getMaster().getServerManager().getOnlineServersList()); - List regions = cluster.getMaster().getRegions(); + List regions = cluster.getMaster().getRegions(); int mActualCount = regions.size(); if (masterCount == 0 || masterCount == SYSTEM_REGIONS) { // 0 means no regions on master. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java index 719b5e1565f..ebd24361a6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileSystemUtilizationChore.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -63,7 +64,7 @@ public class TestFileSystemUtilizationChore { .reportRegionSizesForQuotas((Map) any(Map.class)); final Region region = mockRegionWithSize(regionSizes); - when(rs.getRegions()).thenReturn(Arrays.asList(region)); + Mockito.doReturn(Arrays.asList(region)).when(rs).getRegions(); chore.chore(); } @@ -80,7 +81,7 @@ public class TestFileSystemUtilizationChore { .reportRegionSizesForQuotas((Map) any(Map.class)); final Region region = mockRegionWithSize(regionSizes); - when(rs.getRegions()).thenReturn(Arrays.asList(region)); + Mockito.doReturn(Arrays.asList(region)).when(rs).getRegions(); chore.chore(); } @@ -106,7 +107,7 @@ public class TestFileSystemUtilizationChore { final Region r1 = mockRegionWithSize(r1Sizes); final Region r2 = mockRegionWithSize(r2Sizes); final Region r3 = mockRegionWithSize(r3Sizes); - when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2, r3)); + Mockito.doReturn(Arrays.asList(r1, r2, r3)).when(rs).getRegions(); chore.chore(); } @@ -172,7 +173,7 @@ public class TestFileSystemUtilizationChore { final Region r1 = mockRegionWithSize(Arrays.asList(1024L, 2048L)); final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L)); final Region r3 = mockRegionWithSize(Arrays.asList(10L * 1024L * 1024L)); - when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2, r3, lr1, lr2)); + Mockito.doReturn(Arrays.asList(r1, r2, r3, lr1, lr2)).when(rs).getRegions(); chore.chore(); } @@ -205,7 +206,7 @@ public class TestFileSystemUtilizationChore { final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L)); final Region r3 = mockRegionWithSize(Arrays.asList(10L * 1024L * 1024L)); // lr2 is no longer online, so it should be ignored - when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2, r3, lr1)); + Mockito.doReturn(Arrays.asList(r1, r2, r3, lr1)).when(rs).getRegions(); chore.chore(); } @@ -228,7 +229,7 @@ public class TestFileSystemUtilizationChore { final Region r1 = mockRegionWithSize(r1Sizes); final Region r2 = mockSplitParentRegionWithSize(r2Sizes); - when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2)); + Mockito.doReturn(Arrays.asList(r1, r2)).when(rs).getRegions(); chore.chore(); } @@ -250,7 +251,7 @@ public class TestFileSystemUtilizationChore { final Region r1 = mockRegionWithSize(r1Sizes); final Region r2 = mockRegionReplicaWithSize(r2Sizes); - when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2)); + Mockito.doReturn(Arrays.asList(r1, r2)).when(rs).getRegions(); chore.chore(); } @@ -277,7 +278,7 @@ public class TestFileSystemUtilizationChore { final Region r1 = mockRegionWithHFileLinks(r1StoreFileSizes, r1HFileSizes); final Region r2 = mockRegionWithHFileLinks(r2StoreFileSizes, r2HFileSizes); - when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2)); + Mockito.doReturn(Arrays.asList(r1, r2)).when(rs).getRegions(); chore.chore(); } 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 8f34b194ce8..3865cb31795 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 @@ -270,12 +270,12 @@ public class RegionAsTable implements Table { @Override public Result append(Append append) throws IOException { - return this.region.append(append, HConstants.NO_NONCE, HConstants.NO_NONCE); + return this.region.append(append); } @Override public Result increment(Increment increment) throws IOException { - return this.region.increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE); + return this.region.increment(increment); } @Override 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 674c3e8927e..3aed91c3a28 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 @@ -287,7 +287,7 @@ public class TestAtomicOperation { inc.addColumn(fam1, qual2, amount*2); inc.addColumn(fam2, qual3, amount*3); inc.setDurability(Durability.ASYNC_WAL); - Result result = region.increment(inc, HConstants.NO_NONCE, HConstants.NO_NONCE); + Result result = region.increment(inc); if (result != null) { assertEquals(Bytes.toLong(result.getValue(fam1, qual1))*2, Bytes.toLong(result.getValue(fam1, qual2))); @@ -555,13 +555,13 @@ public class TestAtomicOperation { } public static class AtomicOperation extends Thread { - protected final Region region; + protected final HRegion 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(HRegion region, int numOps, AtomicLong timeStamps, AtomicInteger failures) { this.region = region; this.numOps = numOps; @@ -633,7 +633,7 @@ public class TestAtomicOperation { put.addColumn(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50")); puts[0] = put; testStep = TestStep.PUT_STARTED; - region.batchMutate(puts, HConstants.NO_NONCE, HConstants.NO_NONCE); + region.batchMutate(puts); } } 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 59c256af18f..5c4fb6037b7 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; + HRegion 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 HRegion 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); + HRegion 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 c28e48ba453..06cbf7a51d8 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()); + HRegion 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 5cfa17d3a7e..4a26f76cb0a 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); + HRegion 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); + HRegion 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/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java index d54b58d39c5..95dec3dfa5a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java @@ -51,6 +51,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.mockito.Mockito; /** * Tests a race condition between archiving of compacted files in CompactedHFilesDischarger chore @@ -90,9 +91,9 @@ public class TestCompactionArchiveConcurrentClose { HRegionInfo info = new HRegionInfo(tableName, null, null, false); HRegion region = initHRegion(htd, info); RegionServerServices rss = mock(RegionServerServices.class); - List regions = new ArrayList<>(); + List regions = new ArrayList<>(); regions.add(region); - when(rss.getRegions()).thenReturn(regions); + Mockito.doReturn(regions).when(rss).getRegions(); // Create the cleaner object CompactedHFilesDischarger cleaner = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java index 8afe85e3fdf..0a47762fa9d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java @@ -27,7 +27,6 @@ import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; @@ -58,6 +57,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.mockito.Mockito; /** * Tests that archiving compacted files behaves correctly when encountering exceptions. @@ -99,9 +99,9 @@ public class TestCompactionArchiveIOException { HRegionInfo info = new HRegionInfo(tableName, null, null, false); final HRegion region = initHRegion(htd, info); RegionServerServices rss = mock(RegionServerServices.class); - List regions = new ArrayList<>(); + List regions = new ArrayList<>(); regions.add(region); - when(rss.getRegions()).thenReturn(regions); + Mockito.doReturn(regions).when(rss).getRegions(); // Create the cleaner object final CompactedHFilesDischarger cleaner = 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 ac52ea4fa49..b30a88459ac 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 @@ -138,7 +138,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.getRegions(table); + List regions = rs.getRegions(table); int countBefore = countStoreFilesInFamilies(regions, families); int countBeforeSingleFamily = countStoreFilesInFamily(regions, family); assertTrue(countBefore > 0); // there should be some data files @@ -206,13 +206,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 (HRegion region: regions) { count += region.getStoreFileList(families).size(); } return count; 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 6b01256aa67..9c87ebe43f7 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 @@ -83,7 +83,7 @@ public class TestGetClosestAtOrBefore { TableDescriptorBuilder metaBuilder = UTIL.getMetaTableDescriptorBuilder() .setMemStoreFlushSize(64 * 1024 * 1024); - Region mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO, + HRegion mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO, rootdir, this.conf, metaBuilder.build()); try { // Write rows for three tables 'A', 'B', and 'C'. @@ -195,7 +195,7 @@ public class TestGetClosestAtOrBefore { */ @Test public void testGetClosestRowBefore3() throws IOException{ - Region region = null; + HRegion region = null; byte [] c0 = UTIL.COLUMNS[0]; byte [] c1 = UTIL.COLUMNS[1]; try { @@ -306,7 +306,7 @@ public class TestGetClosestAtOrBefore { /** For HBASE-694 */ @Test public void testGetClosestRowBefore2() throws IOException{ - Region region = null; + HRegion 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 a7793f6045e..24e42bbadc0 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 @@ -5884,7 +5884,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 2b52e33d23c..3665ed8dbd0 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 @@ -87,7 +87,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; - +import org.mockito.Mockito; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @@ -180,9 +180,9 @@ 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.getRegions()).thenReturn(regions); + Mockito.doReturn(regions).when(rss).getRegions(); primaryRegion = HRegion.openHRegion(rootDir, primaryHri, htd, walPrimary, CONF, rss, null); secondaryRegion = HRegion.openHRegion(secondaryHri, htd, null, CONF, rss, null); @@ -1393,9 +1393,9 @@ public class TestHRegionReplayEvents { // Test case 3: compact primary files primaryRegion.compactStores(); - List regions = new ArrayList<>(); + List regions = new ArrayList<>(); regions.add(primaryRegion); - when(rss.getRegions()).thenReturn(regions); + Mockito.doReturn(regions).when(rss).getRegions(); CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, rss, false); cleaner.chore(); secondaryRegion.refreshStoreFiles(); 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 ba0d309bf7b..d4f1dfbe60e 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 @@ -804,12 +804,12 @@ public class TestHeapMemoryManager { } @Override - public void requestFlush(Region region, boolean forceFlushAllStores) { + public void requestFlush(HRegion region, boolean forceFlushAllStores) { this.listener.flushRequested(flushType, region); } @Override - public void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores) { + public void requestDelayedFlush(HRegion 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 7edcf547674..aabfa16433d 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 @@ -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); + HRegion region = hbu.createLocalHRegion(htd, null, null); long ts = EnvironmentEdgeManager.currentTime(); Put p = new Put(T1, ts); 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 323564e3d1a..f443705e1ec 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 @@ -388,17 +388,17 @@ public class TestMajorCompaction { return count; } - private void createStoreFile(final Region region) throws IOException { + private void createStoreFile(final HRegion region) throws IOException { createStoreFile(region, Bytes.toString(COLUMN_FAMILY)); } - private void createStoreFile(final Region region, String family) throws IOException { + private void createStoreFile(final HRegion 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 HRegion 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 e8d60e67c7a..4ffe5d23604 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); + HRegion 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); + HRegion 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); + HRegion 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); + HRegion 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); + HRegion 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); + HRegion 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/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java index d22046c06b4..f1a66e7a56c 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, + HRegion 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 e4f7663269f..49a61c5b145 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, 40 * 1024); // Intialize the region - Region region = initHRegion("testSelectiveFlushWithDataCompaction", conf); + HRegion region = initHRegion("testSelectiveFlushWithDataCompaction", 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)); @@ -324,12 +324,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.getRegions(tableName)) { + for (HRegion region : hrs.getRegions(tableName)) { return Pair.newPair(region, hrs); } } @@ -367,8 +367,8 @@ public class TestPerColumnFamilyFlush { } Thread.sleep(1000); - Pair desiredRegionAndServer = getRegionWithName(TABLENAME); - Region desiredRegion = desiredRegionAndServer.getFirst(); + Pair desiredRegionAndServer = getRegionWithName(TABLENAME); + HRegion desiredRegion = desiredRegionAndServer.getFirst(); assertTrue("Could not find a region which hosts the new region.", desiredRegion != null); // Flush the region selectively. @@ -476,8 +476,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 HRegion desiredRegion = desiredRegionAndServer.getFirst(); assertTrue("Could not find a region which hosts the new region.", desiredRegion != null); LOG.info("Writing to region=" + desiredRegion); 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 b2b03ff12be..b98ac2ac2ad 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.getRegions(TABLE_NAME); - for (Region region : regions) { + List regions = server.getRegions(TABLE_NAME); + for (HRegion 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.getRegions(TABLE_NAME); - for (Region region : regions) { + List regions = server.getRegions(TABLE_NAME); + for (HRegion 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/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 5d2b9c1204a..1a5a91d6449 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 @@ -156,7 +156,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()); + HRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); region.flush(true); openRegion(HTU, getRS(), hriSecondary); @@ -180,7 +180,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()); + HRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); region.flush(true); openRegion(HTU, getRS(), hriSecondary); @@ -250,7 +250,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()); + HRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); region.flush(true); HRegion primaryRegion = (HRegion) region; @@ -442,11 +442,11 @@ 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()); + HRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName()); region.flush(true); } - Region primaryRegion = getRS().getRegion(hriPrimary.getEncodedName()); + HRegion primaryRegion = getRS().getRegion(hriPrimary.getEncodedName()); Assert.assertEquals(3, primaryRegion.getStore(f).getStorefilesCount()); // Refresh store files on the secondary diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java index 5897fe7a941..45536f2b0bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithModifyTable.java @@ -18,10 +18,8 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import org.apache.commons.logging.Log; @@ -31,7 +29,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -39,7 +36,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.RegionSplitter; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -116,9 +112,9 @@ public class TestRegionReplicasWithModifyTable { try { tableName = TableName.valueOf(name.getMethodName()); enableReplicationByModification(tableName, false, 0, 3, 0); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); + List onlineRegions = getRS().getRegions(tableName); + List onlineRegions2 = getSecondaryRS().getRegions(tableName); + List onlineRegions3 = getTertiaryRS().getRegions(tableName); int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); assertEquals("the number of regions should be more than 1", totalRegions, 3); } finally { @@ -137,9 +133,9 @@ public class TestRegionReplicasWithModifyTable { try { tableName = TableName.valueOf(name.getMethodName()); enableReplicationByModification(tableName, false, 0, 3, 10); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); + List onlineRegions = getRS().getRegions(tableName); + List onlineRegions2 = getSecondaryRS().getRegions(tableName); + List onlineRegions3 = getTertiaryRS().getRegions(tableName); int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); assertEquals("the number of regions should be equal to 30", totalRegions, 30); } finally { @@ -153,9 +149,9 @@ public class TestRegionReplicasWithModifyTable { try { tableName = TableName.valueOf(name.getMethodName()); enableReplicationByModification(tableName, true, 2, 3, 0); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); + List onlineRegions = getRS().getRegions(tableName); + List onlineRegions2 = getSecondaryRS().getRegions(tableName); + List onlineRegions3 = getTertiaryRS().getRegions(tableName); int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); assertEquals("the number of regions should be 3", totalRegions, 3); } finally { @@ -169,9 +165,9 @@ public class TestRegionReplicasWithModifyTable { try { tableName = TableName.valueOf(name.getMethodName()); enableReplicationByModification(tableName, true, 3, 2, 0); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); + List onlineRegions = getRS().getRegions(tableName); + List onlineRegions2 = getSecondaryRS().getRegions(tableName); + List onlineRegions3 = getTertiaryRS().getRegions(tableName); int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); assertEquals("the number of regions should be reduced to 2", totalRegions, 2); } finally { @@ -186,9 +182,9 @@ public class TestRegionReplicasWithModifyTable { try { tableName = TableName.valueOf(name.getMethodName()); enableReplicationByModification(tableName, true, 3, 2, 20); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); + List onlineRegions = getRS().getRegions(tableName); + List onlineRegions2 = getSecondaryRS().getRegions(tableName); + List onlineRegions3 = getTertiaryRS().getRegions(tableName); int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); assertEquals("the number of regions should be reduced to 40", totalRegions, 40); } finally { @@ -203,9 +199,9 @@ public class TestRegionReplicasWithModifyTable { try { tableName = TableName.valueOf(name.getMethodName()); enableReplicationByModification(tableName, true, 2, 3, 15); - List onlineRegions = getRS().getRegions(tableName); - List onlineRegions2 = getSecondaryRS().getRegions(tableName); - List onlineRegions3 = getTertiaryRS().getRegions(tableName); + List onlineRegions = getRS().getRegions(tableName); + List onlineRegions2 = getSecondaryRS().getRegions(tableName); + List onlineRegions3 = getTertiaryRS().getRegions(tableName); int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size(); assertEquals("the number of regions should be equal to 45", totalRegions, 3 * 15); } finally { 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 6d8744b06d5..80d170a6719 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 @@ -496,7 +496,7 @@ public class TestRegionServerMetrics { byte[] val = Bytes.toBytes("mobdata"); try { Table table = TEST_UTIL.createTable(htd, new byte[0][0], conf); - Region region = rs.getRegions(tableName).get(0); + HRegion region = rs.getRegions(tableName).get(0); for (int insertCount = 0; insertCount < numHfiles; insertCount++) { Put p = new Put(Bytes.toBytes(insertCount)); p.addColumn(cf, qualifier, val); @@ -515,7 +515,7 @@ public class TestRegionServerMetrics { setMobThreshold(region, cf, 100); // metrics are reset by the region initialization - ((HRegion) region).initialize(); + region.initialize(); region.compact(true); metricsRegionServer.getRegionServerWrapper().forceRecompute(); assertCounter("cellsCountCompactedFromMob", numHfiles); @@ -539,7 +539,7 @@ public class TestRegionServerMetrics { ((HRegion) region).close(); // metrics are reset by the region initialization - ((HRegion) region).initialize(); + region.initialize(); region.compact(true); metricsRegionServer.getRegionServerWrapper().forceRecompute(); // metrics are reset by the region initialization 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 3b66a1df1e4..ffda9640607 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 @@ -129,7 +129,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()); + HRegion 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/TestRegionSplitPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java index 2b79152681f..8713999deff 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 @@ -104,8 +104,8 @@ 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<>(); - Mockito.when(rss.getRegions(TABLENAME)).thenReturn(regions); + final List regions = new ArrayList<>(); + Mockito.doReturn(regions).when(rss).getRegions(TABLENAME); Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss); // Set max size for this 'table'. long maxSplitSize = 1024L; @@ -163,8 +163,8 @@ public class TestRegionSplitPolicy { conf.setFloat("hbase.busy.policy.blockedRequests", 0.1f); RegionServerServices rss = Mockito.mock(RegionServerServices.class); - final List regions = new ArrayList<>(); - Mockito.when(rss.getRegions(TABLENAME)).thenReturn(regions); + final List regions = new ArrayList<>(); + Mockito.doReturn(regions).when(rss).getRegions(TABLENAME); Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss); Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(0L); Mockito.when(mockRegion.getWriteRequestsCount()).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 3a952eee64e..570422ee862 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 @@ -73,7 +73,7 @@ public class TestResettingCounters { throw new IOException("Failed delete of " + path); } } - Region region = HBaseTestingUtility.createRegionAndWAL(hri, path, conf, htd); + HRegion 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 dbf3be015ac..63e549e30e4 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 @@ -321,7 +321,7 @@ public class TestReversibleScanners { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())) .addFamily(new HColumnDescriptor(FAMILYNAME)) .addFamily(new HColumnDescriptor(FAMILYNAME2)); - Region region = TEST_UTIL.createLocalHRegion(htd, null, null); + HRegion region = TEST_UTIL.createLocalHRegion(htd, null, null); loadDataToRegion(region, FAMILYNAME2); // verify row count with forward scan @@ -641,7 +641,7 @@ public class TestReversibleScanners { return nextReadableNum; } - private static void loadDataToRegion(Region region, byte[] additionalFamily) + private static void loadDataToRegion(HRegion 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 4d3a1c3bb88..f9d0d1a4d47 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 = + HRegion 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 = + HRegion 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 72267beecb9..e01b59d885b 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 HRegion 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 67f6f34ae0c..03860f2d58e 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 HRegion region; private Put put; private Delete del; private Random rand; 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 fecfd94cc0e..40abf79d65d 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.HRegion.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 5ac7efb4eb2..04dcb14f4f1 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 @@ -99,7 +99,7 @@ public class TestStoreFileRefresherChore { } } - private Region initHRegion(HTableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId) + private HRegion initHRegion(HTableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId) throws IOException { Configuration conf = TEST_UTIL.getConfiguration(); Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName()); @@ -171,14 +171,14 @@ 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(name.getMethodName()), families); htd.setRegionReplication(2); - 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); + HRegion primary = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 0); + HRegion replica1 = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 1); 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 4d0a16868c8..a012d093de3 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 @@ -141,7 +141,7 @@ public class TestWalAndCompactingMemStoreFlush { String.valueOf(MemoryCompactionPolicy.EAGER)); // Intialize the region - Region region = initHRegion("testSelectiveFlushWithEager", conf); + HRegion region = initHRegion("testSelectiveFlushWithEager", conf); verifyInMemoryFlushSize(region); // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3 for (int i = 1; i <= 1200; i++) { @@ -382,7 +382,7 @@ public class TestWalAndCompactingMemStoreFlush { String.valueOf(MemoryCompactionPolicy.BASIC)); // Initialize the region - Region region = initHRegion("testSelectiveFlushWithIndexCompaction", conf); + HRegion region = initHRegion("testSelectiveFlushWithIndexCompaction", conf); verifyInMemoryFlushSize(region); /*------------------------------------------------------------------------------*/ /* PHASE I - insertions */ 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 5afa52fa98e..773a786170a 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 @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import java.io.IOException; import java.util.ArrayList; @@ -44,7 +43,6 @@ 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.HStoreFile; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -54,6 +52,7 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.mockito.Mockito; @Category({ MediumTests.class, RegionServerTests.class }) public class TestCompactedHFilesDischarger { @@ -76,9 +75,9 @@ 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<>(1); + List regions = new ArrayList<>(1); regions.add(region); - when(rss.getRegions()).thenReturn(regions); + Mockito.doReturn(regions).when(rss).getRegions(); } @After 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 ca6c86b2093..3c1228e2f91 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 @@ -165,7 +165,7 @@ public class TestFlushWithThroughputController { HRegionServer regionServer = hbtu.getRSForFirstRegionInTable(tableName); PressureAwareFlushThroughputController throughputController = (PressureAwareFlushThroughputController) regionServer.getFlushThroughputController(); - for (Region region : regionServer.getRegions()) { + for (HRegion region : regionServer.getRegions()) { 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 8ff18239879..68d71b0958d 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 @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; 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.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Store; @@ -195,7 +196,7 @@ public abstract class AbstractTestLogRolling { assertLogFileSize(log); // flush all regions - for (Region r : server.getOnlineRegionsLocalContext()) { + for (HRegion r : server.getOnlineRegionsLocalContext()) { r.flush(true); } @@ -248,7 +249,7 @@ public abstract class AbstractTestLogRolling { table = createTestTable(getName()); server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); - Region region = server.getRegions(table.getName()).get(0); + HRegion region = server.getRegions(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 096d3cacd9d..63c904d1678 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 @@ -350,7 +350,7 @@ public abstract class AbstractTestWALReplay { Region 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); + HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf); byte [] family = htd.getFamilies().iterator().next().getName(); Path f = new Path(basedir, "hfile"); @@ -1117,7 +1117,7 @@ public abstract class AbstractTestWALReplay { private HRegion r; @Override - public void requestFlush(Region region, boolean force) { + public void requestFlush(HRegion region, boolean force) { try { r.flush(force); } catch (IOException e) { @@ -1126,7 +1126,7 @@ public abstract class AbstractTestWALReplay { } @Override - public void requestDelayedFlush(Region region, long when, boolean forceFlushAllStores) { + public void requestDelayedFlush(HRegion region, long when, boolean forceFlushAllStores) { // TODO Auto-generated method stub } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java index 17416ba53e4..654330cbdc9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java @@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.regionserver.ChunkCreator; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -183,7 +182,7 @@ public class TestFSHLog extends AbstractTestFSWAL { @Override public void run() { try { - Region.FlushResult flushResult = region.flush(true); + HRegion.FlushResult flushResult = region.flush(true); LOG.info("Flush result:" + flushResult.getResult()); LOG.info("Flush succeeded:" + flushResult.isFlushSucceeded()); flushFinished.countDown(); 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 295ac2f65e0..088ad486d3d 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 @@ -47,6 +47,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.HRegion; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; @@ -345,7 +346,7 @@ public class TestLogRolling extends AbstractTestLogRolling { assertTrue(loggedRows.contains("row1005")); // flush all regions - for (Region r : server.getOnlineRegionsLocalContext()) { + for (HRegion 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 80681309c34..910449b3da5 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,6 +44,7 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RpcRetryingCallerImpl; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.wal.WAL.Entry; @@ -247,8 +248,8 @@ public class TestRegionReplicaReplicationEndpoint { for (int i=0; i < NB_SERVERS; i++) { HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(i); - List onlineRegions = rs.getRegions(tableName); - for (Region region : onlineRegions) { + List onlineRegions = rs.getRegions(tableName); + for (HRegion region : onlineRegions) { regions[region.getRegionInfo().getReplicaId()] = region; } } 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 29bce1a2942..499213572f4 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 @@ -307,7 +307,7 @@ public abstract class TestVisibilityLabels { List regionServerThreads = TEST_UTIL.getHBaseCluster() .getRegionServerThreads(); for (RegionServerThread rsThread : regionServerThreads) { - List onlineRegions = rsThread.getRegionServer().getRegions( + List onlineRegions = rsThread.getRegionServer().getRegions( 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().getRegions( + List onlineRegions = rsThread.getRegionServer().getRegions( LABELS_TABLE_NAME); if (onlineRegions.size() > 0) { break; 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 c0cb9e735fc..cf17e528f32 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 @@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.Region; @@ -759,8 +760,8 @@ public final class SnapshotTestingUtils { final TableName tableName) throws IOException, InterruptedException { HRegionServer rs = util.getRSForFirstRegionInTable(tableName); - List onlineRegions = rs.getRegions(tableName); - for (Region region : onlineRegions) { + List onlineRegions = rs.getRegions(tableName); + for (HRegion region : onlineRegions) { region.waitForFlushesAndCompactions(); } // Wait up to 60 seconds for a table to be available.