HBASE-12972 Region, a supportable public/evolving subset of HRegion

This commit is contained in:
Andrew Purtell 2015-03-30 17:12:04 -07:00
parent 55a5a3be33
commit f1f4b66183
148 changed files with 2325 additions and 1916 deletions

View File

@ -19,11 +19,13 @@
package org.apache.hadoop.hbase;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Implementers are Stoppable.
*/
@InterfaceAudience.Private
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface Stoppable {
/**
* Stop this service.

View File

@ -46,9 +46,9 @@ import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.Bu
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.OperationStatus;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.util.Bytes;
@ -113,7 +113,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
RpcCallback<BulkDeleteResponse> done) {
long totalRowsDeleted = 0L;
long totalVersionsDeleted = 0L;
HRegion region = env.getRegion();
Region region = env.getRegion();
int rowBatchSize = request.getRowBatchSize();
Long timestamp = null;
if (request.hasTimestamp()) {
@ -152,7 +152,8 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
for (List<Cell> deleteRow : deleteRows) {
deleteArr[i++] = createDeleteMutation(deleteRow, deleteType, timestamp);
}
OperationStatus[] opStatus = region.batchMutate(deleteArr);
OperationStatus[] opStatus = region.batchMutate(deleteArr, HConstants.NO_NONCE,
HConstants.NO_NONCE);
for (i = 0; i < opStatus.length; i++) {
if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
break;

View File

@ -26,7 +26,7 @@
org.apache.hadoop.hbase.regionserver.HRegionServer;
org.apache.hadoop.hbase.util.Bytes;
org.apache.hadoop.hbase.HRegionInfo;
org.apache.hadoop.hbase.regionserver.HRegion;
org.apache.hadoop.hbase.regionserver.Region;
org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.HBaseConfiguration;
org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -254,7 +254,7 @@
<%for HRegionInfo r: onlineRegions %>
<%java>
HRegion region = regionServer.getFromOnlineRegions(r.getEncodedName());
Region region = regionServer.getFromOnlineRegions(r.getEncodedName());
MetricsRegionWrapper mWrap = region == null ? null: region.getMetrics().getRegionWrapper();
</%java>

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -28,7 +29,8 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
* Defines the set of shared functions implemented by HBase servers (Masters
* and RegionServers).
*/
@InterfaceAudience.Private
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface Server extends Abortable, Stoppable {
/**
* Gets the configuration object for this server.

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
@ -445,7 +445,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
taskReadyLock.wait(checkInterval);
if (server != null) {
// check to see if we have stale recovering regions in our internal memory state
Map<String, HRegion> recoveringRegions = server.getRecoveringRegions();
Map<String, Region> recoveringRegions = server.getRecoveringRegions();
if (!recoveringRegions.isEmpty()) {
// Make a local copy to prevent ConcurrentModificationException when other threads
// modify recoveringRegions

View File

@ -115,7 +115,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
}
}
log.info("Maximum from this region is "
+ env.getRegion().getRegionNameAsString() + ": " + max);
+ env.getRegion().getRegionInfo().getRegionNameAsString() + ": " + max);
done.run(response);
}
@ -168,7 +168,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
}
}
log.info("Minimum from this region is "
+ env.getRegion().getRegionNameAsString() + ": " + min);
+ env.getRegion().getRegionInfo().getRegionNameAsString() + ": " + min);
done.run(response);
}
@ -223,7 +223,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
}
}
log.debug("Sum from this region is "
+ env.getRegion().getRegionNameAsString() + ": " + sum);
+ env.getRegion().getRegionInfo().getRegionNameAsString() + ": " + sum);
done.run(response);
}
@ -274,7 +274,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
}
}
log.info("Row counter from this region is "
+ env.getRegion().getRegionNameAsString() + ": " + counter);
+ env.getRegion().getRegionInfo().getRegionNameAsString() + ": " + counter);
done.run(response);
}

View File

@ -42,11 +42,11 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.regionserver.DeleteTracker;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.Store;
@ -152,7 +152,7 @@ public abstract class BaseRegionObserver implements RegionObserver {
}
@Override
public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, HRegion l, HRegion r)
public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, Region l, Region r)
throws IOException {
}

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
/**
@ -48,28 +48,28 @@ public class BaseRegionServerObserver implements RegionServerObserver {
public void stop(CoprocessorEnvironment env) throws IOException { }
@Override
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, HRegion regionA,
HRegion regionB) throws IOException { }
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, Region regionA,
Region regionB) throws IOException { }
@Override
public void postMerge(ObserverContext<RegionServerCoprocessorEnvironment> c, HRegion regionA,
HRegion regionB, HRegion mergedRegion) throws IOException { }
public void postMerge(ObserverContext<RegionServerCoprocessorEnvironment> c, Region regionA,
Region regionB, Region mergedRegion) throws IOException { }
@Override
public void preMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB, List<Mutation> metaEntries) throws IOException { }
Region regionA, Region regionB, List<Mutation> metaEntries) throws IOException { }
@Override
public void postMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB, HRegion mergedRegion) throws IOException { }
Region regionA, Region regionB, Region mergedRegion) throws IOException { }
@Override
public void preRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB) throws IOException { }
Region regionA, Region regionB) throws IOException { }
@Override
public void postRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB) throws IOException { }
Region regionA, Region regionB) throws IOException { }
@Override
public void preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse;
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.RowProcessorService;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RowProcessor;
import com.google.protobuf.ByteString;
@ -42,7 +42,7 @@ import com.google.protobuf.Service;
/**
* This class demonstrates how to implement atomic read-modify-writes
* using {@link HRegion#processRowsWithLocks} and Coprocessor endpoints.
* using {@link Region#processRowsWithLocks} and Coprocessor endpoints.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
@ -50,7 +50,7 @@ public abstract class BaseRowProcessorEndpoint<S extends Message, T extends Mess
extends RowProcessorService implements CoprocessorService, Coprocessor {
private RegionCoprocessorEnvironment env;
/**
* Pass a processor to HRegion to process multiple rows atomically.
* Pass a processor to region to process multiple rows atomically.
*
* The RowProcessor implementations should be the inner classes of your
* RowProcessorEndpoint. This way the RowProcessor can be class-loaded with
@ -68,7 +68,7 @@ extends RowProcessorService implements CoprocessorService, Coprocessor {
ProcessResponse resultProto = null;
try {
RowProcessor<S,T> processor = constructRowProcessorFromRequest(request);
HRegion region = env.getRegion();
Region region = env.getRegion();
long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;
long nonce = request.hasNonce() ? request.getNonce() : HConstants.NO_NONCE;
region.processRowsWithLocks(processor, nonceGroup, nonce);

View File

@ -26,14 +26,14 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface RegionCoprocessorEnvironment extends CoprocessorEnvironment {
/** @return the region associated with this coprocessor */
HRegion getRegion();
Region getRegion();
/** @return region information for the region this coprocessor is running on */
HRegionInfo getRegionInfo();

View File

@ -42,8 +42,8 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.regionserver.DeleteTracker;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
@ -368,8 +368,8 @@ public interface RegionObserver extends Coprocessor {
* @deprecated Use postCompleteSplit() instead
*/
@Deprecated
void postSplit(final ObserverContext<RegionCoprocessorEnvironment> c, final HRegion l,
final HRegion r) throws IOException;
void postSplit(final ObserverContext<RegionCoprocessorEnvironment> c, final Region l,
final Region r) throws IOException;
/**
* This will be called before PONR step as part of split transaction. Calling
@ -617,7 +617,7 @@ public interface RegionObserver extends Coprocessor {
* called after acquiring the locks on the mutating rows and after applying the proper timestamp
* for each Mutation at the server. The batch may contain Put/Delete. By setting OperationStatus
* of Mutations ({@link MiniBatchOperationInProgress#setOperationStatus(int, OperationStatus)}),
* {@link RegionObserver} can make HRegion to skip these Mutations.
* {@link RegionObserver} can make Region to skip these Mutations.
* @param c the environment provided by the region server
* @param miniBatchOp batch of Mutations getting applied to region.
* @throws IOException if an error occurred on the coprocessor
@ -637,7 +637,7 @@ public interface RegionObserver extends Coprocessor {
/**
* This will be called for region operations where read lock is acquired in
* {@link HRegion#startRegionOperation()}.
* {@link Region#startRegionOperation()}.
* @param ctx
* @param operation The operation is about to be taken on the region
* @throws IOException
@ -646,7 +646,7 @@ public interface RegionObserver extends Coprocessor {
Operation operation) throws IOException;
/**
* Called after releasing read lock in {@link HRegion#closeRegionOperation(Operation)}.
* Called after releasing read lock in {@link Region#closeRegionOperation()}.
* @param ctx
* @param operation
* @throws IOException

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
public interface RegionServerObserver extends Coprocessor {
@ -50,7 +50,7 @@ public interface RegionServerObserver extends Coprocessor {
* @throws IOException
*/
void preMerge(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
final HRegion regionA, final HRegion regionB) throws IOException;
final Region regionA, final Region regionB) throws IOException;
/**
* called after the regions merge.
@ -61,7 +61,7 @@ public interface RegionServerObserver extends Coprocessor {
* @throws IOException
*/
void postMerge(final ObserverContext<RegionServerCoprocessorEnvironment> c,
final HRegion regionA, final HRegion regionB, final HRegion mergedRegion) throws IOException;
final Region regionA, final Region regionB, final Region mergedRegion) throws IOException;
/**
* This will be called before PONR step as part of regions merge transaction. Calling
@ -74,7 +74,7 @@ public interface RegionServerObserver extends Coprocessor {
* @throws IOException
*/
void preMergeCommit(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
final HRegion regionA, final HRegion regionB,
final Region regionA, final Region regionB,
@MetaMutationAnnotation List<Mutation> metaEntries) throws IOException;
/**
@ -86,7 +86,7 @@ public interface RegionServerObserver extends Coprocessor {
* @throws IOException
*/
void postMergeCommit(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
final HRegion regionA, final HRegion regionB, final HRegion mergedRegion) throws IOException;
final Region regionA, final Region regionB, final Region mergedRegion) throws IOException;
/**
* This will be called before the roll back of the regions merge.
@ -96,7 +96,7 @@ public interface RegionServerObserver extends Coprocessor {
* @throws IOException
*/
void preRollBackMerge(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
final HRegion regionA, final HRegion regionB) throws IOException;
final Region regionA, final Region regionB) throws IOException;
/**
* This will be called after the roll back of the regions merge.
@ -106,7 +106,7 @@ public interface RegionServerObserver extends Coprocessor {
* @throws IOException
*/
void postRollBackMerge(final ObserverContext<RegionServerCoprocessorEnvironment> ctx,
final HRegion regionA, final HRegion regionB) throws IOException;
final Region regionA, final Region regionB) throws IOException;
/**
* This will be called before executing user request to roll a region server WAL.

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.MultiHConnection;
@ -56,7 +56,7 @@ public class RegionStateStore {
/** The delimiter for meta columns for replicaIds > 0 */
protected static final char META_REPLICA_ID_DELIMITER = '_';
private volatile HRegion metaRegion;
private volatile Region metaRegion;
private volatile boolean initialized;
private MultiHConnection multiHConnection;
private final Server server;

View File

