HBASE-12972 Region, a supportable public/evolving subset of HRegion
This commit is contained in:
parent
55a5a3be33
commit
f1f4b66183
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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>
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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())) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
||||
}
|
|
@ -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)
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
});
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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()));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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>();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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"));
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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++) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue