HBASE-18183 Region interface cleanup for CP expose.
This commit is contained in:
parent
2e368cf461
commit
b212bf936e
|
@ -2863,6 +2863,10 @@ public final class ProtobufUtil {
|
||||||
return CompactionState.valueOf(state.toString());
|
return CompactionState.valueOf(state.toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static GetRegionInfoResponse.CompactionState createCompactionState(CompactionState state) {
|
||||||
|
return GetRegionInfoResponse.CompactionState.valueOf(state.toString());
|
||||||
|
}
|
||||||
|
|
||||||
public static Optional<Long> toOptionalTimestamp(MajorCompactionTimestampResponse resp) {
|
public static Optional<Long> toOptionalTimestamp(MajorCompactionTimestampResponse resp) {
|
||||||
long timestamp = resp.getCompactionTimestamp();
|
long timestamp = resp.getCompactionTimestamp();
|
||||||
return timestamp == 0 ? Optional.empty() : Optional.of(timestamp);
|
return timestamp == 0 ? Optional.empty() : Optional.of(timestamp);
|
||||||
|
|
|
@ -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.SecureBulkLoadHFilesRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService;
|
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.RegionServerServices;
|
||||||
import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
|
import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
|
||||||
|
|
||||||
|
@ -141,7 +142,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService implements Reg
|
||||||
try {
|
try {
|
||||||
SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager();
|
SecureBulkLoadManager secureBulkLoadManager = this.rsServices.getSecureBulkLoadManager();
|
||||||
BulkLoadHFileRequest bulkLoadHFileRequest = ConvertSecureBulkLoadHFilesRequest(request);
|
BulkLoadHFileRequest bulkLoadHFileRequest = ConvertSecureBulkLoadHFilesRequest(request);
|
||||||
map = secureBulkLoadManager.secureBulkLoadHFiles(this.env.getRegion(),
|
map = secureBulkLoadManager.secureBulkLoadHFiles((HRegion) this.env.getRegion(),
|
||||||
convert(bulkLoadHFileRequest));
|
convert(bulkLoadHFileRequest));
|
||||||
loaded = map != null && !map.isEmpty();
|
loaded = map != null && !map.isEmpty();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
|
|
@ -150,8 +150,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements RegionCopro
|
||||||
for (List<Cell> deleteRow : deleteRows) {
|
for (List<Cell> deleteRow : deleteRows) {
|
||||||
deleteArr[i++] = createDeleteMutation(deleteRow, deleteType, timestamp);
|
deleteArr[i++] = createDeleteMutation(deleteRow, deleteType, timestamp);
|
||||||
}
|
}
|
||||||
OperationStatus[] opStatus = region.batchMutate(deleteArr, HConstants.NO_NONCE,
|
OperationStatus[] opStatus = region.batchMutate(deleteArr);
|
||||||
HConstants.NO_NONCE);
|
|
||||||
for (i = 0; i < opStatus.length; i++) {
|
for (i = 0; i < opStatus.length; i++) {
|
||||||
if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
|
if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
|
||||||
break;
|
break;
|
||||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
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.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
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.RegionServerServices;
|
||||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
||||||
|
@ -446,7 +446,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
||||||
taskReadyLock.wait(checkInterval);
|
taskReadyLock.wait(checkInterval);
|
||||||
if (server != null) {
|
if (server != null) {
|
||||||
// check to see if we have stale recovering regions in our internal memory state
|
// check to see if we have stale recovering regions in our internal memory state
|
||||||
Map<String, Region> recoveringRegions = server.getRecoveringRegions();
|
Map<String, HRegion> recoveringRegions = server.getRecoveringRegions();
|
||||||
if (!recoveringRegions.isEmpty()) {
|
if (!recoveringRegions.isEmpty()) {
|
||||||
// Make a local copy to prevent ConcurrentModificationException when other threads
|
// Make a local copy to prevent ConcurrentModificationException when other threads
|
||||||
// modify recoveringRegions
|
// modify recoveringRegions
|
||||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||||
import org.apache.hadoop.hbase.procedure.ProcedureMember;
|
import org.apache.hadoop.hbase.procedure.ProcedureMember;
|
||||||
import org.apache.hadoop.hbase.procedure.Subprocedure;
|
import org.apache.hadoop.hbase.procedure.Subprocedure;
|
||||||
import org.apache.hadoop.hbase.procedure.flush.RegionServerFlushTableProcedureManager.FlushTableSubprocedurePool;
|
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
|
* 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 static final Log LOG = LogFactory.getLog(FlushTableSubprocedure.class);
|
||||||
|
|
||||||
private final String table;
|
private final String table;
|
||||||
private final List<Region> regions;
|
private final List<HRegion> regions;
|
||||||
private final FlushTableSubprocedurePool taskManager;
|
private final FlushTableSubprocedurePool taskManager;
|
||||||
|
|
||||||
public FlushTableSubprocedure(ProcedureMember member,
|
public FlushTableSubprocedure(ProcedureMember member,
|
||||||
ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
|
ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
|
||||||
List<Region> regions, String table,
|
List<HRegion> regions, String table,
|
||||||
FlushTableSubprocedurePool taskManager) {
|
FlushTableSubprocedurePool taskManager) {
|
||||||
super(member, table, errorListener, wakeFrequency, timeout);
|
super(member, table, errorListener, wakeFrequency, timeout);
|
||||||
this.table = table;
|
this.table = table;
|
||||||
|
@ -54,8 +54,8 @@ public class FlushTableSubprocedure extends Subprocedure {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class RegionFlushTask implements Callable<Void> {
|
private static class RegionFlushTask implements Callable<Void> {
|
||||||
Region region;
|
HRegion region;
|
||||||
RegionFlushTask(Region region) {
|
RegionFlushTask(HRegion region) {
|
||||||
this.region = region;
|
this.region = region;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -90,7 +90,7 @@ public class FlushTableSubprocedure extends Subprocedure {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Add all hfiles already existing in region.
|
// Add all hfiles already existing in region.
|
||||||
for (Region region : regions) {
|
for (HRegion region : regions) {
|
||||||
// submit one task per region for parallelize by region.
|
// submit one task per region for parallelize by region.
|
||||||
taskManager.submitTask(new RegionFlushTask(region));
|
taskManager.submitTask(new RegionFlushTask(region));
|
||||||
monitor.rethrowException();
|
monitor.rethrowException();
|
||||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
|
||||||
import org.apache.hadoop.hbase.procedure.Subprocedure;
|
import org.apache.hadoop.hbase.procedure.Subprocedure;
|
||||||
import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
|
import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
|
||||||
import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
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
|
// check to see if this server is hosting any regions for the table
|
||||||
List<Region> involvedRegions;
|
List<HRegion> involvedRegions;
|
||||||
try {
|
try {
|
||||||
involvedRegions = getRegionsToFlush(table);
|
involvedRegions = getRegionsToFlush(table);
|
||||||
} catch (IOException e1) {
|
} 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.
|
* @return the list of online regions. Empty list is returned if no regions.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private List<Region> getRegionsToFlush(String table) throws IOException {
|
private List<HRegion> getRegionsToFlush(String table) throws IOException {
|
||||||
return rss.getRegions(TableName.valueOf(table));
|
return (List<HRegion>) rss.getRegions(TableName.valueOf(table));
|
||||||
}
|
}
|
||||||
|
|
||||||
public class FlushTableSubprocedureBuilder implements SubprocedureFactory {
|
public class FlushTableSubprocedureBuilder implements SubprocedureFactory {
|
||||||
|
|
|
@ -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
|
// 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
|
// when the test case is not spinning up a cluster
|
||||||
if (regionServerServices == null) return;
|
if (regionServerServices == null) return;
|
||||||
List<Region> onlineRegions = regionServerServices.getRegions();
|
List<HRegion> onlineRegions = (List<HRegion>) regionServerServices.getRegions();
|
||||||
if (onlineRegions == null) return;
|
if (onlineRegions == null) return;
|
||||||
for (Region region : onlineRegions) {
|
for (HRegion region : onlineRegions) {
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Started compacted hfiles cleaner on " + region.getRegionInfo());
|
LOG.trace("Started compacted hfiles cleaner on " + region.getRegionInfo());
|
||||||
}
|
}
|
||||||
for (HStore store : ((HRegion) region).getStores()) {
|
for (HStore store : region.getStores()) {
|
||||||
try {
|
try {
|
||||||
if (useExecutor && regionServerServices != null) {
|
if (useExecutor && regionServerServices != null) {
|
||||||
CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler(
|
CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler(
|
||||||
|
|
|
@ -33,7 +33,7 @@ public interface FlushRequester {
|
||||||
* @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log
|
* @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log
|
||||||
* rolling.
|
* 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
|
* 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
|
* @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log
|
||||||
* rolling.
|
* rolling.
|
||||||
*/
|
*/
|
||||||
void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores);
|
void requestDelayedFlush(HRegion region, long delay, boolean forceFlushAllStores);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Register a FlushRequestListener
|
* Register a FlushRequestListener
|
||||||
|
|
|
@ -102,6 +102,7 @@ import org.apache.hadoop.hbase.TagUtil;
|
||||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||||
import org.apache.hadoop.hbase.client.Append;
|
import org.apache.hadoop.hbase.client.Append;
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
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.Delete;
|
||||||
import org.apache.hadoop.hbase.client.Durability;
|
import org.apache.hadoop.hbase.client.Durability;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
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.Lists;
|
||||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
|
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.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.TextFormat;
|
||||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
|
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.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;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
|
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.RegionEventDescriptor.EventType;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
|
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.
|
||||||
|
*
|
||||||
|
* <p>An Region is defined by its table and its key extent.
|
||||||
|
*
|
||||||
|
* <p>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")
|
@SuppressWarnings("deprecation")
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
|
public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
|
||||||
|
@ -782,7 +798,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this);
|
this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this);
|
||||||
this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper);
|
this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper);
|
||||||
|
|
||||||
Map<String, Region> recoveringRegions = rsServices.getRecoveringRegions();
|
Map<String, HRegion> recoveringRegions = rsServices.getRecoveringRegions();
|
||||||
String encodedName = getRegionInfo().getEncodedName();
|
String encodedName = getRegionInfo().getEncodedName();
|
||||||
if (recoveringRegions != null && recoveringRegions.containsKey(encodedName)) {
|
if (recoveringRegions != null && recoveringRegions.containsKey(encodedName)) {
|
||||||
this.recovering = true;
|
this.recovering = true;
|
||||||
|
@ -1121,7 +1137,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
this.updatesLock.writeLock().unlock();
|
this.updatesLock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public HDFSBlocksDistribution getHDFSBlocksDistribution() {
|
public HDFSBlocksDistribution getHDFSBlocksDistribution() {
|
||||||
HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
|
HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
|
||||||
stores.values().stream().filter(s -> s.getStorefiles() != null)
|
stores.values().stream().filter(s -> s.getStorefiles() != null)
|
||||||
|
@ -1238,7 +1253,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return readRequestsCount.sum();
|
return readRequestsCount.sum();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
|
* Update the read request count for this region
|
||||||
|
* @param i increment
|
||||||
|
*/
|
||||||
public void updateReadRequestsCount(long i) {
|
public void updateReadRequestsCount(long i) {
|
||||||
readRequestsCount.add(i);
|
readRequestsCount.add(i);
|
||||||
}
|
}
|
||||||
|
@ -1253,7 +1271,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return writeRequestsCount.sum();
|
return writeRequestsCount.sum();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
|
* Update the write request count for this region
|
||||||
|
* @param i increment
|
||||||
|
*/
|
||||||
public void updateWriteRequestsCount(long i) {
|
public void updateWriteRequestsCount(long i) {
|
||||||
writeRequestsCount.add(i);
|
writeRequestsCount.add(i);
|
||||||
}
|
}
|
||||||
|
@ -1263,7 +1284,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return memstoreDataSize.get();
|
return memstoreDataSize.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/** @return store services for this region, to access services required by store level needs */
|
||||||
public RegionServicesForStores getRegionServicesForStores() {
|
public RegionServicesForStores getRegionServicesForStores() {
|
||||||
return regionServicesForStores;
|
return regionServicesForStores;
|
||||||
}
|
}
|
||||||
|
@ -1293,7 +1314,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return checkAndMutateChecksFailed.sum();
|
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() {
|
public MetricsRegion getMetrics() {
|
||||||
return metricsRegion;
|
return metricsRegion;
|
||||||
}
|
}
|
||||||
|
@ -1433,12 +1456,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return mvcc.getReadPoint();
|
return mvcc.getReadPoint();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public long getReadpoint(IsolationLevel isolationLevel) {
|
|
||||||
return getReadPoint(isolationLevel);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isLoadingCfsOnDemandDefault() {
|
public boolean isLoadingCfsOnDemandDefault() {
|
||||||
return this.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();
|
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() {
|
public void waitForFlushesAndCompactions() {
|
||||||
synchronized (writestate) {
|
synchronized (writestate) {
|
||||||
if (this.writestate.readOnly) {
|
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() {
|
public void waitForFlushes() {
|
||||||
synchronized (writestate) {
|
synchronized (writestate) {
|
||||||
if (this.writestate.readOnly) {
|
if (this.writestate.readOnly) {
|
||||||
|
@ -1941,7 +1963,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
stores.values().forEach(HStore::triggerMajorCompaction);
|
stores.values().forEach(HStore::triggerMajorCompaction);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
|
* Synchronously compact all stores in the region.
|
||||||
|
* <p>This operation could block for a long time, so don't call it from a
|
||||||
|
* time-sensitive thread.
|
||||||
|
* <p>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 {
|
public void compact(boolean majorCompaction) throws IOException {
|
||||||
if (majorCompaction) {
|
if (majorCompaction) {
|
||||||
triggerMajorCompaction();
|
triggerMajorCompaction();
|
||||||
|
@ -2157,11 +2191,51 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
|
* Flush the cache.
|
||||||
|
*
|
||||||
|
* <p>When this method is called the cache will be flushed unless:
|
||||||
|
* <ol>
|
||||||
|
* <li>the cache is empty</li>
|
||||||
|
* <li>the region is closed.</li>
|
||||||
|
* <li>a flush is already in progress</li>
|
||||||
|
* <li>writes are disabled</li>
|
||||||
|
* </ol>
|
||||||
|
*
|
||||||
|
* <p>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 {
|
public FlushResult flush(boolean force) throws IOException {
|
||||||
return flushcache(force, false);
|
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.
|
* Flush the cache.
|
||||||
*
|
*
|
||||||
|
@ -2805,7 +2879,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
|
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 {
|
public void prepareDelete(Delete delete) throws IOException {
|
||||||
// Check to see if this is a deleteRow insert
|
// Check to see if this is a deleteRow insert
|
||||||
if(delete.getFamilyCellMap().isEmpty()){
|
if(delete.getFamilyCellMap().isEmpty()){
|
||||||
|
@ -2854,7 +2932,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
doBatchMutate(delete);
|
doBatchMutate(delete);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
|
* Set up correct timestamps in the KVs in Delete object.
|
||||||
|
* <p>Caller should have the row and region locks.
|
||||||
|
* @param mutation
|
||||||
|
* @param familyMap
|
||||||
|
* @param byteNow
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> familyMap,
|
public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> familyMap,
|
||||||
byte[] byteNow) throws IOException {
|
byte[] byteNow) throws IOException {
|
||||||
for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
|
for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
|
||||||
|
@ -3043,7 +3128,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce)
|
public OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
// As it stands, this is used for 3 things
|
// 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));
|
return batchMutate(new MutationBatch(mutations, nonceGroup, nonce));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
|
public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
|
||||||
return batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
return batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId)
|
public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (!RegionReplicaUtil.isDefaultReplica(getRegionInfo())
|
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<List<Cell>> cellItr, final byte[] now)
|
public void updateCellTimestamps(final Iterable<List<Cell>> cellItr, final byte[] now)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
for (List<Cell> cells: cellItr) {
|
for (List<Cell> cells: cellItr) {
|
||||||
|
@ -3991,14 +4080,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
store.add(cell, memstoreSize);
|
store.add(cell, memstoreSize);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
|
* Check the collection of families for validity.
|
||||||
|
* @param families
|
||||||
|
* @throws NoSuchColumnFamilyException
|
||||||
|
*/
|
||||||
public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException {
|
public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException {
|
||||||
for (byte[] family : families) {
|
for (byte[] family : families) {
|
||||||
checkFamily(family);
|
checkFamily(family);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
|
* Check the collection of families for valid timestamps
|
||||||
|
* @param familyMap
|
||||||
|
* @param now current timestamp
|
||||||
|
* @throws FailedSanityCheckException
|
||||||
|
*/
|
||||||
public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
|
public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
|
||||||
throws FailedSanityCheckException {
|
throws FailedSanityCheckException {
|
||||||
if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
|
if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
|
||||||
|
@ -5423,8 +5521,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
private void releaseRowLocks(List<RowLock> rowLocks) {
|
||||||
public void releaseRowLocks(List<RowLock> rowLocks) {
|
|
||||||
if (rowLocks != null) {
|
if (rowLocks != null) {
|
||||||
for (int i = 0; i < rowLocks.size(); i++) {
|
for (int i = 0; i < rowLocks.size(); i++) {
|
||||||
rowLocks.get(i).release();
|
rowLocks.get(i).release();
|
||||||
|
@ -5560,13 +5657,67 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return multipleFamilies;
|
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<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
|
public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
|
||||||
BulkLoadListener bulkLoadListener) throws IOException {
|
BulkLoadListener bulkLoadListener) throws IOException {
|
||||||
return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false);
|
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<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths,
|
public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths,
|
||||||
boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile) throws IOException {
|
boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile) throws IOException {
|
||||||
long seqId = -1;
|
long seqId = -1;
|
||||||
|
@ -6875,7 +7026,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
|
public List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
List<Cell> results = new ArrayList<>();
|
List<Cell> results = new ArrayList<>();
|
||||||
|
@ -7167,22 +7317,21 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Result append(Append append) throws IOException {
|
public Result append(Append append) throws IOException {
|
||||||
return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Result append(Append mutation, long nonceGroup, long nonce) throws IOException {
|
public Result append(Append mutation, long nonceGroup, long nonce) throws IOException {
|
||||||
return doDelta(Operation.APPEND, mutation, nonceGroup, nonce, mutation.isReturnResults());
|
return doDelta(Operation.APPEND, mutation, nonceGroup, nonce, mutation.isReturnResults());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Result increment(Increment increment) throws IOException {
|
public Result increment(Increment increment) throws IOException {
|
||||||
return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
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());
|
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();
|
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.
|
||||||
|
*
|
||||||
|
* <p>
|
||||||
|
* 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}.
|
||||||
|
* </p>
|
||||||
|
* @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) {
|
public boolean registerService(com.google.protobuf.Service instance) {
|
||||||
/*
|
/*
|
||||||
* No stacking of instances is allowed for a single service name
|
* No stacking of instances is allowed for a single service name
|
||||||
|
@ -7597,10 +7760,22 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return true;
|
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,
|
public com.google.protobuf.Message execService(com.google.protobuf.RpcController controller,
|
||||||
CoprocessorServiceCall call)
|
CoprocessorServiceCall call) throws IOException {
|
||||||
throws IOException {
|
|
||||||
String serviceName = call.getServiceName();
|
String serviceName = call.getServiceName();
|
||||||
com.google.protobuf.Service service = coprocessorServiceHandlers.get(serviceName);
|
com.google.protobuf.Service service = coprocessorServiceHandlers.get(serviceName);
|
||||||
if (service == null) {
|
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() {
|
public long getOpenSeqNum() {
|
||||||
return this.openSeqNum;
|
return this.openSeqNum;
|
||||||
}
|
}
|
||||||
|
@ -7981,7 +8156,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
return this.maxSeqIdInStores;
|
return this.maxSeqIdInStores;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public long getOldestSeqIdOfStore(byte[] familyName) {
|
public long getOldestSeqIdOfStore(byte[] familyName) {
|
||||||
return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName);
|
return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName);
|
||||||
}
|
}
|
||||||
|
|
|
@ -290,7 +290,7 @@ public class HRegionServer extends HasThread implements
|
||||||
* Map of regions currently being served by this region server. Key is the
|
* Map of regions currently being served by this region server. Key is the
|
||||||
* encoded region name. All access should be synchronized.
|
* encoded region name. All access should be synchronized.
|
||||||
*/
|
*/
|
||||||
protected final Map<String, Region> onlineRegions = new ConcurrentHashMap<>();
|
protected final Map<String, HRegion> onlineRegions = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Map of encoded region names to the DataNode locations they should be hosted on
|
* 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
|
* 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.
|
* previous failed region server) but not reads. A recovering region is also an online region.
|
||||||
*/
|
*/
|
||||||
protected final Map<String, Region> recoveringRegions = Collections
|
protected final Map<String, HRegion> recoveringRegions = Collections
|
||||||
.synchronizedMap(new HashMap<String, Region>());
|
.synchronizedMap(new HashMap<String, HRegion>());
|
||||||
|
|
||||||
// Leases
|
// Leases
|
||||||
protected Leases leases;
|
protected Leases leases;
|
||||||
|
@ -1235,7 +1235,7 @@ public class HRegionServer extends HasThread implements
|
||||||
private boolean areAllUserRegionsOffline() {
|
private boolean areAllUserRegionsOffline() {
|
||||||
if (getNumberOfOnlineRegions() > 2) return false;
|
if (getNumberOfOnlineRegions() > 2) return false;
|
||||||
boolean allUserRegionsOffline = true;
|
boolean allUserRegionsOffline = true;
|
||||||
for (Map.Entry<String, Region> e: this.onlineRegions.entrySet()) {
|
for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
|
||||||
if (!e.getValue().getRegionInfo().isMetaTable()) {
|
if (!e.getValue().getRegionInfo().isMetaTable()) {
|
||||||
allUserRegionsOffline = false;
|
allUserRegionsOffline = false;
|
||||||
break;
|
break;
|
||||||
|
@ -1249,7 +1249,7 @@ public class HRegionServer extends HasThread implements
|
||||||
*/
|
*/
|
||||||
private long getWriteRequestCount() {
|
private long getWriteRequestCount() {
|
||||||
long writeCount = 0;
|
long writeCount = 0;
|
||||||
for (Map.Entry<String, Region> e: this.onlineRegions.entrySet()) {
|
for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
|
||||||
writeCount += e.getValue().getWriteRequestsCount();
|
writeCount += e.getValue().getWriteRequestsCount();
|
||||||
}
|
}
|
||||||
return writeCount;
|
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
|
// improved; Additionally the load balancer will be able to take advantage of a more complete
|
||||||
// history.
|
// history.
|
||||||
MetricsRegionServerWrapper regionServerWrapper = metricsRegionServer.getRegionServerWrapper();
|
MetricsRegionServerWrapper regionServerWrapper = metricsRegionServer.getRegionServerWrapper();
|
||||||
Collection<Region> regions = getOnlineRegionsLocalContext();
|
Collection<HRegion> regions = getOnlineRegionsLocalContext();
|
||||||
long usedMemory = -1L;
|
long usedMemory = -1L;
|
||||||
long maxMemory = -1L;
|
long maxMemory = -1L;
|
||||||
final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage();
|
final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage();
|
||||||
|
@ -1391,7 +1391,7 @@ public class HRegionServer extends HasThread implements
|
||||||
}
|
}
|
||||||
RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder();
|
RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder();
|
||||||
RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
|
RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
|
||||||
for (Region region : regions) {
|
for (HRegion region : regions) {
|
||||||
if (region.getCoprocessorHost() != null) {
|
if (region.getCoprocessorHost() != null) {
|
||||||
Set<String> regionCoprocessors = region.getCoprocessorHost().getCoprocessors();
|
Set<String> regionCoprocessors = region.getCoprocessorHost().getCoprocessors();
|
||||||
Iterator<String> iterator = regionCoprocessors.iterator();
|
Iterator<String> 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
|
// 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
|
// protect against the case where an open comes in after we start the
|
||||||
// iterator of onlineRegions to close all user regions.
|
// iterator of onlineRegions to close all user regions.
|
||||||
for (Map.Entry<String, Region> e : this.onlineRegions.entrySet()) {
|
for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
|
||||||
RegionInfo hri = e.getValue().getRegionInfo();
|
RegionInfo hri = e.getValue().getRegionInfo();
|
||||||
if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
|
if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
|
||||||
&& !closedRegions.contains(hri.getEncodedName())) {
|
&& !closedRegions.contains(hri.getEncodedName())) {
|
||||||
|
@ -1672,7 +1672,7 @@ public class HRegionServer extends HasThread implements
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
RegionLoad createRegionLoad(final Region r, RegionLoad.Builder regionLoadBldr,
|
RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr,
|
||||||
RegionSpecifier.Builder regionSpecifier) throws IOException {
|
RegionSpecifier.Builder regionSpecifier) throws IOException {
|
||||||
byte[] name = r.getRegionInfo().getRegionName();
|
byte[] name = r.getRegionInfo().getRegionName();
|
||||||
int stores = 0;
|
int stores = 0;
|
||||||
|
@ -1740,7 +1740,7 @@ public class HRegionServer extends HasThread implements
|
||||||
* @return An instance of RegionLoad.
|
* @return An instance of RegionLoad.
|
||||||
*/
|
*/
|
||||||
public RegionLoad createRegionLoad(final String encodedRegionName) throws IOException {
|
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;
|
return r != null ? createRegionLoad(r, null, null) : null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1821,9 +1821,9 @@ public class HRegionServer extends HasThread implements
|
||||||
@Override
|
@Override
|
||||||
protected void chore() {
|
protected void chore() {
|
||||||
final StringBuffer whyFlush = new StringBuffer();
|
final StringBuffer whyFlush = new StringBuffer();
|
||||||
for (Region r : this.server.onlineRegions.values()) {
|
for (HRegion r : this.server.onlineRegions.values()) {
|
||||||
if (r == null) continue;
|
if (r == null) continue;
|
||||||
if (((HRegion)r).shouldFlush(whyFlush)) {
|
if (r.shouldFlush(whyFlush)) {
|
||||||
FlushRequester requester = server.getFlushRequester();
|
FlushRequester requester = server.getFlushRequester();
|
||||||
if (requester != null) {
|
if (requester != null) {
|
||||||
long randomDelay = RandomUtils.nextInt(0, RANGE_OF_DELAY) + MIN_DELAY_TIME;
|
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
|
@Override
|
||||||
public void postOpenDeployTasks(final PostOpenDeployContext context)
|
public void postOpenDeployTasks(final PostOpenDeployContext context)
|
||||||
throws KeeperException, IOException {
|
throws KeeperException, IOException {
|
||||||
|
@ -2197,18 +2192,6 @@ public class HRegionServer extends HasThread implements
|
||||||
LOG.debug("Finished post open deploy task for " + r.getRegionInfo().getRegionNameAsString());
|
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
|
@Override
|
||||||
public boolean reportRegionStateTransition(final RegionStateTransitionContext context) {
|
public boolean reportRegionStateTransition(final RegionStateTransitionContext context) {
|
||||||
TransitionCode code = context.getCode();
|
TransitionCode code = context.getCode();
|
||||||
|
@ -2664,10 +2647,10 @@ public class HRegionServer extends HasThread implements
|
||||||
* @param abort Whether we're running an abort.
|
* @param abort Whether we're running an abort.
|
||||||
*/
|
*/
|
||||||
void closeMetaTableRegions(final boolean abort) {
|
void closeMetaTableRegions(final boolean abort) {
|
||||||
Region meta = null;
|
HRegion meta = null;
|
||||||
this.lock.writeLock().lock();
|
this.lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
for (Map.Entry<String, Region> e: onlineRegions.entrySet()) {
|
for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
|
||||||
RegionInfo hri = e.getValue().getRegionInfo();
|
RegionInfo hri = e.getValue().getRegionInfo();
|
||||||
if (hri.isMetaRegion()) {
|
if (hri.isMetaRegion()) {
|
||||||
meta = e.getValue();
|
meta = e.getValue();
|
||||||
|
@ -2689,8 +2672,8 @@ public class HRegionServer extends HasThread implements
|
||||||
void closeUserRegions(final boolean abort) {
|
void closeUserRegions(final boolean abort) {
|
||||||
this.lock.writeLock().lock();
|
this.lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
for (Map.Entry<String, Region> e: this.onlineRegions.entrySet()) {
|
for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
|
||||||
Region r = e.getValue();
|
HRegion r = e.getValue();
|
||||||
if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
|
if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
|
||||||
// Don't update zk with this close transition; pass false.
|
// Don't update zk with this close transition; pass false.
|
||||||
closeRegionIgnoreErrors(r.getRegionInfo(), abort);
|
closeRegionIgnoreErrors(r.getRegionInfo(), abort);
|
||||||
|
@ -2720,7 +2703,7 @@ public class HRegionServer extends HasThread implements
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<String, Region> getRecoveringRegions() {
|
public Map<String, HRegion> getRecoveringRegions() {
|
||||||
return this.recoveringRegions;
|
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;
|
* This method will only work if HRegionServer is in the same JVM as client;
|
||||||
* HRegion cannot be serialized to cross an rpc.
|
* HRegion cannot be serialized to cross an rpc.
|
||||||
*/
|
*/
|
||||||
public Collection<Region> getOnlineRegionsLocalContext() {
|
public Collection<HRegion> getOnlineRegionsLocalContext() {
|
||||||
Collection<Region> regions = this.onlineRegions.values();
|
Collection<HRegion> regions = this.onlineRegions.values();
|
||||||
return Collections.unmodifiableCollection(regions);
|
return Collections.unmodifiableCollection(regions);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void addRegion(Region region) {
|
public void addRegion(HRegion region) {
|
||||||
this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
|
this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
|
||||||
configurationManager.registerObserver(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
|
* biggest. If two regions are the same size, then the last one found wins; i.e. this method
|
||||||
* may NOT return all regions.
|
* may NOT return all regions.
|
||||||
*/
|
*/
|
||||||
SortedMap<Long, Region> getCopyOfOnlineRegionsSortedBySize() {
|
SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
|
||||||
// we'll sort the regions in reverse
|
// we'll sort the regions in reverse
|
||||||
SortedMap<Long, Region> sortedRegions = new TreeMap<>(
|
SortedMap<Long, HRegion> sortedRegions = new TreeMap<>(
|
||||||
new Comparator<Long>() {
|
new Comparator<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public int compare(Long a, Long b) {
|
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.
|
// 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);
|
sortedRegions.put(region.getMemStoreSize(), region);
|
||||||
}
|
}
|
||||||
return sortedRegions;
|
return sortedRegions;
|
||||||
|
@ -3003,10 +2986,10 @@ public class HRegionServer extends HasThread implements
|
||||||
* @return Online regions from <code>tableName</code>
|
* @return Online regions from <code>tableName</code>
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public List<Region> getRegions(TableName tableName) {
|
public List<HRegion> getRegions(TableName tableName) {
|
||||||
List<Region> tableRegions = new ArrayList<>();
|
List<HRegion> tableRegions = new ArrayList<>();
|
||||||
synchronized (this.onlineRegions) {
|
synchronized (this.onlineRegions) {
|
||||||
for (Region region: this.onlineRegions.values()) {
|
for (HRegion region: this.onlineRegions.values()) {
|
||||||
RegionInfo regionInfo = region.getRegionInfo();
|
RegionInfo regionInfo = region.getRegionInfo();
|
||||||
if(regionInfo.getTable().equals(tableName)) {
|
if(regionInfo.getTable().equals(tableName)) {
|
||||||
tableRegions.add(region);
|
tableRegions.add(region);
|
||||||
|
@ -3017,8 +3000,8 @@ public class HRegionServer extends HasThread implements
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<Region> getRegions() {
|
public List<HRegion> getRegions() {
|
||||||
List<Region> allRegions = new ArrayList<>();
|
List<HRegion> allRegions = new ArrayList<>();
|
||||||
synchronized (this.onlineRegions) {
|
synchronized (this.onlineRegions) {
|
||||||
// Return a clone copy of the onlineRegions
|
// Return a clone copy of the onlineRegions
|
||||||
allRegions.addAll(onlineRegions.values());
|
allRegions.addAll(onlineRegions.values());
|
||||||
|
@ -3051,8 +3034,8 @@ public class HRegionServer extends HasThread implements
|
||||||
"skipping.");
|
"skipping.");
|
||||||
LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception);
|
LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception);
|
||||||
}
|
}
|
||||||
Collection<Region> regions = getOnlineRegionsLocalContext();
|
Collection<HRegion> regions = getOnlineRegionsLocalContext();
|
||||||
for (Region region: regions) {
|
for (HRegion region: regions) {
|
||||||
coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
|
coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
|
||||||
try {
|
try {
|
||||||
coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors());
|
coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors());
|
||||||
|
@ -3170,7 +3153,7 @@ public class HRegionServer extends HasThread implements
|
||||||
protected boolean closeAndOfflineRegionForSplitOrMerge(
|
protected boolean closeAndOfflineRegionForSplitOrMerge(
|
||||||
final List<String> regionEncodedName) throws IOException {
|
final List<String> regionEncodedName) throws IOException {
|
||||||
for (int i = 0; i < regionEncodedName.size(); ++i) {
|
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) {
|
if (regionToClose != null) {
|
||||||
Map<byte[], List<HStoreFile>> hstoreFiles = null;
|
Map<byte[], List<HStoreFile>> hstoreFiles = null;
|
||||||
Exception exceptionToThrow = null;
|
Exception exceptionToThrow = null;
|
||||||
|
@ -3232,14 +3215,14 @@ public class HRegionServer extends HasThread implements
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Region getRegion(final String encodedRegionName) {
|
public HRegion getRegion(final String encodedRegionName) {
|
||||||
return this.onlineRegions.get(encodedRegionName);
|
return this.onlineRegions.get(encodedRegionName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean removeRegion(final Region r, ServerName destination) {
|
public boolean removeRegion(final HRegion r, ServerName destination) {
|
||||||
Region toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
|
HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
|
||||||
if (destination != null) {
|
if (destination != null) {
|
||||||
long closeSeqNum = r.getMaxFlushedSeqId();
|
long closeSeqNum = r.getMaxFlushedSeqId();
|
||||||
if (closeSeqNum == HConstants.NO_SEQNUM) {
|
if (closeSeqNum == HConstants.NO_SEQNUM) {
|
||||||
|
@ -3261,20 +3244,20 @@ public class HRegionServer extends HasThread implements
|
||||||
* @return {@link HRegion} for <code>regionName</code>
|
* @return {@link HRegion} for <code>regionName</code>
|
||||||
* @throws NotServingRegionException
|
* @throws NotServingRegionException
|
||||||
*/
|
*/
|
||||||
protected Region getRegion(final byte[] regionName)
|
protected HRegion getRegion(final byte[] regionName)
|
||||||
throws NotServingRegionException {
|
throws NotServingRegionException {
|
||||||
String encodedRegionName = RegionInfo.encodeRegionName(regionName);
|
String encodedRegionName = RegionInfo.encodeRegionName(regionName);
|
||||||
return getRegionByEncodedName(regionName, encodedRegionName);
|
return getRegionByEncodedName(regionName, encodedRegionName);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Region getRegionByEncodedName(String encodedRegionName)
|
public HRegion getRegionByEncodedName(String encodedRegionName)
|
||||||
throws NotServingRegionException {
|
throws NotServingRegionException {
|
||||||
return getRegionByEncodedName(null, encodedRegionName);
|
return getRegionByEncodedName(null, encodedRegionName);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected Region getRegionByEncodedName(byte[] regionName, String encodedRegionName)
|
protected HRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName)
|
||||||
throws NotServingRegionException {
|
throws NotServingRegionException {
|
||||||
Region region = this.onlineRegions.get(encodedRegionName);
|
HRegion region = this.onlineRegions.get(encodedRegionName);
|
||||||
if (region == null) {
|
if (region == null) {
|
||||||
MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
|
MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
|
||||||
if (moveInfo != null) {
|
if (moveInfo != null) {
|
||||||
|
|
|
@ -50,11 +50,11 @@ public interface ImmutableOnlineRegions {
|
||||||
* @return List of Region
|
* @return List of Region
|
||||||
* @throws java.io.IOException
|
* @throws java.io.IOException
|
||||||
*/
|
*/
|
||||||
List<Region> getRegions(TableName tableName) throws IOException;
|
List<? extends Region> getRegions(TableName tableName) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get all online regions in this RS.
|
* Get all online regions in this RS.
|
||||||
* @return List of online Region
|
* @return List of online Region
|
||||||
*/
|
*/
|
||||||
List<Region> getRegions();
|
List<? extends Region> getRegions();
|
||||||
}
|
}
|
||||||
|
|
|
@ -110,7 +110,7 @@ public class IncreasingToUpperBoundRegionSplitPolicy extends ConstantSizeRegionS
|
||||||
TableName tablename = region.getTableDescriptor().getTableName();
|
TableName tablename = region.getTableDescriptor().getTableName();
|
||||||
int tableRegionsCount = 0;
|
int tableRegionsCount = 0;
|
||||||
try {
|
try {
|
||||||
List<Region> hri = rss.getRegions(tablename);
|
List<? extends Region> hri = rss.getRegions(tablename);
|
||||||
tableRegionsCount = hri == null || hri.isEmpty() ? 0 : hri.size();
|
tableRegionsCount = hri == null || hri.isEmpty() ? 0 : hri.size();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.debug("Failed getOnlineRegions " + tablename, e);
|
LOG.debug("Failed getOnlineRegions " + tablename, e);
|
||||||
|
|
|
@ -204,7 +204,7 @@ public class LogRoller extends HasThread implements Closeable {
|
||||||
*/
|
*/
|
||||||
private void scheduleFlush(final byte [] encodedRegionName) {
|
private void scheduleFlush(final byte [] encodedRegionName) {
|
||||||
boolean scheduled = false;
|
boolean scheduled = false;
|
||||||
Region r = this.services.getRegion(Bytes.toString(encodedRegionName));
|
HRegion r = (HRegion) this.services.getRegion(Bytes.toString(encodedRegionName));
|
||||||
FlushRequester requester = null;
|
FlushRequester requester = null;
|
||||||
if (r != null) {
|
if (r != null) {
|
||||||
requester = this.services.getFlushRequester();
|
requester = this.services.getFlushRequester();
|
||||||
|
|
|
@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.DroppedSnapshotException;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
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.shaded.com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
@ -126,8 +126,8 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
* @return true if successful
|
* @return true if successful
|
||||||
*/
|
*/
|
||||||
private boolean flushOneForGlobalPressure() {
|
private boolean flushOneForGlobalPressure() {
|
||||||
SortedMap<Long, Region> regionsBySize = server.getCopyOfOnlineRegionsSortedBySize();
|
SortedMap<Long, HRegion> regionsBySize = server.getCopyOfOnlineRegionsSortedBySize();
|
||||||
Set<Region> excludedRegions = new HashSet<>();
|
Set<HRegion> excludedRegions = new HashSet<>();
|
||||||
|
|
||||||
double secondaryMultiplier
|
double secondaryMultiplier
|
||||||
= ServerRegionReplicaUtil.getRegionReplicaStoreFileRefreshMultiplier(conf);
|
= ServerRegionReplicaUtil.getRegionReplicaStoreFileRefreshMultiplier(conf);
|
||||||
|
@ -136,12 +136,12 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
while (!flushedOne) {
|
while (!flushedOne) {
|
||||||
// Find the biggest region that doesn't have too many storefiles
|
// Find the biggest region that doesn't have too many storefiles
|
||||||
// (might be null!)
|
// (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.
|
// Find the biggest region, total, even if it might have too many flushes.
|
||||||
Region bestAnyRegion = getBiggestMemStoreRegion(
|
HRegion bestAnyRegion = getBiggestMemStoreRegion(
|
||||||
regionsBySize, excludedRegions, false);
|
regionsBySize, excludedRegions, false);
|
||||||
// Find the biggest region that is a secondary region
|
// Find the biggest region that is a secondary region
|
||||||
Region bestRegionReplica = getBiggestMemStoreOfRegionReplica(regionsBySize,
|
HRegion bestRegionReplica = getBiggestMemStoreOfRegionReplica(regionsBySize,
|
||||||
excludedRegions);
|
excludedRegions);
|
||||||
|
|
||||||
if (bestAnyRegion == null && bestRegionReplica == null) {
|
if (bestAnyRegion == null && bestRegionReplica == null) {
|
||||||
|
@ -149,7 +149,7 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
Region regionToFlush;
|
HRegion regionToFlush;
|
||||||
if (bestFlushableRegion != null &&
|
if (bestFlushableRegion != null &&
|
||||||
bestAnyRegion.getMemStoreSize() > 2 * bestFlushableRegion.getMemStoreSize()) {
|
bestAnyRegion.getMemStoreSize() > 2 * bestFlushableRegion.getMemStoreSize()) {
|
||||||
// Even if it's not supposed to be flushed, pick a region if it's more than twice
|
// 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(
|
private HRegion getBiggestMemStoreRegion(
|
||||||
SortedMap<Long, Region> regionsBySize,
|
SortedMap<Long, HRegion> regionsBySize,
|
||||||
Set<Region> excludedRegions,
|
Set<HRegion> excludedRegions,
|
||||||
boolean checkStoreFileCount) {
|
boolean checkStoreFileCount) {
|
||||||
synchronized (regionsInQueue) {
|
synchronized (regionsInQueue) {
|
||||||
for (Region region : regionsBySize.values()) {
|
for (HRegion region : regionsBySize.values()) {
|
||||||
if (excludedRegions.contains(region)) {
|
if (excludedRegions.contains(region)) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (((HRegion)region).writestate.flushing ||
|
if (region.writestate.flushing || !region.writestate.writesEnabled) {
|
||||||
!((HRegion)region).writestate.writesEnabled) {
|
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -307,10 +306,10 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Region getBiggestMemStoreOfRegionReplica(SortedMap<Long, Region> regionsBySize,
|
private HRegion getBiggestMemStoreOfRegionReplica(SortedMap<Long, HRegion> regionsBySize,
|
||||||
Set<Region> excludedRegions) {
|
Set<HRegion> excludedRegions) {
|
||||||
synchronized (regionsInQueue) {
|
synchronized (regionsInQueue) {
|
||||||
for (Region region : regionsBySize.values()) {
|
for (HRegion region : regionsBySize.values()) {
|
||||||
if (excludedRegions.contains(region)) {
|
if (excludedRegions.contains(region)) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
@ -349,8 +348,8 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void requestFlush(Region r, boolean forceFlushAllStores) {
|
public void requestFlush(HRegion r, boolean forceFlushAllStores) {
|
||||||
((HRegion)r).incrementFlushesQueuedCount();
|
r.incrementFlushesQueuedCount();
|
||||||
synchronized (regionsInQueue) {
|
synchronized (regionsInQueue) {
|
||||||
if (!regionsInQueue.containsKey(r)) {
|
if (!regionsInQueue.containsKey(r)) {
|
||||||
// This entry has no delay so it will be added at the top of the flush
|
// 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
|
@Override
|
||||||
public void requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) {
|
public void requestDelayedFlush(HRegion r, long delay, boolean forceFlushAllStores) {
|
||||||
((HRegion)r).incrementFlushesQueuedCount();
|
r.incrementFlushesQueuedCount();
|
||||||
synchronized (regionsInQueue) {
|
synchronized (regionsInQueue) {
|
||||||
if (!regionsInQueue.containsKey(r)) {
|
if (!regionsInQueue.containsKey(r)) {
|
||||||
// This entry has some delay
|
// This entry has some delay
|
||||||
|
@ -430,7 +429,7 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
* not flushed.
|
* not flushed.
|
||||||
*/
|
*/
|
||||||
private boolean flushRegion(final FlushRegionEntry fqe) {
|
private boolean flushRegion(final FlushRegionEntry fqe) {
|
||||||
Region region = fqe.region;
|
HRegion region = fqe.region;
|
||||||
if (!region.getRegionInfo().isMetaRegion() &&
|
if (!region.getRegionInfo().isMetaRegion() &&
|
||||||
isTooManyStoreFiles(region)) {
|
isTooManyStoreFiles(region)) {
|
||||||
if (fqe.isMaximumWait(this.blockingWaitTime)) {
|
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
|
* false, there will be accompanying log messages explaining why the region was
|
||||||
* not flushed.
|
* not flushed.
|
||||||
*/
|
*/
|
||||||
private boolean flushRegion(final Region region, final boolean emergencyFlush,
|
private boolean flushRegion(final HRegion region, final boolean emergencyFlush,
|
||||||
boolean forceFlushAllStores) {
|
boolean forceFlushAllStores) {
|
||||||
synchronized (this.regionsInQueue) {
|
synchronized (this.regionsInQueue) {
|
||||||
FlushRegionEntry fqe = this.regionsInQueue.remove(region);
|
FlushRegionEntry fqe = this.regionsInQueue.remove(region);
|
||||||
|
@ -497,12 +496,11 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
FlushResult flushResult = region.flush(forceFlushAllStores);
|
FlushResult flushResult = region.flush(forceFlushAllStores);
|
||||||
boolean shouldCompact = flushResult.isCompactionNeeded();
|
boolean shouldCompact = flushResult.isCompactionNeeded();
|
||||||
// We just want to check the size
|
// We just want to check the size
|
||||||
boolean shouldSplit = ((HRegion)region).checkSplit() != null;
|
boolean shouldSplit = region.checkSplit() != null;
|
||||||
if (shouldSplit) {
|
if (shouldSplit) {
|
||||||
this.server.compactSplitThread.requestSplit(region);
|
this.server.compactSplitThread.requestSplit(region);
|
||||||
} else if (shouldCompact) {
|
} else if (shouldCompact) {
|
||||||
server.compactSplitThread.requestSystemCompaction((HRegion) region,
|
server.compactSplitThread.requestSystemCompaction(region, Thread.currentThread().getName());
|
||||||
Thread.currentThread().getName());
|
|
||||||
}
|
}
|
||||||
} catch (DroppedSnapshotException ex) {
|
} catch (DroppedSnapshotException ex) {
|
||||||
// Cache flush can fail in a few places. If it fails in a critical
|
// Cache flush can fail in a few places. If it fails in a critical
|
||||||
|
@ -728,7 +726,7 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
* a while.
|
* a while.
|
||||||
*/
|
*/
|
||||||
static class FlushRegionEntry implements FlushQueueEntry {
|
static class FlushRegionEntry implements FlushQueueEntry {
|
||||||
private final Region region;
|
private final HRegion region;
|
||||||
|
|
||||||
private final long createTime;
|
private final long createTime;
|
||||||
private long whenToExpire;
|
private long whenToExpire;
|
||||||
|
@ -736,7 +734,7 @@ class MemStoreFlusher implements FlushRequester {
|
||||||
|
|
||||||
private boolean forceFlushAllStores;
|
private boolean forceFlushAllStores;
|
||||||
|
|
||||||
FlushRegionEntry(final Region r, boolean forceFlushAllStores) {
|
FlushRegionEntry(final HRegion r, boolean forceFlushAllStores) {
|
||||||
this.region = r;
|
this.region = r;
|
||||||
this.createTime = EnvironmentEdgeManager.currentTime();
|
this.createTime = EnvironmentEdgeManager.currentTime();
|
||||||
this.whenToExpire = this.createTime;
|
this.whenToExpire = this.createTime;
|
||||||
|
|
|
@ -214,7 +214,7 @@ class MetricsRegionServerWrapperImpl
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getNumOnlineRegions() {
|
public long getNumOnlineRegions() {
|
||||||
Collection<Region> onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext();
|
Collection<HRegion> onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext();
|
||||||
if (onlineRegionsLocalContext == null) {
|
if (onlineRegionsLocalContext == null) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
@ -754,7 +754,7 @@ class MetricsRegionServerWrapperImpl
|
||||||
long tempMobScanCellsSize = 0;
|
long tempMobScanCellsSize = 0;
|
||||||
long tempBlockedRequestsCount = 0;
|
long tempBlockedRequestsCount = 0;
|
||||||
int regionCount = 0;
|
int regionCount = 0;
|
||||||
for (Region r : regionServer.getOnlineRegionsLocalContext()) {
|
for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
|
||||||
tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL();
|
tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL();
|
||||||
tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL();
|
tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL();
|
||||||
tempReadRequestsCount += r.getReadRequestsCount();
|
tempReadRequestsCount += r.getReadRequestsCount();
|
||||||
|
|
|
@ -32,7 +32,7 @@ public interface OnlineRegions extends ImmutableOnlineRegions {
|
||||||
* Add to online regions.
|
* Add to online regions.
|
||||||
* @param r
|
* @param r
|
||||||
*/
|
*/
|
||||||
void addRegion(final Region r);
|
void addRegion(final HRegion r);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method removes Region corresponding to hri from the Map of onlineRegions.
|
* 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.
|
* @param destination Destination, if any, null otherwise.
|
||||||
* @return True if we removed a region from online list.
|
* @return True if we removed a region from online list.
|
||||||
*/
|
*/
|
||||||
boolean removeRegion(final Region r, ServerName destination);
|
boolean removeRegion(final HRegion r, ServerName destination);
|
||||||
}
|
}
|
||||||
|
|
|
@ -388,13 +388,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
private final AtomicLong nextCallSeq = new AtomicLong(0);
|
private final AtomicLong nextCallSeq = new AtomicLong(0);
|
||||||
private final String scannerName;
|
private final String scannerName;
|
||||||
private final RegionScanner s;
|
private final RegionScanner s;
|
||||||
private final Region r;
|
private final HRegion r;
|
||||||
private final RpcCallback closeCallBack;
|
private final RpcCallback closeCallBack;
|
||||||
private final RpcCallback shippedCallback;
|
private final RpcCallback shippedCallback;
|
||||||
private byte[] rowOfLastPartialResult;
|
private byte[] rowOfLastPartialResult;
|
||||||
private boolean needCursor;
|
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) {
|
RpcCallback closeCallBack, RpcCallback shippedCallback, boolean needCursor) {
|
||||||
this.scannerName = scannerName;
|
this.scannerName = scannerName;
|
||||||
this.s = s;
|
this.s = s;
|
||||||
|
@ -432,7 +432,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
RegionScanner s = rsh.s;
|
RegionScanner s = rsh.s;
|
||||||
LOG.info("Scanner " + this.scannerName + " lease expired on region "
|
LOG.info("Scanner " + this.scannerName + " lease expired on region "
|
||||||
+ s.getRegionInfo().getRegionNameAsString());
|
+ s.getRegionInfo().getRegionNameAsString());
|
||||||
Region region = null;
|
HRegion region = null;
|
||||||
try {
|
try {
|
||||||
region = regionServer.getRegion(s.getRegionInfo().getRegionName());
|
region = regionServer.getRegion(s.getRegionInfo().getRegionName());
|
||||||
if (region != null && region.getCoprocessorHost() != null) {
|
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.
|
* @param cellScanner if non-null, the mutation data -- the Cell content.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private void mutateRows(final Region region,
|
private void mutateRows(final HRegion region,
|
||||||
final List<ClientProtos.Action> actions,
|
final List<ClientProtos.Action> actions,
|
||||||
final CellScanner cellScanner, RegionActionResult.Builder builder) throws IOException {
|
final CellScanner cellScanner, RegionActionResult.Builder builder) throws IOException {
|
||||||
if (!region.getRegionInfo().isMetaTable()) {
|
if (!region.getRegionInfo().isMetaTable()) {
|
||||||
|
@ -600,7 +600,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
* @param compareOp
|
* @param compareOp
|
||||||
* @param comparator @throws IOException
|
* @param comparator @throws IOException
|
||||||
*/
|
*/
|
||||||
private boolean checkAndRowMutate(final Region region, final List<ClientProtos.Action> actions,
|
private boolean checkAndRowMutate(final HRegion region, final List<ClientProtos.Action> actions,
|
||||||
final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
|
final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
|
||||||
CompareOperator op, ByteArrayComparable comparator, RegionActionResult.Builder builder,
|
CompareOperator op, ByteArrayComparable comparator, RegionActionResult.Builder builder,
|
||||||
ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
|
ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
|
||||||
|
@ -656,7 +656,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
* bypassed as indicated by RegionObserver, null otherwise
|
* bypassed as indicated by RegionObserver, null otherwise
|
||||||
* @throws IOException
|
* @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,
|
final MutationProto mutation, final CellScanner cellScanner, long nonceGroup,
|
||||||
ActivePolicyEnforcement spaceQuota)
|
ActivePolicyEnforcement spaceQuota)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
@ -707,7 +707,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
* @return the Result
|
* @return the Result
|
||||||
* @throws IOException
|
* @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,
|
final MutationProto mutation, final CellScanner cells, long nonceGroup,
|
||||||
ActivePolicyEnforcement spaceQuota)
|
ActivePolicyEnforcement spaceQuota)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
@ -763,7 +763,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
* @param context the current RpcCallContext
|
* @param context the current RpcCallContext
|
||||||
* @return Return the <code>cellScanner</code> passed
|
* @return Return the <code>cellScanner</code> passed
|
||||||
*/
|
*/
|
||||||
private List<CellScannable> doNonAtomicRegionMutation(final Region region,
|
private List<CellScannable> doNonAtomicRegionMutation(final HRegion region,
|
||||||
final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner,
|
final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner,
|
||||||
final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup,
|
final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup,
|
||||||
final RegionScannersCloseCallBack closeCallBack, RpcCallContext context,
|
final RegionScannersCloseCallBack closeCallBack, RpcCallContext context,
|
||||||
|
@ -926,11 +926,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
return cellsToReturn;
|
return cellsToReturn;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void checkCellSizeLimit(final Region region, final Mutation m) throws IOException {
|
private void checkCellSizeLimit(final HRegion r, final Mutation m) throws IOException {
|
||||||
if (!(region instanceof HRegion)) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
HRegion r = (HRegion)region;
|
|
||||||
if (r.maxCellSize > 0) {
|
if (r.maxCellSize > 0) {
|
||||||
CellScanner cells = m.cellScanner();
|
CellScanner cells = m.cellScanner();
|
||||||
while (cells.advance()) {
|
while (cells.advance()) {
|
||||||
|
@ -953,7 +949,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
* @param region
|
* @param region
|
||||||
* @param mutations
|
* @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<ClientProtos.Action> mutations,
|
final OperationQuota quota, final List<ClientProtos.Action> mutations,
|
||||||
final CellScanner cells, ActivePolicyEnforcement spaceQuotaEnforcement) {
|
final CellScanner cells, ActivePolicyEnforcement spaceQuotaEnforcement) {
|
||||||
Mutation[] mArray = new Mutation[mutations.size()];
|
Mutation[] mArray = new Mutation[mutations.size()];
|
||||||
|
@ -1050,7 +1046,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
* exceptionMessage if any
|
* exceptionMessage if any
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private OperationStatus [] doReplayBatchOp(final Region region,
|
private OperationStatus [] doReplayBatchOp(final HRegion region,
|
||||||
final List<WALSplitter.MutationReplay> mutations, long replaySeqId) throws IOException {
|
final List<WALSplitter.MutationReplay> mutations, long replaySeqId) throws IOException {
|
||||||
long before = EnvironmentEdgeManager.currentTime();
|
long before = EnvironmentEdgeManager.currentTime();
|
||||||
boolean batchContainsPuts = false, batchContainsDelete = false;
|
boolean batchContainsPuts = false, batchContainsDelete = false;
|
||||||
|
@ -1335,7 +1331,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
return lastBlock;
|
return lastBlock;
|
||||||
}
|
}
|
||||||
|
|
||||||
private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r,
|
private RegionScannerHolder addScanner(String scannerName, RegionScanner s, HRegion r,
|
||||||
boolean needCursor) throws LeaseStillHeldException {
|
boolean needCursor) throws LeaseStillHeldException {
|
||||||
Lease lease = regionServer.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod,
|
Lease lease = regionServer.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod,
|
||||||
new ScannerListener(scannerName));
|
new ScannerListener(scannerName));
|
||||||
|
@ -1363,7 +1359,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
* but failed to find the region
|
* but failed to find the region
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public Region getRegion(
|
public HRegion getRegion(
|
||||||
final RegionSpecifier regionSpecifier) throws IOException {
|
final RegionSpecifier regionSpecifier) throws IOException {
|
||||||
ByteString value = regionSpecifier.getValue();
|
ByteString value = regionSpecifier.getValue();
|
||||||
RegionSpecifierType type = regionSpecifier.getType();
|
RegionSpecifierType type = regionSpecifier.getType();
|
||||||
|
@ -1652,9 +1648,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
try {
|
try {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
requestCount.increment();
|
requestCount.increment();
|
||||||
Map<String, Region> onlineRegions = regionServer.onlineRegions;
|
Map<String, HRegion> onlineRegions = regionServer.onlineRegions;
|
||||||
List<RegionInfo> list = new ArrayList<>(onlineRegions.size());
|
List<RegionInfo> list = new ArrayList<>(onlineRegions.size());
|
||||||
for (Region region: onlineRegions.values()) {
|
for (HRegion region: onlineRegions.values()) {
|
||||||
list.add(region.getRegionInfo());
|
list.add(region.getRegionInfo());
|
||||||
}
|
}
|
||||||
Collections.sort(list, RegionInfo.COMPARATOR);
|
Collections.sort(list, RegionInfo.COMPARATOR);
|
||||||
|
@ -1671,7 +1667,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
try {
|
try {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
requestCount.increment();
|
requestCount.increment();
|
||||||
Region region = getRegion(request.getRegion());
|
HRegion region = getRegion(request.getRegion());
|
||||||
RegionInfo info = region.getRegionInfo();
|
RegionInfo info = region.getRegionInfo();
|
||||||
byte[] bestSplitRow = null;
|
byte[] bestSplitRow = null;
|
||||||
if (request.hasBestSplitRow() && request.getBestSplitRow()) {
|
if (request.hasBestSplitRow() && request.getBestSplitRow()) {
|
||||||
|
@ -1690,7 +1686,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
|
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
|
||||||
builder.setRegionInfo(ProtobufUtil.toRegionInfo(info));
|
builder.setRegionInfo(ProtobufUtil.toRegionInfo(info));
|
||||||
if (request.hasCompactionState() && request.getCompactionState()) {
|
if (request.hasCompactionState() && request.getCompactionState()) {
|
||||||
builder.setCompactionState(region.getCompactionState());
|
builder.setCompactionState(ProtobufUtil.createCompactionState(region.getCompactionState()));
|
||||||
}
|
}
|
||||||
builder.setSplittable(region.isSplittable());
|
builder.setSplittable(region.isSplittable());
|
||||||
builder.setMergeable(region.isMergeable());
|
builder.setMergeable(region.isMergeable());
|
||||||
|
@ -1709,7 +1705,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
public GetRegionLoadResponse getRegionLoad(RpcController controller,
|
public GetRegionLoadResponse getRegionLoad(RpcController controller,
|
||||||
GetRegionLoadRequest request) throws ServiceException {
|
GetRegionLoadRequest request) throws ServiceException {
|
||||||
|
|
||||||
List<Region> regions;
|
List<HRegion> regions;
|
||||||
if (request.hasTableName()) {
|
if (request.hasTableName()) {
|
||||||
TableName tableName = ProtobufUtil.toTableName(request.getTableName());
|
TableName tableName = ProtobufUtil.toTableName(request.getTableName());
|
||||||
regions = regionServer.getRegions(tableName);
|
regions = regionServer.getRegions(tableName);
|
||||||
|
@ -1721,7 +1717,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
|
RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
for (Region region : regions) {
|
for (HRegion region : regions) {
|
||||||
rLoads.add(regionServer.createRegionLoad(region, regionLoadBuilder, regionSpecifier));
|
rLoads.add(regionServer.createRegionLoad(region, regionLoadBuilder, regionSpecifier));
|
||||||
}
|
}
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
@ -1797,7 +1793,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
final GetStoreFileRequest request) throws ServiceException {
|
final GetStoreFileRequest request) throws ServiceException {
|
||||||
try {
|
try {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
Region region = getRegion(request.getRegion());
|
HRegion region = getRegion(request.getRegion());
|
||||||
requestCount.increment();
|
requestCount.increment();
|
||||||
Set<byte[]> columnFamilies;
|
Set<byte[]> columnFamilies;
|
||||||
if (request.getFamilyCount() == 0) {
|
if (request.getFamilyCount() == 0) {
|
||||||
|
@ -1902,7 +1898,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
try {
|
try {
|
||||||
String encodedName = region.getEncodedName();
|
String encodedName = region.getEncodedName();
|
||||||
byte[] encodedNameBytes = region.getEncodedNameAsBytes();
|
byte[] encodedNameBytes = region.getEncodedNameAsBytes();
|
||||||
final Region onlineRegion = regionServer.getRegion(encodedName);
|
final HRegion onlineRegion = regionServer.getRegion(encodedName);
|
||||||
if (onlineRegion != null) {
|
if (onlineRegion != null) {
|
||||||
// The region is already online. This should not happen any more.
|
// The region is already online. This should not happen any more.
|
||||||
String error = "Received OPEN for the region:"
|
String error = "Received OPEN for the region:"
|
||||||
|
@ -2026,7 +2022,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
checkOpen();
|
checkOpen();
|
||||||
String encodedName = region.getEncodedName();
|
String encodedName = region.getEncodedName();
|
||||||
byte[] encodedNameBytes = region.getEncodedNameAsBytes();
|
byte[] encodedNameBytes = region.getEncodedNameAsBytes();
|
||||||
final Region onlineRegion = regionServer.getRegion(encodedName);
|
final HRegion onlineRegion = regionServer.getRegion(encodedName);
|
||||||
|
|
||||||
if (onlineRegion != null) {
|
if (onlineRegion != null) {
|
||||||
LOG.info("Region already online. Skipping warming up " + region);
|
LOG.info("Region already online. Skipping warming up " + region);
|
||||||
|
@ -2077,7 +2073,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
return ReplicateWALEntryResponse.newBuilder().build();
|
return ReplicateWALEntryResponse.newBuilder().build();
|
||||||
}
|
}
|
||||||
ByteString regionName = entries.get(0).getKey().getEncodedRegionName();
|
ByteString regionName = entries.get(0).getKey().getEncodedRegionName();
|
||||||
Region region = regionServer.getRegionByEncodedName(regionName.toStringUtf8());
|
HRegion region = regionServer.getRegionByEncodedName(regionName.toStringUtf8());
|
||||||
RegionCoprocessorHost coprocessorHost =
|
RegionCoprocessorHost coprocessorHost =
|
||||||
ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo())
|
ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo())
|
||||||
? region.getCoprocessorHost()
|
? region.getCoprocessorHost()
|
||||||
|
@ -2133,7 +2129,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
}
|
}
|
||||||
|
|
||||||
//sync wal at the end because ASYNC_WAL is used above
|
//sync wal at the end because ASYNC_WAL is used above
|
||||||
WAL wal = getWAL(region);
|
WAL wal = region.getWAL();
|
||||||
if (wal != null) {
|
if (wal != null) {
|
||||||
wal.sync();
|
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.
|
* Replicate WAL entries on the region server.
|
||||||
*
|
*
|
||||||
|
@ -2258,7 +2250,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
try {
|
try {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
requestCount.increment();
|
requestCount.increment();
|
||||||
Region region = getRegion(request.getRegion());
|
HRegion region = getRegion(request.getRegion());
|
||||||
boolean bypass = false;
|
boolean bypass = false;
|
||||||
boolean loaded = false;
|
boolean loaded = false;
|
||||||
Map<byte[], List<Path>> map = null;
|
Map<byte[], List<Path>> map = null;
|
||||||
|
@ -2328,7 +2320,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
checkOpen();
|
checkOpen();
|
||||||
requestCount.increment();
|
requestCount.increment();
|
||||||
|
|
||||||
Region region = getRegion(request.getRegion());
|
HRegion region = getRegion(request.getRegion());
|
||||||
|
|
||||||
String bulkToken = regionServer.secureBulkLoadManager.prepareBulkLoad(region, request);
|
String bulkToken = regionServer.secureBulkLoadManager.prepareBulkLoad(region, request);
|
||||||
PrepareBulkLoadResponse.Builder builder = PrepareBulkLoadResponse.newBuilder();
|
PrepareBulkLoadResponse.Builder builder = PrepareBulkLoadResponse.newBuilder();
|
||||||
|
@ -2346,7 +2338,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
checkOpen();
|
checkOpen();
|
||||||
requestCount.increment();
|
requestCount.increment();
|
||||||
|
|
||||||
Region region = getRegion(request.getRegion());
|
HRegion region = getRegion(request.getRegion());
|
||||||
|
|
||||||
regionServer.secureBulkLoadManager.cleanupBulkLoad(region, request);
|
regionServer.secureBulkLoadManager.cleanupBulkLoad(region, request);
|
||||||
CleanupBulkLoadResponse response = CleanupBulkLoadResponse.newBuilder().build();
|
CleanupBulkLoadResponse response = CleanupBulkLoadResponse.newBuilder().build();
|
||||||
|
@ -2362,7 +2354,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
try {
|
try {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
requestCount.increment();
|
requestCount.increment();
|
||||||
Region region = getRegion(request.getRegion());
|
HRegion region = getRegion(request.getRegion());
|
||||||
com.google.protobuf.Message result = execServiceOnRegion(region, request.getCall());
|
com.google.protobuf.Message result = execServiceOnRegion(region, request.getCall());
|
||||||
CoprocessorServiceResponse.Builder builder = CoprocessorServiceResponse.newBuilder();
|
CoprocessorServiceResponse.Builder builder = CoprocessorServiceResponse.newBuilder();
|
||||||
builder.setRegion(RequestConverter.buildRegionSpecifier(
|
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 {
|
final ClientProtos.CoprocessorServiceCall serviceCall) throws IOException {
|
||||||
// ignore the passed in controller (from the serialized call)
|
// ignore the passed in controller (from the serialized call)
|
||||||
ServerRpcController execController = new ServerRpcController();
|
ServerRpcController execController = new ServerRpcController();
|
||||||
|
@ -2401,7 +2393,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
requestCount.increment();
|
requestCount.increment();
|
||||||
requestRowActionCount.increment();
|
requestRowActionCount.increment();
|
||||||
rpcGetRequestCount.increment();
|
rpcGetRequestCount.increment();
|
||||||
Region region = getRegion(request.getRegion());
|
HRegion region = getRegion(request.getRegion());
|
||||||
|
|
||||||
GetResponse.Builder builder = GetResponse.newBuilder();
|
GetResponse.Builder builder = GetResponse.newBuilder();
|
||||||
ClientProtos.Get get = request.getGet();
|
ClientProtos.Get get = request.getGet();
|
||||||
|
@ -2567,7 +2559,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
for (RegionAction regionAction : request.getRegionActionList()) {
|
for (RegionAction regionAction : request.getRegionActionList()) {
|
||||||
this.requestRowActionCount.add(regionAction.getActionCount());
|
this.requestRowActionCount.add(regionAction.getActionCount());
|
||||||
OperationQuota quota;
|
OperationQuota quota;
|
||||||
Region region;
|
HRegion region;
|
||||||
regionActionResultBuilder.clear();
|
regionActionResultBuilder.clear();
|
||||||
RegionSpecifier regionSpecifier = regionAction.getRegion();
|
RegionSpecifier regionSpecifier = regionAction.getRegion();
|
||||||
try {
|
try {
|
||||||
|
@ -2702,7 +2694,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
requestCount.increment();
|
requestCount.increment();
|
||||||
requestRowActionCount.increment();
|
requestRowActionCount.increment();
|
||||||
rpcMutateRequestCount.increment();
|
rpcMutateRequestCount.increment();
|
||||||
Region region = getRegion(request.getRegion());
|
HRegion region = getRegion(request.getRegion());
|
||||||
MutateResponse.Builder builder = MutateResponse.newBuilder();
|
MutateResponse.Builder builder = MutateResponse.newBuilder();
|
||||||
MutationProto mutation = request.getMutation();
|
MutationProto mutation = request.getMutation();
|
||||||
if (!region.getRegionInfo().isMetaTable()) {
|
if (!region.getRegionInfo().isMetaTable()) {
|
||||||
|
@ -2892,7 +2884,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
|
|
||||||
private RegionScannerHolder newRegionScanner(ScanRequest request, ScanResponse.Builder builder)
|
private RegionScannerHolder newRegionScanner(ScanRequest request, ScanResponse.Builder builder)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Region region = getRegion(request.getRegion());
|
HRegion region = getRegion(request.getRegion());
|
||||||
ClientProtos.Scan protoScan = request.getScan();
|
ClientProtos.Scan protoScan = request.getScan();
|
||||||
boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand();
|
boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand();
|
||||||
Scan scan = ProtobufUtil.toScan(protoScan);
|
Scan scan = ProtobufUtil.toScan(protoScan);
|
||||||
|
@ -2992,7 +2984,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
long maxQuotaResultSize, int maxResults, int limitOfRows, List<Result> results,
|
long maxQuotaResultSize, int maxResults, int limitOfRows, List<Result> results,
|
||||||
ScanResponse.Builder builder, MutableObject lastBlock, RpcCallContext context)
|
ScanResponse.Builder builder, MutableObject lastBlock, RpcCallContext context)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Region region = rsh.r;
|
HRegion region = rsh.r;
|
||||||
RegionScanner scanner = rsh.s;
|
RegionScanner scanner = rsh.s;
|
||||||
long maxResultSize;
|
long maxResultSize;
|
||||||
if (scanner.getMaxResultSize() > 0) {
|
if (scanner.getMaxResultSize() > 0) {
|
||||||
|
@ -3226,7 +3218,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
||||||
}
|
}
|
||||||
throw new ServiceException(e);
|
throw new ServiceException(e);
|
||||||
}
|
}
|
||||||
Region region = rsh.r;
|
HRegion region = rsh.r;
|
||||||
String scannerName = rsh.scannerName;
|
String scannerName = rsh.scannerName;
|
||||||
Leases.Lease lease;
|
Leases.Lease lease;
|
||||||
try {
|
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 {
|
RpcCallContext context) throws IOException {
|
||||||
if (region.getCoprocessorHost() != null) {
|
if (region.getCoprocessorHost() != null) {
|
||||||
if (region.getCoprocessorHost().preScannerClose(scanner)) {
|
if (region.getCoprocessorHost().preScannerClose(scanner)) {
|
||||||
|
|
|
@ -22,13 +22,12 @@ import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.CellComparator;
|
import org.apache.hadoop.hbase.CellComparator;
|
||||||
import org.apache.hadoop.hbase.CompareOperator;
|
import org.apache.hadoop.hbase.CompareOperator;
|
||||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
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.Append;
|
||||||
|
import org.apache.hadoop.hbase.client.CompactionState;
|
||||||
import org.apache.hadoop.hbase.client.Delete;
|
import org.apache.hadoop.hbase.client.Delete;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
import org.apache.hadoop.hbase.client.Increment;
|
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.Scan;
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
|
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.filter.ByteArrayComparable;
|
||||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
|
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
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.InterfaceAudience;
|
||||||
import org.apache.yetus.audience.InterfaceStability;
|
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
|
* Region is a subset of HRegion with operations required for the {@link RegionCoprocessor
|
||||||
* for each row. A given table consists of one or more Regions.
|
* Coprocessors}. The operations include ability to do mutations, requesting compaction, getting
|
||||||
*
|
* different counters/sizes, locking rows and getting access to {@linkplain Store}s.
|
||||||
* <p>An Region is defined by its table and its key extent.
|
|
||||||
*
|
|
||||||
* <p>Locking at the Region level serves only one purpose: preventing the
|
|
||||||
* region from being closed (and consequently split) while other operations
|
|
||||||
* are ongoing. Each row level operation obtains both a row lock and a region
|
|
||||||
* read lock for the duration of the operation. While a scanner is being
|
|
||||||
* constructed, getScanner holds a read lock. If the scanner is successfully
|
|
||||||
* constructed, it holds a read lock until it is closed. A close takes out a
|
|
||||||
* write lock and consequently will block for ongoing operations and will block
|
|
||||||
* new operations from starting while the close is in progress.
|
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Evolving
|
||||||
|
@ -131,24 +113,11 @@ public interface Region extends ConfigurationObserver {
|
||||||
*/
|
*/
|
||||||
boolean refreshStoreFiles() throws IOException;
|
boolean refreshStoreFiles() throws IOException;
|
||||||
|
|
||||||
/** @return the latest sequence number that was read from storage when this region was opened */
|
|
||||||
long getOpenSeqNum();
|
|
||||||
|
|
||||||
/** @return the max sequence id of flushed data on this region; no edit in memory will have
|
/** @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.
|
* a sequence id that is less that what is returned here.
|
||||||
*/
|
*/
|
||||||
long getMaxFlushedSeqId();
|
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.
|
* 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
|
* @param majorCompactionOnly Only consider HFile that are the result of major compaction
|
||||||
|
@ -162,19 +131,9 @@ public interface Region extends ConfigurationObserver {
|
||||||
*/
|
*/
|
||||||
public Map<byte[], Long> getMaxStoreSeqId();
|
public Map<byte[], Long> getMaxStoreSeqId();
|
||||||
|
|
||||||
/** @return true if loading column families on demand by default */
|
|
||||||
boolean isLoadingCfsOnDemandDefault();
|
|
||||||
|
|
||||||
/** @return readpoint considering given IsolationLevel; pass null for default*/
|
/** @return readpoint considering given IsolationLevel; pass null for default*/
|
||||||
long getReadPoint(IsolationLevel isolationLevel);
|
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
|
* @return The earliest time a store in the region was flushed. All
|
||||||
* other stores in the region would have been flushed either at, or
|
* 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 */
|
/** @return read requests count for this region */
|
||||||
long getReadRequestsCount();
|
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 */
|
/** @return filtered read requests count for this region */
|
||||||
long getFilteredReadRequestsCount();
|
long getFilteredReadRequestsCount();
|
||||||
|
|
||||||
/** @return write request count for this region */
|
/** @return write request count for this region */
|
||||||
long getWriteRequestsCount();
|
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
|
* @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.
|
* 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();
|
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 */
|
/** @return the number of mutations processed bypassing the WAL */
|
||||||
long getNumMutationsWithoutWAL();
|
long getNumMutationsWithoutWAL();
|
||||||
|
|
||||||
|
@ -231,12 +175,6 @@ public interface Region extends ConfigurationObserver {
|
||||||
/** @return the number of failed checkAndMutate guards */
|
/** @return the number of failed checkAndMutate guards */
|
||||||
long getCheckAndMutateChecksFailed();
|
long getCheckAndMutateChecksFailed();
|
||||||
|
|
||||||
/** @return the MetricsRegion for this region */
|
|
||||||
MetricsRegion getMetrics();
|
|
||||||
|
|
||||||
/** @return the block distribution for all Stores managed by this region */
|
|
||||||
HDFSBlocksDistribution getHDFSBlocksDistribution();
|
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////
|
||||||
// Locking
|
// Locking
|
||||||
|
|
||||||
|
@ -259,6 +197,8 @@ public interface Region extends ConfigurationObserver {
|
||||||
* the operation has completed, whether it succeeded or failed.
|
* the operation has completed, whether it succeeded or failed.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
|
// TODO Exposing this and closeRegionOperation() as we have getRowLock() exposed.
|
||||||
|
// Remove if we get rid of exposing getRowLock().
|
||||||
void startRegionOperation() throws IOException;
|
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
|
* 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).
|
* started (the calling thread has already acquired the region-close-guard lock).
|
||||||
*
|
* <p>
|
||||||
|
* The obtained locks should be released after use by {@link RowLock#release()}
|
||||||
|
* <p>
|
||||||
* NOTE: the boolean passed here has changed. It used to be a boolean that
|
* 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
|
* stated whether or not to wait on the lock. Now it is whether it an exclusive
|
||||||
* lock is requested.
|
* lock is requested.
|
||||||
|
@ -319,50 +261,33 @@ public interface Region extends ConfigurationObserver {
|
||||||
* @see #startRegionOperation()
|
* @see #startRegionOperation()
|
||||||
* @see #startRegionOperation(Operation)
|
* @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;
|
RowLock getRowLock(byte[] row, boolean readLock) throws IOException;
|
||||||
|
|
||||||
/**
|
|
||||||
* If the given list of row locks is not null, releases all locks.
|
|
||||||
*/
|
|
||||||
void releaseRowLocks(List<RowLock> rowLocks);
|
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////
|
||||||
// Region operations
|
// Region operations
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Perform one or more append operations on a row.
|
* Perform one or more append operations on a row.
|
||||||
* @param append
|
* @param append
|
||||||
* @param nonceGroup
|
|
||||||
* @param nonce
|
|
||||||
* @return result of the operation
|
* @return result of the operation
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
Result append(Append append, long nonceGroup, long nonce) throws IOException;
|
Result append(Append append) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Perform a batch of mutations.
|
* Perform a batch of mutations.
|
||||||
* <p>
|
* <p>
|
||||||
* Note this supports only Put and Delete mutations and will ignore other types passed.
|
* Note this supports only Put and Delete mutations and will ignore other types passed.
|
||||||
* @param mutations the list of mutations
|
* @param mutations the list of mutations
|
||||||
* @param nonceGroup
|
|
||||||
* @param nonce
|
|
||||||
* @return an array of OperationStatus which internally contains the
|
* @return an array of OperationStatus which internally contains the
|
||||||
* OperationStatusCode and the exceptionMessage if any.
|
* OperationStatusCode and the exceptionMessage if any.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce)
|
OperationStatus[] batchMutate(Mutation[] mutations)
|
||||||
throws IOException;
|
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,
|
* 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
|
* 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<Cell> get(Get get, boolean withCoprocessor) throws IOException;
|
List<Cell> 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<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce) throws IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return an iterator that scans over the HRegion, returning the indicated
|
* Return an iterator that scans over the HRegion, returning the indicated
|
||||||
* columns and rows specified by the {@link Scan}.
|
* 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.
|
* Perform one or more increment operations on a row.
|
||||||
* @param increment
|
* @param increment
|
||||||
* @param nonceGroup
|
|
||||||
* @param nonce
|
|
||||||
* @return result of the operation
|
* @return result of the operation
|
||||||
* @throws IOException
|
* @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
|
* Performs multiple mutations atomically on a single row. Currently
|
||||||
|
@ -496,6 +408,8 @@ public interface Region extends ConfigurationObserver {
|
||||||
* <code>rowsToLock</code> is sorted in order to avoid deadlocks.
|
* <code>rowsToLock</code> is sorted in order to avoid deadlocks.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
|
// TODO Should not be exposing with params nonceGroup, nonce. Change when doing the jira for
|
||||||
|
// Changing processRowsWithLocks and RowProcessor
|
||||||
void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock,
|
void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock,
|
||||||
long nonceGroup, long nonce) throws IOException;
|
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 nonceGroup Optional nonce group of the operation (client Id)
|
||||||
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
|
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
|
||||||
*/
|
*/
|
||||||
|
// 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)
|
void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
|
@ -525,6 +441,8 @@ public interface Region extends ConfigurationObserver {
|
||||||
* @param nonceGroup Optional nonce group of the operation (client Id)
|
* @param nonceGroup Optional nonce group of the operation (client Id)
|
||||||
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
|
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
|
||||||
*/
|
*/
|
||||||
|
// 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)
|
void processRowsWithLocks(RowProcessor<?,?> processor, long timeout, long nonceGroup, long nonce)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
|
@ -535,216 +453,14 @@ public interface Region extends ConfigurationObserver {
|
||||||
*/
|
*/
|
||||||
void put(Put put) throws IOException;
|
void put(Put put) throws IOException;
|
||||||
|
|
||||||
/**
|
|
||||||
* Listener class to enable callers of
|
|
||||||
* bulkLoadHFile() to perform any necessary
|
|
||||||
* pre/post processing of a given bulkload call
|
|
||||||
*/
|
|
||||||
interface BulkLoadListener {
|
|
||||||
/**
|
|
||||||
* Called before an HFile is actually loaded
|
|
||||||
* @param family family being loaded to
|
|
||||||
* @param srcPath path of HFile
|
|
||||||
* @return final path to be used for actual loading
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
String prepareBulkLoad(byte[] family, String srcPath, 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<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> 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<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths,
|
|
||||||
boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile) throws IOException;
|
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////
|
|
||||||
// Coprocessors
|
|
||||||
|
|
||||||
/** @return the coprocessor host */
|
/** @return the coprocessor host */
|
||||||
|
// TODO To be removed by HBASE-18954
|
||||||
RegionCoprocessorHost getCoprocessorHost();
|
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.
|
|
||||||
*
|
|
||||||
* <p>
|
|
||||||
* 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}.
|
|
||||||
* </p>
|
|
||||||
* @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<byte[]> families) throws NoSuchColumnFamilyException;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Check the collection of families for valid timestamps
|
|
||||||
* @param familyMap
|
|
||||||
* @param now current timestamp
|
|
||||||
* @throws FailedSanityCheckException
|
|
||||||
*/
|
|
||||||
void checkTimestamps(Map<byte[], List<Cell>> 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.
|
|
||||||
* <p>Caller should have the row and region locks.
|
|
||||||
* @param mutation
|
|
||||||
* @param familyCellMap
|
|
||||||
* @param now
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> 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<List<Cell>> values, final byte[] now)
|
|
||||||
throws IOException;
|
|
||||||
|
|
||||||
///////////////////////////////////////////////////////////////////////////
|
///////////////////////////////////////////////////////////////////////////
|
||||||
// Flushes, compactions, splits, etc.
|
// Flushes, compactions, splits, etc.
|
||||||
// Wizards only, please
|
// 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.
|
|
||||||
*
|
|
||||||
* <p>When this method is called the cache will be flushed unless:
|
|
||||||
* <ol>
|
|
||||||
* <li>the cache is empty</li>
|
|
||||||
* <li>the region is closed.</li>
|
|
||||||
* <li>a flush is already in progress</li>
|
|
||||||
* <li>writes are disabled</li>
|
|
||||||
* </ol>
|
|
||||||
*
|
|
||||||
* <p>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.
|
|
||||||
* <p>This operation could block for a long time, so don't call it from a
|
|
||||||
* time-sensitive thread.
|
|
||||||
* <p>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.
|
* Trigger major compaction on all stores in the region.
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -770,11 +486,4 @@ public interface Region extends ConfigurationObserver {
|
||||||
*/
|
*/
|
||||||
void requestCompaction(byte[] family, String why, int priority,
|
void requestCompaction(byte[] family, String why, int priority,
|
||||||
CompactionLifeCycleTracker tracker, User user) throws IOException;
|
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();
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -201,7 +201,7 @@ public class RegionCoprocessorHost
|
||||||
/** The region server services */
|
/** The region server services */
|
||||||
RegionServerServices rsServices;
|
RegionServerServices rsServices;
|
||||||
/** The region */
|
/** The region */
|
||||||
Region region;
|
HRegion region;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor
|
* Constructor
|
||||||
|
@ -209,7 +209,7 @@ public class RegionCoprocessorHost
|
||||||
* @param rsServices interface to available region server functionality
|
* @param rsServices interface to available region server functionality
|
||||||
* @param conf the configuration
|
* @param conf the configuration
|
||||||
*/
|
*/
|
||||||
public RegionCoprocessorHost(final Region region,
|
public RegionCoprocessorHost(final HRegion region,
|
||||||
final RegionServerServices rsServices, final Configuration conf) {
|
final RegionServerServices rsServices, final Configuration conf) {
|
||||||
super(rsServices);
|
super(rsServices);
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
|
|
|
@ -22,9 +22,11 @@ import java.security.PrivilegedAction;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
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.security.User;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
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.
|
// 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
|
// 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().
|
// 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);
|
LOG.error("Unable to ask master to merge: " + region_a + ", " + region_b);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -85,15 +85,15 @@ public interface RegionServerServices
|
||||||
* Context for postOpenDeployTasks().
|
* Context for postOpenDeployTasks().
|
||||||
*/
|
*/
|
||||||
class PostOpenDeployContext {
|
class PostOpenDeployContext {
|
||||||
private final Region region;
|
private final HRegion region;
|
||||||
private final long masterSystemTime;
|
private final long masterSystemTime;
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public PostOpenDeployContext(Region region, long masterSystemTime) {
|
public PostOpenDeployContext(HRegion region, long masterSystemTime) {
|
||||||
this.region = region;
|
this.region = region;
|
||||||
this.masterSystemTime = masterSystemTime;
|
this.masterSystemTime = masterSystemTime;
|
||||||
}
|
}
|
||||||
public Region getRegion() {
|
public HRegion getRegion() {
|
||||||
return region;
|
return region;
|
||||||
}
|
}
|
||||||
public long getMasterSystemTime() {
|
public long getMasterSystemTime() {
|
||||||
|
@ -111,18 +111,6 @@ public interface RegionServerServices
|
||||||
*/
|
*/
|
||||||
void postOpenDeployTasks(final PostOpenDeployContext context) throws KeeperException, IOException;
|
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 {
|
class RegionStateTransitionContext {
|
||||||
private final TransitionCode code;
|
private final TransitionCode code;
|
||||||
private final long openSeqNum;
|
private final long openSeqNum;
|
||||||
|
@ -156,20 +144,6 @@ public interface RegionServerServices
|
||||||
*/
|
*/
|
||||||
boolean reportRegionStateTransition(final RegionStateTransitionContext context);
|
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
|
* 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
|
* @return set of recovering regions on the hosting region server
|
||||||
*/
|
*/
|
||||||
Map<String, Region> getRecoveringRegions();
|
Map<String, HRegion> getRecoveringRegions();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Only required for "old" log replay; if it's removed, remove this.
|
* Only required for "old" log replay; if it's removed, remove this.
|
||||||
|
|
|
@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
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.User;
|
||||||
import org.apache.hadoop.hbase.security.UserProvider;
|
import org.apache.hadoop.hbase.security.UserProvider;
|
||||||
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
|
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
|
||||||
|
@ -157,7 +157,7 @@ public class SecureBulkLoadManager {
|
||||||
LOG.info("Cleaned up " + path + " successfully.");
|
LOG.info("Cleaned up " + path + " successfully.");
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<byte[], List<Path>> secureBulkLoadHFiles(final Region region,
|
public Map<byte[], List<Path>> secureBulkLoadHFiles(final HRegion region,
|
||||||
final BulkLoadHFileRequest request) throws IOException {
|
final BulkLoadHFileRequest request) throws IOException {
|
||||||
final List<Pair<byte[], String>> familyPaths = new ArrayList<>(request.getFamilyPathCount());
|
final List<Pair<byte[], String>> familyPaths = new ArrayList<>(request.getFamilyPathCount());
|
||||||
for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
|
for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
|
||||||
|
|
|
@ -22,9 +22,11 @@ import java.security.PrivilegedAction;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
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.security.User;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
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.
|
// 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
|
// 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().
|
// 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());
|
LOG.error("Unable to ask master to split " + parent.getRegionNameAsString());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
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.executor.EventType;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
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);
|
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
|
// Done! Region is closed on this RS
|
||||||
LOG.debug("Closed " + region.getRegionInfo().getRegionNameAsString());
|
LOG.debug("Closed " + region.getRegionInfo().getRegionNameAsString());
|
||||||
|
|
|
@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
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.RegionServerAccounting;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices.PostOpenDeployContext;
|
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.hadoop.hbase.util.CancelableProgressable;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
|
@ -160,7 +162,8 @@ public class OpenRegionHandler extends EventHandler {
|
||||||
cleanupFailedOpen(region);
|
cleanupFailedOpen(region);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
rsServices.reportRegionStateTransition(TransitionCode.FAILED_OPEN, regionInfo);
|
rsServices.reportRegionStateTransition(new RegionStateTransitionContext(
|
||||||
|
TransitionCode.FAILED_OPEN, HConstants.NO_SEQNUM, -1, regionInfo));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -31,8 +31,7 @@ import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
|
||||||
import org.apache.hadoop.hbase.procedure.ProcedureMember;
|
import org.apache.hadoop.hbase.procedure.ProcedureMember;
|
||||||
import org.apache.hadoop.hbase.procedure.Subprocedure;
|
import org.apache.hadoop.hbase.procedure.Subprocedure;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.Region.Operation;
|
import org.apache.hadoop.hbase.regionserver.Region.Operation;
|
||||||
import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager.SnapshotSubprocedurePool;
|
import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager.SnapshotSubprocedurePool;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
|
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 {
|
public class FlushSnapshotSubprocedure extends Subprocedure {
|
||||||
private static final Log LOG = LogFactory.getLog(FlushSnapshotSubprocedure.class);
|
private static final Log LOG = LogFactory.getLog(FlushSnapshotSubprocedure.class);
|
||||||
|
|
||||||
private final List<Region> regions;
|
private final List<HRegion> regions;
|
||||||
private final SnapshotDescription snapshot;
|
private final SnapshotDescription snapshot;
|
||||||
private final SnapshotSubprocedurePool taskManager;
|
private final SnapshotSubprocedurePool taskManager;
|
||||||
private boolean snapshotSkipFlush = false;
|
private boolean snapshotSkipFlush = false;
|
||||||
|
@ -60,7 +59,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure {
|
||||||
|
|
||||||
public FlushSnapshotSubprocedure(ProcedureMember member,
|
public FlushSnapshotSubprocedure(ProcedureMember member,
|
||||||
ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
|
ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
|
||||||
List<Region> regions, SnapshotDescription snapshot,
|
List<HRegion> regions, SnapshotDescription snapshot,
|
||||||
SnapshotSubprocedurePool taskManager) {
|
SnapshotSubprocedurePool taskManager) {
|
||||||
super(member, snapshot.getName(), errorListener, wakeFrequency, timeout);
|
super(member, snapshot.getName(), errorListener, wakeFrequency, timeout);
|
||||||
this.snapshot = snapshot;
|
this.snapshot = snapshot;
|
||||||
|
@ -76,12 +75,12 @@ public class FlushSnapshotSubprocedure extends Subprocedure {
|
||||||
* Callable for adding files to snapshot manifest working dir. Ready for multithreading.
|
* Callable for adding files to snapshot manifest working dir. Ready for multithreading.
|
||||||
*/
|
*/
|
||||||
public static class RegionSnapshotTask implements Callable<Void> {
|
public static class RegionSnapshotTask implements Callable<Void> {
|
||||||
private Region region;
|
private HRegion region;
|
||||||
private boolean skipFlush;
|
private boolean skipFlush;
|
||||||
private ForeignExceptionDispatcher monitor;
|
private ForeignExceptionDispatcher monitor;
|
||||||
private SnapshotDescription snapshotDesc;
|
private SnapshotDescription snapshotDesc;
|
||||||
|
|
||||||
public RegionSnapshotTask(Region region, SnapshotDescription snapshotDesc,
|
public RegionSnapshotTask(HRegion region, SnapshotDescription snapshotDesc,
|
||||||
boolean skipFlush, ForeignExceptionDispatcher monitor) {
|
boolean skipFlush, ForeignExceptionDispatcher monitor) {
|
||||||
this.region = region;
|
this.region = region;
|
||||||
this.skipFlush = skipFlush;
|
this.skipFlush = skipFlush;
|
||||||
|
@ -111,7 +110,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure {
|
||||||
} else {
|
} else {
|
||||||
LOG.debug("Flush Snapshotting region " + region.toString() + " started...");
|
LOG.debug("Flush Snapshotting region " + region.toString() + " started...");
|
||||||
boolean succeeded = false;
|
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++) {
|
for (int i = 0; i < MAX_RETRIES; i++) {
|
||||||
FlushResult res = region.flush(true);
|
FlushResult res = region.flush(true);
|
||||||
if (res.getResult() == FlushResult.Result.CANNOT_FLUSH) {
|
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");
|
throw new IOException("Unable to complete flush after " + MAX_RETRIES + " attempts");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
((HRegion)region).addRegionToSnapshot(snapshotDesc, monitor);
|
region.addRegionToSnapshot(snapshotDesc, monitor);
|
||||||
if (skipFlush) {
|
if (skipFlush) {
|
||||||
LOG.debug("... SkipFlush Snapshotting region " + region.toString() + " completed.");
|
LOG.debug("... SkipFlush Snapshotting region " + region.toString() + " completed.");
|
||||||
} else {
|
} else {
|
||||||
|
@ -162,7 +161,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Add all hfiles already existing in region.
|
// Add all hfiles already existing in region.
|
||||||
for (Region region : regions) {
|
for (HRegion region : regions) {
|
||||||
// submit one task per region for parallelize by region.
|
// submit one task per region for parallelize by region.
|
||||||
taskManager.submitTask(new RegionSnapshotTask(region, snapshot, snapshotSkipFlush, monitor));
|
taskManager.submitTask(new RegionSnapshotTask(region, snapshot, snapshotSkipFlush, monitor));
|
||||||
monitor.rethrowException();
|
monitor.rethrowException();
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
|
||||||
import org.apache.hadoop.hbase.procedure.Subprocedure;
|
import org.apache.hadoop.hbase.procedure.Subprocedure;
|
||||||
import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
|
import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
|
||||||
import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
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 this server is hosting any regions for the snapshots
|
||||||
// check to see if we have regions for the snapshot
|
// check to see if we have regions for the snapshot
|
||||||
List<Region> involvedRegions;
|
List<HRegion> involvedRegions;
|
||||||
try {
|
try {
|
||||||
involvedRegions = getRegionsToSnapshot(snapshot);
|
involvedRegions = getRegionsToSnapshot(snapshot);
|
||||||
} catch (IOException e1) {
|
} catch (IOException e1) {
|
||||||
|
@ -222,12 +223,13 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
|
||||||
* the given snapshot.
|
* the given snapshot.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private List<Region> getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException {
|
private List<HRegion> getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException {
|
||||||
List<Region> onlineRegions = rss.getRegions(TableName.valueOf(snapshot.getTable()));
|
List<HRegion> onlineRegions = (List<HRegion>) rss
|
||||||
Iterator<Region> iterator = onlineRegions.iterator();
|
.getRegions(TableName.valueOf(snapshot.getTable()));
|
||||||
|
Iterator<HRegion> iterator = onlineRegions.iterator();
|
||||||
// remove the non-default regions
|
// remove the non-default regions
|
||||||
while (iterator.hasNext()) {
|
while (iterator.hasNext()) {
|
||||||
Region r = iterator.next();
|
HRegion r = iterator.next();
|
||||||
if (!RegionReplicaUtil.isDefaultReplica(r.getRegionInfo())) {
|
if (!RegionReplicaUtil.isDefaultReplica(r.getRegionInfo())) {
|
||||||
iterator.remove();
|
iterator.remove();
|
||||||
}
|
}
|
||||||
|
|
|
@ -307,7 +307,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
|
||||||
private boolean mutateLabelsRegion(List<Mutation> mutations, OperationStatus[] finalOpStatus)
|
private boolean mutateLabelsRegion(List<Mutation> mutations, OperationStatus[] finalOpStatus)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
OperationStatus[] opStatus = this.labelsRegion.batchMutate(mutations
|
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;
|
int i = 0;
|
||||||
boolean updateZk = false;
|
boolean updateZk = false;
|
||||||
for (OperationStatus status : opStatus) {
|
for (OperationStatus status : opStatus) {
|
||||||
|
|
|
@ -3947,8 +3947,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
||||||
if (server.equals(rs.getServerName())) {
|
if (server.equals(rs.getServerName())) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
Collection<Region> hrs = rs.getOnlineRegionsLocalContext();
|
Collection<HRegion> hrs = rs.getOnlineRegionsLocalContext();
|
||||||
for (Region r: hrs) {
|
for (HRegion r: hrs) {
|
||||||
assertTrue("Region should not be double assigned",
|
assertTrue("Region should not be double assigned",
|
||||||
r.getRegionInfo().getRegionId() != hri.getRegionId());
|
r.getRegionInfo().getRegionId() != hri.getRegionId());
|
||||||
}
|
}
|
||||||
|
|
|
@ -615,7 +615,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
||||||
public void flushcache() throws IOException {
|
public void flushcache() throws IOException {
|
||||||
for (JVMClusterUtil.RegionServerThread t:
|
for (JVMClusterUtil.RegionServerThread t:
|
||||||
this.hbaseCluster.getRegionServers()) {
|
this.hbaseCluster.getRegionServers()) {
|
||||||
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
||||||
r.flush(true);
|
r.flush(true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -628,7 +628,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
||||||
public void flushcache(TableName tableName) throws IOException {
|
public void flushcache(TableName tableName) throws IOException {
|
||||||
for (JVMClusterUtil.RegionServerThread t:
|
for (JVMClusterUtil.RegionServerThread t:
|
||||||
this.hbaseCluster.getRegionServers()) {
|
this.hbaseCluster.getRegionServers()) {
|
||||||
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
||||||
if(r.getTableDescriptor().getTableName().equals(tableName)) {
|
if(r.getTableDescriptor().getTableName().equals(tableName)) {
|
||||||
r.flush(true);
|
r.flush(true);
|
||||||
}
|
}
|
||||||
|
@ -643,7 +643,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
||||||
public void compact(boolean major) throws IOException {
|
public void compact(boolean major) throws IOException {
|
||||||
for (JVMClusterUtil.RegionServerThread t:
|
for (JVMClusterUtil.RegionServerThread t:
|
||||||
this.hbaseCluster.getRegionServers()) {
|
this.hbaseCluster.getRegionServers()) {
|
||||||
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
||||||
r.compact(major);
|
r.compact(major);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -656,7 +656,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
||||||
public void compact(TableName tableName, boolean major) throws IOException {
|
public void compact(TableName tableName, boolean major) throws IOException {
|
||||||
for (JVMClusterUtil.RegionServerThread t:
|
for (JVMClusterUtil.RegionServerThread t:
|
||||||
this.hbaseCluster.getRegionServers()) {
|
this.hbaseCluster.getRegionServers()) {
|
||||||
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
||||||
if(r.getTableDescriptor().getTableName().equals(tableName)) {
|
if(r.getTableDescriptor().getTableName().equals(tableName)) {
|
||||||
r.compact(major);
|
r.compact(major);
|
||||||
}
|
}
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
||||||
import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
|
import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
|
||||||
import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
|
import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
|
||||||
import org.apache.hadoop.hbase.regionserver.FlushRequester;
|
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.HeapMemoryManager;
|
||||||
import org.apache.hadoop.hbase.regionserver.Leases;
|
import org.apache.hadoop.hbase.regionserver.Leases;
|
||||||
import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
|
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.zookeeper.KeeperException;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
|
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;
|
import com.google.protobuf.Service;
|
||||||
|
|
||||||
|
@ -96,7 +96,7 @@ public class MockRegionServerServices implements RegionServerServices {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean removeRegion(Region r, ServerName destination) {
|
public boolean removeRegion(HRegion r, ServerName destination) {
|
||||||
return this.regions.remove(r.getRegionInfo().getEncodedName()) != null;
|
return this.regions.remove(r.getRegionInfo().getEncodedName()) != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -121,15 +121,10 @@ public class MockRegionServerServices implements RegionServerServices {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void addRegion(Region r) {
|
public void addRegion(HRegion r) {
|
||||||
this.regions.put(r.getRegionInfo().getEncodedName(), r);
|
this.regions.put(r.getRegionInfo().getEncodedName(), r);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void postOpenDeployTasks(Region r) throws KeeperException, IOException {
|
|
||||||
addRegion(r);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void postOpenDeployTasks(PostOpenDeployContext context) throws KeeperException,
|
public void postOpenDeployTasks(PostOpenDeployContext context) throws KeeperException,
|
||||||
IOException {
|
IOException {
|
||||||
|
@ -269,7 +264,7 @@ public class MockRegionServerServices implements RegionServerServices {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<String, Region> getRecoveringRegions() {
|
public Map<String, HRegion> getRecoveringRegions() {
|
||||||
// TODO Auto-generated method stub
|
// TODO Auto-generated method stub
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -280,18 +275,6 @@ public class MockRegionServerServices implements RegionServerServices {
|
||||||
return null;
|
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
|
@Override
|
||||||
public boolean reportRegionStateTransition(RegionStateTransitionContext context) {
|
public boolean reportRegionStateTransition(RegionStateTransitionContext context) {
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -31,8 +31,8 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -109,7 +109,7 @@ public class TestGlobalMemStoreSize {
|
||||||
|
|
||||||
for (RegionInfo regionInfo :
|
for (RegionInfo regionInfo :
|
||||||
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
||||||
Region r = server.getRegion(regionInfo.getEncodedName());
|
HRegion r = server.getRegion(regionInfo.getEncodedName());
|
||||||
flush(r, server);
|
flush(r, server);
|
||||||
}
|
}
|
||||||
LOG.info("Post flush on " + server.getServerName());
|
LOG.info("Post flush on " + server.getServerName());
|
||||||
|
@ -125,7 +125,7 @@ public class TestGlobalMemStoreSize {
|
||||||
// our test was running....
|
// our test was running....
|
||||||
for (RegionInfo regionInfo :
|
for (RegionInfo regionInfo :
|
||||||
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
||||||
Region r = server.getRegion(regionInfo.getEncodedName());
|
HRegion r = server.getRegion(regionInfo.getEncodedName());
|
||||||
long l = r.getMemStoreSize();
|
long l = r.getMemStoreSize();
|
||||||
if (l > 0) {
|
if (l > 0) {
|
||||||
// Only meta could have edits at this stage. Give it another flush
|
// Only meta could have edits at this stage. Give it another flush
|
||||||
|
@ -150,7 +150,7 @@ public class TestGlobalMemStoreSize {
|
||||||
* @param server
|
* @param server
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private void flush(final Region r, final HRegionServer server)
|
private void flush(final HRegion r, final HRegionServer server)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
LOG.info("Flush " + r.toString() + " on " + server.getServerName() +
|
LOG.info("Flush " + r.toString() + " on " + server.getServerName() +
|
||||||
", " + r.flush(true) + ", size=" +
|
", " + r.flush(true) + ", size=" +
|
||||||
|
|
|
@ -230,7 +230,7 @@ public class TestHFileArchiving {
|
||||||
List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(tableName);
|
List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(tableName);
|
||||||
// make sure we only have 1 region serving this table
|
// make sure we only have 1 region serving this table
|
||||||
assertEquals(1, servingRegions.size());
|
assertEquals(1, servingRegions.size());
|
||||||
Region region = servingRegions.get(0);
|
HRegion region = servingRegions.get(0);
|
||||||
|
|
||||||
// get the parent RS and monitor
|
// get the parent RS and monitor
|
||||||
HRegionServer hrs = UTIL.getRSForFirstRegionInTable(tableName);
|
HRegionServer hrs = UTIL.getRSForFirstRegionInTable(tableName);
|
||||||
|
@ -241,7 +241,7 @@ public class TestHFileArchiving {
|
||||||
UTIL.loadRegion(region, TEST_FAM);
|
UTIL.loadRegion(region, TEST_FAM);
|
||||||
|
|
||||||
// get the hfiles in the region
|
// get the hfiles in the region
|
||||||
List<Region> regions = hrs.getRegions(tableName);
|
List<HRegion> regions = hrs.getRegions(tableName);
|
||||||
assertEquals("More that 1 region for test table.", 1, regions.size());
|
assertEquals("More that 1 region for test table.", 1, regions.size());
|
||||||
|
|
||||||
region = regions.get(0);
|
region = regions.get(0);
|
||||||
|
@ -309,7 +309,7 @@ public class TestHFileArchiving {
|
||||||
List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(tableName);
|
List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(tableName);
|
||||||
// make sure we only have 1 region serving this table
|
// make sure we only have 1 region serving this table
|
||||||
assertEquals(1, servingRegions.size());
|
assertEquals(1, servingRegions.size());
|
||||||
Region region = servingRegions.get(0);
|
HRegion region = servingRegions.get(0);
|
||||||
|
|
||||||
// get the parent RS and monitor
|
// get the parent RS and monitor
|
||||||
HRegionServer hrs = UTIL.getRSForFirstRegionInTable(tableName);
|
HRegionServer hrs = UTIL.getRSForFirstRegionInTable(tableName);
|
||||||
|
@ -320,7 +320,7 @@ public class TestHFileArchiving {
|
||||||
UTIL.loadRegion(region, TEST_FAM);
|
UTIL.loadRegion(region, TEST_FAM);
|
||||||
|
|
||||||
// get the hfiles in the region
|
// get the hfiles in the region
|
||||||
List<Region> regions = hrs.getRegions(tableName);
|
List<HRegion> regions = hrs.getRegions(tableName);
|
||||||
assertEquals("More that 1 region for test table.", 1, regions.size());
|
assertEquals("More that 1 region for test table.", 1, regions.size());
|
||||||
|
|
||||||
region = regions.get(0);
|
region = regions.get(0);
|
||||||
|
|
|
@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.when;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
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.CompactedHFilesDischarger;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||||
|
@ -179,9 +177,9 @@ public class TestZooKeeperTableArchiveClient {
|
||||||
// create the region
|
// create the region
|
||||||
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM);
|
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM);
|
||||||
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
|
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
|
||||||
List<Region> regions = new ArrayList<>();
|
List<HRegion> regions = new ArrayList<>();
|
||||||
regions.add(region);
|
regions.add(region);
|
||||||
when(rss.getRegions()).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions();
|
||||||
final CompactedHFilesDischarger compactionCleaner =
|
final CompactedHFilesDischarger compactionCleaner =
|
||||||
new CompactedHFilesDischarger(100, stop, rss, false);
|
new CompactedHFilesDischarger(100, stop, rss, false);
|
||||||
loadFlushAndCompact(region, TEST_FAM);
|
loadFlushAndCompact(region, TEST_FAM);
|
||||||
|
@ -232,9 +230,9 @@ public class TestZooKeeperTableArchiveClient {
|
||||||
// create the region
|
// create the region
|
||||||
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM);
|
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM);
|
||||||
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
|
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
|
||||||
List<Region> regions = new ArrayList<>();
|
List<HRegion> regions = new ArrayList<>();
|
||||||
regions.add(region);
|
regions.add(region);
|
||||||
when(rss.getRegions()).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions();
|
||||||
final CompactedHFilesDischarger compactionCleaner =
|
final CompactedHFilesDischarger compactionCleaner =
|
||||||
new CompactedHFilesDischarger(100, stop, rss, false);
|
new CompactedHFilesDischarger(100, stop, rss, false);
|
||||||
loadFlushAndCompact(region, TEST_FAM);
|
loadFlushAndCompact(region, TEST_FAM);
|
||||||
|
@ -244,7 +242,7 @@ public class TestZooKeeperTableArchiveClient {
|
||||||
HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd);
|
HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd);
|
||||||
regions = new ArrayList<>();
|
regions = new ArrayList<>();
|
||||||
regions.add(otherRegion);
|
regions.add(otherRegion);
|
||||||
when(rss.getRegions()).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions();
|
||||||
final CompactedHFilesDischarger compactionCleaner1 = new CompactedHFilesDischarger(100, stop,
|
final CompactedHFilesDischarger compactionCleaner1 = new CompactedHFilesDischarger(100, stop,
|
||||||
rss, false);
|
rss, false);
|
||||||
loadFlushAndCompact(otherRegion, TEST_FAM);
|
loadFlushAndCompact(otherRegion, TEST_FAM);
|
||||||
|
@ -422,7 +420,7 @@ public class TestZooKeeperTableArchiveClient {
|
||||||
* @param columnFamily family for which to add data
|
* @param columnFamily family for which to add data
|
||||||
* @throws IOException
|
* @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 one row in the region
|
||||||
Put p = new Put(Bytes.toBytes("row"));
|
Put p = new Put(Bytes.toBytes("row"));
|
||||||
p.addColumn(columnFamily, Bytes.toBytes("Qual"), Bytes.toBytes("v1"));
|
p.addColumn(columnFamily, Bytes.toBytes("Qual"), Bytes.toBytes("v1"));
|
||||||
|
|
|
@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||||
import org.apache.hadoop.hbase.master.HMaster;
|
import org.apache.hadoop.hbase.master.HMaster;
|
||||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||||
|
@ -472,7 +473,7 @@ public class TestAdmin2 {
|
||||||
+ AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files");
|
+ AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files");
|
||||||
|
|
||||||
// flush all regions
|
// flush all regions
|
||||||
for (Region r : regionServer.getOnlineRegionsLocalContext()) {
|
for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
|
||||||
r.flush(true);
|
r.flush(true);
|
||||||
}
|
}
|
||||||
admin.rollWALWriter(regionServer.getServerName());
|
admin.rollWALWriter(regionServer.getServerName());
|
||||||
|
|
|
@ -41,8 +41,8 @@ import org.apache.hadoop.hbase.RegionLoad;
|
||||||
import org.apache.hadoop.hbase.ServerLoad;
|
import org.apache.hadoop.hbase.ServerLoad;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
|
||||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -140,7 +140,7 @@ public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
|
||||||
+ AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files");
|
+ AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files");
|
||||||
|
|
||||||
// flush all regions
|
// flush all regions
|
||||||
for (Region r : regionServer.getOnlineRegionsLocalContext()) {
|
for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
|
||||||
r.flush(true);
|
r.flush(true);
|
||||||
}
|
}
|
||||||
admin.rollWALWriter(regionServer.getServerName()).join();
|
admin.rollWALWriter(regionServer.getServerName()).join();
|
||||||
|
|
|
@ -110,7 +110,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
|
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
|
||||||
import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
|
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.ScanInfo;
|
||||||
import org.apache.hadoop.hbase.regionserver.Store;
|
import org.apache.hadoop.hbase.regionserver.Store;
|
||||||
import org.apache.hadoop.hbase.regionserver.StoreScanner;
|
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
|
// set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
|
||||||
// in Store.rowAtOrBeforeFromStoreFile
|
// in Store.rowAtOrBeforeFromStoreFile
|
||||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
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 put1 = new Put(firstRow);
|
||||||
Put put2 = new Put(secondRow);
|
Put put2 = new Put(secondRow);
|
||||||
Put put3 = new Put(thirdRow);
|
Put put3 = new Put(thirdRow);
|
||||||
|
|
|
@ -963,7 +963,7 @@ public class TestFromClientSide3 {
|
||||||
private static Region find(final TableName tableName)
|
private static Region find(final TableName tableName)
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
||||||
List<Region> regions = rs.getRegions(tableName);
|
List<HRegion> regions = rs.getRegions(tableName);
|
||||||
assertEquals(1, regions.size());
|
assertEquals(1, regions.size());
|
||||||
return regions.get(0);
|
return regions.get(0);
|
||||||
}
|
}
|
||||||
|
|
|
@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||||
import org.apache.hadoop.hbase.master.HMaster;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
||||||
|
@ -1347,7 +1348,7 @@ public class TestHCM {
|
||||||
assertTrue(!destServerName.equals(metaServerName));
|
assertTrue(!destServerName.equals(metaServerName));
|
||||||
|
|
||||||
//find another row in the cur server that is less than ROW_X
|
//find another row in the cur server that is less than ROW_X
|
||||||
List<Region> regions = curServer.getRegions(TABLE_NAME3);
|
List<HRegion> regions = curServer.getRegions(TABLE_NAME3);
|
||||||
byte[] otherRow = null;
|
byte[] otherRow = null;
|
||||||
for (Region region : regions) {
|
for (Region region : regions) {
|
||||||
if (!region.getRegionInfo().getEncodedName().equals(toMove.getRegionInfo().getEncodedName())
|
if (!region.getRegionInfo().getEncodedName().equals(toMove.getRegionInfo().getEncodedName())
|
||||||
|
|
|
@ -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.quotas.ThrottlingException;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
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.ClientTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -248,11 +247,6 @@ public class TestMetaCache {
|
||||||
exceptions.throwOnScan(this, request);
|
exceptions.throwOnScan(this, request);
|
||||||
return super.scan(controller, request);
|
return super.scan(controller, request);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Region getRegion(
|
|
||||||
final HBaseProtos.RegionSpecifier regionSpecifier) throws IOException {
|
|
||||||
return super.getRegion(regionSpecifier);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static abstract class ExceptionInjector {
|
public static abstract class ExceptionInjector {
|
||||||
|
|
|
@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.regionserver.ChunkCreator;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||||
import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
|
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.RegionCoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||||
|
@ -289,7 +288,7 @@ public class TestCoprocessorInterface {
|
||||||
byte [][] families = { fam1, fam2, fam3 };
|
byte [][] families = { fam1, fam2, fam3 };
|
||||||
|
|
||||||
Configuration hc = initConfig();
|
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++) {
|
for (int i = 0; i < 3; i++) {
|
||||||
HBaseTestCase.addContent(region, fam3);
|
HBaseTestCase.addContent(region, fam3);
|
||||||
|
@ -351,7 +350,7 @@ public class TestCoprocessorInterface {
|
||||||
byte [][] families = { fam1, fam2, fam3 };
|
byte [][] families = { fam1, fam2, fam3 };
|
||||||
|
|
||||||
Configuration hc = initConfig();
|
Configuration hc = initConfig();
|
||||||
Region region = initHRegion(tableName, name.getMethodName(), hc,
|
HRegion region = initHRegion(tableName, name.getMethodName(), hc,
|
||||||
new Class<?>[]{CoprocessorImpl.class}, families);
|
new Class<?>[]{CoprocessorImpl.class}, families);
|
||||||
for (int i = 0; i < 3; i++) {
|
for (int i = 0; i < 3; i++) {
|
||||||
HBaseTestCase.addContent(region, fam3);
|
HBaseTestCase.addContent(region, fam3);
|
||||||
|
@ -378,10 +377,10 @@ public class TestCoprocessorInterface {
|
||||||
assertTrue(((CoprocessorImpl)c).wasCompacted());
|
assertTrue(((CoprocessorImpl)c).wasCompacted());
|
||||||
}
|
}
|
||||||
|
|
||||||
Region reopenRegion(final Region closedRegion, Class<?> ... implClasses)
|
HRegion reopenRegion(final HRegion closedRegion, Class<?> ... implClasses)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
//RegionInfo info = new RegionInfo(tableName, null, null, false);
|
//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
|
// this following piece is a hack. currently a coprocessorHost
|
||||||
// is secretly loaded at OpenRegionHandler. we don't really
|
// is secretly loaded at OpenRegionHandler. we don't really
|
||||||
|
@ -389,7 +388,7 @@ public class TestCoprocessorInterface {
|
||||||
// and set it to region.
|
// and set it to region.
|
||||||
Configuration conf = TEST_UTIL.getConfiguration();
|
Configuration conf = TEST_UTIL.getConfiguration();
|
||||||
RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf);
|
RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf);
|
||||||
((HRegion)r).setCoprocessorHost(host);
|
r.setCoprocessorHost(host);
|
||||||
|
|
||||||
for (Class<?> implClass : implClasses) {
|
for (Class<?> implClass : implClasses) {
|
||||||
host.load((Class<? extends RegionCoprocessor>) implClass, Coprocessor.PRIORITY_USER, conf);
|
host.load((Class<? extends RegionCoprocessor>) implClass, Coprocessor.PRIORITY_USER, conf);
|
||||||
|
@ -405,7 +404,7 @@ public class TestCoprocessorInterface {
|
||||||
return r;
|
return r;
|
||||||
}
|
}
|
||||||
|
|
||||||
Region initHRegion (TableName tableName, String callingMethod,
|
HRegion initHRegion (TableName tableName, String callingMethod,
|
||||||
Configuration conf, Class<?> [] implClasses, byte [][] families)
|
Configuration conf, Class<?> [] implClasses, byte [][] families)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||||
|
@ -419,11 +418,11 @@ public class TestCoprocessorInterface {
|
||||||
.setSplit(false)
|
.setSplit(false)
|
||||||
.build();
|
.build();
|
||||||
Path path = new Path(DIR + callingMethod);
|
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.
|
// this following piece is a hack.
|
||||||
RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf);
|
RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf);
|
||||||
((HRegion)r).setCoprocessorHost(host);
|
r.setCoprocessorHost(host);
|
||||||
|
|
||||||
for (Class<?> implClass : implClasses) {
|
for (Class<?> implClass : implClasses) {
|
||||||
host.load((Class<? extends RegionCoprocessor>) implClass, Coprocessor.PRIORITY_USER, conf);
|
host.load((Class<? extends RegionCoprocessor>) implClass, Coprocessor.PRIORITY_USER, conf);
|
||||||
|
|
|
@ -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 {
|
byte[]... families) throws IOException {
|
||||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
|
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
|
||||||
for (byte[] family : families) {
|
for (byte[] family : families) {
|
||||||
|
@ -239,7 +239,7 @@ public class TestRegionObserverScannerOpenHook {
|
||||||
byte[][] FAMILIES = new byte[][] { A };
|
byte[][] FAMILIES = new byte[][] { A };
|
||||||
|
|
||||||
Configuration conf = HBaseConfiguration.create();
|
Configuration conf = HBaseConfiguration.create();
|
||||||
Region region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
|
HRegion region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
|
||||||
RegionCoprocessorHost h = region.getCoprocessorHost();
|
RegionCoprocessorHost h = region.getCoprocessorHost();
|
||||||
h.load(NoDataFromFlush.class, Coprocessor.PRIORITY_HIGHEST, conf);
|
h.load(NoDataFromFlush.class, Coprocessor.PRIORITY_HIGHEST, conf);
|
||||||
h.load(EmptyRegionObsever.class, Coprocessor.PRIORITY_USER, conf);
|
h.load(EmptyRegionObsever.class, Coprocessor.PRIORITY_USER, conf);
|
||||||
|
@ -323,7 +323,7 @@ public class TestRegionObserverScannerOpenHook {
|
||||||
table.put(put);
|
table.put(put);
|
||||||
|
|
||||||
HRegionServer rs = UTIL.getRSForFirstRegionInTable(desc.getTableName());
|
HRegionServer rs = UTIL.getRSForFirstRegionInTable(desc.getTableName());
|
||||||
List<Region> regions = rs.getRegions(desc.getTableName());
|
List<HRegion> regions = rs.getRegions(desc.getTableName());
|
||||||
assertEquals("More than 1 region serving test table with 1 row", 1, regions.size());
|
assertEquals("More than 1 region serving test table with 1 row", 1, regions.size());
|
||||||
Region region = regions.get(0);
|
Region region = regions.get(0);
|
||||||
admin.flushRegion(region.getRegionInfo().getRegionName());
|
admin.flushRegion(region.getRegionInfo().getRegionName());
|
||||||
|
|
|
@ -71,7 +71,7 @@ import org.junit.rules.TestName;
|
||||||
@Category({FilterTests.class, SmallTests.class})
|
@Category({FilterTests.class, SmallTests.class})
|
||||||
public class TestFilter {
|
public class TestFilter {
|
||||||
private final static Log LOG = LogFactory.getLog(TestFilter.class);
|
private final static Log LOG = LogFactory.getLog(TestFilter.class);
|
||||||
private Region region;
|
private HRegion region;
|
||||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
|
@ -1499,7 +1499,7 @@ public class TestFilter {
|
||||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||||
htd.addFamily(new HColumnDescriptor(family));
|
htd.addFamily(new HColumnDescriptor(family));
|
||||||
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
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);
|
TEST_UTIL.getConfiguration(), htd);
|
||||||
|
|
||||||
for(int i=0; i<5; i++) {
|
for(int i=0; i<5; i++) {
|
||||||
|
@ -2060,7 +2060,7 @@ public class TestFilter {
|
||||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||||
htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
|
htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
|
||||||
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
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);
|
TEST_UTIL.getConfiguration(), htd);
|
||||||
for(int i=0; i<10; i++) {
|
for(int i=0; i<10; i++) {
|
||||||
Put p = new Put(Bytes.toBytes("row" + i));
|
Put p = new Put(Bytes.toBytes("row" + i));
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
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.FilterTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -61,7 +60,7 @@ public class TestInvocationRecordFilter {
|
||||||
private static final String VALUE_PREFIX = "value";
|
private static final String VALUE_PREFIX = "value";
|
||||||
|
|
||||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||||
private Region region;
|
private HRegion region;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
|
|
|
@ -292,7 +292,7 @@ public class TestBlockReorder {
|
||||||
|
|
||||||
int nbTest = 0;
|
int nbTest = 0;
|
||||||
while (nbTest < 10) {
|
while (nbTest < 10) {
|
||||||
final List<Region> regions = targetRs.getRegions(h.getName());
|
final List<HRegion> regions = targetRs.getRegions(h.getName());
|
||||||
final CountDownLatch latch = new CountDownLatch(regions.size());
|
final CountDownLatch latch = new CountDownLatch(regions.size());
|
||||||
// listen for successful log rolls
|
// listen for successful log rolls
|
||||||
final WALActionsListener listener = new WALActionsListener.Base() {
|
final WALActionsListener listener = new WALActionsListener.Base() {
|
||||||
|
@ -301,8 +301,8 @@ public class TestBlockReorder {
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
for (Region region : regions) {
|
for (HRegion region : regions) {
|
||||||
((HRegion)region).getWAL().registerWALActionsListener(listener);
|
region.getWAL().registerWALActionsListener(listener);
|
||||||
}
|
}
|
||||||
|
|
||||||
htu.getAdmin().rollWALWriter(targetRs.getServerName());
|
htu.getAdmin().rollWALWriter(targetRs.getServerName());
|
||||||
|
|
|
@ -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.HFile;
|
||||||
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
|
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
|
||||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
|
@ -114,7 +115,7 @@ public class TestEncodedSeekers {
|
||||||
setBlocksize(BLOCK_SIZE).
|
setBlocksize(BLOCK_SIZE).
|
||||||
setBloomFilterType(BloomType.NONE).
|
setBloomFilterType(BloomType.NONE).
|
||||||
setCompressTags(compressTags);
|
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
|
//write the data, but leave some in the memstore
|
||||||
doPuts(region);
|
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);
|
LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(MIN_VALUE_SIZE, MAX_VALUE_SIZE);
|
||||||
for (int i = 0; i < NUM_ROWS; ++i) {
|
for (int i = 0; i < NUM_ROWS; ++i) {
|
||||||
byte[] key = LoadTestKVGenerator.md5PrefixedKey(i).getBytes();
|
byte[] key = LoadTestKVGenerator.md5PrefixedKey(i).getBytes();
|
||||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
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.Region;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||||
|
@ -65,7 +66,7 @@ public class TestPrefixTree {
|
||||||
|
|
||||||
private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
|
private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
|
||||||
|
|
||||||
private Region region;
|
private HRegion region;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
|
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.regionserver.RegionScanner;
|
||||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
|
@ -48,7 +48,7 @@ import org.junit.experimental.categories.Category;
|
||||||
public class TestSeekBeforeWithReverseScan {
|
public class TestSeekBeforeWithReverseScan {
|
||||||
private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
|
private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
|
||||||
|
|
||||||
private Region region;
|
private HRegion region;
|
||||||
|
|
||||||
private byte[] cfName = Bytes.toBytes("a");
|
private byte[] cfName = Bytes.toBytes("a");
|
||||||
private byte[] cqName = Bytes.toBytes("b");
|
private byte[] cqName = Bytes.toBytes("b");
|
||||||
|
|
|
@ -411,7 +411,7 @@ public class TestCacheOnWrite {
|
||||||
final String cf = "myCF";
|
final String cf = "myCF";
|
||||||
final byte[] cfBytes = Bytes.toBytes(cf);
|
final byte[] cfBytes = Bytes.toBytes(cf);
|
||||||
final int maxVersions = 3;
|
final int maxVersions = 3;
|
||||||
Region region = TEST_UTIL.createTestRegion(table,
|
HRegion region = TEST_UTIL.createTestRegion(table,
|
||||||
new HColumnDescriptor(cf)
|
new HColumnDescriptor(cf)
|
||||||
.setCompressionType(compress)
|
.setCompressionType(compress)
|
||||||
.setBloomFilterType(BLOOM_TYPE)
|
.setBloomFilterType(BLOOM_TYPE)
|
||||||
|
|
|
@ -127,7 +127,7 @@ public class TestForceCacheImportantBlocks {
|
||||||
else assertTrue(stats.getMissCount() > missCount);
|
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) {
|
for (int i = 0; i < NUM_ROWS; ++i) {
|
||||||
Put put = new Put(Bytes.toBytes("row" + i));
|
Put put = new Put(Bytes.toBytes("row" + i));
|
||||||
for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
|
for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
|
||||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||||
|
@ -99,7 +100,7 @@ public class TestScannerSelectionUsingKeyRange {
|
||||||
HTableDescriptor htd = new HTableDescriptor(TABLE);
|
HTableDescriptor htd = new HTableDescriptor(TABLE);
|
||||||
htd.addFamily(hcd);
|
htd.addFamily(hcd);
|
||||||
HRegionInfo info = new HRegionInfo(TABLE);
|
HRegionInfo info = new HRegionInfo(TABLE);
|
||||||
Region region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf,
|
HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf,
|
||||||
htd);
|
htd);
|
||||||
|
|
||||||
for (int iFile = 0; iFile < NUM_FILES; ++iFile) {
|
for (int iFile = 0; iFile < NUM_FILES; ++iFile) {
|
||||||
|
|
|
@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
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.HStore;
|
||||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
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.IOTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -106,7 +106,7 @@ public class TestScannerSelectionUsingTTL {
|
||||||
HTableDescriptor htd = new HTableDescriptor(TABLE);
|
HTableDescriptor htd = new HTableDescriptor(TABLE);
|
||||||
htd.addFamily(hcd);
|
htd.addFamily(hcd);
|
||||||
HRegionInfo info = new HRegionInfo(TABLE);
|
HRegionInfo info = new HRegionInfo(TABLE);
|
||||||
Region region = HBaseTestingUtility.createRegionAndWAL(info,
|
HRegion region = HBaseTestingUtility.createRegionAndWAL(info,
|
||||||
TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, htd);
|
TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, htd);
|
||||||
|
|
||||||
long ts = EnvironmentEdgeManager.currentTime();
|
long ts = EnvironmentEdgeManager.currentTime();
|
||||||
|
@ -150,7 +150,7 @@ public class TestScannerSelectionUsingTTL {
|
||||||
|
|
||||||
// Exercise both compaction codepaths.
|
// Exercise both compaction codepaths.
|
||||||
if (explicitCompaction) {
|
if (explicitCompaction) {
|
||||||
HStore store = (HStore)region.getStore(FAMILY_BYTES);
|
HStore store = region.getStore(FAMILY_BYTES);
|
||||||
store.compactRecentForTestingAssumingDefaultPolicy(totalNumFiles);
|
store.compactRecentForTestingAssumingDefaultPolicy(totalNumFiles);
|
||||||
} else {
|
} else {
|
||||||
region.compact(false);
|
region.compact(false);
|
||||||
|
|
|
@ -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.ClientProtos.ScanResponse;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
|
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.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A mock RegionServer implementation.
|
* A mock RegionServer implementation.
|
||||||
|
@ -264,12 +263,12 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void addRegion(Region r) {
|
public void addRegion(HRegion r) {
|
||||||
// TODO Auto-generated method stub
|
// TODO Auto-generated method stub
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean removeRegion(Region r, ServerName destination) {
|
public boolean removeRegion(HRegion r, ServerName destination) {
|
||||||
// TODO Auto-generated method stub
|
// TODO Auto-generated method stub
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
@ -332,11 +331,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void postOpenDeployTasks(Region r) throws KeeperException, IOException {
|
|
||||||
// TODO Auto-generated method stub
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void postOpenDeployTasks(PostOpenDeployContext context) throws KeeperException,
|
public void postOpenDeployTasks(PostOpenDeployContext context) throws KeeperException,
|
||||||
IOException {
|
IOException {
|
||||||
|
@ -592,7 +586,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<String, Region> getRecoveringRegions() {
|
public Map<String, HRegion> getRecoveringRegions() {
|
||||||
// TODO Auto-generated method stub
|
// TODO Auto-generated method stub
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -608,17 +602,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
||||||
return null;
|
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
|
@Override
|
||||||
public boolean reportRegionStateTransition(RegionStateTransitionContext context) {
|
public boolean reportRegionStateTransition(RegionStateTransitionContext context) {
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
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.shaded.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.testclassification.FlakeyTests;
|
import org.apache.hadoop.hbase.testclassification.FlakeyTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
|
@ -225,9 +224,9 @@ public class TestMasterFailover {
|
||||||
// region server should expire (how it can be verified?)
|
// region server should expire (how it can be verified?)
|
||||||
MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(),
|
MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(),
|
||||||
rs.getServerName(), State.OPENING);
|
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);
|
rs.removeRegion(meta, null);
|
||||||
((HRegion)meta).close();
|
meta.close();
|
||||||
|
|
||||||
log("Aborting master");
|
log("Aborting master");
|
||||||
activeMaster.abort("test-kill");
|
activeMaster.abort("test-kill");
|
||||||
|
|
|
@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
|
||||||
import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer;
|
import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer;
|
||||||
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
|
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
|
||||||
import org.apache.hadoop.hbase.favored.FavoredNodesPlan.Position;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
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
|
// kill a random non-meta server carrying at least one region
|
||||||
killIndex = random.nextInt(SLAVES);
|
killIndex = random.nextInt(SLAVES);
|
||||||
serverToKill = TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getServerName();
|
serverToKill = TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getServerName();
|
||||||
Collection<Region> regs =
|
Collection<HRegion> regs =
|
||||||
TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getOnlineRegionsLocalContext();
|
TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getOnlineRegionsLocalContext();
|
||||||
isNamespaceServer = false;
|
isNamespaceServer = false;
|
||||||
for (Region r : regs) {
|
for (HRegion r : regs) {
|
||||||
if (r.getRegionInfo().getTable().getNamespaceAsString()
|
if (r.getRegionInfo().getTable().getNamespaceAsString()
|
||||||
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
|
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
|
||||||
isNamespaceServer = true;
|
isNamespaceServer = true;
|
||||||
|
|
|
@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.master.LoadBalancer;
|
||||||
import org.apache.hadoop.hbase.master.ServerManager;
|
import org.apache.hadoop.hbase.master.ServerManager;
|
||||||
import org.apache.hadoop.hbase.master.assignment.RegionStates;
|
import org.apache.hadoop.hbase.master.assignment.RegionStates;
|
||||||
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
|
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.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -544,7 +545,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
|
||||||
|
|
||||||
private void compactTable(TableName tableName) throws IOException {
|
private void compactTable(TableName tableName) throws IOException {
|
||||||
for(JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
|
for(JVMClusterUtil.RegionServerThread t : cluster.getRegionServerThreads()) {
|
||||||
for(Region region : t.getRegionServer().getRegions(tableName)) {
|
for(HRegion region : t.getRegionServer().getRegions(tableName)) {
|
||||||
region.compact(true);
|
region.compact(true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
|
||||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -62,7 +62,7 @@ public class TestRegionLocationFinder {
|
||||||
|
|
||||||
for (int i = 0; i < ServerNum; i++) {
|
for (int i = 0; i < ServerNum; i++) {
|
||||||
HRegionServer server = cluster.getRegionServer(i);
|
HRegionServer server = cluster.getRegionServer(i);
|
||||||
for (Region region : server.getRegions(tableName)) {
|
for (HRegion region : server.getRegions(tableName)) {
|
||||||
region.flush(true);
|
region.flush(true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -83,7 +83,7 @@ public class TestRegionLocationFinder {
|
||||||
public void testInternalGetTopBlockLocation() throws Exception {
|
public void testInternalGetTopBlockLocation() throws Exception {
|
||||||
for (int i = 0; i < ServerNum; i++) {
|
for (int i = 0; i < ServerNum; i++) {
|
||||||
HRegionServer server = cluster.getRegionServer(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,
|
// get region's hdfs block distribution by region and RegionLocationFinder,
|
||||||
// they should have same result
|
// they should have same result
|
||||||
HDFSBlocksDistribution blocksDistribution1 = region.getHDFSBlocksDistribution();
|
HDFSBlocksDistribution blocksDistribution1 = region.getHDFSBlocksDistribution();
|
||||||
|
@ -122,7 +122,7 @@ public class TestRegionLocationFinder {
|
||||||
public void testGetTopBlockLocations() throws Exception {
|
public void testGetTopBlockLocations() throws Exception {
|
||||||
for (int i = 0; i < ServerNum; i++) {
|
for (int i = 0; i < ServerNum; i++) {
|
||||||
HRegionServer server = cluster.getRegionServer(i);
|
HRegionServer server = cluster.getRegionServer(i);
|
||||||
for (Region region : server.getRegions(tableName)) {
|
for (HRegion region : server.getRegions(tableName)) {
|
||||||
List<ServerName> servers = finder.getTopBlockLocations(region
|
List<ServerName> servers = finder.getTopBlockLocations(region
|
||||||
.getRegionInfo());
|
.getRegionInfo());
|
||||||
// test table may have empty region
|
// test table may have empty region
|
||||||
|
@ -147,12 +147,12 @@ public class TestRegionLocationFinder {
|
||||||
finder.getCache().invalidateAll();
|
finder.getCache().invalidateAll();
|
||||||
for (int i = 0; i < ServerNum; i++) {
|
for (int i = 0; i < ServerNum; i++) {
|
||||||
HRegionServer server = cluster.getRegionServer(i);
|
HRegionServer server = cluster.getRegionServer(i);
|
||||||
List<Region> regions = server.getRegions(tableName);
|
List<HRegion> regions = server.getRegions(tableName);
|
||||||
if (regions.size() <= 0) {
|
if (regions.size() <= 0) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
List<RegionInfo> regionInfos = new ArrayList<>(regions.size());
|
List<RegionInfo> regionInfos = new ArrayList<>(regions.size());
|
||||||
for (Region region : regions) {
|
for (HRegion region : regions) {
|
||||||
regionInfos.add(region.getRegionInfo());
|
regionInfos.add(region.getRegionInfo());
|
||||||
}
|
}
|
||||||
finder.refreshAndWait(regionInfos);
|
finder.refreshAndWait(regionInfos);
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.*;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.master.HMaster;
|
import org.apache.hadoop.hbase.master.HMaster;
|
||||||
import org.apache.hadoop.hbase.master.LoadBalancer;
|
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.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||||
|
@ -150,7 +151,7 @@ public class TestRegionsOnMasterOptions {
|
||||||
try {
|
try {
|
||||||
Table t = TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY, REGIONS);
|
Table t = TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY, REGIONS);
|
||||||
LOG.info("Server: " + cluster.getMaster().getServerManager().getOnlineServersList());
|
LOG.info("Server: " + cluster.getMaster().getServerManager().getOnlineServersList());
|
||||||
List<Region> regions = cluster.getMaster().getRegions();
|
List<HRegion> regions = cluster.getMaster().getRegions();
|
||||||
int mActualCount = regions.size();
|
int mActualCount = regions.size();
|
||||||
if (masterCount == 0 || masterCount == SYSTEM_REGIONS) {
|
if (masterCount == 0 || masterCount == SYSTEM_REGIONS) {
|
||||||
// 0 means no regions on master.
|
// 0 means no regions on master.
|
||||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.regionserver.Store;
|
||||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.mockito.Mockito;
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
import org.mockito.stubbing.Answer;
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
|
@ -63,7 +64,7 @@ public class TestFileSystemUtilizationChore {
|
||||||
.reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
|
.reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
|
||||||
|
|
||||||
final Region region = mockRegionWithSize(regionSizes);
|
final Region region = mockRegionWithSize(regionSizes);
|
||||||
when(rs.getRegions()).thenReturn(Arrays.asList(region));
|
Mockito.doReturn(Arrays.asList(region)).when(rs).getRegions();
|
||||||
chore.chore();
|
chore.chore();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -80,7 +81,7 @@ public class TestFileSystemUtilizationChore {
|
||||||
.reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
|
.reportRegionSizesForQuotas((Map<RegionInfo,Long>) any(Map.class));
|
||||||
|
|
||||||
final Region region = mockRegionWithSize(regionSizes);
|
final Region region = mockRegionWithSize(regionSizes);
|
||||||
when(rs.getRegions()).thenReturn(Arrays.asList(region));
|
Mockito.doReturn(Arrays.asList(region)).when(rs).getRegions();
|
||||||
chore.chore();
|
chore.chore();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -106,7 +107,7 @@ public class TestFileSystemUtilizationChore {
|
||||||
final Region r1 = mockRegionWithSize(r1Sizes);
|
final Region r1 = mockRegionWithSize(r1Sizes);
|
||||||
final Region r2 = mockRegionWithSize(r2Sizes);
|
final Region r2 = mockRegionWithSize(r2Sizes);
|
||||||
final Region r3 = mockRegionWithSize(r3Sizes);
|
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();
|
chore.chore();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -172,7 +173,7 @@ public class TestFileSystemUtilizationChore {
|
||||||
final Region r1 = mockRegionWithSize(Arrays.asList(1024L, 2048L));
|
final Region r1 = mockRegionWithSize(Arrays.asList(1024L, 2048L));
|
||||||
final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L));
|
final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L));
|
||||||
final Region r3 = mockRegionWithSize(Arrays.asList(10L * 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();
|
chore.chore();
|
||||||
}
|
}
|
||||||
|
@ -205,7 +206,7 @@ public class TestFileSystemUtilizationChore {
|
||||||
final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L));
|
final Region r2 = mockRegionWithSize(Arrays.asList(1024L * 1024L));
|
||||||
final Region r3 = mockRegionWithSize(Arrays.asList(10L * 1024L * 1024L));
|
final Region r3 = mockRegionWithSize(Arrays.asList(10L * 1024L * 1024L));
|
||||||
// lr2 is no longer online, so it should be ignored
|
// 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();
|
chore.chore();
|
||||||
}
|
}
|
||||||
|
@ -228,7 +229,7 @@ public class TestFileSystemUtilizationChore {
|
||||||
|
|
||||||
final Region r1 = mockRegionWithSize(r1Sizes);
|
final Region r1 = mockRegionWithSize(r1Sizes);
|
||||||
final Region r2 = mockSplitParentRegionWithSize(r2Sizes);
|
final Region r2 = mockSplitParentRegionWithSize(r2Sizes);
|
||||||
when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2));
|
Mockito.doReturn(Arrays.asList(r1, r2)).when(rs).getRegions();
|
||||||
chore.chore();
|
chore.chore();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -250,7 +251,7 @@ public class TestFileSystemUtilizationChore {
|
||||||
|
|
||||||
final Region r1 = mockRegionWithSize(r1Sizes);
|
final Region r1 = mockRegionWithSize(r1Sizes);
|
||||||
final Region r2 = mockRegionReplicaWithSize(r2Sizes);
|
final Region r2 = mockRegionReplicaWithSize(r2Sizes);
|
||||||
when(rs.getRegions()).thenReturn(Arrays.asList(r1, r2));
|
Mockito.doReturn(Arrays.asList(r1, r2)).when(rs).getRegions();
|
||||||
chore.chore();
|
chore.chore();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -277,7 +278,7 @@ public class TestFileSystemUtilizationChore {
|
||||||
|
|
||||||
final Region r1 = mockRegionWithHFileLinks(r1StoreFileSizes, r1HFileSizes);
|
final Region r1 = mockRegionWithHFileLinks(r1StoreFileSizes, r1HFileSizes);
|
||||||
final Region r2 = mockRegionWithHFileLinks(r2StoreFileSizes, r2HFileSizes);
|
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();
|
chore.chore();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -270,12 +270,12 @@ public class RegionAsTable implements Table {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Result append(Append append) throws IOException {
|
public Result append(Append append) throws IOException {
|
||||||
return this.region.append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
return this.region.append(append);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Result increment(Increment increment) throws IOException {
|
public Result increment(Increment increment) throws IOException {
|
||||||
return this.region.increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
return this.region.increment(increment);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -287,7 +287,7 @@ public class TestAtomicOperation {
|
||||||
inc.addColumn(fam1, qual2, amount*2);
|
inc.addColumn(fam1, qual2, amount*2);
|
||||||
inc.addColumn(fam2, qual3, amount*3);
|
inc.addColumn(fam2, qual3, amount*3);
|
||||||
inc.setDurability(Durability.ASYNC_WAL);
|
inc.setDurability(Durability.ASYNC_WAL);
|
||||||
Result result = region.increment(inc, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
Result result = region.increment(inc);
|
||||||
if (result != null) {
|
if (result != null) {
|
||||||
assertEquals(Bytes.toLong(result.getValue(fam1, qual1))*2,
|
assertEquals(Bytes.toLong(result.getValue(fam1, qual1))*2,
|
||||||
Bytes.toLong(result.getValue(fam1, qual2)));
|
Bytes.toLong(result.getValue(fam1, qual2)));
|
||||||
|
@ -555,13 +555,13 @@ public class TestAtomicOperation {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class AtomicOperation extends Thread {
|
public static class AtomicOperation extends Thread {
|
||||||
protected final Region region;
|
protected final HRegion region;
|
||||||
protected final int numOps;
|
protected final int numOps;
|
||||||
protected final AtomicLong timeStamps;
|
protected final AtomicLong timeStamps;
|
||||||
protected final AtomicInteger failures;
|
protected final AtomicInteger failures;
|
||||||
protected final Random r = new Random();
|
protected final Random r = new Random();
|
||||||
|
|
||||||
public AtomicOperation(Region region, int numOps, AtomicLong timeStamps,
|
public AtomicOperation(HRegion region, int numOps, AtomicLong timeStamps,
|
||||||
AtomicInteger failures) {
|
AtomicInteger failures) {
|
||||||
this.region = region;
|
this.region = region;
|
||||||
this.numOps = numOps;
|
this.numOps = numOps;
|
||||||
|
@ -633,7 +633,7 @@ public class TestAtomicOperation {
|
||||||
put.addColumn(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
|
put.addColumn(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
|
||||||
puts[0] = put;
|
puts[0] = put;
|
||||||
testStep = TestStep.PUT_STARTED;
|
testStep = TestStep.PUT_STARTED;
|
||||||
region.batchMutate(puts, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
region.batchMutate(puts);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -63,7 +63,7 @@ public class TestBlocksRead {
|
||||||
BloomType.ROW, BloomType.NONE };
|
BloomType.ROW, BloomType.NONE };
|
||||||
|
|
||||||
private static BlockCache blockCache;
|
private static BlockCache blockCache;
|
||||||
Region region = null;
|
HRegion region = null;
|
||||||
private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||||
private final String DIR = TEST_UTIL.getDataTestDir("TestBlocksRead").toString();
|
private final String DIR = TEST_UTIL.getDataTestDir("TestBlocksRead").toString();
|
||||||
private Configuration conf = TEST_UTIL.getConfiguration();
|
private Configuration conf = TEST_UTIL.getConfiguration();
|
||||||
|
@ -88,7 +88,7 @@ public class TestBlocksRead {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @return created and initialized region.
|
* @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 {
|
Configuration conf, String family) throws IOException {
|
||||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
|
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
|
||||||
HColumnDescriptor familyDesc;
|
HColumnDescriptor familyDesc;
|
||||||
|
@ -102,7 +102,7 @@ public class TestBlocksRead {
|
||||||
|
|
||||||
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
||||||
Path path = new Path(DIR + callingMethod);
|
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();
|
blockCache = new CacheConfig(conf).getBlockCache();
|
||||||
return r;
|
return r;
|
||||||
}
|
}
|
||||||
|
|
|
@ -89,7 +89,7 @@ public class TestBlocksScanned extends HBaseTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void _testBlocksScanned(HTableDescriptor table) throws Exception {
|
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);
|
addContent(r, FAMILY, COL);
|
||||||
r.flush(true);
|
r.flush(true);
|
||||||
|
|
||||||
|
|
|
@ -76,7 +76,7 @@ public class TestColumnSeeking {
|
||||||
htd.addFamily(hcd);
|
htd.addFamily(hcd);
|
||||||
HRegionInfo info = new HRegionInfo(table, null, null, false);
|
HRegionInfo info = new HRegionInfo(table, null, null, false);
|
||||||
// Set this so that the archiver writes to the temp dir as well.
|
// 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 {
|
try {
|
||||||
List<String> rows = generateRandomWords(10, "row");
|
List<String> rows = generateRandomWords(10, "row");
|
||||||
List<String> allColumns = generateRandomWords(10, "column");
|
List<String> allColumns = generateRandomWords(10, "column");
|
||||||
|
@ -188,7 +188,7 @@ public class TestColumnSeeking {
|
||||||
htd.addFamily(hcd);
|
htd.addFamily(hcd);
|
||||||
|
|
||||||
HRegionInfo info = new HRegionInfo(table, null, null, false);
|
HRegionInfo info = new HRegionInfo(table, null, null, false);
|
||||||
Region region = TEST_UTIL.createLocalHRegion(info, htd);
|
HRegion region = TEST_UTIL.createLocalHRegion(info, htd);
|
||||||
|
|
||||||
List<String> rows = generateRandomWords(10, "row");
|
List<String> rows = generateRandomWords(10, "row");
|
||||||
List<String> allColumns = generateRandomWords(100, "column");
|
List<String> allColumns = generateRandomWords(100, "column");
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
import org.junit.rules.TestName;
|
import org.junit.rules.TestName;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests a race condition between archiving of compacted files in CompactedHFilesDischarger chore
|
* 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);
|
HRegionInfo info = new HRegionInfo(tableName, null, null, false);
|
||||||
HRegion region = initHRegion(htd, info);
|
HRegion region = initHRegion(htd, info);
|
||||||
RegionServerServices rss = mock(RegionServerServices.class);
|
RegionServerServices rss = mock(RegionServerServices.class);
|
||||||
List<Region> regions = new ArrayList<>();
|
List<HRegion> regions = new ArrayList<>();
|
||||||
regions.add(region);
|
regions.add(region);
|
||||||
when(rss.getRegions()).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions();
|
||||||
|
|
||||||
// Create the cleaner object
|
// Create the cleaner object
|
||||||
CompactedHFilesDischarger cleaner =
|
CompactedHFilesDischarger cleaner =
|
||||||
|
|
|
@ -27,7 +27,6 @@ import static org.mockito.Matchers.eq;
|
||||||
import static org.mockito.Mockito.doThrow;
|
import static org.mockito.Mockito.doThrow;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.spy;
|
import static org.mockito.Mockito.spy;
|
||||||
import static org.mockito.Mockito.when;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
|
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.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
import org.junit.rules.TestName;
|
import org.junit.rules.TestName;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests that archiving compacted files behaves correctly when encountering exceptions.
|
* 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);
|
HRegionInfo info = new HRegionInfo(tableName, null, null, false);
|
||||||
final HRegion region = initHRegion(htd, info);
|
final HRegion region = initHRegion(htd, info);
|
||||||
RegionServerServices rss = mock(RegionServerServices.class);
|
RegionServerServices rss = mock(RegionServerServices.class);
|
||||||
List<Region> regions = new ArrayList<>();
|
List<HRegion> regions = new ArrayList<>();
|
||||||
regions.add(region);
|
regions.add(region);
|
||||||
when(rss.getRegions()).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions();
|
||||||
|
|
||||||
// Create the cleaner object
|
// Create the cleaner object
|
||||||
final CompactedHFilesDischarger cleaner =
|
final CompactedHFilesDischarger cleaner =
|
||||||
|
|
|
@ -138,7 +138,7 @@ public class TestCompactionState {
|
||||||
ht = TEST_UTIL.createTable(table, families);
|
ht = TEST_UTIL.createTable(table, families);
|
||||||
loadData(ht, families, 3000, flushes);
|
loadData(ht, families, 3000, flushes);
|
||||||
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
|
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
|
||||||
List<Region> regions = rs.getRegions(table);
|
List<HRegion> regions = rs.getRegions(table);
|
||||||
int countBefore = countStoreFilesInFamilies(regions, families);
|
int countBefore = countStoreFilesInFamilies(regions, families);
|
||||||
int countBeforeSingleFamily = countStoreFilesInFamily(regions, family);
|
int countBeforeSingleFamily = countStoreFilesInFamily(regions, family);
|
||||||
assertTrue(countBefore > 0); // there should be some data files
|
assertTrue(countBefore > 0); // there should be some data files
|
||||||
|
@ -206,13 +206,13 @@ public class TestCompactionState {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static int countStoreFilesInFamily(
|
private static int countStoreFilesInFamily(
|
||||||
List<Region> regions, final byte[] family) {
|
List<HRegion> regions, final byte[] family) {
|
||||||
return countStoreFilesInFamilies(regions, new byte[][]{family});
|
return countStoreFilesInFamilies(regions, new byte[][]{family});
|
||||||
}
|
}
|
||||||
|
|
||||||
private static int countStoreFilesInFamilies(List<Region> regions, final byte[][] families) {
|
private static int countStoreFilesInFamilies(List<HRegion> regions, final byte[][] families) {
|
||||||
int count = 0;
|
int count = 0;
|
||||||
for (Region region: regions) {
|
for (HRegion region: regions) {
|
||||||
count += region.getStoreFileList(families).size();
|
count += region.getStoreFileList(families).size();
|
||||||
}
|
}
|
||||||
return count;
|
return count;
|
||||||
|
|
|
@ -83,7 +83,7 @@ public class TestGetClosestAtOrBefore {
|
||||||
TableDescriptorBuilder metaBuilder = UTIL.getMetaTableDescriptorBuilder()
|
TableDescriptorBuilder metaBuilder = UTIL.getMetaTableDescriptorBuilder()
|
||||||
.setMemStoreFlushSize(64 * 1024 * 1024);
|
.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());
|
rootdir, this.conf, metaBuilder.build());
|
||||||
try {
|
try {
|
||||||
// Write rows for three tables 'A', 'B', and 'C'.
|
// Write rows for three tables 'A', 'B', and 'C'.
|
||||||
|
@ -195,7 +195,7 @@ public class TestGetClosestAtOrBefore {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testGetClosestRowBefore3() throws IOException{
|
public void testGetClosestRowBefore3() throws IOException{
|
||||||
Region region = null;
|
HRegion region = null;
|
||||||
byte [] c0 = UTIL.COLUMNS[0];
|
byte [] c0 = UTIL.COLUMNS[0];
|
||||||
byte [] c1 = UTIL.COLUMNS[1];
|
byte [] c1 = UTIL.COLUMNS[1];
|
||||||
try {
|
try {
|
||||||
|
@ -306,7 +306,7 @@ public class TestGetClosestAtOrBefore {
|
||||||
/** For HBASE-694 */
|
/** For HBASE-694 */
|
||||||
@Test
|
@Test
|
||||||
public void testGetClosestRowBefore2() throws IOException{
|
public void testGetClosestRowBefore2() throws IOException{
|
||||||
Region region = null;
|
HRegion region = null;
|
||||||
byte [] c0 = UTIL.COLUMNS[0];
|
byte [] c0 = UTIL.COLUMNS[0];
|
||||||
try {
|
try {
|
||||||
TableName tn = TableName.valueOf(testName.getMethodName());
|
TableName tn = TableName.valueOf(testName.getMethodName());
|
||||||
|
|
|
@ -5884,7 +5884,7 @@ public class TestHRegion {
|
||||||
when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
|
when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
|
||||||
|
|
||||||
// add the region to recovering regions
|
// add the region to recovering regions
|
||||||
HashMap<String, Region> recoveringRegions = Maps.newHashMap();
|
HashMap<String, HRegion> recoveringRegions = Maps.newHashMap();
|
||||||
recoveringRegions.put(region.getRegionInfo().getEncodedName(), null);
|
recoveringRegions.put(region.getRegionInfo().getEncodedName(), null);
|
||||||
when(rss.getRecoveringRegions()).thenReturn(recoveringRegions);
|
when(rss.getRecoveringRegions()).thenReturn(recoveringRegions);
|
||||||
|
|
||||||
|
|
|
@ -87,7 +87,7 @@ import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
import org.junit.rules.TestName;
|
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.common.collect.Lists;
|
||||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
|
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.ProtobufUtil;
|
||||||
|
@ -180,9 +180,9 @@ public class TestHRegionReplayEvents {
|
||||||
when(rss.getExecutorService()).thenReturn(es);
|
when(rss.getExecutorService()).thenReturn(es);
|
||||||
primaryRegion = HRegion.createHRegion(primaryHri, rootDir, CONF, htd, walPrimary);
|
primaryRegion = HRegion.createHRegion(primaryHri, rootDir, CONF, htd, walPrimary);
|
||||||
primaryRegion.close();
|
primaryRegion.close();
|
||||||
List<Region> regions = new ArrayList<>();
|
List<HRegion> regions = new ArrayList<>();
|
||||||
regions.add(primaryRegion);
|
regions.add(primaryRegion);
|
||||||
when(rss.getRegions()).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions();
|
||||||
|
|
||||||
primaryRegion = HRegion.openHRegion(rootDir, primaryHri, htd, walPrimary, CONF, rss, null);
|
primaryRegion = HRegion.openHRegion(rootDir, primaryHri, htd, walPrimary, CONF, rss, null);
|
||||||
secondaryRegion = HRegion.openHRegion(secondaryHri, htd, null, 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
|
// Test case 3: compact primary files
|
||||||
primaryRegion.compactStores();
|
primaryRegion.compactStores();
|
||||||
List<Region> regions = new ArrayList<>();
|
List<HRegion> regions = new ArrayList<>();
|
||||||
regions.add(primaryRegion);
|
regions.add(primaryRegion);
|
||||||
when(rss.getRegions()).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions();
|
||||||
CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, rss, false);
|
CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, rss, false);
|
||||||
cleaner.chore();
|
cleaner.chore();
|
||||||
secondaryRegion.refreshStoreFiles();
|
secondaryRegion.refreshStoreFiles();
|
||||||
|
|
|
@ -804,12 +804,12 @@ public class TestHeapMemoryManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void requestFlush(Region region, boolean forceFlushAllStores) {
|
public void requestFlush(HRegion region, boolean forceFlushAllStores) {
|
||||||
this.listener.flushRequested(flushType, region);
|
this.listener.flushRequested(flushType, region);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores) {
|
public void requestDelayedFlush(HRegion region, long delay, boolean forceFlushAllStores) {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -196,7 +196,7 @@ public class TestKeepDeletes {
|
||||||
// KEEP_DELETED_CELLS is NOT enabled
|
// KEEP_DELETED_CELLS is NOT enabled
|
||||||
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3,
|
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3,
|
||||||
HConstants.FOREVER, KeepDeletedCells.FALSE);
|
HConstants.FOREVER, KeepDeletedCells.FALSE);
|
||||||
Region region = hbu.createLocalHRegion(htd, null, null);
|
HRegion region = hbu.createLocalHRegion(htd, null, null);
|
||||||
|
|
||||||
long ts = EnvironmentEdgeManager.currentTime();
|
long ts = EnvironmentEdgeManager.currentTime();
|
||||||
Put p = new Put(T1, ts);
|
Put p = new Put(T1, ts);
|
||||||
|
|
|
@ -388,17 +388,17 @@ public class TestMajorCompaction {
|
||||||
return count;
|
return count;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void createStoreFile(final Region region) throws IOException {
|
private void createStoreFile(final HRegion region) throws IOException {
|
||||||
createStoreFile(region, Bytes.toString(COLUMN_FAMILY));
|
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);
|
Table loader = new RegionAsTable(region);
|
||||||
HBaseTestCase.addContent(loader, family);
|
HBaseTestCase.addContent(loader, family);
|
||||||
region.flush(true);
|
region.flush(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void createSmallerStoreFile(final Region region) throws IOException {
|
private void createSmallerStoreFile(final HRegion region) throws IOException {
|
||||||
Table loader = new RegionAsTable(region);
|
Table loader = new RegionAsTable(region);
|
||||||
HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY), ("" +
|
HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY), ("" +
|
||||||
"bbb").getBytes(), null);
|
"bbb").getBytes(), null);
|
||||||
|
|
|
@ -68,7 +68,7 @@ public class TestMinVersions {
|
||||||
public void testGetClosestBefore() throws Exception {
|
public void testGetClosestBefore() throws Exception {
|
||||||
HTableDescriptor htd =
|
HTableDescriptor htd =
|
||||||
hbu.createTableDescriptor(name.getMethodName(), 1, 1000, 1, KeepDeletedCells.FALSE);
|
hbu.createTableDescriptor(name.getMethodName(), 1, 1000, 1, KeepDeletedCells.FALSE);
|
||||||
Region region = hbu.createLocalHRegion(htd, null, null);
|
HRegion region = hbu.createLocalHRegion(htd, null, null);
|
||||||
try {
|
try {
|
||||||
|
|
||||||
// 2s in the past
|
// 2s in the past
|
||||||
|
@ -118,7 +118,7 @@ public class TestMinVersions {
|
||||||
// keep 3 versions minimum
|
// keep 3 versions minimum
|
||||||
HTableDescriptor htd =
|
HTableDescriptor htd =
|
||||||
hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.FALSE);
|
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
|
// 2s in the past
|
||||||
long ts = EnvironmentEdgeManager.currentTime() - 2000;
|
long ts = EnvironmentEdgeManager.currentTime() - 2000;
|
||||||
|
|
||||||
|
@ -173,7 +173,7 @@ public class TestMinVersions {
|
||||||
public void testDelete() throws Exception {
|
public void testDelete() throws Exception {
|
||||||
HTableDescriptor htd =
|
HTableDescriptor htd =
|
||||||
hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.FALSE);
|
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
|
// 2s in the past
|
||||||
long ts = EnvironmentEdgeManager.currentTime() - 2000;
|
long ts = EnvironmentEdgeManager.currentTime() - 2000;
|
||||||
|
@ -232,7 +232,7 @@ public class TestMinVersions {
|
||||||
public void testMemStore() throws Exception {
|
public void testMemStore() throws Exception {
|
||||||
HTableDescriptor htd =
|
HTableDescriptor htd =
|
||||||
hbu.createTableDescriptor(name.getMethodName(), 2, 1000, 1, KeepDeletedCells.FALSE);
|
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
|
// 2s in the past
|
||||||
long ts = EnvironmentEdgeManager.currentTime() - 2000;
|
long ts = EnvironmentEdgeManager.currentTime() - 2000;
|
||||||
|
@ -308,7 +308,7 @@ public class TestMinVersions {
|
||||||
// 1 version minimum, 1000 versions maximum, ttl = 1s
|
// 1 version minimum, 1000 versions maximum, ttl = 1s
|
||||||
HTableDescriptor htd =
|
HTableDescriptor htd =
|
||||||
hbu.createTableDescriptor(name.getMethodName(), 2, 1000, 1, KeepDeletedCells.FALSE);
|
hbu.createTableDescriptor(name.getMethodName(), 2, 1000, 1, KeepDeletedCells.FALSE);
|
||||||
Region region = hbu.createLocalHRegion(htd, null, null);
|
HRegion region = hbu.createLocalHRegion(htd, null, null);
|
||||||
try {
|
try {
|
||||||
|
|
||||||
// 2s in the past
|
// 2s in the past
|
||||||
|
@ -400,7 +400,7 @@ public class TestMinVersions {
|
||||||
public void testFilters() throws Exception {
|
public void testFilters() throws Exception {
|
||||||
HTableDescriptor htd =
|
HTableDescriptor htd =
|
||||||
hbu.createTableDescriptor(name.getMethodName(), 2, 1000, 1, KeepDeletedCells.FALSE);
|
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];
|
final byte [] c1 = COLUMNS[1];
|
||||||
|
|
||||||
// 2s in the past
|
// 2s in the past
|
||||||
|
|
|
@ -145,7 +145,7 @@ public class TestMultiColumnScanner {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMultiColumnScanner() throws IOException {
|
public void testMultiColumnScanner() throws IOException {
|
||||||
Region region = TEST_UTIL.createTestRegion(TABLE_NAME,
|
HRegion region = TEST_UTIL.createTestRegion(TABLE_NAME,
|
||||||
new HColumnDescriptor(FAMILY)
|
new HColumnDescriptor(FAMILY)
|
||||||
.setCompressionType(comprAlgo)
|
.setCompressionType(comprAlgo)
|
||||||
.setBloomFilterType(bloomType)
|
.setBloomFilterType(bloomType)
|
||||||
|
|
|
@ -130,7 +130,7 @@ public class TestPerColumnFamilyFlush {
|
||||||
conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
|
conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
|
||||||
40 * 1024);
|
40 * 1024);
|
||||||
// Intialize the region
|
// 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
|
// Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
|
||||||
for (int i = 1; i <= 1200; i++) {
|
for (int i = 1; i <= 1200; i++) {
|
||||||
region.put(createPut(1, i));
|
region.put(createPut(1, i));
|
||||||
|
@ -324,12 +324,12 @@ public class TestPerColumnFamilyFlush {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Find the (first) region which has the specified name.
|
// Find the (first) region which has the specified name.
|
||||||
private static Pair<Region, HRegionServer> getRegionWithName(TableName tableName) {
|
private static Pair<HRegion, HRegionServer> getRegionWithName(TableName tableName) {
|
||||||
MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
|
MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
|
||||||
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
|
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
|
||||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||||
HRegionServer hrs = rsts.get(i).getRegionServer();
|
HRegionServer hrs = rsts.get(i).getRegionServer();
|
||||||
for (Region region : hrs.getRegions(tableName)) {
|
for (HRegion region : hrs.getRegions(tableName)) {
|
||||||
return Pair.newPair(region, hrs);
|
return Pair.newPair(region, hrs);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -367,8 +367,8 @@ public class TestPerColumnFamilyFlush {
|
||||||
}
|
}
|
||||||
Thread.sleep(1000);
|
Thread.sleep(1000);
|
||||||
|
|
||||||
Pair<Region, HRegionServer> desiredRegionAndServer = getRegionWithName(TABLENAME);
|
Pair<HRegion, HRegionServer> desiredRegionAndServer = getRegionWithName(TABLENAME);
|
||||||
Region desiredRegion = desiredRegionAndServer.getFirst();
|
HRegion desiredRegion = desiredRegionAndServer.getFirst();
|
||||||
assertTrue("Could not find a region which hosts the new region.", desiredRegion != null);
|
assertTrue("Could not find a region which hosts the new region.", desiredRegion != null);
|
||||||
|
|
||||||
// Flush the region selectively.
|
// Flush the region selectively.
|
||||||
|
@ -476,8 +476,8 @@ public class TestPerColumnFamilyFlush {
|
||||||
try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
|
try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
|
||||||
admin.flush(TableName.NAMESPACE_TABLE_NAME);
|
admin.flush(TableName.NAMESPACE_TABLE_NAME);
|
||||||
}
|
}
|
||||||
Pair<Region, HRegionServer> desiredRegionAndServer = getRegionWithName(tableName);
|
Pair<HRegion, HRegionServer> desiredRegionAndServer = getRegionWithName(tableName);
|
||||||
final Region desiredRegion = desiredRegionAndServer.getFirst();
|
final HRegion desiredRegion = desiredRegionAndServer.getFirst();
|
||||||
assertTrue("Could not find a region which hosts the new region.", desiredRegion != null);
|
assertTrue("Could not find a region which hosts the new region.", desiredRegion != null);
|
||||||
LOG.info("Writing to region=" + desiredRegion);
|
LOG.info("Writing to region=" + desiredRegion);
|
||||||
|
|
||||||
|
|
|
@ -114,8 +114,8 @@ public class TestRegionFavoredNodes {
|
||||||
// them as favored nodes through the region.
|
// them as favored nodes through the region.
|
||||||
for (int i = 0; i < REGION_SERVERS; i++) {
|
for (int i = 0; i < REGION_SERVERS; i++) {
|
||||||
HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i);
|
HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i);
|
||||||
List<Region> regions = server.getRegions(TABLE_NAME);
|
List<HRegion> regions = server.getRegions(TABLE_NAME);
|
||||||
for (Region region : regions) {
|
for (HRegion region : regions) {
|
||||||
List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName>favoredNodes =
|
List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName>favoredNodes =
|
||||||
new ArrayList<>(3);
|
new ArrayList<>(3);
|
||||||
String encodedRegionName = region.getRegionInfo().getEncodedName();
|
String encodedRegionName = region.getRegionInfo().getEncodedName();
|
||||||
|
@ -142,8 +142,8 @@ public class TestRegionFavoredNodes {
|
||||||
// they are consistent with the favored nodes for that region.
|
// they are consistent with the favored nodes for that region.
|
||||||
for (int i = 0; i < REGION_SERVERS; i++) {
|
for (int i = 0; i < REGION_SERVERS; i++) {
|
||||||
HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i);
|
HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(i);
|
||||||
List<Region> regions = server.getRegions(TABLE_NAME);
|
List<HRegion> regions = server.getRegions(TABLE_NAME);
|
||||||
for (Region region : regions) {
|
for (HRegion region : regions) {
|
||||||
List<String> files = region.getStoreFileList(new byte[][]{COLUMN_FAMILY});
|
List<String> files = region.getStoreFileList(new byte[][]{COLUMN_FAMILY});
|
||||||
for (String file : files) {
|
for (String file : files) {
|
||||||
FileStatus status = TEST_UTIL.getDFSCluster().getFileSystem().
|
FileStatus status = TEST_UTIL.getDFSCluster().getFileSystem().
|
||||||
|
|
|
@ -156,7 +156,7 @@ public class TestRegionReplicas {
|
||||||
// assert that we can read back from primary
|
// assert that we can read back from primary
|
||||||
Assert.assertEquals(1000, HTU.countRows(table));
|
Assert.assertEquals(1000, HTU.countRows(table));
|
||||||
// flush so that region replica can read
|
// flush so that region replica can read
|
||||||
Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName());
|
HRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName());
|
||||||
region.flush(true);
|
region.flush(true);
|
||||||
|
|
||||||
openRegion(HTU, getRS(), hriSecondary);
|
openRegion(HTU, getRS(), hriSecondary);
|
||||||
|
@ -180,7 +180,7 @@ public class TestRegionReplicas {
|
||||||
// assert that we can read back from primary
|
// assert that we can read back from primary
|
||||||
Assert.assertEquals(1000, HTU.countRows(table));
|
Assert.assertEquals(1000, HTU.countRows(table));
|
||||||
// flush so that region replica can read
|
// flush so that region replica can read
|
||||||
Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName());
|
HRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName());
|
||||||
region.flush(true);
|
region.flush(true);
|
||||||
|
|
||||||
openRegion(HTU, getRS(), hriSecondary);
|
openRegion(HTU, getRS(), hriSecondary);
|
||||||
|
@ -250,7 +250,7 @@ public class TestRegionReplicas {
|
||||||
Assert.assertEquals(1000, HTU.countRows(table));
|
Assert.assertEquals(1000, HTU.countRows(table));
|
||||||
// flush so that region replica can read
|
// flush so that region replica can read
|
||||||
LOG.info("Flushing primary region");
|
LOG.info("Flushing primary region");
|
||||||
Region region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName());
|
HRegion region = getRS().getRegionByEncodedName(hriPrimary.getEncodedName());
|
||||||
region.flush(true);
|
region.flush(true);
|
||||||
HRegion primaryRegion = (HRegion) region;
|
HRegion primaryRegion = (HRegion) region;
|
||||||
|
|
||||||
|
@ -442,11 +442,11 @@ public class TestRegionReplicas {
|
||||||
LOG.info("Loading data to primary region");
|
LOG.info("Loading data to primary region");
|
||||||
for (int i = 0; i < 3; ++i) {
|
for (int i = 0; i < 3; ++i) {
|
||||||
HTU.loadNumericRows(table, f, i * 1000, (i + 1) * 1000);
|
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.flush(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
Region primaryRegion = getRS().getRegion(hriPrimary.getEncodedName());
|
HRegion primaryRegion = getRS().getRegion(hriPrimary.getEncodedName());
|
||||||
Assert.assertEquals(3, primaryRegion.getStore(f).getStorefilesCount());
|
Assert.assertEquals(3, primaryRegion.getStore(f).getStorefilesCount());
|
||||||
|
|
||||||
// Refresh store files on the secondary
|
// Refresh store files on the secondary
|
||||||
|
|
|
@ -18,10 +18,8 @@
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
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.HConstants;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
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.client.Table;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
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.apache.hadoop.hbase.util.RegionSplitter;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Ignore;
|
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
|
@ -116,9 +112,9 @@ public class TestRegionReplicasWithModifyTable {
|
||||||
try {
|
try {
|
||||||
tableName = TableName.valueOf(name.getMethodName());
|
tableName = TableName.valueOf(name.getMethodName());
|
||||||
enableReplicationByModification(tableName, false, 0, 3, 0);
|
enableReplicationByModification(tableName, false, 0, 3, 0);
|
||||||
List<Region> onlineRegions = getRS().getRegions(tableName);
|
List<HRegion> onlineRegions = getRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
||||||
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
||||||
assertEquals("the number of regions should be more than 1", totalRegions, 3);
|
assertEquals("the number of regions should be more than 1", totalRegions, 3);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -137,9 +133,9 @@ public class TestRegionReplicasWithModifyTable {
|
||||||
try {
|
try {
|
||||||
tableName = TableName.valueOf(name.getMethodName());
|
tableName = TableName.valueOf(name.getMethodName());
|
||||||
enableReplicationByModification(tableName, false, 0, 3, 10);
|
enableReplicationByModification(tableName, false, 0, 3, 10);
|
||||||
List<Region> onlineRegions = getRS().getRegions(tableName);
|
List<HRegion> onlineRegions = getRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
||||||
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
||||||
assertEquals("the number of regions should be equal to 30", totalRegions, 30);
|
assertEquals("the number of regions should be equal to 30", totalRegions, 30);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -153,9 +149,9 @@ public class TestRegionReplicasWithModifyTable {
|
||||||
try {
|
try {
|
||||||
tableName = TableName.valueOf(name.getMethodName());
|
tableName = TableName.valueOf(name.getMethodName());
|
||||||
enableReplicationByModification(tableName, true, 2, 3, 0);
|
enableReplicationByModification(tableName, true, 2, 3, 0);
|
||||||
List<Region> onlineRegions = getRS().getRegions(tableName);
|
List<HRegion> onlineRegions = getRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
||||||
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
||||||
assertEquals("the number of regions should be 3", totalRegions, 3);
|
assertEquals("the number of regions should be 3", totalRegions, 3);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -169,9 +165,9 @@ public class TestRegionReplicasWithModifyTable {
|
||||||
try {
|
try {
|
||||||
tableName = TableName.valueOf(name.getMethodName());
|
tableName = TableName.valueOf(name.getMethodName());
|
||||||
enableReplicationByModification(tableName, true, 3, 2, 0);
|
enableReplicationByModification(tableName, true, 3, 2, 0);
|
||||||
List<Region> onlineRegions = getRS().getRegions(tableName);
|
List<HRegion> onlineRegions = getRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
||||||
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
||||||
assertEquals("the number of regions should be reduced to 2", totalRegions, 2);
|
assertEquals("the number of regions should be reduced to 2", totalRegions, 2);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -186,9 +182,9 @@ public class TestRegionReplicasWithModifyTable {
|
||||||
try {
|
try {
|
||||||
tableName = TableName.valueOf(name.getMethodName());
|
tableName = TableName.valueOf(name.getMethodName());
|
||||||
enableReplicationByModification(tableName, true, 3, 2, 20);
|
enableReplicationByModification(tableName, true, 3, 2, 20);
|
||||||
List<Region> onlineRegions = getRS().getRegions(tableName);
|
List<HRegion> onlineRegions = getRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
||||||
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
||||||
assertEquals("the number of regions should be reduced to 40", totalRegions, 40);
|
assertEquals("the number of regions should be reduced to 40", totalRegions, 40);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -203,9 +199,9 @@ public class TestRegionReplicasWithModifyTable {
|
||||||
try {
|
try {
|
||||||
tableName = TableName.valueOf(name.getMethodName());
|
tableName = TableName.valueOf(name.getMethodName());
|
||||||
enableReplicationByModification(tableName, true, 2, 3, 15);
|
enableReplicationByModification(tableName, true, 2, 3, 15);
|
||||||
List<Region> onlineRegions = getRS().getRegions(tableName);
|
List<HRegion> onlineRegions = getRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions2 = getSecondaryRS().getRegions(tableName);
|
||||||
List<Region> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
List<HRegion> onlineRegions3 = getTertiaryRS().getRegions(tableName);
|
||||||
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
int totalRegions = onlineRegions.size() + onlineRegions2.size() + onlineRegions3.size();
|
||||||
assertEquals("the number of regions should be equal to 45", totalRegions, 3 * 15);
|
assertEquals("the number of regions should be equal to 45", totalRegions, 3 * 15);
|
||||||
} finally {
|
} finally {
|
||||||
|
|
|
@ -496,7 +496,7 @@ public class TestRegionServerMetrics {
|
||||||
byte[] val = Bytes.toBytes("mobdata");
|
byte[] val = Bytes.toBytes("mobdata");
|
||||||
try {
|
try {
|
||||||
Table table = TEST_UTIL.createTable(htd, new byte[0][0], conf);
|
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++) {
|
for (int insertCount = 0; insertCount < numHfiles; insertCount++) {
|
||||||
Put p = new Put(Bytes.toBytes(insertCount));
|
Put p = new Put(Bytes.toBytes(insertCount));
|
||||||
p.addColumn(cf, qualifier, val);
|
p.addColumn(cf, qualifier, val);
|
||||||
|
@ -515,7 +515,7 @@ public class TestRegionServerMetrics {
|
||||||
|
|
||||||
setMobThreshold(region, cf, 100);
|
setMobThreshold(region, cf, 100);
|
||||||
// metrics are reset by the region initialization
|
// metrics are reset by the region initialization
|
||||||
((HRegion) region).initialize();
|
region.initialize();
|
||||||
region.compact(true);
|
region.compact(true);
|
||||||
metricsRegionServer.getRegionServerWrapper().forceRecompute();
|
metricsRegionServer.getRegionServerWrapper().forceRecompute();
|
||||||
assertCounter("cellsCountCompactedFromMob", numHfiles);
|
assertCounter("cellsCountCompactedFromMob", numHfiles);
|
||||||
|
@ -539,7 +539,7 @@ public class TestRegionServerMetrics {
|
||||||
((HRegion) region).close();
|
((HRegion) region).close();
|
||||||
|
|
||||||
// metrics are reset by the region initialization
|
// metrics are reset by the region initialization
|
||||||
((HRegion) region).initialize();
|
region.initialize();
|
||||||
region.compact(true);
|
region.compact(true);
|
||||||
metricsRegionServer.getRegionServerWrapper().forceRecompute();
|
metricsRegionServer.getRegionServerWrapper().forceRecompute();
|
||||||
// metrics are reset by the region initialization
|
// metrics are reset by the region initialization
|
||||||
|
|
|
@ -129,7 +129,7 @@ public class TestRegionServerNoMaster {
|
||||||
/** Flush the given region in the mini cluster. Since no master, we cannot use HBaseAdmin.flush() */
|
/** 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 {
|
public static void flushRegion(HBaseTestingUtility HTU, HRegionInfo regionInfo) throws IOException {
|
||||||
for (RegionServerThread rst : HTU.getMiniHBaseCluster().getRegionServerThreads()) {
|
for (RegionServerThread rst : HTU.getMiniHBaseCluster().getRegionServerThreads()) {
|
||||||
Region region = rst.getRegionServer().getRegionByEncodedName(regionInfo.getEncodedName());
|
HRegion region = rst.getRegionServer().getRegionByEncodedName(regionInfo.getEncodedName());
|
||||||
if (region != null) {
|
if (region != null) {
|
||||||
region.flush(true);
|
region.flush(true);
|
||||||
return;
|
return;
|
||||||
|
|
|
@ -104,8 +104,8 @@ public class TestRegionSplitPolicy {
|
||||||
// Now make it so the mock region has a RegionServerService that will
|
// Now make it so the mock region has a RegionServerService that will
|
||||||
// return 'online regions'.
|
// return 'online regions'.
|
||||||
RegionServerServices rss = Mockito.mock(RegionServerServices.class);
|
RegionServerServices rss = Mockito.mock(RegionServerServices.class);
|
||||||
final List<Region> regions = new ArrayList<>();
|
final List<HRegion> regions = new ArrayList<>();
|
||||||
Mockito.when(rss.getRegions(TABLENAME)).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions(TABLENAME);
|
||||||
Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
|
Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
|
||||||
// Set max size for this 'table'.
|
// Set max size for this 'table'.
|
||||||
long maxSplitSize = 1024L;
|
long maxSplitSize = 1024L;
|
||||||
|
@ -163,8 +163,8 @@ public class TestRegionSplitPolicy {
|
||||||
conf.setFloat("hbase.busy.policy.blockedRequests", 0.1f);
|
conf.setFloat("hbase.busy.policy.blockedRequests", 0.1f);
|
||||||
|
|
||||||
RegionServerServices rss = Mockito.mock(RegionServerServices.class);
|
RegionServerServices rss = Mockito.mock(RegionServerServices.class);
|
||||||
final List<Region> regions = new ArrayList<>();
|
final List<HRegion> regions = new ArrayList<>();
|
||||||
Mockito.when(rss.getRegions(TABLENAME)).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions(TABLENAME);
|
||||||
Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
|
Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
|
||||||
Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(0L);
|
Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(0L);
|
||||||
Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(0L);
|
Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(0L);
|
||||||
|
|
|
@ -73,7 +73,7 @@ public class TestResettingCounters {
|
||||||
throw new IOException("Failed delete of " + path);
|
throw new IOException("Failed delete of " + path);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
Region region = HBaseTestingUtility.createRegionAndWAL(hri, path, conf, htd);
|
HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, path, conf, htd);
|
||||||
try {
|
try {
|
||||||
Increment odd = new Increment(rows[0]);
|
Increment odd = new Increment(rows[0]);
|
||||||
odd.setDurability(Durability.SKIP_WAL);
|
odd.setDurability(Durability.SKIP_WAL);
|
||||||
|
|
|
@ -321,7 +321,7 @@ public class TestReversibleScanners {
|
||||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()))
|
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()))
|
||||||
.addFamily(new HColumnDescriptor(FAMILYNAME))
|
.addFamily(new HColumnDescriptor(FAMILYNAME))
|
||||||
.addFamily(new HColumnDescriptor(FAMILYNAME2));
|
.addFamily(new HColumnDescriptor(FAMILYNAME2));
|
||||||
Region region = TEST_UTIL.createLocalHRegion(htd, null, null);
|
HRegion region = TEST_UTIL.createLocalHRegion(htd, null, null);
|
||||||
loadDataToRegion(region, FAMILYNAME2);
|
loadDataToRegion(region, FAMILYNAME2);
|
||||||
|
|
||||||
// verify row count with forward scan
|
// verify row count with forward scan
|
||||||
|
@ -641,7 +641,7 @@ public class TestReversibleScanners {
|
||||||
return nextReadableNum;
|
return nextReadableNum;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void loadDataToRegion(Region region, byte[] additionalFamily)
|
private static void loadDataToRegion(HRegion region, byte[] additionalFamily)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
for (int i = 0; i < ROWSIZE; i++) {
|
for (int i = 0; i < ROWSIZE; i++) {
|
||||||
Put put = new Put(ROWS[i]);
|
Put put = new Put(ROWS[i]);
|
||||||
|
|
|
@ -85,7 +85,7 @@ public class TestRowTooBig {
|
||||||
final HRegionInfo hri =
|
final HRegionInfo hri =
|
||||||
new HRegionInfo(htd.getTableName(), HConstants.EMPTY_END_ROW,
|
new HRegionInfo(htd.getTableName(), HConstants.EMPTY_END_ROW,
|
||||||
HConstants.EMPTY_END_ROW);
|
HConstants.EMPTY_END_ROW);
|
||||||
Region region =
|
HRegion region =
|
||||||
HBaseTestingUtility.createRegionAndWAL(hri, rootRegionDir, HTU.getConfiguration(), htd);
|
HBaseTestingUtility.createRegionAndWAL(hri, rootRegionDir, HTU.getConfiguration(), htd);
|
||||||
try {
|
try {
|
||||||
// Add 5 cells to memstore
|
// Add 5 cells to memstore
|
||||||
|
@ -132,7 +132,7 @@ public class TestRowTooBig {
|
||||||
final HRegionInfo hri =
|
final HRegionInfo hri =
|
||||||
new HRegionInfo(htd.getTableName(), HConstants.EMPTY_END_ROW,
|
new HRegionInfo(htd.getTableName(), HConstants.EMPTY_END_ROW,
|
||||||
HConstants.EMPTY_END_ROW);
|
HConstants.EMPTY_END_ROW);
|
||||||
Region region =
|
HRegion region =
|
||||||
HBaseTestingUtility.createRegionAndWAL(hri, rootRegionDir, HTU.getConfiguration(), htd);
|
HBaseTestingUtility.createRegionAndWAL(hri, rootRegionDir, HTU.getConfiguration(), htd);
|
||||||
try {
|
try {
|
||||||
// Add to memstore
|
// Add to memstore
|
||||||
|
|
|
@ -75,7 +75,7 @@ public class TestScanWithBloomError {
|
||||||
private static final String QUALIFIER_PREFIX = "qual";
|
private static final String QUALIFIER_PREFIX = "qual";
|
||||||
private static final byte[] ROW_BYTES = Bytes.toBytes(ROW);
|
private static final byte[] ROW_BYTES = Bytes.toBytes(ROW);
|
||||||
private static NavigableSet<Integer> allColIds = new TreeSet<>();
|
private static NavigableSet<Integer> allColIds = new TreeSet<>();
|
||||||
private Region region;
|
private HRegion region;
|
||||||
private BloomType bloomType;
|
private BloomType bloomType;
|
||||||
private FileSystem fs;
|
private FileSystem fs;
|
||||||
private Configuration conf;
|
private Configuration conf;
|
||||||
|
|
|
@ -107,7 +107,7 @@ public class TestSeekOptimizations {
|
||||||
private static final int[] MAX_VERSIONS_VALUES = new int[] { 1, 2 };
|
private static final int[] MAX_VERSIONS_VALUES = new int[] { 1, 2 };
|
||||||
|
|
||||||
// Instance variables
|
// Instance variables
|
||||||
private Region region;
|
private HRegion region;
|
||||||
private Put put;
|
private Put put;
|
||||||
private Delete del;
|
private Delete del;
|
||||||
private Random rand;
|
private Random rand;
|
||||||
|
|
|
@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult;
|
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.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
@ -120,7 +120,7 @@ public class TestSplitWalDataLoss {
|
||||||
Matchers.<Collection<HStore>> any());
|
Matchers.<Collection<HStore>> any());
|
||||||
// Find region key; don't pick up key for hbase:meta by mistake.
|
// Find region key; don't pick up key for hbase:meta by mistake.
|
||||||
String key = null;
|
String key = null;
|
||||||
for (Map.Entry<String, Region> entry: rs.onlineRegions.entrySet()) {
|
for (Map.Entry<String, HRegion> entry: rs.onlineRegions.entrySet()) {
|
||||||
if (entry.getValue().getRegionInfo().getTable().equals(this.tableName)) {
|
if (entry.getValue().getRegionInfo().getTable().equals(this.tableName)) {
|
||||||
key = entry.getKey();
|
key = entry.getKey();
|
||||||
break;
|
break;
|
||||||
|
|
|
@ -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 {
|
throws IOException {
|
||||||
Configuration conf = TEST_UTIL.getConfiguration();
|
Configuration conf = TEST_UTIL.getConfiguration();
|
||||||
Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
|
Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
|
||||||
|
@ -171,14 +171,14 @@ public class TestStoreFileRefresherChore {
|
||||||
byte[] qf = Bytes.toBytes("cq");
|
byte[] qf = Bytes.toBytes("cq");
|
||||||
|
|
||||||
HRegionServer regionServer = mock(HRegionServer.class);
|
HRegionServer regionServer = mock(HRegionServer.class);
|
||||||
List<Region> regions = new ArrayList<>();
|
List<HRegion> regions = new ArrayList<>();
|
||||||
when(regionServer.getOnlineRegionsLocalContext()).thenReturn(regions);
|
when(regionServer.getOnlineRegionsLocalContext()).thenReturn(regions);
|
||||||
when(regionServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration());
|
when(regionServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration());
|
||||||
|
|
||||||
HTableDescriptor htd = getTableDesc(TableName.valueOf(name.getMethodName()), families);
|
HTableDescriptor htd = getTableDesc(TableName.valueOf(name.getMethodName()), families);
|
||||||
htd.setRegionReplication(2);
|
htd.setRegionReplication(2);
|
||||||
Region primary = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 0);
|
HRegion 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 replica1 = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 1);
|
||||||
regions.add(primary);
|
regions.add(primary);
|
||||||
regions.add(replica1);
|
regions.add(replica1);
|
||||||
|
|
||||||
|
|
|
@ -141,7 +141,7 @@ public class TestWalAndCompactingMemStoreFlush {
|
||||||
String.valueOf(MemoryCompactionPolicy.EAGER));
|
String.valueOf(MemoryCompactionPolicy.EAGER));
|
||||||
|
|
||||||
// Intialize the region
|
// Intialize the region
|
||||||
Region region = initHRegion("testSelectiveFlushWithEager", conf);
|
HRegion region = initHRegion("testSelectiveFlushWithEager", conf);
|
||||||
verifyInMemoryFlushSize(region);
|
verifyInMemoryFlushSize(region);
|
||||||
// Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
|
// Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
|
||||||
for (int i = 1; i <= 1200; i++) {
|
for (int i = 1; i <= 1200; i++) {
|
||||||
|
@ -382,7 +382,7 @@ public class TestWalAndCompactingMemStoreFlush {
|
||||||
String.valueOf(MemoryCompactionPolicy.BASIC));
|
String.valueOf(MemoryCompactionPolicy.BASIC));
|
||||||
|
|
||||||
// Initialize the region
|
// Initialize the region
|
||||||
Region region = initHRegion("testSelectiveFlushWithIndexCompaction", conf);
|
HRegion region = initHRegion("testSelectiveFlushWithIndexCompaction", conf);
|
||||||
verifyInMemoryFlushSize(region);
|
verifyInMemoryFlushSize(region);
|
||||||
/*------------------------------------------------------------------------------*/
|
/*------------------------------------------------------------------------------*/
|
||||||
/* PHASE I - insertions */
|
/* PHASE I - insertions */
|
||||||
|
|
|
@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.when;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
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.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
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.RegionScanner;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
|
@ -54,6 +52,7 @@ import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
@Category({ MediumTests.class, RegionServerTests.class })
|
@Category({ MediumTests.class, RegionServerTests.class })
|
||||||
public class TestCompactedHFilesDischarger {
|
public class TestCompactedHFilesDischarger {
|
||||||
|
@ -76,9 +75,9 @@ public class TestCompactedHFilesDischarger {
|
||||||
Path path = testUtil.getDataTestDir(getClass().getSimpleName());
|
Path path = testUtil.getDataTestDir(getClass().getSimpleName());
|
||||||
region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(), htd);
|
region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(), htd);
|
||||||
rss = mock(RegionServerServices.class);
|
rss = mock(RegionServerServices.class);
|
||||||
List<Region> regions = new ArrayList<>(1);
|
List<HRegion> regions = new ArrayList<>(1);
|
||||||
regions.add(region);
|
regions.add(region);
|
||||||
when(rss.getRegions()).thenReturn(regions);
|
Mockito.doReturn(regions).when(rss).getRegions();
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
|
|
@ -165,7 +165,7 @@ public class TestFlushWithThroughputController {
|
||||||
HRegionServer regionServer = hbtu.getRSForFirstRegionInTable(tableName);
|
HRegionServer regionServer = hbtu.getRSForFirstRegionInTable(tableName);
|
||||||
PressureAwareFlushThroughputController throughputController =
|
PressureAwareFlushThroughputController throughputController =
|
||||||
(PressureAwareFlushThroughputController) regionServer.getFlushThroughputController();
|
(PressureAwareFlushThroughputController) regionServer.getFlushThroughputController();
|
||||||
for (Region region : regionServer.getRegions()) {
|
for (HRegion region : regionServer.getRegions()) {
|
||||||
region.flush(true);
|
region.flush(true);
|
||||||
}
|
}
|
||||||
assertEquals(0.0, regionServer.getFlushPressure(), EPSILON);
|
assertEquals(0.0, regionServer.getFlushPressure(), EPSILON);
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.regionserver.Store;
|
import org.apache.hadoop.hbase.regionserver.Store;
|
||||||
|
@ -195,7 +196,7 @@ public abstract class AbstractTestLogRolling {
|
||||||
assertLogFileSize(log);
|
assertLogFileSize(log);
|
||||||
|
|
||||||
// flush all regions
|
// flush all regions
|
||||||
for (Region r : server.getOnlineRegionsLocalContext()) {
|
for (HRegion r : server.getOnlineRegionsLocalContext()) {
|
||||||
r.flush(true);
|
r.flush(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -248,7 +249,7 @@ public abstract class AbstractTestLogRolling {
|
||||||
table = createTestTable(getName());
|
table = createTestTable(getName());
|
||||||
|
|
||||||
server = TEST_UTIL.getRSForFirstRegionInTable(table.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());
|
final WAL log = server.getWAL(region.getRegionInfo());
|
||||||
Store s = region.getStore(HConstants.CATALOG_FAMILY);
|
Store s = region.getStore(HConstants.CATALOG_FAMILY);
|
||||||
|
|
||||||
|
|
|
@ -350,7 +350,7 @@ public abstract class AbstractTestWALReplay {
|
||||||
Region region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
|
Region region2 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
|
||||||
HBaseTestingUtility.closeRegionAndWAL(region2);
|
HBaseTestingUtility.closeRegionAndWAL(region2);
|
||||||
WAL wal = createWAL(this.conf, hbaseRootDir, logName);
|
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();
|
byte [] family = htd.getFamilies().iterator().next().getName();
|
||||||
Path f = new Path(basedir, "hfile");
|
Path f = new Path(basedir, "hfile");
|
||||||
|
@ -1117,7 +1117,7 @@ public abstract class AbstractTestWALReplay {
|
||||||
private HRegion r;
|
private HRegion r;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void requestFlush(Region region, boolean force) {
|
public void requestFlush(HRegion region, boolean force) {
|
||||||
try {
|
try {
|
||||||
r.flush(force);
|
r.flush(force);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
@ -1126,7 +1126,7 @@ public abstract class AbstractTestWALReplay {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void requestDelayedFlush(Region region, long when, boolean forceFlushAllStores) {
|
public void requestDelayedFlush(HRegion region, long when, boolean forceFlushAllStores) {
|
||||||
// TODO Auto-generated method stub
|
// TODO Auto-generated method stub
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.regionserver.ChunkCreator;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
|
import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
|
||||||
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
|
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.MediumTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -183,7 +182,7 @@ public class TestFSHLog extends AbstractTestFSWAL {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
try {
|
try {
|
||||||
Region.FlushResult flushResult = region.flush(true);
|
HRegion.FlushResult flushResult = region.flush(true);
|
||||||
LOG.info("Flush result:" + flushResult.getResult());
|
LOG.info("Flush result:" + flushResult.getResult());
|
||||||
LOG.info("Flush succeeded:" + flushResult.isFlushSucceeded());
|
LOG.info("Flush succeeded:" + flushResult.isFlushSucceeded());
|
||||||
flushFinished.countDown();
|
flushFinished.countDown();
|
||||||
|
|
|
@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
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.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
|
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
|
||||||
|
@ -345,7 +346,7 @@ public class TestLogRolling extends AbstractTestLogRolling {
|
||||||
assertTrue(loggedRows.contains("row1005"));
|
assertTrue(loggedRows.contains("row1005"));
|
||||||
|
|
||||||
// flush all regions
|
// flush all regions
|
||||||
for (Region r : server.getOnlineRegionsLocalContext()) {
|
for (HRegion r : server.getOnlineRegionsLocalContext()) {
|
||||||
try {
|
try {
|
||||||
r.flush(true);
|
r.flush(true);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.RegionLocator;
|
||||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerImpl;
|
import org.apache.hadoop.hbase.client.RpcRetryingCallerImpl;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
|
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.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||||
|
@ -247,8 +248,8 @@ public class TestRegionReplicaReplicationEndpoint {
|
||||||
|
|
||||||
for (int i=0; i < NB_SERVERS; i++) {
|
for (int i=0; i < NB_SERVERS; i++) {
|
||||||
HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(i);
|
HRegionServer rs = HTU.getMiniHBaseCluster().getRegionServer(i);
|
||||||
List<Region> onlineRegions = rs.getRegions(tableName);
|
List<HRegion> onlineRegions = rs.getRegions(tableName);
|
||||||
for (Region region : onlineRegions) {
|
for (HRegion region : onlineRegions) {
|
||||||
regions[region.getRegionInfo().getReplicaId()] = region;
|
regions[region.getRegionInfo().getReplicaId()] = region;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -307,7 +307,7 @@ public abstract class TestVisibilityLabels {
|
||||||
List<RegionServerThread> regionServerThreads = TEST_UTIL.getHBaseCluster()
|
List<RegionServerThread> regionServerThreads = TEST_UTIL.getHBaseCluster()
|
||||||
.getRegionServerThreads();
|
.getRegionServerThreads();
|
||||||
for (RegionServerThread rsThread : regionServerThreads) {
|
for (RegionServerThread rsThread : regionServerThreads) {
|
||||||
List<Region> onlineRegions = rsThread.getRegionServer().getRegions(
|
List<HRegion> onlineRegions = rsThread.getRegionServer().getRegions(
|
||||||
LABELS_TABLE_NAME);
|
LABELS_TABLE_NAME);
|
||||||
if (onlineRegions.size() > 0) {
|
if (onlineRegions.size() > 0) {
|
||||||
rsThread.getRegionServer().abort("Aborting ");
|
rsThread.getRegionServer().abort("Aborting ");
|
||||||
|
@ -341,7 +341,7 @@ public abstract class TestVisibilityLabels {
|
||||||
for (RegionServerThread rsThread : regionServerThreads) {
|
for (RegionServerThread rsThread : regionServerThreads) {
|
||||||
while (true) {
|
while (true) {
|
||||||
if (!rsThread.getRegionServer().isAborted()) {
|
if (!rsThread.getRegionServer().isAborted()) {
|
||||||
List<Region> onlineRegions = rsThread.getRegionServer().getRegions(
|
List<HRegion> onlineRegions = rsThread.getRegionServer().getRegions(
|
||||||
LABELS_TABLE_NAME);
|
LABELS_TABLE_NAME);
|
||||||
if (onlineRegions.size() > 0) {
|
if (onlineRegions.size() > 0) {
|
||||||
break;
|
break;
|
||||||
|
|
|
@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.io.HFileLink;
|
||||||
import org.apache.hadoop.hbase.master.HMaster;
|
import org.apache.hadoop.hbase.master.HMaster;
|
||||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
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.HRegionFileSystem;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
|
@ -759,8 +760,8 @@ public final class SnapshotTestingUtils {
|
||||||
final TableName tableName)
|
final TableName tableName)
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
HRegionServer rs = util.getRSForFirstRegionInTable(tableName);
|
HRegionServer rs = util.getRSForFirstRegionInTable(tableName);
|
||||||
List<Region> onlineRegions = rs.getRegions(tableName);
|
List<HRegion> onlineRegions = rs.getRegions(tableName);
|
||||||
for (Region region : onlineRegions) {
|
for (HRegion region : onlineRegions) {
|
||||||
region.waitForFlushesAndCompactions();
|
region.waitForFlushesAndCompactions();
|
||||||
}
|
}
|
||||||
// Wait up to 60 seconds for a table to be available.
|
// Wait up to 60 seconds for a table to be available.
|
||||||
|
|
Loading…
Reference in New Issue