@ -28,7 +28,8 @@ import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.procedure.ProcedureMember;
import org.apache.hadoop.hbase.procedure.Subprocedure;
import org.apache.hadoop.hbase.procedure.flush.RegionServerFlushTableProcedureManager.FlushTableSubprocedurePool;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
/**
* This flush region implementation uses the distributed procedure framework to flush
@ -40,12 +41,12 @@ public class FlushTableSubprocedure extends Subprocedure {
private static final Log LOG = LogFactory.getLog(FlushTableSubprocedure.class);
private final String table;
private final List<HRegion> regions;
private final List<Region> regions;
private final FlushTableSubprocedurePool taskManager;
public FlushTableSubprocedure(ProcedureMember member,
ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
List<HRegion> regions, String table,
List<Region> regions, String table,
FlushTableSubprocedurePool taskManager) {
super(member, table, errorListener, wakeFrequency, timeout);
this.table = table;
@ -54,8 +55,8 @@ public class FlushTableSubprocedure extends Subprocedure {
}
private static class RegionFlushTask implements Callable<Void> {
HRegion region;
RegionFlushTask(HRegion region) {
Region region;
RegionFlushTask(Region region) {
this.region = region;
}
@ -65,7 +66,7 @@ public class FlushTableSubprocedure extends Subprocedure {
region.startRegionOperation();
try {
LOG.debug("Flush region " + region.toString() + " started...");
region.flushcache();
region.flush(true);
} finally {
LOG.debug("Closing region operation on " + region);
region.closeRegionOperation();
@ -89,7 +90,7 @@ public class FlushTableSubprocedure extends Subprocedure {
}
// Add all hfiles already existing in region.
for (HRegion region : regions) {
for (Region region : regions) {
// submit one task per region for parallelize by region.
taskManager.submitTask(new RegionFlushTask(region));
monitor.rethrowException();

View File

@ -44,8 +44,8 @@ import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
import org.apache.hadoop.hbase.procedure.Subprocedure;
import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -137,7 +137,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur
}
// check to see if this server is hosting any regions for the table
List<HRegion> involvedRegions;
List<Region> involvedRegions;
try {
involvedRegions = getRegionsToFlush(table);
} catch (IOException e1) {
@ -172,7 +172,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur
* @return the list of online regions. Empty list is returned if no regions.
* @throws IOException
*/
private List<HRegion> getRegionsToFlush(String table) throws IOException {
private List<Region> getRegionsToFlush(String table) throws IOException {
return rss.getOnlineRegions(TableName.valueOf(table));
}

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.RpcScheduler;
import org.apache.hadoop.hbase.ipc.RequestContext;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.security.UserGroupInformation;
@ -129,7 +129,7 @@ public class RegionServerQuotaManager {
* @return the OperationQuota
* @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
*/
public OperationQuota checkQuota(final HRegion region,
public OperationQuota checkQuota(final Region region,
final OperationQuota.OperationType type) throws IOException, ThrottlingException {
switch (type) {
case SCAN: return checkQuota(region, 0, 0, 1);
@ -148,7 +148,7 @@ public class RegionServerQuotaManager {
* @return the OperationQuota
* @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
*/
public OperationQuota checkQuota(final HRegion region,
public OperationQuota checkQuota(final Region region,
final List<ClientProtos.Action> actions) throws IOException, ThrottlingException {
int numWrites = 0;
int numReads = 0;
@ -173,7 +173,7 @@ public class RegionServerQuotaManager {
* @return the OperationQuota
* @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
*/
private OperationQuota checkQuota(final HRegion region,
private OperationQuota checkQuota(final Region region,
final int numWrites, final int numReads, final int numScans)
throws IOException, ThrottlingException {
UserGroupInformation ugi;

View File

@ -176,10 +176,11 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
if (hasRegion != null && (Boolean)hasRegion.invoke(param, (Object[])null)) {
Method getRegion = methodMap.get("getRegion").get(rpcArgClass);
regionSpecifier = (RegionSpecifier)getRegion.invoke(param, (Object[])null);
HRegion region = rpcServices.getRegion(regionSpecifier);
Region region = rpcServices.getRegion(regionSpecifier);
if (region.getRegionInfo().isSystemTable()) {
if (LOG.isTraceEnabled()) {
LOG.trace("High priority because region=" + region.getRegionNameAsString());
LOG.trace("High priority because region=" +
region.getRegionInfo().getRegionNameAsString());
}
return HConstants.SYSTEMTABLE_QOS;
}

View File

@ -218,8 +218,8 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
return queueLists.toString();
}
public synchronized void requestRegionsMerge(final HRegion a,
final HRegion b, final boolean forcible) {
public synchronized void requestRegionsMerge(final Region a,
final Region b, final boolean forcible) {
try {
mergePool.execute(new RegionMergeRequest(a, b, this.server, forcible));
if (LOG.isDebugEnabled()) {
@ -232,10 +232,10 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
}
}
public synchronized boolean requestSplit(final HRegion r) {
public synchronized boolean requestSplit(final Region r) {
// don't split regions that are blocking
if (shouldSplitRegion() && r.getCompactPriority() >= Store.PRIORITY_USER) {
byte[] midKey = r.checkSplit();
if (shouldSplitRegion() && ((HRegion)r).getCompactPriority() >= Store.PRIORITY_USER) {
byte[] midKey = ((HRegion)r).checkSplit();
if (midKey != null) {
requestSplit(r, midKey);
return true;
@ -244,12 +244,12 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
return false;
}
public synchronized void requestSplit(final HRegion r, byte[] midKey) {
public synchronized void requestSplit(final Region r, byte[] midKey) {
if (midKey == null) {
LOG.debug("Region " + r.getRegionNameAsString() +
LOG.debug("Region " + r.getRegionInfo().getRegionNameAsString() +
" not splittable because midkey=null");
if (r.shouldForceSplit()) {
r.clearSplit();
if (((HRegion)r).shouldForceSplit()) {
((HRegion)r).clearSplit();
}
return;
}
@ -264,36 +264,36 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
}
@Override
public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why)
public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why)
throws IOException {
return requestCompaction(r, why, null);
}
@Override
public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why,
List<Pair<CompactionRequest, Store>> requests) throws IOException {
return requestCompaction(r, why, Store.NO_PRIORITY, requests);
}
@Override
public synchronized CompactionRequest requestCompaction(final HRegion r, final Store s,
public synchronized CompactionRequest requestCompaction(final Region r, final Store s,
final String why, CompactionRequest request) throws IOException {
return requestCompaction(r, s, why, Store.NO_PRIORITY, request);
}
@Override
public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why,
int p, List<Pair<CompactionRequest, Store>> requests) throws IOException {
return requestCompactionInternal(r, why, p, requests, true);
}
private List<CompactionRequest> requestCompactionInternal(final HRegion r, final String why,
private List<CompactionRequest> requestCompactionInternal(final Region r, final String why,
int p, List<Pair<CompactionRequest, Store>> requests, boolean selectNow) throws IOException {
// not a special compaction request, so make our own list
List<CompactionRequest> ret = null;
if (requests == null) {
ret = selectNow ? new ArrayList<CompactionRequest>(r.getStores().size()) : null;
for (Store s : r.getStores().values()) {
for (Store s : r.getStores()) {
CompactionRequest cr = requestCompactionInternal(r, s, why, p, null, selectNow);
if (selectNow) ret.add(cr);
}
@ -307,30 +307,30 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
return ret;
}
public CompactionRequest requestCompaction(final HRegion r, final Store s,
public CompactionRequest requestCompaction(final Region r, final Store s,
final String why, int priority, CompactionRequest request) throws IOException {
return requestCompactionInternal(r, s, why, priority, request, true);
}
public synchronized void requestSystemCompaction(
final HRegion r, final String why) throws IOException {
final Region r, final String why) throws IOException {
requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false);
}
public void requestSystemCompaction(
final HRegion r, final Store s, final String why) throws IOException {
final Region r, final Store s, final String why) throws IOException {
requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false);
}
/**
* @param r HRegion store belongs to
* @param r region store belongs to
* @param s Store to request compaction on
* @param why Why compaction requested -- used in debug messages
* @param priority override the default priority (NO_PRIORITY == decide)
* @param request custom compaction request. Can be <tt>null</tt> in which case a simple
* compaction will be used.
*/
private synchronized CompactionRequest requestCompactionInternal(final HRegion r, final Store s,
private synchronized CompactionRequest requestCompactionInternal(final Region r, final Store s,
final String why, int priority, CompactionRequest request, boolean selectNow)
throws IOException {
if (this.server.isStopped()
@ -358,12 +358,12 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
return selectNow ? compaction.getRequest() : null;
}
private CompactionContext selectCompaction(final HRegion r, final Store s,
private CompactionContext selectCompaction(final Region r, final Store s,
int priority, CompactionRequest request) throws IOException {
CompactionContext compaction = s.requestCompaction(priority, request);
if (compaction == null) {
if(LOG.isDebugEnabled()) {
LOG.debug("Not compacting " + r.getRegionNameAsString() +
LOG.debug("Not compacting " + r.getRegionInfo().getRegionNameAsString() +
" because compaction request was cancelled");
}
return null;
@ -454,11 +454,11 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
private int queuedPriority;
private ThreadPoolExecutor parent;
public CompactionRunner(Store store, HRegion region,
public CompactionRunner(Store store, Region region,
CompactionContext compaction, ThreadPoolExecutor parent) {
super();
this.store = store;
this.region = region;
this.region = (HRegion)region;
this.compaction = compaction;
this.queuedPriority = (this.compaction == null)
? store.getCompactPriority() : compaction.getRequest().getPriority();

View File

@ -34,7 +34,7 @@ public interface CompactionRequestor {
* compactions were started
* @throws IOException
*/
List<CompactionRequest> requestCompaction(final HRegion r, final String why)
List<CompactionRequest> requestCompaction(final Region r, final String why)
throws IOException;
/**
@ -48,7 +48,7 @@ public interface CompactionRequestor {
* @throws IOException
*/
List<CompactionRequest> requestCompaction(
final HRegion r, final String why, List<Pair<CompactionRequest, Store>> requests
final Region r, final String why, List<Pair<CompactionRequest, Store>> requests
)
throws IOException;
@ -56,13 +56,13 @@ public interface CompactionRequestor {
* @param r Region to compact
* @param s Store within region to compact
* @param why Why compaction was requested -- used in debug messages
* @param request custom compaction request for the {@link HRegion} and {@link Store}. Custom
* @param request custom compaction request for the {@link Region} and {@link Store}. Custom
* request must be <tt>null</tt> or be constructed with matching region and store.
* @return The created {@link CompactionRequest} or <tt>null</tt> if no compaction was started.
* @throws IOException
*/
CompactionRequest requestCompaction(
final HRegion r, final Store s, final String why, CompactionRequest request
final Region r, final Store s, final String why, CompactionRequest request
) throws IOException;
/**
@ -77,7 +77,7 @@ public interface CompactionRequestor {
* @throws IOException
*/
List<CompactionRequest> requestCompaction(
final HRegion r, final String why, int pri, List<Pair<CompactionRequest, Store>> requests
final Region r, final String why, int pri, List<Pair<CompactionRequest, Store>> requests
) throws IOException;
/**
@ -85,12 +85,12 @@ public interface CompactionRequestor {
* @param s Store within region to compact
* @param why Why compaction was requested -- used in debug messages
* @param pri Priority of this compaction. minHeap. <=0 is critical
* @param request custom compaction request to run. {@link Store} and {@link HRegion} for the
* @param request custom compaction request to run. {@link Store} and {@link Region} for the
* request must match the region and store specified here.
* @return The created {@link CompactionRequest} or <tt>null</tt> if no compaction was started
* @throws IOException
*/
CompactionRequest requestCompaction(
final HRegion r, final Store s, final String why, int pri, CompactionRequest request
final Region r, final Store s, final String why, int pri, CompactionRequest request
) throws IOException;
}

View File

@ -55,7 +55,7 @@ public class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy {
boolean force = region.shouldForceSplit();
boolean foundABigStore = false;
for (Store store : region.getStores().values()) {
for (Store store : region.getStores()) {
// If any of the stores are unable to split (eg they contain reference files)
// then don't split
if ((!store.canSplit())) {

View File

@ -21,15 +21,18 @@ package org.apache.hadoop.hbase.regionserver;
import java.net.InetSocketAddress;
import java.util.List;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName;
/**
* Abstraction that allows different modules in RegionServer to update/get
* the favored nodes information for regions.
*/
@InterfaceAudience.Private
interface FavoredNodesForRegion {
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface FavoredNodesForRegion {
/**
* Used to update the favored nodes mapping when required.
* @param encodedRegionName

View File

@ -32,5 +32,5 @@ public interface FlushRequestListener {
* @param type The type of flush. (ie. Whether a normal flush or flush because of global heap preassure)
* @param region The region for which flush is requested
*/
void flushRequested(FlushType type, HRegion region);
void flushRequested(FlushType type, Region region);
}

View File

@ -29,21 +29,21 @@ public interface FlushRequester {
/**
* Tell the listener the cache needs to be flushed.
*
* @param region the HRegion requesting the cache flush
* @param region the Region requesting the cache flush
* @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log
* rolling.
*/
void requestFlush(HRegion region, boolean forceFlushAllStores);
void requestFlush(Region region, boolean forceFlushAllStores);
/**
* Tell the listener the cache needs to be flushed after a delay
*
* @param region the HRegion requesting the cache flush
* @param region the Region requesting the cache flush
* @param delay after how much time should the flush happen
* @param forceFlushAllStores whether we want to flush all stores. e.g., when request from log
* rolling.
*/
void requestDelayedFlush(HRegion region, long delay, boolean forceFlushAllStores);
void requestDelayedFlush(Region region, long delay, boolean forceFlushAllStores);
/**
* Register a FlushRequestListener

View File

@ -253,8 +253,7 @@ public class HRegionServer extends HasThread implements
* Map of regions currently being served by this region server. Key is the
* encoded region name. All access should be synchronized.
*/
protected final Map<String, HRegion> onlineRegions =
new ConcurrentHashMap<String, HRegion>();
protected final Map<String, Region> onlineRegions = new ConcurrentHashMap<String, Region>();
/**
* Map of encoded region names to the DataNode locations they should be hosted on
@ -272,8 +271,8 @@ public class HRegionServer extends HasThread implements
* Set of regions currently being in recovering state which means it can accept writes(edits from
* previous failed region server) but not reads. A recovering region is also an online region.
*/
protected final Map<String, HRegion> recoveringRegions = Collections
.synchronizedMap(new HashMap<String, HRegion>());
protected final Map<String, Region> recoveringRegions = Collections
.synchronizedMap(new HashMap<String, Region>());
// Leases
protected Leases leases;
@ -1073,7 +1072,7 @@ public class HRegionServer extends HasThread implements
private boolean areAllUserRegionsOffline() {
if (getNumberOfOnlineRegions() > 2) return false;
boolean allUserRegionsOffline = true;
for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
for (Map.Entry<String, Region> e: this.onlineRegions.entrySet()) {
if (!e.getValue().getRegionInfo().isMetaTable()) {
allUserRegionsOffline = false;
break;
@ -1087,7 +1086,7 @@ public class HRegionServer extends HasThread implements
*/
private long getWriteRequestCount() {
int writeCount = 0;
for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
for (Map.Entry<String, Region> e: this.onlineRegions.entrySet()) {
writeCount += e.getValue().getWriteRequestsCount();
}
return writeCount;
@ -1133,10 +1132,9 @@ public class HRegionServer extends HasThread implements
// Instead they should be stored in an HBase table so that external visibility into HBase is
// improved; Additionally the load balancer will be able to take advantage of a more complete
// history.
MetricsRegionServerWrapper regionServerWrapper = this.metricsRegionServer.getRegionServerWrapper();
Collection<HRegion> regions = getOnlineRegionsLocalContext();
MemoryUsage memory =
ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
MetricsRegionServerWrapper regionServerWrapper = metricsRegionServer.getRegionServerWrapper();
Collection<Region> regions = getOnlineRegionsLocalContext();
MemoryUsage memory = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
ClusterStatusProtos.ServerLoad.Builder serverLoad =
ClusterStatusProtos.ServerLoad.newBuilder();
@ -1151,7 +1149,7 @@ public class HRegionServer extends HasThread implements
}
RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder();
RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
for (HRegion region : regions) {
for (Region region : regions) {
serverLoad.addRegionLoads(createRegionLoad(region, regionLoadBldr, regionSpecifier));
for (String coprocessor :
getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors()) {
@ -1186,7 +1184,7 @@ public class HRegionServer extends HasThread implements
String getOnlineRegionsAsPrintableString() {
StringBuilder sb = new StringBuilder();
for (HRegion r: this.onlineRegions.values()) {
for (Region r: this.onlineRegions.values()) {
if (sb.length() > 0) sb.append(", ");
sb.append(r.getRegionInfo().getEncodedName());
}
@ -1222,7 +1220,7 @@ public class HRegionServer extends HasThread implements
// Ensure all user regions have been sent a close. Use this to
// protect against the case where an open comes in after we start the
// iterator of onlineRegions to close all user regions.
for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
for (Map.Entry<String, Region> e : this.onlineRegions.entrySet()) {
HRegionInfo hri = e.getValue().getRegionInfo();
if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
&& !closedRegions.contains(hri.getEncodedName())) {
@ -1378,44 +1376,37 @@ public class HRegionServer extends HasThread implements
*
* @throws IOException
*/
private RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr,
private RegionLoad createRegionLoad(final Region r, RegionLoad.Builder regionLoadBldr,
RegionSpecifier.Builder regionSpecifier) throws IOException {
byte[] name = r.getRegionName();
byte[] name = r.getRegionInfo().getRegionName();
int stores = 0;
int storefiles = 0;
int storeUncompressedSizeMB = 0;
int storefileSizeMB = 0;
int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
int memstoreSizeMB = (int) (r.getMemstoreSize() / 1024 / 1024);
int storefileIndexSizeMB = 0;
int rootIndexSizeKB = 0;
int totalStaticIndexSizeKB = 0;
int totalStaticBloomSizeKB = 0;
long totalCompactingKVs = 0;
long currentCompactedKVs = 0;
synchronized (r.stores) {
stores += r.stores.size();
for (Store store : r.stores.values()) {
storefiles += store.getStorefilesCount();
storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed()
/ 1024 / 1024);
storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
CompactionProgress progress = store.getCompactionProgress();
if (progress != null) {
totalCompactingKVs += progress.totalCompactingKVs;
currentCompactedKVs += progress.currentCompactedKVs;
}
rootIndexSizeKB +=
(int) (store.getStorefilesIndexSize() / 1024);
totalStaticIndexSizeKB +=
(int) (store.getTotalStaticIndexSize() / 1024);
totalStaticBloomSizeKB +=
(int) (store.getTotalStaticBloomSize() / 1024);
List<Store> storeList = r.getStores();
stores += storeList.size();
for (Store store : storeList) {
storefiles += store.getStorefilesCount();
storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed() / 1024 / 1024);
storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
CompactionProgress progress = store.getCompactionProgress();
if (progress != null) {
totalCompactingKVs += progress.totalCompactingKVs;
currentCompactedKVs += progress.currentCompactedKVs;
}
rootIndexSizeKB += (int) (store.getStorefilesIndexSize() / 1024);
totalStaticIndexSizeKB += (int) (store.getTotalStaticIndexSize() / 1024);
totalStaticBloomSizeKB += (int) (store.getTotalStaticBloomSize() / 1024);
}
float dataLocality =
r.getHDFSBlocksDistribution().getBlockLocalityIndex(serverName.getHostname());
if (regionLoadBldr == null) {
@ -1426,8 +1417,7 @@ public class HRegionServer extends HasThread implements
}
regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
regionSpecifier.setValue(ByteStringer.wrap(name));
r.setCompleteSequenceId(regionLoadBldr)
.setRegionSpecifier(regionSpecifier.build())
regionLoadBldr.setRegionSpecifier(regionSpecifier.build())
.setStores(stores)
.setStorefiles(storefiles)
.setStoreUncompressedSizeMB(storeUncompressedSizeMB)
@ -1437,12 +1427,13 @@ public class HRegionServer extends HasThread implements
.setRootIndexSizeKB(rootIndexSizeKB)
.setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
.setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
.setReadRequestsCount(r.readRequestsCount.get())
.setWriteRequestsCount(r.writeRequestsCount.get())
.setReadRequestsCount(r.getReadRequestsCount())
.setWriteRequestsCount(r.getWriteRequestsCount())
.setTotalCompactingKVs(totalCompactingKVs)
.setCurrentCompactedKVs(currentCompactedKVs)
.setDataLocality(dataLocality)
.setLastMajorCompactionTs(r.getOldestHfileTs(true));
((HRegion)r).setCompleteSequenceId(regionLoadBldr);
return regionLoadBldr.build();
}
@ -1452,8 +1443,7 @@ public class HRegionServer extends HasThread implements
* @return An instance of RegionLoad.
*/
public RegionLoad createRegionLoad(final String encodedRegionName) throws IOException {
HRegion r = null;
r = this.onlineRegions.get(encodedRegionName);
Region r = onlineRegions.get(encodedRegionName);
return r != null ? createRegionLoad(r, null, null) : null;
}
@ -1482,10 +1472,10 @@ public class HRegionServer extends HasThread implements
@Override
protected void chore() {
for (HRegion r : this.instance.onlineRegions.values()) {
for (Region r : this.instance.onlineRegions.values()) {
if (r == null)
continue;
for (Store s : r.getStores().values()) {
for (Store s : r.getStores()) {
try {
long multiplier = s.getCompactionCheckMultiplier();
assert multiplier > 0;
@ -1496,7 +1486,7 @@ public class HRegionServer extends HasThread implements
+ " requests compaction");
} else if (s.isMajorCompaction()) {
if (majorCompactPriority == DEFAULT_PRIORITY
|| majorCompactPriority > r.getCompactPriority()) {
|| majorCompactPriority > ((HRegion)r).getCompactPriority()) {
this.instance.compactSplitThread.requestCompaction(r, s, getName()
+ " requests major compaction; use default priority", null);
} else {
@ -1525,15 +1515,15 @@ public class HRegionServer extends HasThread implements
@Override
protected void chore() {
for (HRegion r : this.server.onlineRegions.values()) {
for (Region r : this.server.onlineRegions.values()) {
if (r == null)
continue;
if (r.shouldFlush()) {
if (((HRegion)r).shouldFlush()) {
FlushRequester requester = server.getFlushRequester();
if (requester != null) {
long randomDelay = RandomUtils.nextInt(RANGE_OF_DELAY) + MIN_DELAY_TIME;
LOG.info(getName() + " requesting flush for region " + r.getRegionNameAsString() +
" after a delay of " + randomDelay);
LOG.info(getName() + " requesting flush for region " +
r.getRegionInfo().getRegionNameAsString() + " after a delay of " + randomDelay);
//Throttle the flushes by putting a delay. If we don't throttle, and there
//is a balanced write-load on the regions in a table, we might end up
//overwhelming the filesystem with too many flushes at once.
@ -1839,12 +1829,12 @@ public class HRegionServer extends HasThread implements
}
@Override
public void postOpenDeployTasks(final HRegion r)
throws KeeperException, IOException {
public void postOpenDeployTasks(final Region r) throws KeeperException, IOException {
Preconditions.checkArgument(r instanceof HRegion, "r must be an HRegion");
rpcServices.checkOpen();
LOG.info("Post open deploy tasks for " + r.getRegionNameAsString());
LOG.info("Post open deploy tasks for " + r.getRegionInfo().getRegionNameAsString());
// Do checks to see if we need to compact (references or too many files)
for (Store s : r.getStores().values()) {
for (Store s : r.getStores()) {
if (s.hasReferences() || s.needsCompaction()) {
this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region");
}
@ -1852,7 +1842,8 @@ public class HRegionServer extends HasThread implements
long openSeqNum = r.getOpenSeqNum();
if (openSeqNum == HConstants.NO_SEQNUM) {
// If we opened a region, we should have read some sequence number from it.
LOG.error("No sequence number found when opening " + r.getRegionNameAsString());
LOG.error("No sequence number found when opening " +
r.getRegionInfo().getRegionNameAsString());
openSeqNum = 0;
}
@ -1863,12 +1854,12 @@ public class HRegionServer extends HasThread implements
if (!reportRegionStateTransition(
TransitionCode.OPENED, openSeqNum, r.getRegionInfo())) {
throw new IOException("Failed to report opened region to master: "
+ r.getRegionNameAsString());
+ r.getRegionInfo().getRegionNameAsString());
}
triggerFlushInPrimaryRegion(r);
triggerFlushInPrimaryRegion((HRegion)r);
LOG.debug("Finished post open deploy task for " + r.getRegionNameAsString());
LOG.debug("Finished post open deploy task for " + r.getRegionInfo().getRegionNameAsString());
}
@Override
@ -2272,10 +2263,10 @@ public class HRegionServer extends HasThread implements
* @param abort Whether we're running an abort.
*/
void closeMetaTableRegions(final boolean abort) {
HRegion meta = null;
Region meta = null;
this.lock.writeLock().lock();
try {
for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
for (Map.Entry<String, Region> e: onlineRegions.entrySet()) {
HRegionInfo hri = e.getValue().getRegionInfo();
if (hri.isMetaRegion()) {
meta = e.getValue();
@ -2297,8 +2288,8 @@ public class HRegionServer extends HasThread implements
void closeUserRegions(final boolean abort) {
this.lock.writeLock().lock();
try {
for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
HRegion r = e.getValue();
for (Map.Entry<String, Region> e: this.onlineRegions.entrySet()) {
Region r = e.getValue();
if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
// Don't update zk with this close transition; pass false.
closeRegionIgnoreErrors(r.getRegionInfo(), abort);
@ -2328,7 +2319,7 @@ public class HRegionServer extends HasThread implements
}
@Override
public Map<String, HRegion> getRecoveringRegions() {
public Map<String, Region> getRecoveringRegions() {
return this.recoveringRegions;
}
@ -2359,13 +2350,13 @@ public class HRegionServer extends HasThread implements
* This method will only work if HRegionServer is in the same JVM as client;
* HRegion cannot be serialized to cross an rpc.
*/
public Collection<HRegion> getOnlineRegionsLocalContext() {
Collection<HRegion> regions = this.onlineRegions.values();
public Collection<Region> getOnlineRegionsLocalContext() {
Collection<Region> regions = this.onlineRegions.values();
return Collections.unmodifiableCollection(regions);
}
@Override
public void addToOnlineRegions(HRegion region) {
public void addToOnlineRegions(Region region) {
this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
configurationManager.registerObserver(region);
}
@ -2375,9 +2366,9 @@ public class HRegionServer extends HasThread implements
* biggest. If two regions are the same size, then the last one found wins; i.e. this method
* may NOT return all regions.
*/
SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
SortedMap<Long, Region> getCopyOfOnlineRegionsSortedBySize() {
// we'll sort the regions in reverse
SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
SortedMap<Long, Region> sortedRegions = new TreeMap<Long, Region>(
new Comparator<Long>() {
@Override
public int compare(Long a, Long b) {
@ -2385,8 +2376,8 @@ public class HRegionServer extends HasThread implements
}
});
// Copy over all regions. Regions are sorted by size with biggest first.
for (HRegion region : this.onlineRegions.values()) {
sortedRegions.put(region.memstoreSize.get(), region);
for (Region region : this.onlineRegions.values()) {
sortedRegions.put(region.getMemstoreSize(), region);
}
return sortedRegions;
}
@ -2412,7 +2403,7 @@ public class HRegionServer extends HasThread implements
*/
protected HRegionInfo[] getMostLoadedRegions() {
ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
for (HRegion r : onlineRegions.values()) {
for (Region r : onlineRegions.values()) {
if (!r.isAvailable()) {
continue;
}
@ -2608,10 +2599,10 @@ public class HRegionServer extends HasThread implements
* @return Online regions from <code>tableName</code>
*/
@Override
public List<HRegion> getOnlineRegions(TableName tableName) {
List<HRegion> tableRegions = new ArrayList<HRegion>();
public List<Region> getOnlineRegions(TableName tableName) {
List<Region> tableRegions = new ArrayList<Region>();
synchronized (this.onlineRegions) {
for (HRegion region: this.onlineRegions.values()) {
for (Region region: this.onlineRegions.values()) {
HRegionInfo regionInfo = region.getRegionInfo();
if(regionInfo.getTable().equals(tableName)) {
tableRegions.add(region);
@ -2630,7 +2621,7 @@ public class HRegionServer extends HasThread implements
public Set<TableName> getOnlineTables() {
Set<TableName> tables = new HashSet<TableName>();
synchronized (this.onlineRegions) {
for (HRegion region: this.onlineRegions.values()) {
for (Region region: this.onlineRegions.values()) {
tables.add(region.getTableDesc().getTableName());
}
}
@ -2647,8 +2638,8 @@ public class HRegionServer extends HasThread implements
"skipping.");
LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception);
}
Collection<HRegion> regions = getOnlineRegionsLocalContext();
for (HRegion region: regions) {
Collection<Region> regions = getOnlineRegionsLocalContext();
for (Region region: regions) {
coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
try {
coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors());
@ -2698,7 +2689,7 @@ public class HRegionServer extends HasThread implements
protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
throws NotServingRegionException {
//Check for permissions to close.
HRegion actualRegion = this.getFromOnlineRegions(encodedName);
Region actualRegion = this.getFromOnlineRegions(encodedName);
if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
try {
actualRegion.getCoprocessorHost().preClose(false);
@ -2759,7 +2750,7 @@ public class HRegionServer extends HasThread implements
* @return HRegion for the passed binary <code>regionName</code> or null if
* named region is not member of the online regions.
*/
public HRegion getOnlineRegion(final byte[] regionName) {
public Region getOnlineRegion(final byte[] regionName) {
String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
return this.onlineRegions.get(encodedRegionName);
}
@ -2769,14 +2760,14 @@ public class HRegionServer extends HasThread implements
}
@Override
public HRegion getFromOnlineRegions(final String encodedRegionName) {
public Region getFromOnlineRegions(final String encodedRegionName) {
return this.onlineRegions.get(encodedRegionName);
}
@Override
public boolean removeFromOnlineRegions(final HRegion r, ServerName destination) {
HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
public boolean removeFromOnlineRegions(final Region r, ServerName destination) {
Region toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
if (destination != null) {
try {
@ -2808,20 +2799,20 @@ public class HRegionServer extends HasThread implements
* @return {@link HRegion} for <code>regionName</code>
* @throws NotServingRegionException
*/
protected HRegion getRegion(final byte[] regionName)
protected Region getRegion(final byte[] regionName)
throws NotServingRegionException {
String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
return getRegionByEncodedName(regionName, encodedRegionName);
}
public HRegion getRegionByEncodedName(String encodedRegionName)
public Region getRegionByEncodedName(String encodedRegionName)
throws NotServingRegionException {
return getRegionByEncodedName(null, encodedRegionName);
}
protected HRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName)
protected Region getRegionByEncodedName(byte[] regionName, String encodedRegionName)
throws NotServingRegionException {
HRegion region = this.onlineRegions.get(encodedRegionName);
Region region = this.onlineRegions.get(encodedRegionName);
if (region == null) {
MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
if (moveInfo != null) {
@ -3077,17 +3068,17 @@ public class HRegionServer extends HasThread implements
* @throws KeeperException
* @throws IOException
*/
private void updateRecoveringRegionLastFlushedSequenceId(HRegion r) throws KeeperException,
private void updateRecoveringRegionLastFlushedSequenceId(Region r) throws KeeperException,
IOException {
if (!r.isRecovering()) {
// return immdiately for non-recovering regions
return;
}
HRegionInfo region = r.getRegionInfo();
HRegionInfo regionInfo = r.getRegionInfo();
ZooKeeperWatcher zkw = getZooKeeper();
String previousRSName = this.getLastFailedRSFromZK(region.getEncodedName());
Map<byte[], Long> maxSeqIdInStores = r.getMaxStoreSeqIdForLogReplay();
String previousRSName = this.getLastFailedRSFromZK(regionInfo.getEncodedName());
Map<byte[], Long> maxSeqIdInStores = r.getMaxStoreSeqId();
long minSeqIdForLogReplay = -1;
for (Long storeSeqIdForReplay : maxSeqIdInStores.values()) {
if (minSeqIdForLogReplay == -1 || storeSeqIdForReplay < minSeqIdForLogReplay) {
@ -3098,7 +3089,7 @@ public class HRegionServer extends HasThread implements
try {
long lastRecordedFlushedSequenceId = -1;
String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
region.getEncodedName());
regionInfo.getEncodedName());
// recovering-region level
byte[] data;
try {
@ -3107,7 +3098,7 @@ public class HRegionServer extends HasThread implements
throw new InterruptedIOException();
}
if (data != null) {
lastRecordedFlushedSequenceId = ZKSplitLog.parseLastFlushedSequenceIdFrom(data);
lastRecordedFlushedSequenceId = ZKSplitLog.parseLastFlushedSequenceIdFrom(data);
}
if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) {
ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay));
@ -3117,14 +3108,14 @@ public class HRegionServer extends HasThread implements
nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
ZKUtil.setData(zkw, nodePath,
ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores));
LOG.debug("Update last flushed sequence id of region " + region.getEncodedName() + " for "
+ previousRSName);
LOG.debug("Update last flushed sequence id of region " + regionInfo.getEncodedName() +
" for " + previousRSName);
} else {
LOG.warn("Can't find failed region server for recovering region " +
region.getEncodedName());
regionInfo.getEncodedName());
}
} catch (NoNodeException ignore) {
LOG.debug("Region " + region.getEncodedName() +
LOG.debug("Region " + regionInfo.getEncodedName() +
" must have completed recovery because its recovery znode has been removed", ignore);
}
}
@ -3241,8 +3232,8 @@ public class HRegionServer extends HasThread implements
@Override
public double getCompactionPressure() {
double max = 0;
for (HRegion region : onlineRegions.values()) {
for (Store store : region.getStores().values()) {
for (Region region : onlineRegions.values()) {
for (Store store : region.getStores()) {
double normCount = store.getCompactionPressure();
if (normCount > max) {
max = normCount;

View File

@ -324,7 +324,7 @@ public class HeapMemoryManager {
}
@Override
public void flushRequested(FlushType type, HRegion region) {
public void flushRequested(FlushType type, Region region) {
switch (type) {
case ABOVE_HIGHER_MARK:
blockedFlushCount.incrementAndGet();

View File

@ -72,7 +72,7 @@ extends ConstantSizeRegionSplitPolicy {
// Get size to check
long sizeToCheck = getSizeToCheck(tableRegionsCount);
for (Store store : region.getStores().values()) {
for (Store store : region.getStores()) {
// If any of the stores is unable to split (eg they contain reference files)
// then don't split
if ((!store.canSplit())) {
@ -114,7 +114,7 @@ extends ConstantSizeRegionSplitPolicy {
TableName tablename = this.region.getTableDesc().getTableName();
int tableRegionsCount = 0;
try {
List<HRegion> hri = rss.getOnlineRegions(tablename);
List<Region> hri = rss.getOnlineRegions(tablename);
tableRegionsCount = hri == null || hri.isEmpty()? 0: hri.size();
} catch (IOException e) {
LOG.debug("Failed getOnlineRegions " + tablename, e);

View File

@ -164,7 +164,7 @@ public class LogRoller extends HasThread {
*/
private void scheduleFlush(final byte [] encodedRegionName) {
boolean scheduled = false;
HRegion r = this.services.getFromOnlineRegions(Bytes.toString(encodedRegionName));
Region r = this.services.getFromOnlineRegions(Bytes.toString(encodedRegionName));
FlushRequester requester = null;
if (r != null) {
requester = this.services.getFlushRequester();

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -79,8 +80,8 @@ class MemStoreFlusher implements FlushRequester {
// a corresponding entry in the other.
private final BlockingQueue<FlushQueueEntry> flushQueue =
new DelayQueue<FlushQueueEntry>();
private final Map<HRegion, FlushRegionEntry> regionsInQueue =
new HashMap<HRegion, FlushRegionEntry>();
private final Map<Region, FlushRegionEntry> regionsInQueue =
new HashMap<Region, FlushRegionEntry>();
private AtomicBoolean wakeupPending = new AtomicBoolean();
private final long threadWakeFrequency;
@ -139,10 +140,8 @@ class MemStoreFlusher implements FlushRequester {
* @return true if successful
*/
private boolean flushOneForGlobalPressure() {
SortedMap<Long, HRegion> regionsBySize =
server.getCopyOfOnlineRegionsSortedBySize();
Set<HRegion> excludedRegions = new HashSet<HRegion>();
SortedMap<Long, Region> regionsBySize = server.getCopyOfOnlineRegionsSortedBySize();
Set<Region> excludedRegions = new HashSet<Region>();
double secondaryMultiplier
= ServerRegionReplicaUtil.getRegionReplicaStoreFileRefreshMultiplier(conf);
@ -151,13 +150,12 @@ class MemStoreFlusher implements FlushRequester {
while (!flushedOne) {
// Find the biggest region that doesn't have too many storefiles
// (might be null!)
HRegion bestFlushableRegion = getBiggestMemstoreRegion(
regionsBySize, excludedRegions, true);
Region bestFlushableRegion = getBiggestMemstoreRegion(regionsBySize, excludedRegions, true);
// Find the biggest region, total, even if it might have too many flushes.
HRegion bestAnyRegion = getBiggestMemstoreRegion(
Region bestAnyRegion = getBiggestMemstoreRegion(
regionsBySize, excludedRegions, false);
// Find the biggest region that is a secondary region
HRegion bestRegionReplica = getBiggestMemstoreOfRegionReplica(regionsBySize,
Region bestRegionReplica = getBiggestMemstoreOfRegionReplica(regionsBySize,
excludedRegions);
if (bestAnyRegion == null && bestRegionReplica == null) {
@ -165,19 +163,20 @@ class MemStoreFlusher implements FlushRequester {
return false;
}
HRegion regionToFlush;
Region regionToFlush;
if (bestFlushableRegion != null &&
bestAnyRegion.memstoreSize.get() > 2 * bestFlushableRegion.memstoreSize.get()) {
bestAnyRegion.getMemstoreSize() > 2 * bestFlushableRegion.getMemstoreSize()) {
// Even if it's not supposed to be flushed, pick a region if it's more than twice
// as big as the best flushable one - otherwise when we're under pressure we make
// lots of little flushes and cause lots of compactions, etc, which just makes
// life worse!
if (LOG.isDebugEnabled()) {
LOG.debug("Under global heap pressure: " + "Region "
+ bestAnyRegion.getRegionNameAsString() + " has too many " + "store files, but is "
+ TraditionalBinaryPrefix.long2String(bestAnyRegion.memstoreSize.get(), "", 1)
+ bestAnyRegion.getRegionInfo().getRegionNameAsString()
+ " has too many " + "store files, but is "
+ TraditionalBinaryPrefix.long2String(bestAnyRegion.getMemstoreSize(), "", 1)
+ " vs best flushable region's "
+ TraditionalBinaryPrefix.long2String(bestFlushableRegion.memstoreSize.get(), "", 1)
+ TraditionalBinaryPrefix.long2String(bestFlushableRegion.getMemstoreSize(), "", 1)
+ ". Choosing the bigger.");
}
regionToFlush = bestAnyRegion;
@ -190,14 +189,14 @@ class MemStoreFlusher implements FlushRequester {
}
Preconditions.checkState(
(regionToFlush != null && regionToFlush.memstoreSize.get() > 0) ||
(bestRegionReplica != null && bestRegionReplica.memstoreSize.get() > 0));
(regionToFlush != null && regionToFlush.getMemstoreSize() > 0) ||
(bestRegionReplica != null && bestRegionReplica.getMemstoreSize() > 0));
if (regionToFlush == null ||
(bestRegionReplica != null &&
ServerRegionReplicaUtil.isRegionReplicaStoreFileRefreshEnabled(conf) &&
(bestRegionReplica.memstoreSize.get()
> secondaryMultiplier * regionToFlush.memstoreSize.get()))) {
(bestRegionReplica.getMemstoreSize()
> secondaryMultiplier * regionToFlush.getMemstoreSize()))) {
LOG.info("Refreshing storefiles of region " + regionToFlush +
" due to global heap pressure. memstore size=" + StringUtils.humanReadableInt(
server.getRegionServerAccounting().getGlobalMemstoreSize()));
@ -212,7 +211,7 @@ class MemStoreFlusher implements FlushRequester {
+ "Total Memstore size="
+ humanReadableInt(server.getRegionServerAccounting().getGlobalMemstoreSize())
+ ", Region memstore size="
+ humanReadableInt(regionToFlush.memstoreSize.get()));
+ humanReadableInt(regionToFlush.getMemstoreSize()));
flushedOne = flushRegion(regionToFlush, true, true);
if (!flushedOne) {
@ -289,17 +288,18 @@ class MemStoreFlusher implements FlushRequester {
}
}
private HRegion getBiggestMemstoreRegion(
SortedMap<Long, HRegion> regionsBySize,
Set<HRegion> excludedRegions,
private Region getBiggestMemstoreRegion(
SortedMap<Long, Region> regionsBySize,
Set<Region> excludedRegions,
boolean checkStoreFileCount) {
synchronized (regionsInQueue) {
for (HRegion region : regionsBySize.values()) {
for (Region region : regionsBySize.values()) {
if (excludedRegions.contains(region)) {
continue;
}
if (region.writestate.flushing || !region.writestate.writesEnabled) {
if (((HRegion)region).writestate.flushing ||
!((HRegion)region).writestate.writesEnabled) {
continue;
}
@ -312,10 +312,10 @@ class MemStoreFlusher implements FlushRequester {
return null;
}
private HRegion getBiggestMemstoreOfRegionReplica(SortedMap<Long, HRegion> regionsBySize,
Set<HRegion> excludedRegions) {
private Region getBiggestMemstoreOfRegionReplica(SortedMap<Long, Region> regionsBySize,
Set<Region> excludedRegions) {
synchronized (regionsInQueue) {
for (HRegion region : regionsBySize.values()) {
for (Region region : regionsBySize.values()) {
if (excludedRegions.contains(region)) {
continue;
}
@ -330,7 +330,7 @@ class MemStoreFlusher implements FlushRequester {
return null;
}
private boolean refreshStoreFilesAndReclaimMemory(HRegion region) {
private boolean refreshStoreFilesAndReclaimMemory(Region region) {
try {
return region.refreshStoreFiles();
} catch (IOException e) {
@ -356,7 +356,7 @@ class MemStoreFlusher implements FlushRequester {
}
@Override
public void requestFlush(HRegion r, boolean forceFlushAllStores) {
public void requestFlush(Region r, boolean forceFlushAllStores) {
synchronized (regionsInQueue) {
if (!regionsInQueue.containsKey(r)) {
// This entry has no delay so it will be added at the top of the flush
@ -369,7 +369,7 @@ class MemStoreFlusher implements FlushRequester {
}
@Override
public void requestDelayedFlush(HRegion r, long delay, boolean forceFlushAllStores) {
public void requestDelayedFlush(Region r, long delay, boolean forceFlushAllStores) {
synchronized (regionsInQueue) {
if (!regionsInQueue.containsKey(r)) {
// This entry has some delay
@ -435,19 +435,19 @@ class MemStoreFlusher implements FlushRequester {
* not flushed.
*/
private boolean flushRegion(final FlushRegionEntry fqe) {
HRegion region = fqe.region;
Region region = fqe.region;
if (!region.getRegionInfo().isMetaRegion() &&
isTooManyStoreFiles(region)) {
if (fqe.isMaximumWait(this.blockingWaitTime)) {
LOG.info("Waited " + (EnvironmentEdgeManager.currentTime() - fqe.createTime) +
"ms on a compaction to clean up 'too many store files'; waited " +
"long enough... proceeding with flush of " +
region.getRegionNameAsString());
region.getRegionInfo().getRegionNameAsString());
} else {
// If this is first time we've been put off, then emit a log message.
if (fqe.getRequeueCount() <= 0) {
// Note: We don't impose blockingStoreFiles constraint on meta regions
LOG.warn("Region " + region.getRegionNameAsString() + " has too many " +
LOG.warn("Region " + region.getRegionInfo().getRegionNameAsString() + " has too many " +
"store files; delaying flush up to " + this.blockingWaitTime + "ms");
if (!this.server.compactSplitThread.requestSplit(region)) {
try {
@ -456,9 +456,8 @@ class MemStoreFlusher implements FlushRequester {
} catch (IOException e) {
e = e instanceof RemoteException ?
((RemoteException)e).unwrapRemoteException() : e;
LOG.error(
"Cache flush failed for region " + Bytes.toStringBinary(region.getRegionName()),
e);
LOG.error("Cache flush failed for region " +
Bytes.toStringBinary(region.getRegionInfo().getRegionName()), e);
}
}
}
@ -485,7 +484,7 @@ class MemStoreFlusher implements FlushRequester {
* false, there will be accompanying log messages explaining why the region was
* not flushed.
*/
private boolean flushRegion(final HRegion region, final boolean emergencyFlush,
private boolean flushRegion(final Region region, final boolean emergencyFlush,
boolean forceFlushAllStores) {
long startTime = 0;
synchronized (this.regionsInQueue) {
@ -509,10 +508,10 @@ class MemStoreFlusher implements FlushRequester {
lock.readLock().lock();
try {
notifyFlushRequest(region, emergencyFlush);
HRegion.FlushResult flushResult = region.flushcache(forceFlushAllStores);
FlushResult flushResult = region.flush(forceFlushAllStores);
boolean shouldCompact = flushResult.isCompactionNeeded();
// We just want to check the size
boolean shouldSplit = region.checkSplit() != null;
boolean shouldSplit = ((HRegion)region).checkSplit() != null;
if (shouldSplit) {
this.server.compactSplitThread.requestSplit(region);
} else if (shouldCompact) {
@ -535,8 +534,9 @@ class MemStoreFlusher implements FlushRequester {
ex = ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex;
LOG.error(
"Cache flush failed"
+ (region != null ? (" for region " + Bytes.toStringBinary(region.getRegionName()))
: ""), ex);
+ (region != null ? (" for region " +
Bytes.toStringBinary(region.getRegionInfo().getRegionName()))
: ""), ex);
if (!server.checkFileSystem()) {
return false;
}
@ -547,7 +547,7 @@ class MemStoreFlusher implements FlushRequester {
return true;
}
private void notifyFlushRequest(HRegion region, boolean emergencyFlush) {
private void notifyFlushRequest(Region region, boolean emergencyFlush) {
FlushType type = FlushType.NORMAL;
if (emergencyFlush) {
type = isAboveHighWaterMark() ? FlushType.ABOVE_HIGHER_MARK : FlushType.ABOVE_LOWER_MARK;
@ -563,8 +563,8 @@ class MemStoreFlusher implements FlushRequester {
}
}
private boolean isTooManyStoreFiles(HRegion region) {
for (Store store : region.stores.values()) {
private boolean isTooManyStoreFiles(Region region) {
for (Store store : region.getStores()) {
if (store.hasTooManyStoreFiles()) {
return true;
}
@ -719,7 +719,7 @@ class MemStoreFlusher implements FlushRequester {
* a while.
*/
static class FlushRegionEntry implements FlushQueueEntry {
private final HRegion region;
private final Region region;
private final long createTime;
private long whenToExpire;
@ -727,7 +727,7 @@ class MemStoreFlusher implements FlushRequester {
private boolean forceFlushAllStores;
FlushRegionEntry(final HRegion r, boolean forceFlushAllStores) {
FlushRegionEntry(final Region r, boolean forceFlushAllStores) {
this.region = r;
this.createTime = EnvironmentEdgeManager.currentTime();
this.whenToExpire = this.createTime;
@ -789,7 +789,7 @@ class MemStoreFlusher implements FlushRequester {
@Override
public String toString() {
return "[flush region " + Bytes.toStringBinary(region.getRegionName()) + "]";
return "[flush region "+Bytes.toStringBinary(region.getRegionInfo().getRegionName())+"]";
}
@Override

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
@ -166,7 +167,7 @@ class MetricsRegionServerWrapperImpl
@Override
public long getNumOnlineRegions() {
Collection<HRegion> onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext();
Collection<Region> onlineRegionsLocalContext = regionServer.getOnlineRegionsLocalContext();
if (onlineRegionsLocalContext == null) {
return 0;
}
@ -452,16 +453,17 @@ class MetricsRegionServerWrapperImpl
long tempMajorCompactedCellsSize = 0;
long tempBlockedRequestsCount = 0L;
for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
tempNumMutationsWithoutWAL += r.numMutationsWithoutWAL.get();
tempDataInMemoryWithoutWAL += r.dataInMemoryWithoutWAL.get();
tempReadRequestsCount += r.readRequestsCount.get();
tempWriteRequestsCount += r.writeRequestsCount.get();
tempCheckAndMutateChecksFailed += r.checkAndMutateChecksFailed.get();
tempCheckAndMutateChecksPassed += r.checkAndMutateChecksPassed.get();
for (Region r : regionServer.getOnlineRegionsLocalContext()) {
tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL();
tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL();
tempReadRequestsCount += r.getReadRequestsCount();
tempWriteRequestsCount += r.getWriteRequestsCount();
tempCheckAndMutateChecksFailed += r.getCheckAndMutateChecksFailed();
tempCheckAndMutateChecksPassed += r.getCheckAndMutateChecksPassed();
tempBlockedRequestsCount += r.getBlockedRequestsCount();
tempNumStores += r.stores.size();
for (Store store : r.stores.values()) {
List<Store> storeList = r.getStores();
tempNumStores += storeList.size();
for (Store store : storeList) {
tempNumStoreFiles += store.getStorefilesCount();
tempMemstoreSize += store.getMemStoreSize();
tempStoreFileSize += store.getStorefilesSize();

View File

@ -22,46 +22,49 @@ import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
/**
* Interface to Map of online regions. In the Map, the key is the region's
* encoded name and the value is an {@link HRegion} instance.
* encoded name and the value is an {@link Region} instance.
*/
@InterfaceAudience.Private
interface OnlineRegions extends Server {
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface OnlineRegions extends Server {
/**
* Add to online regions.
* @param r
*/
void addToOnlineRegions(final HRegion r);
void addToOnlineRegions(final Region r);
/**
* This method removes HRegion corresponding to hri from the Map of onlineRegions.
* This method removes Region corresponding to hri from the Map of onlineRegions.
*
* @param r Region to remove.
* @param destination Destination, if any, null otherwise.
* @return True if we removed a region from online list.
*/
boolean removeFromOnlineRegions(final HRegion r, ServerName destination);
boolean removeFromOnlineRegions(final Region r, ServerName destination);
/**
* Return {@link HRegion} instance.
* Only works if caller is in same context, in same JVM. HRegion is not
* Return {@link Region} instance.
* Only works if caller is in same context, in same JVM. Region is not
* serializable.
* @param encodedRegionName
* @return HRegion for the passed encoded <code>encodedRegionName</code> or
* @return Region for the passed encoded <code>encodedRegionName</code> or
* null if named region is not member of the online regions.
*/
HRegion getFromOnlineRegions(String encodedRegionName);
Region getFromOnlineRegions(String encodedRegionName);
/**
* Get all online regions of a table in this RS.
* @param tableName
* @return List of HRegion
* @return List of Region
* @throws java.io.IOException
*/
List<HRegion> getOnlineRegions(TableName tableName) throws IOException;
List<Region> getOnlineRegions(TableName tableName) throws IOException;
}

View File

@ -151,9 +151,10 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.quotas.OperationQuota;
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -163,6 +164,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
@ -215,9 +217,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
private static class RegionScannerHolder {
private RegionScanner s;
private long nextCallSeq = 0L;
private HRegion r;
private Region r;
public RegionScannerHolder(RegionScanner s, HRegion r) {
public RegionScannerHolder(RegionScanner s, Region r) {
this.s = s;
this.r = r;
}
@ -242,7 +244,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
LOG.info("Scanner " + this.scannerName + " lease expired on region "
+ s.getRegionInfo().getRegionNameAsString());
try {
HRegion region = regionServer.getRegion(s.getRegionInfo().getRegionName());
Region region = regionServer.getRegion(s.getRegionInfo().getRegionName());
if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().preScannerClose(s);
}
@ -362,7 +364,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @param cellScanner if non-null, the mutation data -- the Cell content.
* @throws IOException
*/
private ClientProtos.RegionLoadStats mutateRows(final HRegion region,
private ClientProtos.RegionLoadStats mutateRows(final Region region,
final List<ClientProtos.Action> actions,
final CellScanner cellScanner) throws IOException {
if (!region.getRegionInfo().isMetaTable()) {
@ -390,7 +392,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
region.mutateRow(rm);
return region.getRegionStats();
return ((HRegion)region).getRegionStats();
}
/**
@ -405,7 +407,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @param compareOp
* @param comparator @throws IOException
*/
private boolean checkAndRowMutate(final HRegion region, final List<ClientProtos.Action> actions,
private boolean checkAndRowMutate(final Region region, final List<ClientProtos.Action> actions,
final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
CompareOp compareOp, ByteArrayComparable comparator) throws IOException {
if (!region.getRegionInfo().isMetaTable()) {
@ -445,7 +447,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* bypassed as indicated by RegionObserver, null otherwise
* @throws IOException
*/
private Result append(final HRegion region, final OperationQuota quota, final MutationProto m,
private Result append(final Region region, final OperationQuota quota, final MutationProto m,
final CellScanner cellScanner, long nonceGroup) throws IOException {
long before = EnvironmentEdgeManager.currentTime();
Append append = ProtobufUtil.toAppend(m, cellScanner);
@ -482,7 +484,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @return the Result
* @throws IOException
*/
private Result increment(final HRegion region, final OperationQuota quota,
private Result increment(final Region region, final OperationQuota quota,
final MutationProto mutation, final CellScanner cells, long nonceGroup)
throws IOException {
long before = EnvironmentEdgeManager.currentTime();
@ -523,7 +525,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* method returns as a 'result'.
* @return Return the <code>cellScanner</code> passed
*/
private List<CellScannable> doNonAtomicRegionMutation(final HRegion region,
private List<CellScannable> doNonAtomicRegionMutation(final Region region,
final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner,
final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup) {
// Gather up CONTIGUOUS Puts and Deletes in this mutations List. Idea is that rather than do
@ -622,7 +624,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @param region
* @param mutations
*/
private void doBatchOp(final RegionActionResult.Builder builder, final HRegion region,
private void doBatchOp(final RegionActionResult.Builder builder, final Region region,
final OperationQuota quota, final List<ClientProtos.Action> mutations,
final CellScanner cells) {
Mutation[] mArray = new Mutation[mutations.size()];
@ -648,7 +650,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
regionServer.cacheFlusher.reclaimMemStoreMemory();
}
OperationStatus codes[] = region.batchMutate(mArray);
OperationStatus codes[] = region.batchMutate(mArray, HConstants.NO_NONCE,
HConstants.NO_NONCE);
for (i = 0; i < codes.length; i++) {
int index = mutations.get(i).getIndex();
Exception e = null;
@ -670,7 +673,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
case SUCCESS:
builder.addResultOrException(getResultOrException(
ClientProtos.Result.getDefaultInstance(), index, region.getRegionStats()));
ClientProtos.Result.getDefaultInstance(), index,
((HRegion)region).getRegionStats()));
break;
}
}
@ -700,7 +704,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* exceptionMessage if any
* @throws IOException
*/
private OperationStatus [] doReplayBatchOp(final HRegion region,
private OperationStatus [] doReplayBatchOp(final Region region,
final List<WALSplitter.MutationReplay> mutations, long replaySeqId) throws IOException {
long before = EnvironmentEdgeManager.currentTime();
boolean batchContainsPuts = false, batchContainsDelete = false;
@ -720,26 +724,27 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
for (Cell metaCell : metaCells) {
CompactionDescriptor compactionDesc = WALEdit.getCompaction(metaCell);
boolean isDefaultReplica = RegionReplicaUtil.isDefaultReplica(region.getRegionInfo());
HRegion hRegion = (HRegion)region;
if (compactionDesc != null) {
// replay the compaction. Remove the files from stores only if we are the primary
// region replica (thus own the files)
region.replayWALCompactionMarker(compactionDesc, !isDefaultReplica, isDefaultReplica,
hRegion.replayWALCompactionMarker(compactionDesc, !isDefaultReplica, isDefaultReplica,
replaySeqId);
continue;
}
FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(metaCell);
if (flushDesc != null && !isDefaultReplica) {
region.replayWALFlushMarker(flushDesc, replaySeqId);
hRegion.replayWALFlushMarker(flushDesc, replaySeqId);
continue;
}
RegionEventDescriptor regionEvent = WALEdit.getRegionEventDescriptor(metaCell);
if (regionEvent != null && !isDefaultReplica) {
region.replayWALRegionEventMarker(regionEvent);
hRegion.replayWALRegionEventMarker(regionEvent);
continue;
}
BulkLoadDescriptor bulkLoadEvent = WALEdit.getBulkLoadDescriptor(metaCell);
if (bulkLoadEvent != null) {
region.replayWALBulkLoadEventMarker(bulkLoadEvent);
hRegion.replayWALBulkLoadEventMarker(bulkLoadEvent);
continue;
}
}
@ -852,7 +857,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return 0L;
}
long addScanner(RegionScanner s, HRegion r) throws LeaseStillHeldException {
long addScanner(RegionScanner s, Region r) throws LeaseStillHeldException {
long scannerId = this.scannerIdGen.incrementAndGet();
String scannerName = String.valueOf(scannerId);
@ -873,7 +878,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @throws IOException if the specifier is not null,
* but failed to find the region
*/
HRegion getRegion(
Region getRegion(
final RegionSpecifier regionSpecifier) throws IOException {
return regionServer.getRegionByEncodedName(regionSpecifier.getValue().toByteArray(),
ProtobufUtil.getRegionEncodedName(regionSpecifier));
@ -1006,7 +1011,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion());
// Can be null if we're calling close on a region that's not online
final HRegion region = regionServer.getFromOnlineRegions(encodedRegionName);
final Region region = regionServer.getFromOnlineRegions(encodedRegionName);
if ((region != null) && (region .getCoprocessorHost() != null)) {
region.getCoprocessorHost().preClose(false);
}
@ -1035,9 +1040,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
HRegion region = getRegion(request.getRegion());
Region region = getRegion(request.getRegion());
region.startRegionOperation(Operation.COMPACT_REGION);
LOG.info("Compacting " + region.getRegionNameAsString());
LOG.info("Compacting " + region.getRegionInfo().getRegionNameAsString());
boolean major = false;
byte [] family = null;
Store store = null;
@ -1046,7 +1051,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
store = region.getStore(family);
if (store == null) {
throw new ServiceException(new IOException("column family " + Bytes.toString(family)
+ " does not exist in region " + region.getRegionNameAsString()));
+ " does not exist in region " + region.getRegionInfo().getRegionNameAsString()));
}
}
if (request.hasMajor()) {
@ -1063,7 +1068,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
String familyLogMsg = (family != null)?" for column family: " + Bytes.toString(family):"";
if (LOG.isTraceEnabled()) {
LOG.trace("User-triggered compaction requested for region "
+ region.getRegionNameAsString() + familyLogMsg);
+ region.getRegionInfo().getRegionNameAsString() + familyLogMsg);
}
String log = "User-triggered " + (major ? "major " : "") + "compaction" + familyLogMsg;
if(family != null) {
@ -1093,8 +1098,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
HRegion region = getRegion(request.getRegion());
LOG.info("Flushing " + region.getRegionNameAsString());
Region region = getRegion(request.getRegion());
LOG.info("Flushing " + region.getRegionInfo().getRegionNameAsString());
boolean shouldFlush = true;
if (request.hasIfOlderThanTs()) {
shouldFlush = region.getEarliestFlushTimeForAllStores() < request.getIfOlderThanTs();
@ -1104,7 +1109,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
boolean writeFlushWalMarker = request.hasWriteFlushWalMarker() ?
request.getWriteFlushWalMarker() : false;
long startTime = EnvironmentEdgeManager.currentTime();
HRegion.FlushResult flushResult = region.flushcache(true, writeFlushWalMarker);
// Go behind the curtain so we can manage writing of the flush WAL marker
HRegion.FlushResultImpl flushResult = (HRegion.FlushResultImpl)
((HRegion)region).flushcache(true, writeFlushWalMarker);
if (flushResult.isFlushSucceeded()) {
long endTime = EnvironmentEdgeManager.currentTime();
regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
@ -1117,7 +1124,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
builder.setFlushed(flushResult.isFlushSucceeded());
builder.setWroteFlushWalMarker(flushResult.wroteFlushWalMarker);
}
builder.setLastFlushTime( region.getEarliestFlushTimeForAllStores());
builder.setLastFlushTime(region.getEarliestFlushTimeForAllStores());
return builder.build();
} catch (DroppedSnapshotException ex) {
// Cache flush can fail in a few places. If it fails in a critical
@ -1138,9 +1145,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
Map<String, HRegion> onlineRegions = regionServer.onlineRegions;
Map<String, Region> onlineRegions = regionServer.onlineRegions;
List<HRegionInfo> list = new ArrayList<HRegionInfo>(onlineRegions.size());
for (HRegion region: onlineRegions.values()) {
for (Region region: onlineRegions.values()) {
list.add(region.getRegionInfo());
}
Collections.sort(list);
@ -1157,7 +1164,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
HRegion region = getRegion(request.getRegion());
Region region = getRegion(request.getRegion());
HRegionInfo info = region.getRegionInfo();
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
builder.setRegionInfo(HRegionInfo.convert(info));
@ -1198,11 +1205,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
final GetStoreFileRequest request) throws ServiceException {
try {
checkOpen();
HRegion region = getRegion(request.getRegion());
Region region = getRegion(request.getRegion());
requestCount.increment();
Set<byte[]> columnFamilies;
if (request.getFamilyCount() == 0) {
columnFamilies = region.getStores().keySet();
columnFamilies = region.getTableDesc().getFamiliesKeys();
} else {
columnFamilies = new TreeSet<byte[]>(Bytes.BYTES_RAWCOMPARATOR);
for (ByteString cf: request.getFamilyList()) {
@ -1235,8 +1242,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
HRegion regionA = getRegion(request.getRegionA());
HRegion regionB = getRegion(request.getRegionB());
Region regionA = getRegion(request.getRegionA());
Region regionB = getRegion(request.getRegionB());
boolean forcible = request.getForcible();
regionA.startRegionOperation(Operation.MERGE_REGION);
regionB.startRegionOperation(Operation.MERGE_REGION);
@ -1247,13 +1254,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
LOG.info("Receiving merging request for " + regionA + ", " + regionB
+ ",forcible=" + forcible);
long startTime = EnvironmentEdgeManager.currentTime();
HRegion.FlushResult flushResult = regionA.flushcache();
FlushResult flushResult = regionA.flush(true);
if (flushResult.isFlushSucceeded()) {
long endTime = EnvironmentEdgeManager.currentTime();
regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
}
startTime = EnvironmentEdgeManager.currentTime();
flushResult = regionB.flushcache();
flushResult = regionB.flush(true);
if (flushResult.isFlushSucceeded()) {
long endTime = EnvironmentEdgeManager.currentTime();
regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
@ -1346,7 +1353,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
String encodedName = region.getEncodedName();
byte[] encodedNameBytes = region.getEncodedNameAsBytes();
final HRegion onlineRegion = regionServer.getFromOnlineRegions(encodedName);
final Region onlineRegion = regionServer.getFromOnlineRegions(encodedName);
if (onlineRegion != null) {
// The region is already online. This should not happen any more.
String error = "Received OPEN for the region:"
@ -1456,7 +1463,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
String encodedName = region.getEncodedName();
byte[] encodedNameBytes = region.getEncodedNameAsBytes();
final HRegion onlineRegion = regionServer.getFromOnlineRegions(encodedName);
final Region onlineRegion = regionServer.getFromOnlineRegions(encodedName);
if (onlineRegion != null) {
LOG.info("Region already online. Skipping warming up " + region);
@ -1507,7 +1514,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return ReplicateWALEntryResponse.newBuilder().build();
}
ByteString regionName = entries.get(0).getKey().getEncodedRegionName();
HRegion region = regionServer.getRegionByEncodedName(regionName.toStringUtf8());
Region region = regionServer.getRegionByEncodedName(regionName.toStringUtf8());
RegionCoprocessorHost coprocessorHost =
ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo())
? region.getCoprocessorHost()
@ -1558,12 +1565,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
//sync wal at the end because ASYNC_WAL is used above
region.syncWal();
WAL wal = getWAL(region);
if (wal != null) {
wal.sync();
}
if (coprocessorHost != null) {
for (Pair<WALKey, WALEdit> wal : walEntries) {
coprocessorHost.postWALRestore(region.getRegionInfo(), wal.getFirst(),
wal.getSecond());
for (Pair<WALKey, WALEdit> entry : walEntries) {
coprocessorHost.postWALRestore(region.getRegionInfo(), entry.getFirst(),
entry.getSecond());
}
}
return ReplicateWALEntryResponse.newBuilder().build();
@ -1577,6 +1587,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
WAL getWAL(Region region) {
return ((HRegion)region).getWAL();
}
/**
* Replicate WAL entries on the region server.
*
@ -1640,15 +1654,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
HRegion region = getRegion(request.getRegion());
Region region = getRegion(request.getRegion());
region.startRegionOperation(Operation.SPLIT_REGION);
if (region.getRegionInfo().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
throw new IOException("Can't split replicas directly. "
+ "Replicas are auto-split when their primary is split.");
}
LOG.info("Splitting " + region.getRegionNameAsString());
LOG.info("Splitting " + region.getRegionInfo().getRegionNameAsString());
long startTime = EnvironmentEdgeManager.currentTime();
HRegion.FlushResult flushResult = region.flushcache();
FlushResult flushResult = region.flush(true);
if (flushResult.isFlushSucceeded()) {
long endTime = EnvironmentEdgeManager.currentTime();
regionServer.metricsRegionServer.updateFlushTime(endTime - startTime);
@ -1657,8 +1671,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
if (request.hasSplitPoint()) {
splitPoint = request.getSplitPoint().toByteArray();
}
region.forceSplit(splitPoint);
regionServer.compactSplitThread.requestSplit(region, region.checkSplit());
((HRegion)region).forceSplit(splitPoint);
regionServer.compactSplitThread.requestSplit(region, ((HRegion)region).checkSplit());
return SplitRegionResponse.newBuilder().build();
} catch (IOException ie) {
throw new ServiceException(ie);
@ -1707,7 +1721,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
HRegion region = getRegion(request.getRegion());
Region region = getRegion(request.getRegion());
List<Pair<byte[], String>> familyPaths = new ArrayList<Pair<byte[], String>>();
for (FamilyPath familyPath: request.getFamilyPathList()) {
familyPaths.add(new Pair<byte[], String>(familyPath.getFamily().toByteArray(),
@ -1719,7 +1733,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
boolean loaded = false;
if (!bypass) {
loaded = region.bulkLoadHFiles(familyPaths, request.getAssignSeqNum());
loaded = region.bulkLoadHFiles(familyPaths, request.getAssignSeqNum(), null);
}
if (region.getCoprocessorHost() != null) {
loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
@ -1738,12 +1752,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
HRegion region = getRegion(request.getRegion());
Region region = getRegion(request.getRegion());
Message result = execServiceOnRegion(region, request.getCall());
CoprocessorServiceResponse.Builder builder =
CoprocessorServiceResponse.newBuilder();
builder.setRegion(RequestConverter.buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, region.getRegionName()));
RegionSpecifierType.REGION_NAME, region.getRegionInfo().getRegionName()));
builder.setValue(
builder.getValueBuilder().setName(result.getClass().getName())
.setValue(result.toByteString()));
@ -1753,7 +1767,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
private Message execServiceOnRegion(HRegion region,
private Message execServiceOnRegion(Region region,
final ClientProtos.CoprocessorServiceCall serviceCall) throws IOException {
// ignore the passed in controller (from the serialized call)
ServerRpcController execController = new ServerRpcController();
@ -1779,7 +1793,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
HRegion region = getRegion(request.getRegion());
Region region = getRegion(request.getRegion());
GetResponse.Builder builder = GetResponse.newBuilder();
ClientProtos.Get get = request.getGet();
@ -1871,7 +1885,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
for (RegionAction regionAction : request.getRegionActionList()) {
this.requestCount.add(regionAction.getActionCount());
OperationQuota quota;
HRegion region;
Region region;
regionActionResultBuilder.clear();
try {
region = getRegion(regionAction.getRegion());
@ -1946,14 +1960,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
try {
checkOpen();
requestCount.increment();
HRegion region = getRegion(request.getRegion());
Region region = getRegion(request.getRegion());
MutateResponse.Builder builder = MutateResponse.newBuilder();
MutationProto mutation = request.getMutation();
if (!region.getRegionInfo().isMetaTable()) {
regionServer.cacheFlusher.reclaimMemStoreMemory();
}
long nonceGroup = request.hasNonceGroup()
? request.getNonceGroup() : HConstants.NO_NONCE;
long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;
Result r = null;
Boolean processed = null;
MutationType type = mutation.getMutateType();
@ -2090,7 +2103,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
requestCount.increment();
int ttl = 0;
HRegion region = null;
Region region = null;
RegionScanner scanner = null;
RegionScannerHolder rsh = null;
boolean moreResults = true;
@ -2129,7 +2142,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
isSmallScan = scan.isSmall();
region.prepareScanner(scan);
if (!scan.hasFamilies()) {
// Adding all families to scanner
for (byte[] family: region.getTableDesc().getFamiliesKeys()) {
scan.addFamily(family);
}
}
if (region.getCoprocessorHost() != null) {
scanner = region.getCoprocessorHost().preScannerOpen(scan);
}
@ -2273,7 +2292,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
builder.setMoreResultsInRegion(false);
}
}
region.readRequestsCount.add(i);
region.updateReadRequestsCount(i);
region.getMetrics().updateScanNext(totalCellSize);
} finally {
region.closeRegionOperation();

View File

@ -0,0 +1,680 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.IsolationLevel;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
/**
* Regions store data for a certain region of a table. It stores all columns
* for each row. A given table consists of one or more Regions.
*
* <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)
@InterfaceStability.Evolving
public interface Region extends ConfigurationObserver {
///////////////////////////////////////////////////////////////////////////
// Region state
/** @return region information for this region */
HRegionInfo getRegionInfo();
/** @return table descriptor for this region */
HTableDescriptor getTableDesc();
/** @return true if region is available (not closed and not closing) */
boolean isAvailable();
/** @return true if region is closed */
boolean isClosed();
/** @return True if closing process has started */
boolean isClosing();
/** @return True if region is in recovering state */
boolean isRecovering();
/** @return True if region is read only */
boolean isReadOnly();
/**
* Return the list of Stores managed by this region
* <p>Use with caution. Exposed for use of fixup utilities.
* @return a list of the Stores managed by this region
*/
List<Store> getStores();
/**
* Return the Store for the given family
* <p>Use with caution. Exposed for use of fixup utilities.
* @return the Store for the given family
*/
Store getStore(byte[] family);
/** @return list of store file names for the given families */
List<String> getStoreFileList(byte [][] columns);
/**
* Check the region's underlying store files, open the files that have not
* been opened yet, and remove the store file readers for store files no
* longer available.
* @throws IOException
*/
boolean refreshStoreFiles() throws IOException;
/** @return the latest sequence number that was read from storage when this region was opened */
long getOpenSeqNum();
/** @return the max sequence id of flushed data on this region */
long getMaxFlushedSeqId();
/** @return the oldest sequence id found in the store for the given family */
public long getOldestSeqIdOfStore(byte[] familyName);
/**
* This can be used to determine the last time all files of this region were major compacted.
* @param majorCompactioOnly Only consider HFile that are the result of major compaction
* @return the timestamp of the oldest HFile for all stores of this region
*/
long getOldestHfileTs(boolean majorCompactioOnly) throws IOException;
/**
* @return map of column family names to max sequence id that was read from storage when this
* region was opened
*/
public Map<byte[], Long> getMaxStoreSeqId();
/** @return true if loading column families on demand by default */
boolean isLoadingCfsOnDemandDefault();
/** @return readpoint considering given IsolationLevel */
long getReadpoint(IsolationLevel isolationLevel);
/**
* @return The earliest time a store in the region was flushed. All
* other stores in the region would have been flushed either at, or
* after this time.
*/
long getEarliestFlushTimeForAllStores();
///////////////////////////////////////////////////////////////////////////
// Metrics
/** @return read requests count for this region */
long getReadRequestsCount();
/**
* Update the read request count for this region
* @param i increment
*/
void updateReadRequestsCount(long i);
/** @return write request count for this region */
long getWriteRequestsCount();
/**
* Update the write request count for this region
* @param i increment
*/
void updateWriteRequestsCount(long i);
/** @return memstore size for this region, in bytes */
long getMemstoreSize();
/** @return the number of mutations processed bypassing the WAL */
long getNumMutationsWithoutWAL();
/** @return the size of data processed bypassing the WAL, in bytes */
long getDataInMemoryWithoutWAL();
/** @return the number of blocked requests */
long getBlockedRequestsCount();
/** @return the number of checkAndMutate guards that passed */
long getCheckAndMutateChecksPassed();
/** @return the number of failed checkAndMutate guards */
long getCheckAndMutateChecksFailed();
/** @return the MetricsRegion for this region */
MetricsRegion getMetrics();
/** @return the block distribution for all Stores managed by this region */
HDFSBlocksDistribution getHDFSBlocksDistribution();
///////////////////////////////////////////////////////////////////////////
// Locking
// Region read locks
/**
* Operation enum is used in {@link Region#startRegionOperation} to provide context for
* various checks before any region operation begins.
*/
enum Operation {
ANY, GET, PUT, DELETE, SCAN, APPEND, INCREMENT, SPLIT_REGION, MERGE_REGION, BATCH_MUTATE,
REPLAY_BATCH_MUTATE, COMPACT_REGION, REPLAY_EVENT
}
/**
* This method needs to be called before any public call that reads or
* modifies data.
* Acquires a read lock and checks if the region is closing or closed.
* <p>{@link #closeRegionOperation} MUST then always be called after
* the operation has completed, whether it succeeded or failed.
* @throws IOException
*/
void startRegionOperation() throws IOException;
/**
* This method needs to be called before any public call that reads or
* modifies data.
* Acquires a read lock and checks if the region is closing or closed.
* <p>{@link #closeRegionOperation} MUST then always be called after
* the operation has completed, whether it succeeded or failed.
* @param op The operation is about to be taken on the region
* @throws IOException
*/
void startRegionOperation(Operation op) throws IOException;
/**
* Closes the region operation lock.
* @throws IOException
*/
void closeRegionOperation() throws IOException;
// Row write locks
/**
* Row lock held by a given thread.
* One thread may acquire multiple locks on the same row simultaneously.
* The locks must be released by calling release() from the same thread.
*/
public interface RowLock {
/**
* Release the given lock. If there are no remaining locks held by the current thread
* then unlock the row and allow other threads to acquire the lock.
* @throws IllegalArgumentException if called by a different thread than the lock owning
* thread
*/
void release();
}
/**
* Tries to acquire a lock on the given row.
* @param waitForLock if true, will block until the lock is available.
* Otherwise, just tries to obtain the lock and returns
* false if unavailable.
* @return the row lock if acquired,
* null if waitForLock was false and the lock was not acquired
* @throws IOException if waitForLock was true and the lock could not be acquired after waiting
*/
RowLock getRowLock(byte[] row, boolean waitForLock) throws IOException;
/**
* If the given list of row locks is not null, releases all locks.
*/
void releaseRowLocks(List<RowLock> rowLocks);
///////////////////////////////////////////////////////////////////////////
// Region operations
/**
* Perform one or more append operations on a row.
* @param append
* @param nonceGroup
* @param nonce
* @return result of the operation
* @throws IOException
*/
Result append(Append append, long nonceGroup, long nonce) throws IOException;
/**
* Perform a batch of mutations.
* <p>
* Note this supports only Put and Delete mutations and will ignore other types passed.
* @param mutations the list of mutations
* @param nonceGroup
* @param nonce
* @return an array of OperationStatus which internally contains the
* OperationStatusCode and the exceptionMessage if any.
* @throws IOException
*/
OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce)
throws IOException;
/**
* Replay a batch of mutations.
* @param mutations mutations to replay.
* @param replaySeqId
* @return an array of OperationStatus which internally contains the
* OperationStatusCode and the exceptionMessage if any.
* @throws IOException
*/
OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId) throws IOException;
/**
* Atomically checks if a row/family/qualifier value matches the expected val
* If it does, it performs the row mutations. If the passed value is null, t
* is for the lack of column (ie: non-existence)
* @param row to check
* @param family column family to check
* @param qualifier column qualifier to check
* @param compareOp the comparison operator
* @param comparator
* @param mutation
* @param writeToWAL
* @return true if mutation was applied, false otherwise
* @throws IOException
*/
boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier, CompareOp compareOp,
ByteArrayComparable comparator, Mutation mutation, boolean writeToWAL) throws IOException;
/**
* Atomically checks if a row/family/qualifier value matches the expected val
* If it does, it performs the row mutations. If the passed value is null, t
* is for the lack of column (ie: non-existence)
* @param row to check
* @param family column family to check
* @param qualifier column qualifier to check
* @param compareOp the comparison operator
* @param comparator
* @param mutations
* @param writeToWAL
* @return true if mutation was applied, false otherwise
* @throws IOException
*/
boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier, CompareOp compareOp,
ByteArrayComparable comparator, RowMutations mutations, boolean writeToWAL)
throws IOException;
/**
* Deletes the specified cells/row.
* @param delete
* @throws IOException
*/
void delete(Delete delete) throws IOException;
/**
* Do a get based on the get parameter.
* @param get query parameters
* @return result of the operation
*/
Result get(Get get) throws IOException;
/**
* Do a get based on the get parameter.
* @param get query parameters
* @param withCoprocessor invoke coprocessor or not. We don't want to
* always invoke cp.
* @return list of cells resulting from the operation
*/
List<Cell> get(Get get, boolean withCoprocessor) throws IOException;
/**
* Return all the data for the row that matches <i>row</i> exactly,
* or the one that immediately preceeds it, at or immediately before
* <i>ts</i>.
* @param row
* @param family
* @return result of the operation
* @throws IOException
*/
Result getClosestRowBefore(byte[] row, byte[] family) throws IOException;
/**
* Return an iterator that scans over the HRegion, returning the indicated
* columns and rows specified by the {@link Scan}.
* <p>
* This Iterator must be closed by the caller.
*
* @param scan configured {@link Scan}
* @return RegionScanner
* @throws IOException read exceptions
*/
RegionScanner getScanner(Scan scan) throws IOException;
/**
* Perform one or more increment operations on a row.
* @param increment
* @param nonceGroup
* @param nonce
* @return result of the operation
* @throws IOException
*/
Result increment(Increment increment, long nonceGroup, long nonce) throws IOException;
/**
* Performs multiple mutations atomically on a single row. Currently
* {@link Put} and {@link Delete} are supported.
*
* @param mutations object that specifies the set of mutations to perform atomically
* @throws IOException
*/
void mutateRow(RowMutations mutations) throws IOException;
/**
* Perform atomic mutations within the region.
*
* @param mutations The list of mutations to perform.
* <code>mutations</code> can contain operations for multiple rows.
* Caller has to ensure that all rows are contained in this region.
* @param rowsToLock Rows to lock
* @param nonceGroup Optional nonce group of the operation (client Id)
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
* If multiple rows are locked care should be taken that
* <code>rowsToLock</code> is sorted in order to avoid deadlocks.
* @throws IOException
*/
void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock,
long nonceGroup, long nonce) throws IOException;
/**
* Performs atomic multiple reads and writes on a given row.
*
* @param processor The object defines the reads and writes to a row.
*/
void processRowsWithLocks(RowProcessor<?,?> processor) throws IOException;
/**
* Performs atomic multiple reads and writes on a given row.
*
* @param processor The object defines the reads and writes to a row.
* @param nonceGroup Optional nonce group of the operation (client Id)
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
*/
void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce)
throws IOException;
/**
* Performs atomic multiple reads and writes on a given row.
*
* @param processor The object defines the reads and writes to a row.
* @param timeout The timeout of the processor.process() execution
* Use a negative number to switch off the time bound
* @param nonceGroup Optional nonce group of the operation (client Id)
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
*/
void processRowsWithLocks(RowProcessor<?,?> processor, long timeout, long nonceGroup, long nonce)
throws IOException;
/**
* Puts some data in the table.
* @param put
* @throws IOException
*/
void put(Put put) throws IOException;
/**
* Listener class to enable callers of
* bulkLoadHFile() to perform any necessary
* pre/post processing of a given bulkload call
*/
interface BulkLoadListener {
/**
* Called before an HFile is actually loaded
* @param family family being loaded to
* @param srcPath path of HFile
* @return final path to be used for actual loading
* @throws IOException
*/
String prepareBulkLoad(byte[] family, String srcPath) throws IOException;
/**
* Called after a successful HFile load
* @param family family being loaded to
* @param srcPath path of HFile
* @throws IOException
*/
void doneBulkLoad(byte[] family, String srcPath) throws IOException;
/**
* Called after a failed HFile load
* @param family family being loaded to
* @param srcPath path of HFile
* @throws IOException
*/
void failedBulkLoad(byte[] family, String srcPath) throws IOException;
}
/**
* Attempts to atomically load a group of hfiles. This is critical for loading
* rows with multiple column families atomically.
*
* @param familyPaths List of Pair<byte[] column family, String hfilePath>
* @param bulkLoadListener Internal hooks enabling massaging/preparation of a
* file about to be bulk loaded
* @param assignSeqId
* @return true if successful, false if failed recoverably
* @throws IOException if failed unrecoverably.
*/
boolean bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
BulkLoadListener bulkLoadListener) throws IOException;
///////////////////////////////////////////////////////////////////////////
// Coprocessors
/** @return the coprocessor host */
RegionCoprocessorHost getCoprocessorHost();
/**
* Executes a single protocol buffer coprocessor endpoint {@link Service} method using
* the registered protocol handlers. {@link Service} implementations must be registered via the
* {@link Region#registerService(com.google.protobuf.Service)}
* method before they are available.
*
* @param controller an {@code RpcContoller} implementation to pass to the invoked service
* @param call a {@code CoprocessorServiceCall} instance identifying the service, method,
* and parameters for the method invocation
* @return a protocol buffer {@code Message} instance containing the method's result
* @throws IOException if no registered service handler is found or an error
* occurs during the invocation
* @see org.apache.hadoop.hbase.regionserver.Region#registerService(com.google.protobuf.Service)
*/
Message execService(RpcController controller, CoprocessorServiceCall call) throws IOException;
/**
* Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to
* be available for handling
* {@link Region#execService(com.google.protobuf.RpcController,
* org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall)}} calls.
*
* <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(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 HConstants#LATEST_TIMESTAMP} with the
* provided current timestamp.
* @param values
* @param now
*/
void updateCellTimestamps(final Iterable<List<Cell>> values, final byte[] now)
throws IOException;
///////////////////////////////////////////////////////////////////////////
// Flushes, compactions, splits, etc.
// Wizards only, please
interface FlushResult {
enum Result {
FLUSHED_NO_COMPACTION_NEEDED,
FLUSHED_COMPACTION_NEEDED,
// Special case where a flush didn't run because there's nothing in the memstores. Used when
// bulk loading to know when we can still load even if a flush didn't happen.
CANNOT_FLUSH_MEMSTORE_EMPTY,
CANNOT_FLUSH
}
/** @return the detailed result code */
Result getResult();
/** @return true if the memstores were flushed, else false */
boolean isFlushSucceeded();
/** @return True if the flush requested a compaction, else false */
boolean isCompactionNeeded();
}
/**
* Flush the cache.
*
* <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
* @throws DroppedSnapshotException Thrown when abort is required
* because a snapshot was not properly persisted.
*/
FlushResult flush(boolean force) throws IOException;
/**
* Synchronously compact all stores in the region.
* <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.
* <p>
* Compaction will be performed asynchronously to this call by the RegionServer's
* CompactSplitThread. See also {@link Store#triggerMajorCompaction()}
* @throws IOException
*/
void triggerMajorCompaction() throws IOException;
/**
* @return if a given region is in compaction now.
*/
CompactionState getCompactionState();
/** Wait for all current flushes and compactions of the region to complete */
void waitForFlushesAndCompactions();
}

View File

@ -77,7 +77,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.wal.WALKey;
@ -88,7 +88,7 @@ import org.apache.hadoop.hbase.util.Pair;
/**
* Implements the coprocessor environment and runtime support for coprocessors
* loaded within a {@link HRegion}.
* loaded within a {@link Region}.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
@ -106,7 +106,7 @@ public class RegionCoprocessorHost
static class RegionEnvironment extends CoprocessorHost.Environment
implements RegionCoprocessorEnvironment {
private HRegion region;
private Region region;
private RegionServerServices rsServices;
ConcurrentMap<String, Object> sharedData;
private static final int LATENCY_BUFFER_SIZE = 100;
@ -121,7 +121,7 @@ public class RegionCoprocessorHost
* @param priority chaining priority
*/
public RegionEnvironment(final Coprocessor impl, final int priority,
final int seq, final Configuration conf, final HRegion region,
final int seq, final Configuration conf, final Region region,
final RegionServerServices services, final ConcurrentMap<String, Object> sharedData) {
super(impl, priority, seq, conf);
this.region = region;
@ -139,7 +139,7 @@ public class RegionCoprocessorHost
/** @return the region */
@Override
public HRegion getRegion() {
public Region getRegion() {
return region;
}
@ -209,7 +209,7 @@ public class RegionCoprocessorHost
/** The region server services */
RegionServerServices rsServices;
/** The region */
HRegion region;
Region region;
/**
* Constructor
@ -217,7 +217,7 @@ public class RegionCoprocessorHost
* @param rsServices interface to available region server functionality
* @param conf the configuration
*/
public RegionCoprocessorHost(final HRegion region,
public RegionCoprocessorHost(final Region region,
final RegionServerServices rsServices, final Configuration conf) {
super(rsServices);
this.conf = conf;
@ -707,7 +707,7 @@ public class RegionCoprocessorHost
* @param r the new right-hand daughter region
* @throws IOException
*/
public void postSplit(final HRegion l, final HRegion r) throws IOException {
public void postSplit(final Region l, final Region r) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
@Override
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)

View File

@ -42,10 +42,10 @@ class RegionMergeRequest implements Runnable {
private final boolean forcible;
private TableLock tableLock;
RegionMergeRequest(HRegion a, HRegion b, HRegionServer hrs, boolean forcible) {
RegionMergeRequest(Region a, Region b, HRegionServer hrs, boolean forcible) {
Preconditions.checkNotNull(hrs);
this.region_a = a;
this.region_b = b;
this.region_a = (HRegion)a;
this.region_b = (HRegion)b;
this.server = hrs;
this.forcible = forcible;
}
@ -71,7 +71,8 @@ class RegionMergeRequest implements Runnable {
//acquire a shared read lock on the table, so that table schema modifications
//do not happen concurrently
tableLock = server.getTableLockManager().readLock(region_a.getTableDesc().getTableName()
, "MERGE_REGIONS:" + region_a.getRegionNameAsString() + ", " + region_b.getRegionNameAsString());
, "MERGE_REGIONS:" + region_a.getRegionInfo().getRegionNameAsString() + ", " +
region_b.getRegionInfo().getRegionNameAsString());
try {
tableLock.acquire();
} catch (IOException ex) {
@ -134,7 +135,7 @@ class RegionMergeRequest implements Runnable {
LOG.error("Could not release the table lock (something is really wrong). "
+ "Aborting this server to avoid holding the lock forever.");
this.server.abort("Abort; we got an error when releasing the table lock "
+ "on " + region_a.getRegionNameAsString());
+ "on " + region_a.getRegionInfo().getRegionNameAsString());
}
}
}

View File

@ -141,14 +141,14 @@ public class RegionMergeTransaction {
* @param b region b to merge
* @param forcible if false, we will only merge adjacent regions
*/
public RegionMergeTransaction(final HRegion a, final HRegion b,
public RegionMergeTransaction(final Region a, final Region b,
final boolean forcible) {
if (a.getRegionInfo().compareTo(b.getRegionInfo()) <= 0) {
this.region_a = a;
this.region_b = b;
this.region_a = (HRegion)a;
this.region_b = (HRegion)b;
} else {
this.region_a = b;
this.region_b = a;
this.region_a = (HRegion)b;
this.region_b = (HRegion)a;
}
this.forcible = forcible;
this.mergesdir = region_a.getRegionFileSystem().getMergesDir();
@ -173,8 +173,8 @@ public class RegionMergeTransaction {
}
if (!forcible && !HRegionInfo.areAdjacent(region_a.getRegionInfo(),
region_b.getRegionInfo())) {
String msg = "Skip merging " + this.region_a.getRegionNameAsString()
+ " and " + this.region_b.getRegionNameAsString()
String msg = "Skip merging " + this.region_a.getRegionInfo().getRegionNameAsString()
+ " and " + this.region_b.getRegionInfo().getRegionNameAsString()
+ ", because they are not adjacent.";
LOG.info(msg);
return false;
@ -184,18 +184,19 @@ public class RegionMergeTransaction {
}
try {
boolean regionAHasMergeQualifier = hasMergeQualifierInMeta(services,
region_a.getRegionName());
region_a.getRegionInfo().getRegionName());
if (regionAHasMergeQualifier ||
hasMergeQualifierInMeta(services, region_b.getRegionName())) {
LOG.debug("Region " + (regionAHasMergeQualifier ? region_a.getRegionNameAsString()
: region_b.getRegionNameAsString())
hasMergeQualifierInMeta(services, region_b.getRegionInfo().getRegionName())) {
LOG.debug("Region " + (regionAHasMergeQualifier ?
region_a.getRegionInfo().getRegionNameAsString() :
region_b.getRegionInfo().getRegionNameAsString())
+ " is not mergeable because it has merge qualifier in META");
return false;
}
} catch (IOException e) {
LOG.warn("Failed judging whether merge transaction is available for "
+ region_a.getRegionNameAsString() + " and "
+ region_b.getRegionNameAsString(), e);
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
+ region_b.getRegionInfo().getRegionNameAsString(), e);
return false;
}
@ -254,7 +255,7 @@ public class RegionMergeTransaction {
HRegion createMergedRegion(final Server server,
final RegionServerServices services) throws IOException {
LOG.info("Starting merge of " + region_a + " and "
+ region_b.getRegionNameAsString() + ", forcible=" + forcible);
+ region_b.getRegionInfo().getRegionNameAsString() + ", forcible=" + forcible);
if ((server != null && server.isStopped())
|| (services != null && services.isStopping())) {
throw new IOException("Server is stopped or stopping");
@ -487,7 +488,7 @@ public class RegionMergeTransaction {
boolean stopped = server != null && server.isStopped();
boolean stopping = services != null && services.isStopping();
if (stopped || stopping) {
LOG.info("Not opening merged region " + merged.getRegionNameAsString()
LOG.info("Not opening merged region " + merged.getRegionInfo().getRegionNameAsString()
+ " because stopping=" + stopping + ", stopped=" + stopped);
return;
}
@ -588,7 +589,7 @@ public class RegionMergeTransaction {
this.region_a.initialize();
} catch (IOException e) {
LOG.error("Failed rollbacking CLOSED_REGION_A of region "
+ this.region_a.getRegionNameAsString(), e);
+ this.region_a.getRegionInfo().getRegionNameAsString(), e);
throw new RuntimeException(e);
}
break;
@ -603,7 +604,7 @@ public class RegionMergeTransaction {
this.region_b.initialize();
} catch (IOException e) {
LOG.error("Failed rollbacking CLOSED_REGION_A of region "
+ this.region_b.getRegionNameAsString(), e);
+ this.region_b.getRegionInfo().getRegionNameAsString(), e);
throw new RuntimeException(e);
}
break;

View File

@ -24,9 +24,11 @@ import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.master.TableLockManager;
@ -40,9 +42,9 @@ import com.google.protobuf.Service;
/**
* Services provided by {@link HRegionServer}
*/
@InterfaceAudience.Private
public interface RegionServerServices
extends OnlineRegions, FavoredNodesForRegion {
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegion {
/**
* @return True if this regionserver is stopping.
*/
@ -85,8 +87,7 @@ public interface RegionServerServices
* @throws KeeperException
* @throws IOException
*/
void postOpenDeployTasks(final HRegion r)
throws KeeperException, IOException;
void postOpenDeployTasks(final Region r) throws KeeperException, IOException;
/**
* Notify master that a handler requests to change a region state
@ -127,7 +128,7 @@ public interface RegionServerServices
/**
* @return set of recovering regions on the hosting region server
*/
Map<String, HRegion> getRecoveringRegions();
Map<String, Region> getRecoveringRegions();
/**
* Only required for "old" log replay; if it's removed, remove this.

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Map;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@ -74,11 +74,11 @@ public abstract class RegionSplitPolicy extends Configured {
if (explicitSplitPoint != null) {
return explicitSplitPoint;
}
Map<byte[], Store> stores = region.getStores();
List<Store> stores = region.getStores();
byte[] splitPointFromLargestStore = null;
long largestStoreSize = 0;
for (Store s : stores.values()) {
for (Store s : stores) {
byte[] splitPoint = s.getSplitPoint();
long storeSize = s.getSize();
if (splitPoint != null && largestStoreSize < storeSize) {

View File

@ -42,9 +42,9 @@ class SplitRequest implements Runnable {
private final HRegionServer server;
private TableLock tableLock;
SplitRequest(HRegion region, byte[] midKey, HRegionServer hrs) {
SplitRequest(Region region, byte[] midKey, HRegionServer hrs) {
Preconditions.checkNotNull(hrs);
this.parent = region;
this.parent = (HRegion)region;
this.midKey = midKey;
this.server = hrs;
}
@ -69,7 +69,7 @@ class SplitRequest implements Runnable {
//acquire a shared read lock on the table, so that table schema modifications
//do not happen concurrently
tableLock = server.getTableLockManager().readLock(parent.getTableDesc().getTableName()
, "SPLIT_REGION:" + parent.getRegionNameAsString());
, "SPLIT_REGION:" + parent.getRegionInfo().getRegionNameAsString());
try {
tableLock.acquire();
} catch (IOException ex) {
@ -87,22 +87,22 @@ class SplitRequest implements Runnable {
if (this.server.isStopping() || this.server.isStopped()) {
LOG.info(
"Skip rollback/cleanup of failed split of "
+ parent.getRegionNameAsString() + " because server is"
+ parent.getRegionInfo().getRegionNameAsString() + " because server is"
+ (this.server.isStopping() ? " stopping" : " stopped"), e);
return;
}
try {
LOG.info("Running rollback/cleanup of failed split of " +
parent.getRegionNameAsString() + "; " + e.getMessage(), e);
parent.getRegionInfo().getRegionNameAsString() + "; " + e.getMessage(), e);
if (st.rollback(this.server, this.server)) {
LOG.info("Successful rollback of failed split of " +
parent.getRegionNameAsString());
parent.getRegionInfo().getRegionNameAsString());
} else {
this.server.abort("Abort; we got an error after point-of-no-return");
}
} catch (RuntimeException ee) {
String msg = "Failed rollback of failed split of " +
parent.getRegionNameAsString() + " -- aborting server";
parent.getRegionInfo().getRegionNameAsString() + " -- aborting server";
// If failed rollback, kill this server to avoid having a hole in table.
LOG.info(msg, ee);
this.server.abort(msg + " -- Cause: " + ee.getMessage());
@ -133,7 +133,7 @@ class SplitRequest implements Runnable {
server.metricsRegionServer.incrSplitSuccess();
// Log success
LOG.info("Region split, hbase:meta updated, and report to master. Parent="
+ parent.getRegionNameAsString() + ", new regions: "
+ parent.getRegionInfo().getRegionNameAsString() + ", new regions: "
+ st.getFirstDaughter().getRegionNameAsString() + ", "
+ st.getSecondDaughter().getRegionNameAsString() + ". Split took "
+ StringUtils.formatTimeDiff(EnvironmentEdgeManager.currentTime(), startTime));
@ -151,7 +151,7 @@ class SplitRequest implements Runnable {
LOG.error("Could not release the table lock (something is really wrong). "
+ "Aborting this server to avoid holding the lock forever.");
this.server.abort("Abort; we got an error when releasing the table lock "
+ "on " + parent.getRegionNameAsString());
+ "on " + parent.getRegionInfo().getRegionNameAsString());
}
}
}

View File

@ -197,8 +197,8 @@ public class SplitTransaction {
* @param r Region to split
* @param splitrow Row to split around
*/
public SplitTransaction(final HRegion r, final byte [] splitrow) {
this.parent = r;
public SplitTransaction(final Region r, final byte [] splitrow) {
this.parent = (HRegion)r;
this.splitrow = splitrow;
this.journal.add(new JournalEntry(JournalEntryType.STARTED));
}
@ -259,7 +259,7 @@ public class SplitTransaction {
* Call {@link #rollback(Server, RegionServerServices)}
* @return Regions created
*/
/* package */PairOfSameType<HRegion> createDaughters(final Server server,
/* package */PairOfSameType<Region> createDaughters(final Server server,
final RegionServerServices services) throws IOException {
LOG.info("Starting split of region " + this.parent);
if ((server != null && server.isStopped()) ||
@ -287,14 +287,14 @@ public class SplitTransaction {
server.getConfiguration().getLong("hbase.regionserver.fileSplitTimeout",
this.fileSplitTimeout);
PairOfSameType<HRegion> daughterRegions = stepsBeforePONR(server, services, testing);
PairOfSameType<Region> daughterRegions = stepsBeforePONR(server, services, testing);
List<Mutation> metaEntries = new ArrayList<Mutation>();
if (this.parent.getCoprocessorHost() != null) {
if (this.parent.getCoprocessorHost().
preSplitBeforePONR(this.splitrow, metaEntries)) {
throw new IOException("Coprocessor bypassing region "
+ this.parent.getRegionNameAsString() + " split.");
+ this.parent.getRegionInfo().getRegionNameAsString() + " split.");
}
try {
for (Mutation p : metaEntries) {
@ -338,12 +338,12 @@ public class SplitTransaction {
return daughterRegions;
}
public PairOfSameType<HRegion> stepsBeforePONR(final Server server,
public PairOfSameType<Region> stepsBeforePONR(final Server server,
final RegionServerServices services, boolean testing) throws IOException {
if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_SPLIT,
parent.getRegionInfo(), hri_a, hri_b)) {
throw new IOException("Failed to get ok from master to split "
+ parent.getRegionNameAsString());
+ parent.getRegionInfo().getRegionNameAsString());
}
this.journal.add(new JournalEntry(JournalEntryType.SET_SPLITTING));
@ -392,7 +392,7 @@ public class SplitTransaction {
this.journal.add(new JournalEntry(JournalEntryType.STARTED_REGION_A_CREATION));
assertReferenceFileCount(expectedReferences.getFirst(),
this.parent.getRegionFileSystem().getSplitsDir(this.hri_a));
HRegion a = this.parent.createDaughterRegionFromSplits(this.hri_a);
Region a = this.parent.createDaughterRegionFromSplits(this.hri_a);
assertReferenceFileCount(expectedReferences.getFirst(),
new Path(this.parent.getRegionFileSystem().getTableDir(), this.hri_a.getEncodedName()));
@ -400,11 +400,11 @@ public class SplitTransaction {
this.journal.add(new JournalEntry(JournalEntryType.STARTED_REGION_B_CREATION));
assertReferenceFileCount(expectedReferences.getSecond(),
this.parent.getRegionFileSystem().getSplitsDir(this.hri_b));
HRegion b = this.parent.createDaughterRegionFromSplits(this.hri_b);
Region b = this.parent.createDaughterRegionFromSplits(this.hri_b);
assertReferenceFileCount(expectedReferences.getSecond(),
new Path(this.parent.getRegionFileSystem().getTableDir(), this.hri_b.getEncodedName()));
return new PairOfSameType<HRegion>(a, b);
return new PairOfSameType<Region>(a, b);
}
void assertReferenceFileCount(int expectedReferenceFileCount, Path dir)
@ -425,7 +425,7 @@ public class SplitTransaction {
* Call {@link #rollback(Server, RegionServerServices)}
*/
/* package */void openDaughters(final Server server,
final RegionServerServices services, HRegion a, HRegion b)
final RegionServerServices services, Region a, Region b)
throws IOException {
boolean stopped = server != null && server.isStopped();
boolean stopping = services != null && services.isStopping();
@ -438,8 +438,8 @@ public class SplitTransaction {
" because stopping=" + stopping + ", stopped=" + stopped);
} else {
// Open daughters in parallel.
DaughterOpener aOpener = new DaughterOpener(server, a);
DaughterOpener bOpener = new DaughterOpener(server, b);
DaughterOpener aOpener = new DaughterOpener(server, (HRegion)a);
DaughterOpener bOpener = new DaughterOpener(server, (HRegion)b);
aOpener.start();
bOpener.start();
try {
@ -485,18 +485,18 @@ public class SplitTransaction {
* @throws IOException
* @see #rollback(Server, RegionServerServices)
*/
public PairOfSameType<HRegion> execute(final Server server,
public PairOfSameType<Region> execute(final Server server,
final RegionServerServices services)
throws IOException {
PairOfSameType<HRegion> regions = createDaughters(server, services);
PairOfSameType<Region> regions = createDaughters(server, services);
if (this.parent.getCoprocessorHost() != null) {
this.parent.getCoprocessorHost().preSplitAfterPONR();
}
return stepsAfterPONR(server, services, regions);
}
public PairOfSameType<HRegion> stepsAfterPONR(final Server server,
final RegionServerServices services, PairOfSameType<HRegion> regions)
public PairOfSameType<Region> stepsAfterPONR(final Server server,
final RegionServerServices services, PairOfSameType<Region> regions)
throws IOException {
openDaughters(server, services, regions.getFirst(), regions.getSecond());
journal.add(new JournalEntry(JournalEntryType.BEFORE_POST_SPLIT_HOOK));
@ -749,7 +749,7 @@ public class SplitTransaction {
this.parent.initialize();
} catch (IOException e) {
LOG.error("Failed rollbacking CLOSED_PARENT_REGION of region " +
this.parent.getRegionNameAsString(), e);
this.parent.getRegionInfo().getRegionNameAsString(), e);
throw new RuntimeException(e);
}
break;

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
@ -48,7 +49,7 @@ import org.apache.hadoop.hbase.util.Pair;
* Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or
* more StoreFiles, which stretch backwards over time.
*/
@InterfaceAudience.Private
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface Store extends HeapSize, StoreConfigInformation, PropagatingConfigurationObserver {
@ -63,7 +64,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
Collection<StoreFile> getStorefiles();
/**
* Close all the readers We don't need to worry about subsequent requests because the HRegion
* Close all the readers We don't need to worry about subsequent requests because the Region
* holds a write lock that will prevent any more reads or writes.
* @return the {@link StoreFile StoreFiles} that were previously being used.
* @throws IOException on failure
@ -241,7 +242,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
void assertBulkLoadHFileOk(Path srcPath) throws IOException;
/**
* This method should only be called from HRegion. It is assumed that the ranges of values in the
* This method should only be called from Region. It is assumed that the ranges of values in the
* HFile fit within the stores assigned region. (assertBulkLoadHFileOk checks this)
*
* @param srcPathStr

View File

@ -84,8 +84,8 @@ public class StorefileRefresherChore extends ScheduledChore {
@Override
protected void chore() {
for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
if (!r.writestate.isReadOnly()) {
for (Region r : regionServer.getOnlineRegionsLocalContext()) {
if (!r.isReadOnly()) {
// skip checking for this region if it can accept writes
continue;
}
@ -98,7 +98,7 @@ public class StorefileRefresherChore extends ScheduledChore {
lastRefreshTimes.put(encodedName, time);
}
try {
for (Store store : r.getStores().values()) {
for (Store store : r.getStores()) {
// TODO: some stores might see new data from flush, while others do not which
// MIGHT break atomic edits across column families. We can fix this with setting
// mvcc read numbers that we know every store has seen
@ -110,12 +110,12 @@ public class StorefileRefresherChore extends ScheduledChore {
// Store files have a TTL in the archive directory. If we fail to refresh for that long, we stop serving reads
if (isRegionStale(encodedName, time)) {
r.setReadsEnabled(false); // stop serving reads
((HRegion)r).setReadsEnabled(false); // stop serving reads
}
continue;
}
lastRefreshTimes.put(encodedName, time);
r.setReadsEnabled(true); // restart serving reads
((HRegion)r).setReadsEnabled(true); // restart serving reads
}
// remove closed regions

View File

@ -90,7 +90,7 @@ public class CloseRegionHandler extends EventHandler {
LOG.debug("Processing close of " + name);
String encodedRegionName = regionInfo.getEncodedName();
// Check that this region is being served here
HRegion region = this.rsServices.getFromOnlineRegions(encodedRegionName);
HRegion region = (HRegion)rsServices.getFromOnlineRegions(encodedRegionName);
if (region == null) {
LOG.warn("Received CLOSE for region " + name + " but currently not serving - ignoring");
// TODO: do better than a simple warning
@ -119,7 +119,7 @@ public class CloseRegionHandler extends EventHandler {
rsServices.reportRegionStateTransition(TransitionCode.CLOSED, regionInfo);
// Done! Region is closed on this RS
LOG.debug("Closed " + region.getRegionNameAsString());
LOG.debug("Closed " + region.getRegionInfo().getRegionNameAsString());
} finally {
this.rsServices.getRegionsInTransitionInRS().
remove(this.regionInfo.getEncodedNameAsBytes(), Boolean.FALSE);

View File

@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
public class FinishRegionRecoveringHandler extends EventHandler {
@ -45,9 +46,9 @@ public class FinishRegionRecoveringHandler extends EventHandler {
@Override
public void process() throws IOException {
HRegion region = this.rss.getRecoveringRegions().remove(regionName);
Region region = this.rss.getRecoveringRegions().remove(regionName);
if (region != null) {
region.setRecovering(false);
((HRegion)region).setRecovering(false);
LOG.info(path + " deleted; " + regionName + " recovered.");
}
}

View File

@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.procedure.ProcedureMember;
import org.apache.hadoop.hbase.procedure.Subprocedure;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager.SnapshotSubprocedurePool;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
@ -45,14 +47,14 @@ import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
public class FlushSnapshotSubprocedure extends Subprocedure {
private static final Log LOG = LogFactory.getLog(FlushSnapshotSubprocedure.class);
private final List<HRegion> regions;
private final List<Region> regions;
private final SnapshotDescription snapshot;
private final SnapshotSubprocedurePool taskManager;
private boolean snapshotSkipFlush = false;
public FlushSnapshotSubprocedure(ProcedureMember member,
ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
List<HRegion> regions, SnapshotDescription snapshot,
List<Region> regions, SnapshotDescription snapshot,
SnapshotSubprocedurePool taskManager) {
super(member, snapshot.getName(), errorListener, wakeFrequency, timeout);
this.snapshot = snapshot;
@ -68,8 +70,8 @@ public class FlushSnapshotSubprocedure extends Subprocedure {
* Callable for adding files to snapshot manifest working dir. Ready for multithreading.
*/
private class RegionSnapshotTask implements Callable<Void> {
HRegion region;
RegionSnapshotTask(HRegion region) {
Region region;
RegionSnapshotTask(Region region) {
this.region = region;
}
@ -94,9 +96,9 @@ public class FlushSnapshotSubprocedure extends Subprocedure {
LOG.debug("take snapshot without flush memstore first");
} else {
LOG.debug("Flush Snapshotting region " + region.toString() + " started...");
region.flushcache();
region.flush(true);
}
region.addRegionToSnapshot(snapshot, monitor);
((HRegion)region).addRegionToSnapshot(snapshot, monitor);
if (snapshotSkipFlush) {
LOG.debug("... SkipFlush Snapshotting region " + region.toString() + " completed.");
} else {
@ -126,7 +128,7 @@ public class FlushSnapshotSubprocedure extends Subprocedure {
}
// Add all hfiles already existing in region.
for (HRegion region : regions) {
for (Region region : regions) {
// submit one task per region for parallelize by region.
taskManager.submitTask(new RegionSnapshotTask(region));
monitor.rethrowException();

View File

@ -49,8 +49,8 @@ import org.apache.hadoop.hbase.procedure.Subprocedure;
import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -160,7 +160,7 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
// check to see if this server is hosting any regions for the snapshots
// check to see if we have regions for the snapshot
List<HRegion> involvedRegions;
List<Region> involvedRegions;
try {
involvedRegions = getRegionsToSnapshot(snapshot);
} catch (IOException e1) {
@ -220,12 +220,12 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
* the given snapshot.
* @throws IOException
*/
private List<HRegion> getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException {
List<HRegion> onlineRegions = rss.getOnlineRegions(TableName.valueOf(snapshot.getTable()));
Iterator<HRegion> iterator = onlineRegions.iterator();
private List<Region> getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException {
List<Region> onlineRegions = rss.getOnlineRegions(TableName.valueOf(snapshot.getTable()));
Iterator<Region> iterator = onlineRegions.iterator();
// remove the non-default regions
while (iterator.hasNext()) {
HRegion r = iterator.next();
Region r = iterator.next();
if (!RegionReplicaUtil.isDefaultReplica(r.getRegionInfo())) {
iterator.remove();
}

View File

@ -61,9 +61,9 @@ import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
@ -328,7 +328,7 @@ public class AccessControlLists {
* Returns {@code true} if the given region is part of the {@code _acl_}
* metadata table.
*/
static boolean isAclRegion(HRegion region) {
static boolean isAclRegion(Region region) {
return ACL_TABLE_NAME.equals(region.getTableDesc().getTableName());
}
@ -347,8 +347,7 @@ public class AccessControlLists {
* @return a map of the permissions for this table.
* @throws IOException
*/
static Map<byte[], ListMultimap<String,TablePermission>> loadAll(
HRegion aclRegion)
static Map<byte[], ListMultimap<String,TablePermission>> loadAll(Region aclRegion)
throws IOException {
if (!isAclRegion(aclRegion)) {

View File

@ -86,10 +86,10 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest;
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.Store;
@ -199,7 +199,7 @@ public class AccessController extends BaseMasterAndRegionObserver
// This boolean having relevance only in the Master.
private volatile boolean aclTabAvailable = false;
public HRegion getRegion() {
public Region getRegion() {
return regionEnv != null ? regionEnv.getRegion() : null;
}
@ -208,7 +208,7 @@ public class AccessController extends BaseMasterAndRegionObserver
}
void initialize(RegionCoprocessorEnvironment e) throws IOException {
final HRegion region = e.getRegion();
final Region region = e.getRegion();
Configuration conf = e.getConfiguration();
Map<byte[], ListMultimap<String,TablePermission>> tables =
AccessControlLists.loadAll(region);
@ -1347,7 +1347,7 @@ public class AccessController extends BaseMasterAndRegionObserver
public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e)
throws IOException {
RegionCoprocessorEnvironment env = e.getEnvironment();
final HRegion region = env.getRegion();
final Region region = env.getRegion();
if (region == null) {
LOG.error("NULL region from RegionCoprocessorEnvironment in preOpen()");
} else {
@ -1363,7 +1363,7 @@ public class AccessController extends BaseMasterAndRegionObserver
@Override
public void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {
RegionCoprocessorEnvironment env = c.getEnvironment();
final HRegion region = env.getRegion();
final Region region = env.getRegion();
if (region == null) {
LOG.error("NULL region from RegionCoprocessorEnvironment in postOpen()");
return;
@ -1467,7 +1467,7 @@ public class AccessController extends BaseMasterAndRegionObserver
throw new RuntimeException("Unhandled operation " + opType);
}
AuthResult authResult = permissionGranted(opType, user, env, families, Action.READ);
HRegion region = getRegion(env);
Region region = getRegion(env);
TableName table = getTableName(region);
Map<ByteRange, Integer> cfVsMaxVersions = Maps.newHashMap();
for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) {
@ -2312,19 +2312,19 @@ public class AccessController extends BaseMasterAndRegionObserver
return AccessControlProtos.AccessControlService.newReflectiveService(this);
}
private HRegion getRegion(RegionCoprocessorEnvironment e) {
private Region getRegion(RegionCoprocessorEnvironment e) {
return e.getRegion();
}
private TableName getTableName(RegionCoprocessorEnvironment e) {
HRegion region = e.getRegion();
Region region = e.getRegion();
if (region != null) {
return getTableName(region);
}
return null;
}
private TableName getTableName(HRegion region) {
private TableName getTableName(Region region) {
HRegionInfo regionInfo = region.getRegionInfo();
if (regionInfo != null) {
return regionInfo.getTable();
@ -2428,31 +2428,31 @@ public class AccessController extends BaseMasterAndRegionObserver
}
@Override
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, HRegion regionA,
HRegion regionB) throws IOException {
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, Region regionA,
Region regionB) throws IOException {
requirePermission("mergeRegions", regionA.getTableDesc().getTableName(), null, null,
Action.ADMIN);
}
@Override
public void postMerge(ObserverContext<RegionServerCoprocessorEnvironment> c, HRegion regionA,
HRegion regionB, HRegion mergedRegion) throws IOException { }
public void postMerge(ObserverContext<RegionServerCoprocessorEnvironment> c, Region regionA,
Region regionB, Region mergedRegion) throws IOException { }
@Override
public void preMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB, List<Mutation> metaEntries) throws IOException { }
Region regionA, Region regionB, List<Mutation> metaEntries) throws IOException { }
@Override
public void postMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB, HRegion mergedRegion) throws IOException { }
Region regionA, Region regionB, Region mergedRegion) throws IOException { }
@Override
public void preRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB) throws IOException { }
Region regionA, Region regionB) throws IOException { }
@Override
public void postRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB) throws IOException { }
Region regionA, Region regionB) throws IOException { }
@Override
public void preRollWALWriterRequest(ObserverContext<RegionServerCoprocessorEnvironment> ctx)

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.security.access;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -49,7 +50,8 @@ import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBu
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadResponse;
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest;
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Region.BulkLoadListener;
import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
@ -236,7 +238,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
return;
}
HRegion region = env.getRegion();
Region region = env.getRegion();
boolean bypass = false;
if (region.getCoprocessorHost() != null) {
try {
@ -353,7 +355,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
return this;
}
private static class SecureBulkLoadListener implements HRegion.BulkLoadListener {
private static class SecureBulkLoadListener implements BulkLoadListener {
// Target filesystem
private FileSystem fs;
private String stagingDir;

View File

@ -44,6 +44,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagType;
@ -55,8 +56,8 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.util.StreamUtils;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.OperationStatus;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
@ -76,7 +77,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
private AtomicInteger ordinalCounter = new AtomicInteger(-1);
private Configuration conf;
private HRegion labelsRegion;
private Region labelsRegion;
private VisibilityLabelsCache labelsCache;
private List<ScanLabelGenerator> scanLabelGenerators;
private List<String> superUsers;
@ -196,7 +197,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
return new Pair<Map<String, Integer>, Map<String, List<Integer>>>(labels, userAuths);
}
protected void addSystemLabel(HRegion region, Map<String, Integer> labels,
protected void addSystemLabel(Region region, Map<String, Integer> labels,
Map<String, List<Integer>> userAuths) throws IOException {
if (!labels.containsKey(SYSTEM_LABEL)) {
Put p = new Put(Bytes.toBytes(SYSTEM_LABEL_ORDINAL));
@ -307,7 +308,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
private boolean mutateLabelsRegion(List<Mutation> mutations, OperationStatus[] finalOpStatus)
throws IOException {
OperationStatus[] opStatus = this.labelsRegion.batchMutate(mutations
.toArray(new Mutation[mutations.size()]));
.toArray(new Mutation[mutations.size()]), HConstants.NO_NONCE, HConstants.NO_NONCE);
int i = 0;
boolean updateZk = false;
for (OperationStatus status : opStatus) {

View File

@ -90,10 +90,10 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.Visibil
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.DeleteTracker;
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
import org.apache.hadoop.hbase.regionserver.OperationStatus;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
import org.apache.hadoop.hbase.security.AccessDeniedException;
@ -513,7 +513,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
if (!initialized) {
throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!");
}
HRegion region = e.getEnvironment().getRegion();
Region region = e.getEnvironment().getRegion();
Authorizations authorizations = null;
try {
authorizations = scan.getAuthorizations();
@ -547,7 +547,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
public DeleteTracker postInstantiateDeleteTracker(
ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
throws IOException {
HRegion region = ctx.getEnvironment().getRegion();
Region region = ctx.getEnvironment().getRegion();
TableName table = region.getRegionInfo().getTable();
if (table.isSystemTable()) {
return delTracker;
@ -611,7 +611,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
if (!initialized) {
throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!");
}
HRegion region = e.getEnvironment().getRegion();
Region region = e.getEnvironment().getRegion();
Authorizations authorizations = null;
try {
authorizations = get.getAuthorizations();

View File

@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUs
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
@ -308,7 +308,7 @@ public class VisibilityUtils {
return false;
}
public static Filter createVisibilityLabelFilter(HRegion region, Authorizations authorizations)
public static Filter createVisibilityLabelFilter(Region region, Authorizations authorizations)
throws IOException {
Map<ByteRange, Integer> cfVsMaxVersions = new HashMap<ByteRange, Integer>();
for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) {

View File

@ -167,7 +167,7 @@ public class SnapshotManifest {
// 2. iterate through all the stores in the region
LOG.debug("Creating references for hfiles");
for (Store store : region.getStores().values()) {
for (Store store : region.getStores()) {
// 2.1. build the snapshot reference for the store
Object familyData = visitor.familyOpen(regionData, store.getFamily().getName());
monitor.rethrowException();

View File

@ -60,7 +60,7 @@ public class WriteSinkCoprocessor extends BaseRegionObserver {
@Override
public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
regionName = e.getEnvironment().getRegion().getRegionNameAsString();
regionName = e.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString();
}

View File

@ -206,16 +206,17 @@ class HMerge {
if ((currentSize + nextSize) <= (maxFilesize / 2)) {
// We merge two adjacent regions if their total size is less than
// one half of the desired maximum size
LOG.info("Merging regions " + currentRegion.getRegionNameAsString() +
" and " + nextRegion.getRegionNameAsString());
LOG.info("Merging regions " + currentRegion.getRegionInfo().getRegionNameAsString() +
" and " + nextRegion.getRegionInfo().getRegionNameAsString());
HRegion mergedRegion =
HRegion.mergeAdjacent(currentRegion, nextRegion);
updateMeta(currentRegion.getRegionName(), nextRegion.getRegionName(),
mergedRegion);
updateMeta(currentRegion.getRegionInfo().getRegionName(),
nextRegion.getRegionInfo().getRegionName(), mergedRegion);
break;
}
LOG.info("not merging regions " + Bytes.toStringBinary(currentRegion.getRegionName())
+ " and " + Bytes.toStringBinary(nextRegion.getRegionName()));
LOG.info("not merging regions " +
Bytes.toStringBinary(currentRegion.getRegionInfo().getRegionName()) +
" and " + Bytes.toStringBinary(nextRegion.getRegionInfo().getRegionName()));
currentRegion.close();
currentRegion = nextRegion;
currentSize = nextSize;
@ -342,7 +343,7 @@ class HMerge {
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: "
+ Bytes.toStringBinary(newRegion.getRegionName()));
+ Bytes.toStringBinary(newRegion.getRegionInfo().getRegionName()));
}
}
}

View File

@ -173,11 +173,11 @@ public class Merge extends Configured implements Tool {
throws IOException {
if (info1 == null) {
throw new IOException("Could not find " + Bytes.toStringBinary(region1) + " in " +
Bytes.toStringBinary(meta.getRegionName()));
Bytes.toStringBinary(meta.getRegionInfo().getRegionName()));
}
if (info2 == null) {
throw new IOException("Could not find " + Bytes.toStringBinary(region2) + " in " +
Bytes.toStringBinary(meta.getRegionName()));
Bytes.toStringBinary(meta.getRegionInfo().getRegionName()));
}
HRegion merged = null;
HRegion r1 = HRegion.openHRegion(info1, htd, utils.getLog(info1), getConf());

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
@ -231,7 +232,7 @@ public abstract class HBaseTestCase extends TestCase {
* @throws IOException
* @return count of what we added.
*/
public static long addContent(final HRegion r, final byte [] columnFamily, final byte[] column)
public static long addContent(final Region r, final byte [] columnFamily, final byte[] column)
throws IOException {
byte [] startKey = r.getRegionInfo().getStartKey();
byte [] endKey = r.getRegionInfo().getEndKey();
@ -243,8 +244,7 @@ public abstract class HBaseTestCase extends TestCase {
startKeyBytes, endKey, -1);
}
public static long addContent(final HRegion r, final byte [] columnFamily)
throws IOException {
public static long addContent(final Region r, final byte [] columnFamily) throws IOException {
return addContent(r, columnFamily, null);
}
@ -440,6 +440,10 @@ public abstract class HBaseTestCase extends TestCase {
this.region = HRegion;
}
public HRegionIncommon(final Region region) {
this.region = (HRegion)region;
}
public void put(Put put) throws IOException {
region.put(put);
}
@ -470,7 +474,7 @@ public abstract class HBaseTestCase extends TestCase {
}
public void flushcache() throws IOException {
this.region.flushcache();
this.region.flush(true);
}
}

View File

@ -90,6 +90,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
@ -286,6 +287,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return htu;
}
/**
* Close both the region {@code r} and it's underlying WAL. For use in tests.
*/
public static void closeRegionAndWAL(final Region r) throws IOException {
closeRegionAndWAL((HRegion)r);
}
/**
* Close both the HRegion {@code r} and it's underlying WAL. For use in tests.
*/
@ -2131,6 +2139,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return loadRegion(r, f, false);
}
public int loadRegion(final Region r, final byte[] f) throws IOException {
return loadRegion((HRegion)r, f);
}
/**
* Load region with rows from 'aaa' to 'zzz'.
* @param r Region
@ -2152,8 +2164,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
Put put = new Put(k);
put.setDurability(Durability.SKIP_WAL);
put.add(f, null, k);
if (r.getWAL() == null) put.setDurability(Durability.SKIP_WAL);
if (r.getWAL() == null) {
put.setDurability(Durability.SKIP_WAL);
}
int preRowCount = rowCount;
int pause = 10;
int maxPause = 1000;
@ -2169,7 +2182,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
}
if (flush) {
r.flushcache();
r.flush(true);
}
}
return rowCount;
@ -2204,11 +2217,21 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
}
public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow)
throws IOException {
verifyNumericRows((HRegion)region, f, startRow, endRow);
}
public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow)
throws IOException {
verifyNumericRows(region, f, startRow, endRow, true);
}
public void verifyNumericRows(Region region, final byte[] f, int startRow, int endRow,
final boolean present) throws IOException {
verifyNumericRows((HRegion)region, f, startRow, endRow, present);
}
public void verifyNumericRows(HRegion region, final byte[] f, int startRow, int endRow,
final boolean present) throws IOException {
for (int i = startRow; i < endRow; i++) {
@ -3755,10 +3778,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
if (server.equals(rs.getServerName())) {
continue;
}
Collection<HRegion> hrs = rs.getOnlineRegionsLocalContext();
for (HRegion r: hrs) {
Collection<Region> hrs = rs.getOnlineRegionsLocalContext();
for (Region r: hrs) {
assertTrue("Region should not be double assigned",
r.getRegionId() != hri.getRegionId());
r.getRegionInfo().getRegionId() != hri.getRegionId());
}
}
return; // good, we are happy

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
@ -535,8 +536,8 @@ public class MiniHBaseCluster extends HBaseCluster {
public void flushcache() throws IOException {
for (JVMClusterUtil.RegionServerThread t:
this.hbaseCluster.getRegionServers()) {
for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
r.flushcache();
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
r.flush(true);
}
}
}
@ -548,9 +549,9 @@ public class MiniHBaseCluster extends HBaseCluster {
public void flushcache(TableName tableName) throws IOException {
for (JVMClusterUtil.RegionServerThread t:
this.hbaseCluster.getRegionServers()) {
for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
if(r.getTableDesc().getTableName().equals(tableName)) {
r.flushcache();
r.flush(true);
}
}
}
@ -563,8 +564,8 @@ public class MiniHBaseCluster extends HBaseCluster {
public void compact(boolean major) throws IOException {
for (JVMClusterUtil.RegionServerThread t:
this.hbaseCluster.getRegionServers()) {
for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
r.compactStores(major);
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
r.compact(major);
}
}
}
@ -576,9 +577,9 @@ public class MiniHBaseCluster extends HBaseCluster {
public void compact(TableName tableName, boolean major) throws IOException {
for (JVMClusterUtil.RegionServerThread t:
this.hbaseCluster.getRegionServers()) {
for(HRegion r: t.getRegionServer().getOnlineRegionsLocalContext()) {
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
if(r.getTableDesc().getTableName().equals(tableName)) {
r.compactStores(major);
r.compact(major);
}
}
}
@ -615,9 +616,9 @@ public class MiniHBaseCluster extends HBaseCluster {
List<HRegion> ret = new ArrayList<HRegion>();
for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
HRegionServer hrs = rst.getRegionServer();
for (HRegion region : hrs.getOnlineRegionsLocalContext()) {
for (Region region : hrs.getOnlineRegionsLocalContext()) {
if (region.getTableDesc().getTableName().equals(tableName)) {
ret.add(region);
ret.add((HRegion)region);
}
}
}
@ -643,8 +644,7 @@ public class MiniHBaseCluster extends HBaseCluster {
int count = 0;
for (JVMClusterUtil.RegionServerThread rst: getRegionServerThreads()) {
HRegionServer hrs = rst.getRegionServer();
HRegion metaRegion =
hrs.getOnlineRegion(regionName);
Region metaRegion = hrs.getOnlineRegion(regionName);
if (metaRegion != null) {
index = count;
break;
@ -662,7 +662,7 @@ public class MiniHBaseCluster extends HBaseCluster {
// should hold some regions. Please refer to #countServedRegions
// to see how we find out all regions.
HMaster master = getMaster();
HRegion region = master.getOnlineRegion(regionName);
Region region = master.getOnlineRegion(regionName);
if (region != null) {
return master.getServerName();
}
@ -712,9 +712,9 @@ public class MiniHBaseCluster extends HBaseCluster {
ArrayList<HRegion> ret = new ArrayList<HRegion>();
for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
HRegionServer hrs = rst.getRegionServer();
for (HRegion region : hrs.getOnlineRegions(tableName)) {
for (Region region : hrs.getOnlineRegions(tableName)) {
if (region.getTableDesc().getTableName().equals(tableName)) {
ret.add(region);
ret.add((HRegion)region);
}
}
}

View File

@ -40,9 +40,9 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
import org.apache.hadoop.hbase.regionserver.Leases;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
@ -59,7 +59,7 @@ import com.google.protobuf.Service;
*/
public class MockRegionServerServices implements RegionServerServices {
protected static final Log LOG = LogFactory.getLog(MockRegionServerServices.class);
private final Map<String, HRegion> regions = new HashMap<String, HRegion>();
private final Map<String, Region> regions = new HashMap<String, Region>();
private final ConcurrentSkipListMap<byte[], Boolean> rit =
new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
private HFileSystem hfs = null;
@ -90,17 +90,17 @@ public class MockRegionServerServices implements RegionServerServices {
}
@Override
public boolean removeFromOnlineRegions(HRegion r, ServerName destination) {
public boolean removeFromOnlineRegions(Region r, ServerName destination) {
return this.regions.remove(r.getRegionInfo().getEncodedName()) != null;
}
@Override
public HRegion getFromOnlineRegions(String encodedRegionName) {
public Region getFromOnlineRegions(String encodedRegionName) {
return this.regions.get(encodedRegionName);
}
@Override
public List<HRegion> getOnlineRegions(TableName tableName) throws IOException {
public List<Region> getOnlineRegions(TableName tableName) throws IOException {
return null;
}
@ -110,13 +110,12 @@ public class MockRegionServerServices implements RegionServerServices {
}
@Override
public void addToOnlineRegions(HRegion r) {
public void addToOnlineRegions(Region r) {
this.regions.put(r.getRegionInfo().getEncodedName(), r);
}
@Override
public void postOpenDeployTasks(HRegion r)
throws KeeperException, IOException {
public void postOpenDeployTasks(Region r) throws KeeperException, IOException {
addToOnlineRegions(r);
}
@ -258,7 +257,7 @@ public class MockRegionServerServices implements RegionServerServices {
}
@Override
public Map<String, HRegion> getRecoveringRegions() {
public Map<String, Region> getRecoveringRegions() {
// TODO Auto-generated method stub
return null;
}

View File

@ -31,8 +31,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -90,7 +90,7 @@ public class TestGlobalMemStoreSize {
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
globalMemStoreSize +=
server.getFromOnlineRegions(regionInfo.getEncodedName()).
getMemstoreSize().get();
getMemstoreSize();
}
assertEquals(server.getRegionServerAccounting().getGlobalMemstoreSize(),
globalMemStoreSize);
@ -104,7 +104,7 @@ public class TestGlobalMemStoreSize {
for (HRegionInfo regionInfo :
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
HRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName());
Region r = server.getFromOnlineRegions(regionInfo.getEncodedName());
flush(r, server);
}
LOG.info("Post flush on " + server.getServerName());
@ -120,14 +120,14 @@ public class TestGlobalMemStoreSize {
// our test was running....
for (HRegionInfo regionInfo :
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
HRegion r = server.getFromOnlineRegions(regionInfo.getEncodedName());
long l = r.getMemstoreSize().longValue();
Region r = server.getFromOnlineRegions(regionInfo.getEncodedName());
long l = r.getMemstoreSize();
if (l > 0) {
// Only meta could have edits at this stage. Give it another flush
// clear them.
assertTrue(regionInfo.isMetaRegion());
LOG.info(r.toString() + " " + l + ", reflushing");
r.flushcache();
r.flush(true);
}
}
}
@ -145,10 +145,10 @@ public class TestGlobalMemStoreSize {
* @param server
* @throws IOException
*/
private void flush(final HRegion r, final HRegionServer server)
private void flush(final Region r, final HRegionServer server)
throws IOException {
LOG.info("Flush " + r.toString() + " on " + server.getServerName() +
", " + r.flushcache() + ", size=" +
", " + r.flush(true) + ", size=" +
server.getRegionServerAccounting().getGlobalMemstoreSize());
}

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
@ -290,13 +291,14 @@ public class TestIOFencing {
long startWaitTime = System.currentTimeMillis();
while (compactingRegion.getEarliestFlushTimeForAllStores() <= lastFlushTime ||
compactingRegion.countStoreFiles() <= 1) {
LOG.info("Waiting for the region to flush " + compactingRegion.getRegionNameAsString());
LOG.info("Waiting for the region to flush " +
compactingRegion.getRegionInfo().getRegionNameAsString());
Thread.sleep(1000);
assertTrue("Timed out waiting for the region to flush",
System.currentTimeMillis() - startWaitTime < 30000);
}
assertTrue(compactingRegion.countStoreFiles() > 1);
final byte REGION_NAME[] = compactingRegion.getRegionName();
final byte REGION_NAME[] = compactingRegion.getRegionInfo().getRegionName();
LOG.info("Asking for compaction");
((HBaseAdmin)admin).majorCompact(TABLE_NAME.getName());
LOG.info("Waiting for compaction to be about to start");
@ -314,7 +316,7 @@ public class TestIOFencing {
Waiter.waitFor(c, 60000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
HRegion newRegion = newServer.getOnlineRegion(REGION_NAME);
Region newRegion = newServer.getOnlineRegion(REGION_NAME);
return newRegion != null && !newRegion.isRecovering();
}
});

View File

@ -42,8 +42,8 @@ import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -231,7 +231,7 @@ public class TestHFileArchiving {
List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(TABLE_NAME);
// make sure we only have 1 region serving this table
assertEquals(1, servingRegions.size());
HRegion region = servingRegions.get(0);
Region region = servingRegions.get(0);
// get the parent RS and monitor
HRegionServer hrs = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
@ -242,7 +242,7 @@ public class TestHFileArchiving {
UTIL.loadRegion(region, TEST_FAM);
// get the hfiles in the region
List<HRegion> regions = hrs.getOnlineRegions(TABLE_NAME);
List<Region> regions = hrs.getOnlineRegions(TABLE_NAME);
assertEquals("More that 1 region for test table.", 1, regions.size());
region = regions.get(0);
@ -257,7 +257,8 @@ public class TestHFileArchiving {
clearArchiveDirectory();
// then get the current store files
List<String> storeFiles = getRegionStoreFiles(region);
byte[][]columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]);
List<String> storeFiles = region.getStoreFileList(columns);
// then delete the table so the hfiles get archived
UTIL.deleteTable(TABLE_NAME);
@ -310,7 +311,7 @@ public class TestHFileArchiving {
List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(TABLE_NAME);
// make sure we only have 1 region serving this table
assertEquals(1, servingRegions.size());
HRegion region = servingRegions.get(0);
Region region = servingRegions.get(0);
// get the parent RS and monitor
HRegionServer hrs = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
@ -321,7 +322,7 @@ public class TestHFileArchiving {
UTIL.loadRegion(region, TEST_FAM);
// get the hfiles in the region
List<HRegion> regions = hrs.getOnlineRegions(TABLE_NAME);
List<Region> regions = hrs.getOnlineRegions(TABLE_NAME);
assertEquals("More that 1 region for test table.", 1, regions.size());
region = regions.get(0);
@ -336,7 +337,8 @@ public class TestHFileArchiving {
clearArchiveDirectory();
// then get the current store files
List<String> storeFiles = getRegionStoreFiles(region);
byte[][]columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]);
List<String> storeFiles = region.getStoreFileList(columns);
// then delete the table so the hfiles get archived
UTIL.getHBaseAdmin().deleteColumn(TABLE_NAME, TEST_FAM);
@ -449,19 +451,4 @@ public class TestHFileArchiving {
}
return fileNames;
}
private List<String> getRegionStoreFiles(final HRegion region) throws IOException {
Path regionDir = region.getRegionFileSystem().getRegionDir();
FileSystem fs = region.getRegionFileSystem().getFileSystem();
List<String> storeFiles = getAllFileNames(fs, regionDir);
// remove all the non-storefile named files for the region
for (int i = 0; i < storeFiles.size(); i++) {
String file = storeFiles.get(i);
if (file.contains(HRegionFileSystem.REGION_INFO_FILE) || file.contains("wal")) {
storeFiles.remove(i--);
}
}
storeFiles.remove(HRegionFileSystem.REGION_INFO_FILE);
return storeFiles;
}
}

View File

@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
@ -170,7 +170,7 @@ public class TestZooKeeperTableArchiveClient {
// create the region
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM);
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
Region region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
loadFlushAndCompact(region, TEST_FAM);
@ -220,12 +220,12 @@ public class TestZooKeeperTableArchiveClient {
// create the region
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM);
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
Region region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
loadFlushAndCompact(region, TEST_FAM);
// create the another table that we don't archive
hcd = new HColumnDescriptor(TEST_FAM);
HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd);
Region otherRegion = UTIL.createTestRegion(otherTable, hcd);
loadFlushAndCompact(otherRegion, TEST_FAM);
// get the current hfiles in the archive directory
@ -379,7 +379,7 @@ public class TestZooKeeperTableArchiveClient {
return allFiles;
}
private void loadFlushAndCompact(HRegion region, byte[] family) throws IOException {
private void loadFlushAndCompact(Region region, byte[] family) throws IOException {
// create two hfiles in the region
createHFileInRegion(region, family);
createHFileInRegion(region, family);
@ -391,7 +391,7 @@ public class TestZooKeeperTableArchiveClient {
// compact the two files into one file to get files in the archive
LOG.debug("Compacting stores");
region.compactStores(true);
region.compact(true);
}
/**
@ -400,13 +400,13 @@ public class TestZooKeeperTableArchiveClient {
* @param columnFamily family for which to add data
* @throws IOException
*/
private void createHFileInRegion(HRegion region, byte[] columnFamily) throws IOException {
private void createHFileInRegion(Region region, byte[] columnFamily) throws IOException {
// put one row in the region
Put p = new Put(Bytes.toBytes("row"));
p.add(columnFamily, Bytes.toBytes("Qual"), Bytes.toBytes("v1"));
region.put(p);
// flush the region to make a store file
region.flushcache();
region.flush(true);
}
/**

View File

@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
@ -53,6 +52,7 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -545,11 +545,8 @@ public class TestAdmin2 {
+ DefaultWALProvider.getNumRolledLogFiles(regionServer.getWAL(null)) + " log files");
// flush all regions
List<HRegion> regions = new ArrayList<HRegion>(regionServer
.getOnlineRegionsLocalContext());
for (HRegion r : regions) {
r.flushcache();
for (Region r : regionServer.getOnlineRegionsLocalContext()) {
r.flush(true);
}
admin.rollWALWriter(regionServer.getServerName());
int count = DefaultWALProvider.getNumRolledLogFiles(regionServer.getWAL(null));

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -91,7 +92,7 @@ public class TestClientPushback {
HTable table = (HTable) conn.getTable(tablename);
HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
HRegion region = rs.getOnlineRegions(tablename).get(0);
Region region = rs.getOnlineRegions(tablename).get(0);
LOG.debug("Writing some data to "+tablename);
// write some data
@ -101,7 +102,7 @@ public class TestClientPushback {
table.flushCommits();
// get the current load on RS. Hopefully memstore isn't flushed since we wrote the the data
int load = (int)((region.addAndGetGlobalMemstoreSize(0) * 100) / flushSizeBytes);
int load = (int)((((HRegion)region).addAndGetGlobalMemstoreSize(0) * 100) / flushSizeBytes);
LOG.debug("Done writing some data to "+tablename);
// get the stats for the region hosting our table
@ -114,7 +115,7 @@ public class TestClientPushback {
assertNotNull( "No stats configured for the client!", stats);
// get the names so we can query the stats
ServerName server = rs.getServerName();
byte[] regionName = region.getRegionName();
byte[] regionName = region.getRegionInfo().getRegionName();
// check to see we found some load on the memstore
ServerStatistics serverStats = stats.getServerStatsForTesting(server);
@ -125,8 +126,8 @@ public class TestClientPushback {
// check that the load reported produces a nonzero delay
long backoffTime = backoffPolicy.getBackoffTime(server, regionName, serverStats);
assertNotEquals("Reported load does not produce a backoff", backoffTime, 0);
LOG.debug("Backoff calculated for " + region.getRegionNameAsString() + " @ " + server +
" is " + backoffTime);
LOG.debug("Backoff calculated for " + region.getRegionInfo().getRegionNameAsString() + " @ " +
server + " is " + backoffTime);
// Reach into the connection and submit work directly to AsyncProcess so we can
// monitor how long the submission was delayed via a callback

View File

@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateR
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -4180,7 +4181,7 @@ public class TestFromClientSide {
// set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
// in Store.rowAtOrBeforeFromStoreFile
String regionName = table.getRegionLocations().firstKey().getEncodedName();
HRegion region =
Region region =
TEST_UTIL.getRSForFirstRegionInTable(tableAname).getFromOnlineRegions(regionName);
Put put1 = new Put(firstRow);
Put put2 = new Put(secondRow);
@ -4199,7 +4200,7 @@ public class TestFromClientSide {
table.put(put2);
table.put(put3);
table.put(put4);
region.flushcache();
region.flush(true);
Result result = null;
// Test before first that null is returned
@ -5051,8 +5052,9 @@ public class TestFromClientSide {
HTable table = TEST_UTIL.createTable(tableName, FAMILY);
// get the block cache and region
String regionName = table.getRegionLocations().firstKey().getEncodedName();
HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
Store store = region.getStores().values().iterator().next();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName)
.getFromOnlineRegions(regionName);
Store store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true);
@ -5087,7 +5089,7 @@ public class TestFromClientSide {
assertEquals(startBlockMiss, cache.getStats().getMissCount());
// flush the data
System.out.println("Flushing cache");
region.flushcache();
region.flush(true);
// expect one more block in cache, no change in hits/misses
long expectedBlockCount = startBlockCount + 1;
long expectedBlockHits = startBlockHits;
@ -5114,7 +5116,7 @@ public class TestFromClientSide {
assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
// flush, one new block
System.out.println("Flushing cache");
region.flushcache();
region.flush(true);
assertEquals(++expectedBlockCount, cache.getBlockCount());
assertEquals(expectedBlockHits, cache.getStats().getHitCount());
assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
@ -5122,7 +5124,7 @@ public class TestFromClientSide {
System.out.println("Compacting");
assertEquals(2, store.getStorefilesCount());
store.triggerMajorCompaction();
region.compactStores();
region.compact(true);
waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max
assertEquals(1, store.getStorefilesCount());
expectedBlockCount -= 2; // evicted two blocks, cached none

View File

@ -64,8 +64,8 @@ import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterBase;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -989,9 +989,9 @@ public class TestHCM {
ServerName destServerName = destServer.getServerName();
//find another row in the cur server that is less than ROW_X
List<HRegion> regions = curServer.getOnlineRegions(TABLE_NAME3);
List<Region> regions = curServer.getOnlineRegions(TABLE_NAME3);
byte[] otherRow = null;
for (HRegion region : regions) {
for (Region region : regions) {
if (!region.getRegionInfo().getEncodedName().equals(toMove.getRegionInfo().getEncodedName())
&& Bytes.BYTES_COMPARATOR.compare(region.getRegionInfo().getStartKey(), ROW_X) < 0) {
otherRow = region.getRegionInfo().getStartKey();

View File

@ -33,9 +33,9 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest;
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.RpcCallback;
@ -87,9 +87,9 @@ implements Coprocessor, CoprocessorService {
int sumResult = 0;
InternalScanner scanner = null;
try {
HRegion region = this.env.getRegion();
Region region = this.env.getRegion();
// for the last region in the table, return null to test null handling
if (Bytes.equals(region.getEndKey(), HConstants.EMPTY_END_ROW)) {
if (Bytes.equals(region.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW)) {
done.run(null);
return;
}
@ -123,6 +123,6 @@ implements Coprocessor, CoprocessorService {
}
done.run(SumResponse.newBuilder().setSum(sumResult).build());
LOG.info("Returning sum " + sumResult + " for region " +
Bytes.toStringBinary(env.getRegion().getRegionName()));
Bytes.toStringBinary(env.getRegion().getRegionInfo().getRegionName()));
}
}

View File

@ -34,9 +34,9 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest;
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.RpcCallback;
@ -88,9 +88,9 @@ implements Coprocessor, CoprocessorService {
int sumResult = 0;
InternalScanner scanner = null;
try {
HRegion region = this.env.getRegion();
Region region = this.env.getRegion();
// throw an exception for requests to the last region in the table, to test error handling
if (Bytes.equals(region.getEndKey(), HConstants.EMPTY_END_ROW)) {
if (Bytes.equals(region.getRegionInfo().getEndKey(), HConstants.EMPTY_END_ROW)) {
throw new DoNotRetryIOException("An expected exception");
}
scanner = region.getScanner(scan);

View File

@ -54,12 +54,12 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.Leases;
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Region.Operation;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.Store;
@ -151,8 +151,8 @@ public class SimpleRegionObserver extends BaseRegionObserver {
// from external packages
RegionCoprocessorEnvironment re = (RegionCoprocessorEnvironment)e;
Leases leases = re.getRegionServerServices().getLeases();
leases.createLease(re.getRegion().getRegionNameAsString(), 2000, null);
leases.cancelLease(re.getRegion().getRegionNameAsString());
leases.createLease(re.getRegion().getRegionInfo().getRegionNameAsString(), 2000, null);
leases.cancelLease(re.getRegion().getRegionInfo().getRegionNameAsString());
}
@Override
@ -229,7 +229,7 @@ public class SimpleRegionObserver extends BaseRegionObserver {
}
@Override
public void postSplit(ObserverContext<RegionCoprocessorEnvironment> c, HRegion l, HRegion r) {
public void postSplit(ObserverContext<RegionCoprocessorEnvironment> c, Region l, Region r) {
ctPostSplit.incrementAndGet();
}

View File

@ -20,11 +20,10 @@ package org.apache.hadoop.hbase.coprocessor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.TestServerCustomProtocol;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -163,12 +162,12 @@ public class TestClassLoading {
// verify that the coprocessors were loaded
boolean foundTableRegion=false;
boolean found1 = true, found2 = true, found2_k1 = true, found2_k2 = true, found2_k3 = true;
Map<HRegion, Set<ClassLoader>> regionsActiveClassLoaders =
new HashMap<HRegion, Set<ClassLoader>>();
Map<Region, Set<ClassLoader>> regionsActiveClassLoaders =
new HashMap<Region, Set<ClassLoader>>();
MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster();
for (HRegion region:
for (Region region:
hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionNameAsString().startsWith(tableName.getNameAsString())) {
if (region.getRegionInfo().getRegionNameAsString().startsWith(tableName.getNameAsString())) {
foundTableRegion = true;
CoprocessorEnvironment env;
env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName1);
@ -206,7 +205,7 @@ public class TestClassLoading {
//check if region active classloaders are shared across all RS regions
Set<ClassLoader> externalClassLoaders = new HashSet<ClassLoader>(
CoprocessorClassLoader.getAllCached());
for (Map.Entry<HRegion, Set<ClassLoader>> regionCP : regionsActiveClassLoaders.entrySet()) {
for (Map.Entry<Region, Set<ClassLoader>> regionCP : regionsActiveClassLoaders.entrySet()) {
assertTrue("Some CP classloaders for region " + regionCP.getKey() + " are not cached."
+ " ClassLoader Cache:" + externalClassLoaders
+ " Region ClassLoaders:" + regionCP.getValue(),
@ -235,9 +234,8 @@ public class TestClassLoading {
// verify that the coprocessor was loaded
boolean found = false;
MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster();
for (HRegion region:
hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionNameAsString().startsWith(cpName3)) {
for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionInfo().getRegionNameAsString().startsWith(cpName3)) {
found = (region.getCoprocessorHost().findCoprocessor(cpName3) != null);
}
}
@ -261,9 +259,8 @@ public class TestClassLoading {
// verify that the coprocessor was loaded correctly
boolean found = false;
MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster();
for (HRegion region:
hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionNameAsString().startsWith(cpName4)) {
for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionInfo().getRegionNameAsString().startsWith(cpName4)) {
Coprocessor cp = region.getCoprocessorHost().findCoprocessor(cpName4);
if (cp != null) {
found = true;
@ -333,9 +330,8 @@ public class TestClassLoading {
found6_k4 = false;
MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster();
for (HRegion region:
hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionNameAsString().startsWith(tableName.getNameAsString())) {
for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionInfo().getRegionNameAsString().startsWith(tableName.getNameAsString())) {
found_1 = found_1 ||
(region.getCoprocessorHost().findCoprocessor(cpName1) != null);
found_2 = found_2 ||
@ -422,9 +418,8 @@ public class TestClassLoading {
boolean found1 = false, found2 = false, found2_k1 = false,
found2_k2 = false, found2_k3 = false;
MiniHBaseCluster hbase = TEST_UTIL.getHBaseCluster();
for (HRegion region:
hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionNameAsString().startsWith(tableName.getNameAsString())) {
for (Region region: hbase.getRegionServer(0).getOnlineRegionsLocalContext()) {
if (region.getRegionInfo().getRegionNameAsString().startsWith(tableName.getNameAsString())) {
CoprocessorEnvironment env;
env = region.getCoprocessorHost().findCoprocessorEnvironment(cpName1);
if (env != null) {

View File

@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
@ -231,7 +232,7 @@ public class TestCoprocessorInterface {
preSplitWithSplitRowCalled = true;
}
@Override
public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, HRegion l, HRegion r) {
public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, Region l, Region r) {
postSplitCalled = true;
}
@ -297,20 +298,19 @@ public class TestCoprocessorInterface {
byte [][] families = { fam1, fam2, fam3 };
Configuration hc = initSplit();
HRegion region = initHRegion(tableName, name.getMethodName(), hc,
Region region = initHRegion(tableName, name.getMethodName(), hc,
new Class<?>[]{}, families);
for (int i = 0; i < 3; i++) {
HBaseTestCase.addContent(region, fam3);
region.flushcache();
region.flush(true);
}
region.compactStores();
byte [] splitRow = region.checkSplit();
region.compact(false);
byte [] splitRow = ((HRegion)region).checkSplit();
assertNotNull(splitRow);
HRegion [] regions = split(region, splitRow);
Region [] regions = split(region, splitRow);
for (int i = 0; i < regions.length; i++) {
regions[i] = reopenRegion(regions[i], CoprocessorImpl.class, CoprocessorII.class);
}
@ -336,7 +336,7 @@ public class TestCoprocessorInterface {
// now have all Environments fail
for (int i = 0; i < regions.length; i++) {
try {
byte [] r = regions[i].getStartKey();
byte [] r = regions[i].getRegionInfo().getStartKey();
if (r == null || r.length <= 0) {
// Its the start row. Can't ask for null. Ask for minimal key instead.
r = new byte [] {0};
@ -376,19 +376,19 @@ public class TestCoprocessorInterface {
byte [][] families = { fam1, fam2, fam3 };
Configuration hc = initSplit();
HRegion region = initHRegion(tableName, name.getMethodName(), hc,
Region region = initHRegion(tableName, name.getMethodName(), hc,
new Class<?>[]{CoprocessorImpl.class}, families);
for (int i = 0; i < 3; i++) {
HBaseTestCase.addContent(region, fam3);
region.flushcache();
region.flush(true);
}
region.compactStores();
region.compact(false);
byte [] splitRow = region.checkSplit();
byte [] splitRow = ((HRegion)region).checkSplit();
assertNotNull(splitRow);
HRegion [] regions = split(region, splitRow);
Region [] regions = split(region, splitRow);
for (int i = 0; i < regions.length; i++) {
regions[i] = reopenRegion(regions[i], CoprocessorImpl.class);
}
@ -423,10 +423,10 @@ public class TestCoprocessorInterface {
}
}
HRegion reopenRegion(final HRegion closedRegion, Class<?> ... implClasses)
Region reopenRegion(final Region closedRegion, Class<?> ... implClasses)
throws IOException {
//HRegionInfo info = new HRegionInfo(tableName, null, null, false);
HRegion r = HRegion.openHRegion(closedRegion, null);
Region r = HRegion.openHRegion(closedRegion, null);
// this following piece is a hack. currently a coprocessorHost
// is secretly loaded at OpenRegionHandler. we don't really
@ -434,7 +434,7 @@ public class TestCoprocessorInterface {
// and set it to region.
Configuration conf = TEST_UTIL.getConfiguration();
RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf);
r.setCoprocessorHost(host);
((HRegion)r).setCoprocessorHost(host);
for (Class<?> implClass : implClasses) {
host.load(implClass, Coprocessor.PRIORITY_USER, conf);
@ -450,7 +450,7 @@ public class TestCoprocessorInterface {
return r;
}
HRegion initHRegion (TableName tableName, String callingMethod,
Region initHRegion (TableName tableName, String callingMethod,
Configuration conf, Class<?> [] implClasses, byte [][] families)
throws IOException {
HTableDescriptor htd = new HTableDescriptor(tableName);
@ -459,11 +459,11 @@ public class TestCoprocessorInterface {
}
HRegionInfo info = new HRegionInfo(tableName, null, null, false);
Path path = new Path(DIR + callingMethod);
HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);
Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);
// this following piece is a hack.
RegionCoprocessorHost host = new RegionCoprocessorHost(r, null, conf);
r.setCoprocessorHost(host);
((HRegion)r).setCoprocessorHost(host);
for (Class<?> implClass : implClasses) {
host.load(implClass, Coprocessor.PRIORITY_USER, conf);
@ -497,10 +497,8 @@ public class TestCoprocessorInterface {
return TEST_UTIL.getConfiguration();
}
private HRegion [] split(final HRegion r, final byte [] splitRow)
throws IOException {
HRegion[] regions = new HRegion[2];
private Region [] split(final Region r, final byte [] splitRow) throws IOException {
Region[] regions = new Region[2];
SplitTransaction st = new SplitTransaction(r, splitRow);
int i = 0;
@ -513,18 +511,18 @@ public class TestCoprocessorInterface {
Server mockServer = Mockito.mock(Server.class);
when(mockServer.getConfiguration()).thenReturn(
TEST_UTIL.getConfiguration());
PairOfSameType<HRegion> daughters = st.execute(mockServer, null);
for (HRegion each_daughter: daughters) {
PairOfSameType<Region> daughters = st.execute(mockServer, null);
for (Region each_daughter: daughters) {
regions[i] = each_daughter;
i++;
}
} catch (IOException ioe) {
LOG.info("Split transaction of " + r.getRegionNameAsString() +
LOG.info("Split transaction of " + r.getRegionInfo().getRegionNameAsString() +
" failed:" + ioe.getMessage());
assertTrue(false);
} catch (RuntimeException e) {
LOG.info("Failed rollback of failed split of " +
r.getRegionNameAsString() + e.getMessage());
r.getRegionInfo().getRegionNameAsString() + e.getMessage());
}
assertTrue(i == 2);

View File

@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.ScanType;
@ -143,7 +144,7 @@ public class TestRegionObserverScannerOpenHook {
}
}
HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
Region initHRegion(byte[] tableName, String callingMethod, Configuration conf,
byte[]... families) throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
for (byte[] family : families) {
@ -170,7 +171,7 @@ public class TestRegionObserverScannerOpenHook {
byte[][] FAMILIES = new byte[][] { A };
Configuration conf = HBaseConfiguration.create();
HRegion region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
Region region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
RegionCoprocessorHost h = region.getCoprocessorHost();
h.load(NoDataFromScan.class, Coprocessor.PRIORITY_HIGHEST, conf);
h.load(EmptyRegionObsever.class, Coprocessor.PRIORITY_USER, conf);
@ -195,7 +196,7 @@ public class TestRegionObserverScannerOpenHook {
byte[][] FAMILIES = new byte[][] { A };
Configuration conf = HBaseConfiguration.create();
HRegion region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
Region region = initHRegion(TABLE, getClass().getName(), conf, FAMILIES);
RegionCoprocessorHost h = region.getCoprocessorHost();
h.load(NoDataFromFlush.class, Coprocessor.PRIORITY_HIGHEST, conf);
h.load(EmptyRegionObsever.class, Coprocessor.PRIORITY_USER, conf);
@ -204,7 +205,7 @@ public class TestRegionObserverScannerOpenHook {
Put put = new Put(ROW);
put.add(A, A, A);
region.put(put);
region.flushcache();
region.flush(true);
Get get = new Get(ROW);
Result r = region.get(get);
assertNull(
@ -272,10 +273,10 @@ public class TestRegionObserverScannerOpenHook {
table.put(put);
HRegionServer rs = UTIL.getRSForFirstRegionInTable(desc.getTableName());
List<HRegion> regions = rs.getOnlineRegions(desc.getTableName());
List<Region> regions = rs.getOnlineRegions(desc.getTableName());
assertEquals("More than 1 region serving test table with 1 row", 1, regions.size());
HRegion region = regions.get(0);
admin.flushRegion(region.getRegionName());
Region region = regions.get(0);
admin.flushRegion(region.getRegionInfo().getRegionName());
CountDownLatch latch = ((CompactionCompletionNotifyingRegion)region)
.getCompactionStateChangeLatch();
@ -283,7 +284,7 @@ public class TestRegionObserverScannerOpenHook {
put = new Put(Bytes.toBytes("anotherrow"));
put.add(A, A, A);
table.put(put);
admin.flushRegion(region.getRegionName());
admin.flushRegion(region.getRegionInfo().getRegionName());
// run a compaction, which normally would should get rid of the data
// wait for the compaction checker to complete

View File

@ -36,10 +36,10 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@ -90,7 +90,7 @@ public class TestRegionServerObserver {
desc.addFamily(new HColumnDescriptor(FAM));
admin.createTable(desc, new byte[][] { Bytes.toBytes("row") });
assertFalse(regionServerObserver.wasRegionMergeCalled());
List<HRegion> regions = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME));
List<Region> regions = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME));
admin.mergeRegions(regions.get(0).getRegionInfo().getEncodedNameAsBytes(), regions.get(1)
.getRegionInfo().getEncodedNameAsBytes(), true);
int regionsCount = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)).size();
@ -130,18 +130,18 @@ public class TestRegionServerObserver {
}
@Override
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, HRegion regionA,
HRegion regionB) throws IOException {
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, Region regionA,
Region regionB) throws IOException {
preMergeCalled = true;
}
@Override
public void preMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB, List<Mutation> metaEntries) throws IOException {
Region regionA, Region regionB, List<Mutation> metaEntries) throws IOException {
preMergeBeforePONRCalled = true;
RegionServerCoprocessorEnvironment environment = ctx.getEnvironment();
HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
List<HRegion> onlineRegions =
List<Region> onlineRegions =
rs.getOnlineRegions(TableName.valueOf("testRegionServerObserver_2"));
rmt = new RegionMergeTransaction(onlineRegions.get(0), onlineRegions.get(1), true);
if (!rmt.prepare(rs)) {
@ -159,7 +159,7 @@ public class TestRegionServerObserver {
@Override
public void postMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB, HRegion mr) throws IOException {
Region regionA, Region regionB, Region mr) throws IOException {
preMergeAfterPONRCalled = true;
RegionServerCoprocessorEnvironment environment = ctx.getEnvironment();
HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
@ -168,19 +168,19 @@ public class TestRegionServerObserver {
@Override
public void preRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB) throws IOException {
Region regionA, Region regionB) throws IOException {
preRollBackMergeCalled = true;
}
@Override
public void postRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB) throws IOException {
Region regionA, Region regionB) throws IOException {
postRollBackMergeCalled = true;
}
@Override
public void postMerge(ObserverContext<RegionServerCoprocessorEnvironment> c, HRegion regionA,
HRegion regionB, HRegion mergedRegion) throws IOException {
public void postMerge(ObserverContext<RegionServerCoprocessorEnvironment> c, Region regionA,
Region regionB, Region mergedRegion) throws IOException {
postMergeCalled = true;
}

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.filter.FilterList.Operator;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.testclassification.FilterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@ -67,7 +68,7 @@ import com.google.common.base.Throwables;
@Category({FilterTests.class, SmallTests.class})
public class TestFilter {
private final static Log LOG = LogFactory.getLog(TestFilter.class);
private HRegion region;
private Region region;
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
//
@ -165,7 +166,7 @@ public class TestFilter {
}
// Flush
this.region.flushcache();
this.region.flush(true);
// Insert second half (reverse families)
for(byte [] ROW : ROWS_ONE) {
@ -242,7 +243,7 @@ public class TestFilter {
this.region.put(p);
}
// Flush
this.region.flushcache();
this.region.flush(true);
// Insert second half (reverse families)
for (byte[] ROW : ROWS_THREE) {
@ -1451,7 +1452,7 @@ public class TestFilter {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("TestFilter"));
htd.addFamily(new HColumnDescriptor(family));
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
Region testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
TEST_UTIL.getConfiguration(), htd);
for(int i=0; i<5; i++) {
@ -1460,7 +1461,7 @@ public class TestFilter {
p.add(family, qualifier, Bytes.toBytes(String.valueOf(111+i)));
testRegion.put(p);
}
testRegion.flushcache();
testRegion.flush(true);
// rows starting with "b"
PrefixFilter pf = new PrefixFilter(new byte[] {'b'}) ;
@ -1486,8 +1487,8 @@ public class TestFilter {
assertEquals(2, resultCount);
scanner.close();
WAL wal = testRegion.getWAL();
testRegion.close();
WAL wal = ((HRegion)testRegion).getWAL();
((HRegion)testRegion).close();
wal.close();
}
@ -1820,7 +1821,7 @@ public class TestFilter {
p.setDurability(Durability.SKIP_WAL);
p.add(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]);
this.region.put(p);
this.region.flushcache();
this.region.flush(true);
// Set of KVs (page: 1; pageSize: 1) - the first set of 1 column per row
KeyValue [] expectedKVs = {
@ -2011,7 +2012,7 @@ public class TestFilter {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testNestedFilterListWithSCVF"));
htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
Region testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
TEST_UTIL.getConfiguration(), htd);
for(int i=0; i<10; i++) {
Put p = new Put(Bytes.toBytes("row" + i));
@ -2019,7 +2020,7 @@ public class TestFilter {
p.add(FAMILIES[0], columnStatus, Bytes.toBytes(i%2));
testRegion.put(p);
}
testRegion.flushcache();
testRegion.flush(true);
// 1. got rows > "row4"
Filter rowFilter = new RowFilter(CompareOp.GREATER,new BinaryComparator(Bytes.toBytes("row4")));
Scan s1 = new Scan();
@ -2095,8 +2096,8 @@ public class TestFilter {
results.clear();
}
assertFalse(NextState.hasMoreValues(scanner.next(results)));
WAL wal = testRegion.getWAL();
testRegion.close();
WAL wal = ((HRegion)testRegion).getWAL();
((HRegion)testRegion).close();
wal.close();
}
}

View File

@ -35,10 +35,11 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.testclassification.FilterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -61,7 +62,7 @@ public class TestInvocationRecordFilter {
private static final String VALUE_PREFIX = "value";
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private HRegion region;
private Region region;
@Before
public void setUp() throws Exception {
@ -79,7 +80,7 @@ public class TestInvocationRecordFilter {
Bytes.toBytes(VALUE_PREFIX + i));
}
this.region.put(put);
this.region.flushcache();
this.region.flush(true);
}
@Test
@ -151,8 +152,8 @@ public class TestInvocationRecordFilter {
@After
public void tearDown() throws Exception {
WAL wal = region.getWAL();
region.close();
WAL wal = ((HRegion)region).getWAL();
((HRegion)region).close();
wal.close();
}

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -285,7 +286,7 @@ public class TestBlockReorder {
int nbTest = 0;
while (nbTest < 10) {
final List<HRegion> regions = targetRs.getOnlineRegions(h.getName());
final List<Region> regions = targetRs.getOnlineRegions(h.getName());
final CountDownLatch latch = new CountDownLatch(regions.size());
// listen for successful log rolls
final WALActionsListener listener = new WALActionsListener.Base() {
@ -294,8 +295,8 @@ public class TestBlockReorder {
latch.countDown();
}
};
for (HRegion region : regions) {
region.getWAL().registerWALActionsListener(listener);
for (Region region : regions) {
((HRegion)region).getWAL().registerWALActionsListener(listener);
}
htu.getHBaseAdmin().rollWALWriter(targetRs.getServerName());
@ -308,8 +309,8 @@ public class TestBlockReorder {
"tests fail, it's probably because we should still be waiting.");
Thread.currentThread().interrupt();
}
for (HRegion region : regions) {
region.getWAL().unregisterWALActionsListener(listener);
for (Region region : regions) {
((HRegion)region).getWAL().unregisterWALActionsListener(listener);
}
// We need a sleep as the namenode is informed asynchronously

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -113,7 +113,7 @@ public class TestEncodedSeekers {
setBlocksize(BLOCK_SIZE).
setBloomFilterType(BloomType.NONE).
setCompressTags(compressTags);
HRegion region = testUtil.createTestRegion(TABLE_NAME, hcd);
Region region = testUtil.createTestRegion(TABLE_NAME, hcd);
//write the data, but leave some in the memstore
doPuts(region);
@ -122,10 +122,9 @@ public class TestEncodedSeekers {
doGets(region);
//verify correctness again after compacting
region.compactStores();
region.compact(false);
doGets(region);
Map<DataBlockEncoding, Integer> encodingCounts = cache.getEncodingCountsForTest();
// Ensure that compactions don't pollute the cache with unencoded blocks
@ -138,7 +137,7 @@ public class TestEncodedSeekers {
}
private void doPuts(HRegion region) throws IOException{
private void doPuts(Region region) throws IOException{
LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(MIN_VALUE_SIZE, MAX_VALUE_SIZE);
for (int i = 0; i < NUM_ROWS; ++i) {
byte[] key = LoadTestKVGenerator.md5PrefixedKey(i).getBytes();
@ -162,13 +161,13 @@ public class TestEncodedSeekers {
region.put(put);
}
if (i % NUM_ROWS_PER_FLUSH == 0) {
region.flushcache();
region.flush(true);
}
}
}
private void doGets(HRegion region) throws IOException{
private void doGets(Region region) throws IOException{
for (int i = 0; i < NUM_ROWS; ++i) {
final byte[] rowKey = LoadTestKVGenerator.md5PrefixedKey(i).getBytes();
for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {

View File

@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@ -66,7 +66,7 @@ public class TestPrefixTree {
private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
private HRegion region;
private Region region;
@Before
public void setUp() throws Exception {
@ -101,7 +101,7 @@ public class TestPrefixTree {
put = new Put(row4_bytes);
put.addColumn(fam, qual2, Bytes.toBytes("c2-value-3"));
region.put(put);
region.flushcache();
region.flush(true);
String[] rows = new String[3];
rows[0] = row1;
rows[1] = row2;
@ -182,7 +182,7 @@ public class TestPrefixTree {
region.put(new Put(Bytes.toBytes("obj29")).addColumn(fam, qual1, Bytes.toBytes("whatever")));
region.put(new Put(Bytes.toBytes("obj2")).addColumn(fam, qual1, Bytes.toBytes("whatever")));
region.put(new Put(Bytes.toBytes("obj3")).addColumn(fam, qual1, Bytes.toBytes("whatever")));
region.flushcache();
region.flush(true);
Scan scan = new Scan(Bytes.toBytes("obj29995"));
RegionScanner scanner = region.getScanner(scan);
List<Cell> cells = new ArrayList<Cell>();

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -433,7 +434,7 @@ public class TestCacheOnWrite {
final String cf = "myCF";
final byte[] cfBytes = Bytes.toBytes(cf);
final int maxVersions = 3;
HRegion region = TEST_UTIL.createTestRegion(table,
Region region = TEST_UTIL.createTestRegion(table,
new HColumnDescriptor(cf)
.setCompressionType(compress)
.setBloomFilterType(BLOOM_TYPE)
@ -467,18 +468,18 @@ public class TestCacheOnWrite {
p.setDurability(Durability.ASYNC_WAL);
region.put(p);
}
region.flushcache();
region.flush(true);
}
clearBlockCache(blockCache);
assertEquals(0, blockCache.getBlockCount());
region.compactStores();
region.compact(false);
LOG.debug("compactStores() returned");
for (CachedBlock block: blockCache) {
assertNotEquals(BlockType.ENCODED_DATA, block.getBlockType());
assertNotEquals(BlockType.DATA, block.getBlockType());
}
region.close();
((HRegion)region).close();
}
@Test

View File

@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Before;
import org.junit.Test;
@ -111,7 +111,7 @@ public class TestForceCacheImportantBlocks {
setBloomFilterType(BLOOM_TYPE);
hcd.setBlocksize(BLOCK_SIZE);
hcd.setBlockCacheEnabled(cfCacheEnabled);
HRegion region = TEST_UTIL.createTestRegion(TABLE, hcd);
Region region = TEST_UTIL.createTestRegion(TABLE, hcd);
BlockCache cache = region.getStore(hcd.getName()).getCacheConfig().getBlockCache();
CacheStats stats = cache.getStats();
writeTestData(region);
@ -128,7 +128,7 @@ public class TestForceCacheImportantBlocks {
else assertTrue(stats.getMissCount() > missCount);
}
private void writeTestData(HRegion region) throws IOException {
private void writeTestData(Region region) throws IOException {
for (int i = 0; i < NUM_ROWS; ++i) {
Put put = new Put(Bytes.toBytes("row" + i));
for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
@ -139,7 +139,7 @@ public class TestForceCacheImportantBlocks {
}
region.put(put);
if ((i + 1) % ROWS_PER_HFILE == 0) {
region.flushcache();
region.flush(true);
}
}
}

View File

@ -36,11 +36,11 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.Test;
@ -100,7 +100,7 @@ public class TestScannerSelectionUsingKeyRange {
HTableDescriptor htd = new HTableDescriptor(TABLE);
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(TABLE);
HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf,
Region region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf,
htd);
for (int iFile = 0; iFile < NUM_FILES; ++iFile) {
@ -112,7 +112,7 @@ public class TestScannerSelectionUsingKeyRange {
}
region.put(put);
}
region.flushcache();
region.flush(true);
}
Scan scan = new Scan(Bytes.toBytes("aaa"), Bytes.toBytes("aaz"));

View File

@ -35,10 +35,10 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -107,10 +107,8 @@ public class TestScannerSelectionUsingTTL {
HTableDescriptor htd = new HTableDescriptor(TABLE);
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(TABLE);
HRegion region =
HBaseTestingUtility.createRegionAndWAL(info,
TEST_UTIL.getDataTestDir(info.getEncodedName()),
conf, htd);
Region region = HBaseTestingUtility.createRegionAndWAL(info,
TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, htd);
long ts = EnvironmentEdgeManager.currentTime();
long version = 0; //make sure each new set of Put's have a new ts
@ -128,7 +126,7 @@ public class TestScannerSelectionUsingTTL {
}
region.put(put);
}
region.flushcache();
region.flush(true);
version++;
}
@ -156,7 +154,7 @@ public class TestScannerSelectionUsingTTL {
HStore store = (HStore)region.getStore(FAMILY_BYTES);
store.compactRecentForTestingAssumingDefaultPolicy(totalNumFiles);
} else {
region.compactStores();
region.compact(false);
}
HBaseTestingUtility.closeRegionAndWAL(region);

View File

@ -38,14 +38,11 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MapReduceTests;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -54,7 +51,7 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.Tool;
@ -249,11 +246,11 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
@Override
public void prePut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit,
Durability durability) throws IOException {
HRegion region = e.getEnvironment().getRegion();
Region region = e.getEnvironment().getRegion();
if (!region.getRegionInfo().isMetaTable()
&& !region.getRegionInfo().getTable().isSystemTable()) {
if (put.getAttribute(TEST_ATR_KEY) != null) {
LOG.debug("allow any put to happen " + region.getRegionNameAsString());
LOG.debug("allow any put to happen " + region.getRegionInfo().getRegionNameAsString());
} else {
e.bypass();
}

View File

@ -41,12 +41,11 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -159,7 +158,7 @@ public class TestImportTSVWithTTLs implements Configurable {
@Override
public void prePut(ObserverContext<RegionCoprocessorEnvironment> e, Put put, WALEdit edit,
Durability durability) throws IOException {
HRegion region = e.getEnvironment().getRegion();
Region region = e.getEnvironment().getRegion();
if (!region.getRegionInfo().isMetaTable()
&& !region.getRegionInfo().getTable().isSystemTable()) {
// The put carries the TTL attribute

View File

@ -99,6 +99,7 @@ import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
import org.apache.hadoop.hbase.regionserver.Leases;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
@ -255,12 +256,12 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
public void addToOnlineRegions(HRegion r) {
public void addToOnlineRegions(Region r) {
// TODO Auto-generated method stub
}
@Override
public boolean removeFromOnlineRegions(HRegion r, ServerName destination) {
public boolean removeFromOnlineRegions(Region r, ServerName destination) {
// TODO Auto-generated method stub
return false;
}
@ -334,8 +335,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
public void postOpenDeployTasks(HRegion r)
throws KeeperException, IOException {
public void postOpenDeployTasks(Region r) throws KeeperException, IOException {
// TODO Auto-generated method stub
}
@ -531,7 +531,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
public List<HRegion> getOnlineRegions(TableName tableName) throws IOException {
public List<Region> getOnlineRegions(TableName tableName) throws IOException {
// TODO Auto-generated method stub
return null;
}
@ -583,7 +583,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
@Override
public Map<String, HRegion> getRecoveringRegions() {
public Map<String, Region> getRecoveringRegions() {
// TODO Auto-generated method stub
return null;
}

View File

@ -34,17 +34,15 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({MasterTests.class, MediumTests.class})
@ -233,8 +231,8 @@ public class TestAssignmentListener {
admin.majorCompact(TABLE_NAME);
mergeable = 0;
for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) {
for (HRegion region: regionThread.getRegionServer().getOnlineRegions(TABLE_NAME)) {
mergeable += region.isMergeable() ? 1 : 0;
for (Region region: regionThread.getRegionServer().getOnlineRegions(TABLE_NAME)) {
mergeable += ((HRegion)region).isMergeable() ? 1 : 0;
}
}
}

View File

@ -90,6 +90,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -1479,7 +1480,7 @@ public class TestDistributedLogSplitting {
}
LOG.debug("adding data to rs = " + rst.getName() +
" region = "+ hri.getRegionNameAsString());
HRegion region = hrs.getOnlineRegion(hri.getRegionName());
Region region = hrs.getOnlineRegion(hri.getRegionName());
assertTrue(region != null);
putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
}
@ -1500,7 +1501,7 @@ public class TestDistributedLogSplitting {
}
LOG.debug("adding data to rs = " + mt.getName() +
" region = "+ hri.getRegionNameAsString());
HRegion region = hrs.getOnlineRegion(hri.getRegionName());
Region region = hrs.getOnlineRegion(hri.getRegionName());
assertTrue(region != null);
putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
}
@ -1614,7 +1615,7 @@ public class TestDistributedLogSplitting {
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
}
private void putData(HRegion region, byte[] startRow, int numRows, byte [] qf,
private void putData(Region region, byte[] startRow, int numRows, byte [] qf,
byte [] ...families)
throws IOException {
for(int i = 0; i < numRows; i++) {

View File

@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
@ -77,10 +77,10 @@ public class TestGetLastFlushedSequenceId {
table.flushCommits();
MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
HRegion region = null;
Region region = null;
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
HRegionServer hrs = rsts.get(i).getRegionServer();
for (HRegion r : hrs.getOnlineRegions(tableName)) {
for (Region r : hrs.getOnlineRegions(tableName)) {
region = r;
break;
}

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.FlakeyTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -383,9 +384,9 @@ public class TestMasterFailover {
// region server should expire (how it can be verified?)
MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(),
rs.getServerName(), State.PENDING_OPEN);
HRegion meta = rs.getFromOnlineRegions(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
Region meta = rs.getFromOnlineRegions(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
rs.removeFromOnlineRegions(meta, null);
meta.close();
((HRegion)meta).close();
log("Aborting master");
activeMaster.abort("test-kill");

View File

@ -55,8 +55,8 @@ import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan;
import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan.Position;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -189,10 +189,10 @@ public class TestRegionPlacement {
// kill a random non-meta server carrying at least one region
killIndex = random.nextInt(SLAVES);
serverToKill = TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getServerName();
Collection<HRegion> regs =
Collection<Region> regs =
TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getOnlineRegionsLocalContext();
isNamespaceServer = false;
for (HRegion r : regs) {
for (Region r : regs) {
if (r.getRegionInfo().getTable().getNamespaceAsString()
.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
isNamespaceServer = true;
@ -416,8 +416,7 @@ public class TestRegionPlacement {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
for (int i = 0; i < SLAVES; i++) {
HRegionServer rs = cluster.getRegionServer(i);
for (HRegion region: rs.getOnlineRegions(
TableName.valueOf("testRegionAssignment"))) {
for (Region region: rs.getOnlineRegions(TableName.valueOf("testRegionAssignment"))) {
InetSocketAddress[] favoredSocketAddress = rs.getFavoredNodesForRegion(
region.getRegionInfo().getEncodedName());
List<ServerName> favoredServerList = plan.getAssignmentMap().get(region.getRegionInfo());
@ -446,7 +445,7 @@ public class TestRegionPlacement {
assertNotNull(addrFromPlan);
assertTrue("Region server " + rs.getServerName().getHostAndPort()
+ " has the " + positions[j] +
" for region " + region.getRegionNameAsString() + " is " +
" for region " + region.getRegionInfo().getRegionNameAsString() + " is " +
addrFromRS + " which is inconsistent with the plan "
+ addrFromPlan, addrFromRS.equals(addrFromPlan));
}

View File

@ -374,7 +374,7 @@ public class TestTableLockManager {
try {
HRegion region = TEST_UTIL.getSplittableRegion(tableName, -1);
if (region != null) {
byte[] regionName = region.getRegionName();
byte[] regionName = region.getRegionInfo().getRegionName();
admin.flushRegion(regionName);
admin.compactRegion(regionName);
admin.splitRegion(regionName);

View File

@ -72,6 +72,7 @@ import org.apache.hadoop.hbase.quotas.QuotaExceededException;
import org.apache.hadoop.hbase.quotas.QuotaUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -281,7 +282,7 @@ public class TestNamespaceAuditor {
@Override
public synchronized void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
HRegion regionA, HRegion regionB) throws IOException {
Region regionA, Region regionB) throws IOException {
triggered = true;
notifyAll();
if (shouldFailMerge) {
@ -458,7 +459,8 @@ public class TestNamespaceAuditor {
observer = (CustomObserver) actualRegion.getCoprocessorHost().findCoprocessor(
CustomObserver.class.getName());
assertNotNull(observer);
ADMIN.split(tableOne, getSplitKey(actualRegion.getStartKey(), actualRegion.getEndKey()));
ADMIN.split(tableOne, getSplitKey(actualRegion.getRegionInfo().getStartKey(),
actualRegion.getRegionInfo().getEndKey()));
observer.postSplit.await();
// Make sure no regions have been added.
List<HRegionInfo> hris = ADMIN.getTableRegions(tableOne);

View File

@ -74,7 +74,7 @@ public class NoOpScanPolicyObserver extends BaseRegionObserver {
public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
Store store, final Scan scan, final NavigableSet<byte[]> targetCols, KeyValueScanner s)
throws IOException {
HRegion r = c.getEnvironment().getRegion();
Region r = c.getEnvironment().getRegion();
return scan.isReversed() ? new ReversedStoreScanner(store,
store.getScanInfo(), scan, targetCols, r.getReadpoint(scan
.getIsolationLevel())) : new StoreScanner(store,

View File

@ -81,7 +81,7 @@ public class TestAtomicOperation {
static final Log LOG = LogFactory.getLog(TestAtomicOperation.class);
@Rule public TestName name = new TestName();
HRegion region = null;
Region region = null;
private HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
// Test names
@ -102,7 +102,7 @@ public class TestAtomicOperation {
@After
public void teardown() throws IOException {
if (region != null) {
region.close();
((HRegion)region).close();
region = null;
}
}
@ -126,11 +126,11 @@ public class TestAtomicOperation {
a.setReturnResults(false);
a.add(fam1, qual1, Bytes.toBytes(v1));
a.add(fam1, qual2, Bytes.toBytes(v2));
assertNull(region.append(a));
assertNull(region.append(a, HConstants.NO_NONCE, HConstants.NO_NONCE));
a = new Append(row);
a.add(fam1, qual1, Bytes.toBytes(v2));
a.add(fam1, qual2, Bytes.toBytes(v1));
Result result = region.append(a);
Result result = region.append(a, HConstants.NO_NONCE, HConstants.NO_NONCE);
assertEquals(0, Bytes.compareTo(Bytes.toBytes(v1+v2), result.getValue(fam1, qual1)));
assertEquals(0, Bytes.compareTo(Bytes.toBytes(v2+v1), result.getValue(fam1, qual2)));
}
@ -217,12 +217,12 @@ public class TestAtomicOperation {
*/
public static class Incrementer extends Thread {
private final HRegion region;
private final Region region;
private final int numIncrements;
private final int amount;
public Incrementer(HRegion region,
public Incrementer(Region region,
int threadNumber, int amount, int numIncrements) {
this.region = region;
this.numIncrements = numIncrements;
@ -239,7 +239,7 @@ public class TestAtomicOperation {
inc.addColumn(fam1, qual2, amount*2);
inc.addColumn(fam2, qual3, amount*3);
inc.setDurability(Durability.ASYNC_WAL);
region.increment(inc);
region.increment(inc, HConstants.NO_NONCE, HConstants.NO_NONCE);
// verify: Make sure we only see completed increments
Get g = new Get(row);
@ -277,7 +277,7 @@ public class TestAtomicOperation {
a.add(fam1, qual2, val);
a.add(fam2, qual3, val);
a.setDurability(Durability.ASYNC_WAL);
region.append(a);
region.append(a, HConstants.NO_NONCE, HConstants.NO_NONCE);
Get g = new Get(row);
Result result = region.get(g);
@ -341,9 +341,9 @@ public class TestAtomicOperation {
if (i%10==0) {
synchronized(region) {
LOG.debug("flushing");
region.flushcache();
region.flush(true);
if (i%100==0) {
region.compactStores();
region.compact(false);
}
}
}
@ -434,9 +434,9 @@ public class TestAtomicOperation {
if (i%10==0) {
synchronized(region) {
LOG.debug("flushing");
region.flushcache();
region.flush(true);
if (i%100==0) {
region.compactStores();
region.compact(false);
}
}
}
@ -461,7 +461,7 @@ public class TestAtomicOperation {
p.add(fam1, qual1, value2);
mrm.add(p);
}
region.mutateRowsWithLocks(mrm, rowsToLock);
region.mutateRowsWithLocks(mrm, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE);
op ^= true;
// check: should always see exactly one column
Scan s = new Scan(row);
@ -500,13 +500,13 @@ public class TestAtomicOperation {
}
public static class AtomicOperation extends Thread {
protected final HRegion region;
protected final Region region;
protected final int numOps;
protected final AtomicLong timeStamps;
protected final AtomicInteger failures;
protected final Random r = new Random();
public AtomicOperation(HRegion region, int numOps, AtomicLong timeStamps,
public AtomicOperation(Region region, int numOps, AtomicLong timeStamps,
AtomicInteger failures) {
this.region = region;
this.numOps = numOps;
@ -541,14 +541,14 @@ public class TestAtomicOperation {
conf.setClass(HConstants.REGION_IMPL, MockHRegion.class, HeapSize.class);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName))
.addFamily(new HColumnDescriptor(family));
final MockHRegion region = (MockHRegion) TEST_UTIL.createLocalHRegion(htd, null, null);
final Region region = TEST_UTIL.createLocalHRegion(htd, null, null);
Put[] puts = new Put[1];
Put put = new Put(Bytes.toBytes("r1"));
put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10"));
puts[0] = put;
region.batchMutate(puts);
region.batchMutate(puts, HConstants.NO_NONCE, HConstants.NO_NONCE);
MultithreadedTestUtil.TestContext ctx =
new MultithreadedTestUtil.TestContext(conf);
ctx.addThread(new PutThread(ctx, region));
@ -569,8 +569,8 @@ public class TestAtomicOperation {
}
private class PutThread extends TestThread {
private MockHRegion region;
PutThread(TestContext ctx, MockHRegion region) {
private Region region;
PutThread(TestContext ctx, Region region) {
super(ctx);
this.region = region;
}
@ -581,13 +581,13 @@ public class TestAtomicOperation {
put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
puts[0] = put;
testStep = TestStep.PUT_STARTED;
region.batchMutate(puts);
region.batchMutate(puts, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
}
private class CheckAndPutThread extends TestThread {
private MockHRegion region;
CheckAndPutThread(TestContext ctx, MockHRegion region) {
private Region region;
CheckAndPutThread(TestContext ctx, Region region) {
super(ctx);
this.region = region;
}
@ -622,10 +622,10 @@ public class TestAtomicOperation {
return new WrappedRowLock(super.getRowLockInternal(row, waitForLock));
}
public class WrappedRowLock extends RowLock {
public class WrappedRowLock extends RowLockImpl {
private WrappedRowLock(RowLock rowLock) {
super(rowLock.context);
setContext(((RowLockImpl)rowLock).getContext());
}
@Override

View File

@ -69,7 +69,7 @@ public class TestBlocksRead extends HBaseTestCase {
return conf;
}
HRegion region = null;
Region region = null;
private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final String DIR = TEST_UTIL.getDataTestDir("TestBlocksRead").toString();
@ -98,7 +98,7 @@ public class TestBlocksRead extends HBaseTestCase {
* @throws IOException
* @return created and initialized region.
*/
private HRegion initHRegion(byte[] tableName, String callingMethod,
private Region initHRegion(byte[] tableName, String callingMethod,
HBaseConfiguration conf, String family) throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
HColumnDescriptor familyDesc;
@ -112,7 +112,7 @@ public class TestBlocksRead extends HBaseTestCase {
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
Path path = new Path(DIR + callingMethod);
HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);
Region r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);
blockCache = new CacheConfig(conf).getBlockCache();
return r;
}
@ -234,7 +234,7 @@ public class TestBlocksRead extends HBaseTestCase {
putData(FAMILY, "row", "col5", 5);
putData(FAMILY, "row", "col6", 6);
putData(FAMILY, "row", "col7", 7);
region.flushcache();
region.flush(true);
// Expected block reads: 1
// The top block has the KV we are
@ -287,12 +287,12 @@ public class TestBlocksRead extends HBaseTestCase {
// File 1
putData(FAMILY, "row", "col1", 1);
putData(FAMILY, "row", "col2", 2);
region.flushcache();
region.flush(true);
// File 2
putData(FAMILY, "row", "col1", 3);
putData(FAMILY, "row", "col2", 4);
region.flushcache();
region.flush(true);
// Expected blocks read: 1.
// File 2's top block is also the KV we are
@ -312,7 +312,7 @@ public class TestBlocksRead extends HBaseTestCase {
// File 3: Add another column
putData(FAMILY, "row", "col3", 5);
region.flushcache();
region.flush(true);
// Expected blocks read: 1
// File 3's top block has the "col3" KV we are
@ -331,7 +331,7 @@ public class TestBlocksRead extends HBaseTestCase {
// File 4: Delete the entire row.
deleteFamily(FAMILY, "row", 6);
region.flushcache();
region.flush(true);
// For ROWCOL Bloom filter: Expected blocks read: 2.
// For ROW Bloom filter: Expected blocks read: 3.
@ -347,14 +347,14 @@ public class TestBlocksRead extends HBaseTestCase {
// File 5: Delete
deleteFamily(FAMILY, "row", 10);
region.flushcache();
region.flush(true);
// File 6: some more puts, but with timestamps older than the
// previous delete.
putData(FAMILY, "row", "col1", 7);
putData(FAMILY, "row", "col2", 8);
putData(FAMILY, "row", "col3", 9);
region.flushcache();
region.flush(true);
// Baseline expected blocks read: 6. [HBASE-4532]
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 6, 7, 7);
@ -364,7 +364,7 @@ public class TestBlocksRead extends HBaseTestCase {
putData(FAMILY, "row", "col1", 11);
putData(FAMILY, "row", "col2", 12);
putData(FAMILY, "row", "col3", 13);
region.flushcache();
region.flush(true);
// Expected blocks read: 8. [HBASE-4585, HBASE-13109]
@ -394,7 +394,7 @@ public class TestBlocksRead extends HBaseTestCase {
try {
putData(FAMILY, "row", "col1", 1);
putData(FAMILY, "row", "col2", 2);
region.flushcache();
region.flush(true);
// Execute a scan with caching turned off
// Expected blocks stored: 0
@ -441,7 +441,7 @@ public class TestBlocksRead extends HBaseTestCase {
putData(FAMILY, "row", "col" + i, i);
}
putData(FAMILY, "row", "col99", 201);
region.flushcache();
region.flush(true);
kvs = getData(FAMILY, "row", Arrays.asList("col0"), 2);
assertEquals(0, kvs.length);

View File

@ -90,10 +90,9 @@ public class TestBlocksScanned extends HBaseTestCase {
}
private void _testBlocksScanned(HTableDescriptor table) throws Exception {
HRegion r = createNewHRegion(table, START_KEY, END_KEY,
TEST_UTIL.getConfiguration());
Region r = createNewHRegion(table, START_KEY, END_KEY, TEST_UTIL.getConfiguration());
addContent(r, FAMILY, COL);
r.flushcache();
r.flush(true);
CacheStats stats = new CacheConfig(TEST_UTIL.getConfiguration()).getBlockCache().getStats();
long before = stats.getHitCount() + stats.getMissCount();

View File

@ -130,25 +130,26 @@ public class TestBulkLoad {
};
context.checking(expection);
testRegionWithFamiliesAndSpecifiedTableName(tableName, family1)
.bulkLoadHFiles(familyPaths, false);
.bulkLoadHFiles(familyPaths, false, null);
}
@Test
public void bulkHLogShouldThrowNoErrorAndWriteMarkerWithBlankInput() throws IOException {
testRegionWithFamilies(family1).bulkLoadHFiles(new ArrayList<Pair<byte[], String>>(), false);
testRegionWithFamilies(family1).bulkLoadHFiles(new ArrayList<Pair<byte[], String>>(),
false, null);
}
@Test
public void shouldBulkLoadSingleFamilyHLog() throws IOException {
context.checking(callOnce);
testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1), false);
testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1), false, null);
}
@Test
public void shouldBulkLoadManyFamilyHLog() throws IOException {
context.checking(callOnce);
testRegionWithFamilies(family1, family2).bulkLoadHFiles(withFamilyPathsFor(family1, family2),
false);
false, null);
}
@Test
@ -156,31 +157,32 @@ public class TestBulkLoad {
context.checking(callOnce);
TableName tableName = TableName.valueOf("test", "test");
testRegionWithFamiliesAndSpecifiedTableName(tableName, family1, family2)
.bulkLoadHFiles(withFamilyPathsFor(family1, family2), false);
.bulkLoadHFiles(withFamilyPathsFor(family1, family2), false, null);
}
@Test(expected = DoNotRetryIOException.class)
public void shouldCrashIfBulkLoadFamiliesNotInTable() throws IOException {
testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1, family2), false);
testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1, family2), false,
null);
}
@Test(expected = DoNotRetryIOException.class)
public void bulkHLogShouldThrowErrorWhenFamilySpecifiedAndHFileExistsButNotInTableDescriptor()
throws IOException {
testRegionWithFamilies().bulkLoadHFiles(withFamilyPathsFor(family1), false);
testRegionWithFamilies().bulkLoadHFiles(withFamilyPathsFor(family1), false, null);
}
@Test(expected = DoNotRetryIOException.class)
public void shouldThrowErrorIfBadFamilySpecifiedAsFamilyPath() throws IOException {
testRegionWithFamilies()
.bulkLoadHFiles(asList(withInvalidColumnFamilyButProperHFileLocation(family1)),
false);
false, null);
}
@Test(expected = FileNotFoundException.class)
public void shouldThrowErrorIfHFileDoesNotExist() throws IOException {
List<Pair<byte[], String>> list = asList(withMissingHFileForFamily(family1));
testRegionWithFamilies(family1).bulkLoadHFiles(list, false);
testRegionWithFamilies(family1).bulkLoadHFiles(list, false, null);
}
private Pair<byte[], String> withMissingHFileForFamily(byte[] family) {

View File

@ -74,7 +74,7 @@ public class TestColumnSeeking {
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(table, null, null, false);
// Set this so that the archiver writes to the temp dir as well.
HRegion region = TEST_UTIL.createLocalHRegion(info, htd);
Region region = TEST_UTIL.createLocalHRegion(info, htd);
try {
List<String> rows = generateRandomWords(10, "row");
List<String> allColumns = generateRandomWords(10, "column");
@ -125,17 +125,17 @@ public class TestColumnSeeking {
region.put(p);
if (Math.random() < flushPercentage) {
LOG.info("Flushing... ");
region.flushcache();
region.flush(true);
}
if (Math.random() < minorPercentage) {
LOG.info("Minor compacting... ");
region.compactStores(false);
region.compact(false);
}
if (Math.random() < majorPercentage) {
LOG.info("Major compacting... ");
region.compactStores(true);
region.compact(true);
}
}
}
@ -186,7 +186,7 @@ public class TestColumnSeeking {
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(table, null, null, false);
HRegion region = TEST_UTIL.createLocalHRegion(info, htd);
Region region = TEST_UTIL.createLocalHRegion(info, htd);
List<String> rows = generateRandomWords(10, "row");
List<String> allColumns = generateRandomWords(100, "column");
@ -238,17 +238,17 @@ public class TestColumnSeeking {
region.put(p);
if (Math.random() < flushPercentage) {
LOG.info("Flushing... ");
region.flushcache();
region.flush(true);
}
if (Math.random() < minorPercentage) {
LOG.info("Minor compacting... ");
region.compactStores(false);
region.compact(false);
}
if (Math.random() < majorPercentage) {
LOG.info("Major compacting... ");
region.compactStores(true);
region.compact(true);
}
}

Some files were not shown because too many files have changed in this diff Show More