HBASE-14551 Procedure v2 - Reimplement split (Stephen Yuan Jiang)

This commit is contained in:
Stephen Yuan Jiang 2016-10-27 21:45:41 -07:00
parent 738ff821dd
commit e108a4f815
43 changed files with 7043 additions and 1022 deletions

View File

@ -165,7 +165,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;

View File

@ -104,6 +104,8 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@ -1719,6 +1721,33 @@ public final class ProtobufUtil {
}
}
/**
* A helper to close a region for split
* using admin protocol.
*
* @param controller RPC controller
* @param admin Admin service
* @param server the RS that hosts the target region
* @param parentRegionInfo the target region info
* @return true if the region is closed
* @throws IOException
*/
public static boolean closeRegionForSplit(
final RpcController controller,
final AdminService.BlockingInterface admin,
final ServerName server,
final HRegionInfo parentRegionInfo) throws IOException {
CloseRegionForSplitRequest closeRegionForSplitRequest =
ProtobufUtil.buildCloseRegionForSplitRequest(server, parentRegionInfo);
try {
CloseRegionForSplitResponse response =
admin.closeRegionForSplit(controller, closeRegionForSplitRequest);
return ResponseConverter.isClosed(response);
} catch (ServiceException se) {
throw getRemoteException(se);
}
}
/**
* A helper to warmup a region given a region name
* using admin protocol
@ -3062,6 +3091,23 @@ public final class ProtobufUtil {
return builder.build();
}
/**
* Create a CloseRegionForSplitRequest for a given region
*
* @param server the RS server that hosts the region
* @param parentRegionInfo the info of the region to close
* @return a CloseRegionForSplitRequest
*/
public static CloseRegionForSplitRequest buildCloseRegionForSplitRequest(
final ServerName server,
final HRegionInfo parentRegionInfo) {
CloseRegionForSplitRequest.Builder builder = CloseRegionForSplitRequest.newBuilder();
RegionSpecifier parentRegion = RequestConverter.buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, parentRegionInfo.getRegionName());
builder.setRegion(parentRegion);
return builder.build();
}
/**
* Create a CloseRegionRequest for a given encoded region name
*

View File

@ -106,6 +106,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOr
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
@ -850,7 +851,7 @@ public final class RequestConverter {
return ubuilder.build();
}
/**
/**
* Create a WarmupRegionRequest for a given region name
*
* @param regionInfo Region we are warming up
@ -1061,6 +1062,19 @@ public final class RequestConverter {
return builder.build();
}
public static SplitTableRegionRequest buildSplitTableRegionRequest(
final HRegionInfo regionInfo,
final byte[] splitPoint,
final long nonceGroup,
final long nonce) {
SplitTableRegionRequest.Builder builder = SplitTableRegionRequest.newBuilder();
builder.setRegionInfo(HRegionInfo.convert(regionInfo));
builder.setSplitRow(UnsafeByteOperations.unsafeWrap(splitPoint));
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
/**
* Create a protocol buffer AssignRegionRequest
*

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.SingleResponse;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
@ -251,6 +252,18 @@ public final class ResponseConverter {
return proto.getClosed();
}
/**
* Check if the region is closed from a CloseRegionForSplitResponse
*
* @param proto the CloseRegionForSplitResponse
* @return the region close state
*/
public static boolean isClosed
(final CloseRegionForSplitResponse proto) {
if (proto == null || !proto.hasClosed()) return false;
return proto.getClosed();
}
/**
* A utility to build a GetServerInfoResponse.
*

View File

@ -117,6 +117,18 @@ message CloseRegionResponse {
required bool closed = 1;
}
/**
* Closes the specified region and create
* child region.
*/
message CloseRegionForSplitRequest {
required RegionSpecifier region = 1;
}
message CloseRegionForSplitResponse {
required bool closed = 1;
}
/**
* Flushes the MemStore of the specified region.
* <p>
@ -274,6 +286,9 @@ service AdminService {
rpc CloseRegion(CloseRegionRequest)
returns(CloseRegionResponse);
rpc CloseRegionForSplit(CloseRegionForSplitRequest)
returns(CloseRegionForSplitResponse);
rpc FlushRegion(FlushRegionRequest)
returns(FlushRegionResponse);

View File

@ -277,6 +277,27 @@ message DispatchMergingRegionsStateData {
optional bool forcible = 4;
}
enum SplitTableRegionState {
SPLIT_TABLE_REGION_PREPARE = 1;
SPLIT_TABLE_REGION_PRE_OPERATION = 2;
SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE = 3;
SPLIT_TABLE_REGION_CLOSED_PARENT_REGION = 4;
SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 5;
SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 6;
SPLIT_TABLE_REGION_UPDATE_META = 7;
SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 8;
SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 9;
SPLIT_TABLE_REGION_POST_OPERATION = 10;
}
message SplitTableRegionStateData {
required UserInformation user_info = 1;
required TableName table_name = 2;
required RegionInfo parent_region_info = 3;
optional bytes split_row = 4;
repeated RegionInfo child_region_info = 5;
}
message ServerCrashStateData {
required ServerName server_name = 1;
optional bool distributed_log_replay = 2;

View File

@ -26,6 +26,7 @@ option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
import "HBase.proto";
import "Master.proto";
import "ClusterStatus.proto";
message RegionServerStartupRequest {
@ -126,6 +127,20 @@ message ReportRegionStateTransitionResponse {
optional string error_message = 1;
}
/**
* Splits the specified region.
*/
message SplitTableRegionRequest {
required RegionInfo region_info = 1;
required bytes split_row = 2;
optional uint64 nonce_group = 3 [default = 0];
optional uint64 nonce = 4 [default = 0];
}
message SplitTableRegionResponse {
optional uint64 proc_id = 1;
}
service RegionServerStatusService {
/** Called when a region server first starts. */
rpc RegionServerStartup(RegionServerStartupRequest)
@ -155,4 +170,16 @@ service RegionServerStatusService {
*/
rpc ReportRegionStateTransition(ReportRegionStateTransitionRequest)
returns(ReportRegionStateTransitionResponse);
/**
* Split region
*/
rpc SplitRegion(SplitTableRegionRequest)
returns(SplitTableRegionResponse);
/**
* Get procedure result
*/
rpc getProcedureResult(GetProcedureResultRequest)
returns(GetProcedureResultResponse);
}

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@ -80,7 +81,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RSGroupAdmi
import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupRequest;
import org.apache.hadoop.hbase.protobuf.generated.RSGroupAdminProtos.RemoveRSGroupResponse;
@InterfaceAudience.Private
public class RSGroupAdminEndpoint extends RSGroupAdminService
implements CoprocessorService, Coprocessor, MasterObserver {
@ -1098,4 +1098,42 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService
public void postListProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx,
List<ProcedureInfo> procInfoList) throws IOException {
}
@Override
public void preSplitRegion(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow) throws IOException {
}
@Override
public void preSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow) throws IOException {
}
@Override
public void postCompletedSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> c,
final HRegionInfo regionInfoA,
final HRegionInfo regionInfoB) throws IOException {
}
@Override
public void preSplitRegionBeforePONRAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final byte[] splitKey,
final List<Mutation> metaEntries) throws IOException {
}
@Override
public void preSplitRegionAfterPONRAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
}
@Override
public void preRollBackSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
}
}

View File

@ -37,11 +37,13 @@ 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.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.regionserver.Region;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
@ -794,4 +796,42 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver
public void preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name)
throws IOException {
}
@Override
public void preSplitRegion(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow) throws IOException {
}
@Override
public void preSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow) throws IOException {
}
@Override
public void postCompletedSplitRegionAction(
ObserverContext<MasterCoprocessorEnvironment> c,
final HRegionInfo regionInfoA,
final HRegionInfo regionInfoB) throws IOException {
}
@Override
public void preSplitRegionBeforePONRAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final byte[] splitKey,
final List<Mutation> metaEntries) throws IOException {
}
@Override
public void preSplitRegionAfterPONRAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
}
@Override
public void preRollBackSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
}
}

View File

@ -37,11 +37,13 @@ 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.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.regionserver.Region;
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.CONFIG})
@InterfaceStability.Evolving
@ -816,6 +818,44 @@ public class BaseMasterObserver implements MasterObserver {
final boolean newValue, final MasterSwitchType switchType) throws IOException {
}
@Override
public void preSplitRegion(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow) throws IOException {
}
@Override
public void preSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow) throws IOException {
}
@Override
public void postCompletedSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> c,
final HRegionInfo regionInfoA,
final HRegionInfo regionInfoB) throws IOException {
}
@Override
public void preSplitRegionBeforePONRAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final byte[] splitKey,
final List<Mutation> metaEntries) throws IOException {
}
@Override
public void preSplitRegionAfterPONRAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
}
@Override
public void preRollBackSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
}
@Override
public void preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {

View File

@ -37,6 +37,7 @@ 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.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@ -1132,6 +1133,76 @@ public interface MasterObserver extends Coprocessor {
void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final MasterSwitchType switchType) throws IOException;
/**
* Called before the split region procedure is called.
* @param c the environment to interact with the framework and master
* @param tableName the table where the region belongs to
* @param splitRow split point
* @throws IOException if an error occurred on the coprocessor
*/
void preSplitRegion(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow)
throws IOException;
/**
* Called before the region is split.
* @param c the environment to interact with the framework and master
* @param tableName the table where the region belongs to
* @param splitRow split point
* @throws IOException if an error occurred on the coprocessor
*/
void preSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow)
throws IOException;
/**
* Called after the region is split.
* @param c the environment to interact with the framework and master
* @param regionInfoA the left daughter region
* @param regionInfoB the right daughter region
* @throws IOException if an error occurred on the coprocessor
*/
void postCompletedSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> c,
final HRegionInfo regionInfoA,
final HRegionInfo regionInfoB) throws IOException;
/**
* This will be called before PONR step as part of split transaction. Calling
* {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} rollback the split
* @param ctx the environment to interact with the framework and master
* @param splitKey
* @param metaEntries
* @throws IOException
*/
void preSplitRegionBeforePONRAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final byte[] splitKey,
final List<Mutation> metaEntries) throws IOException;
/**
* This will be called after PONR step as part of split transaction
* Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
* effect in this hook.
* @param ctx the environment to interact with the framework and master
* @throws IOException
*/
void preSplitRegionAfterPONRAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException;
/**
* This will be called before the roll back of the split region is completed
* @param ctx the environment to interact with the framework and master
* @throws IOException
*/
void preRollBackSplitRegionAction(final ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException;
/**
* Called prior to modifying the flag used to enable/disable region balancing.
* @param ctx the coprocessor instance's environment

View File

@ -359,7 +359,10 @@ public interface RegionObserver extends Coprocessor {
* @param c the environment provided by the region server
* (e.getRegion() returns the parent region)
* @throws IOException if an error occurred on the coprocessor
*
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c, byte[] splitRow)
throws IOException;
@ -383,32 +386,43 @@ public interface RegionObserver extends Coprocessor {
* @param splitKey
* @param metaEntries
* @throws IOException
*/
*
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
void preSplitBeforePONR(final ObserverContext<RegionCoprocessorEnvironment> ctx,
byte[] splitKey, List<Mutation> metaEntries) throws IOException;
/**
* This will be called after PONR step as part of split transaction
* Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
* effect in this hook.
* @param ctx
* @throws IOException
*/
*
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
void preSplitAfterPONR(final ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException;
/**
* This will be called before the roll back of the split region is completed
* @param ctx
* @throws IOException
*/
*
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
void preRollBackSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx) throws IOException;
/**
* This will be called after the roll back of the split region is completed
* @param ctx
* @throws IOException
*/
*
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
void postRollBackSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx)
throws IOException;

View File

@ -61,7 +61,6 @@ import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.RegionStateListener;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@ -2432,6 +2431,39 @@ public class AssignmentManager {
return null;
}
public void assignDaughterRegions(
final HRegionInfo parentHRI,
final HRegionInfo daughterAHRI,
final HRegionInfo daughterBHRI) throws InterruptedException, IOException {
//Offline the parent region
regionOffline(parentHRI, State.SPLIT);
//Set daughter regions to offline
regionStates.prepareAssignDaughters(daughterAHRI, daughterBHRI);
// Assign daughter regions
invokeAssign(daughterAHRI);
invokeAssign(daughterBHRI);
Callable<Object> splitReplicasCallable = new Callable<Object>() {
@Override
public Object call() {
doSplittingOfReplicas(parentHRI, daughterAHRI, daughterBHRI);
return null;
}
};
threadPoolExecutorService.submit(splitReplicasCallable);
// wait for assignment completion
ArrayList<HRegionInfo> regionAssignSet = new ArrayList<HRegionInfo>(2);
regionAssignSet.add(daughterAHRI);
regionAssignSet.add(daughterBHRI);
while (!waitForAssignment(regionAssignSet, true, regionAssignSet.size(),
Long.MAX_VALUE)) {
LOG.debug("some user regions are still in transition: " + regionAssignSet);
}
}
private String onRegionSplit(final RegionState current, final HRegionInfo hri,
final ServerName serverName, final RegionStateTransition transition) {
// The region must be splitting on this server, and the daughters must be in
@ -2866,7 +2898,7 @@ public class AssignmentManager {
* (d) Other scenarios should be handled similarly as for
* region open/close
*/
protected String onRegionTransition(final ServerName serverName,
public String onRegionTransition(final ServerName serverName,
final RegionStateTransition transition) {
TransitionCode code = transition.getTransitionCode();
HRegionInfo hri = HRegionInfo.convert(transition.getRegionInfo(0));

View File

@ -111,6 +111,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.SplitTableRegionProcedure;
import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.mob.MobConstants;
@ -1352,6 +1353,30 @@ public class HMaster extends HRegionServer implements MasterServices {
return procId;
}
@Override
public long splitRegion(
final HRegionInfo regionInfo,
final byte[] splitRow,
final long nonceGroup,
final long nonce) throws IOException {
checkInitialized();
if (cpHost != null) {
cpHost.preSplitRegion(regionInfo.getTable(), splitRow);
}
LOG.info(getClientIdAuditPrefix() + " Split region " + regionInfo);
// Execute the operation asynchronously
long procId = this.procedureExecutor.submitProcedure(
new SplitTableRegionProcedure(
procedureExecutor.getEnvironment(), regionInfo.getTable(), regionInfo, splitRow),
nonceGroup,
nonce);
return procId;
}
void move(final byte[] encodedRegionName,
final byte[] destServerName) throws HBaseIOException {
RegionState regionState = assignmentManager.getRegionStates().

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@ -833,6 +834,114 @@ public class MasterCoprocessorHost
});
}
/**
* Invoked just before calling the split region procedure
* @param tableName the table where the region belongs to
* @param splitRow the split point
* @throws IOException
*/
public void preSplitRegion(
final TableName tableName,
final byte[] splitRow) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preSplitRegion(ctx, tableName, splitRow);
}
});
}
/**
* Invoked just before a split
* @param tableName the table where the region belongs to
* @param splitRow the split point
* @param user the user
* @throws IOException
*/
public void preSplitRegionAction(
final TableName tableName,
final byte[] splitRow,
final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preSplitRegionAction(ctx, tableName, splitRow);
}
});
}
/**
* Invoked just after a split
* @param regionInfoA the new left-hand daughter region
* @param regionInfoB the new right-hand daughter region
* @param user the user
* @throws IOException
*/
public void postCompletedSplitRegionAction(
final HRegionInfo regionInfoA,
final HRegionInfo regionInfoB,
final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.postCompletedSplitRegionAction(ctx, regionInfoA, regionInfoB);
}
});
}
/**
* This will be called before PONR step as part of split table region procedure.
* @param splitKey
* @param metaEntries
* @param user the user
* @throws IOException
*/
public boolean preSplitBeforePONRAction(
final byte[] splitKey,
final List<Mutation> metaEntries,
final User user) throws IOException {
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preSplitRegionBeforePONRAction(ctx, splitKey, metaEntries);
}
});
}
/**
* This will be called after PONR step as part of split table region procedure.
* @param user the user
* @throws IOException
*/
public void preSplitAfterPONRAction(final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preSplitRegionAfterPONRAction(ctx);
}
});
}
/**
* Invoked just before the rollback of a failed split is started
* @param user the user
* @throws IOException
*/
public void preRollBackSplitAction(final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation(user) {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preRollBackSplitRegionAction(ctx);
}
});
}
public boolean preBalanceSwitch(final boolean b) throws IOException {
return execOperationWithResult(b, coprocessors.isEmpty() ? null :
new CoprocessorOperationWithResult<Boolean>() {

View File

@ -83,6 +83,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.AccessController;
@ -549,6 +551,22 @@ public class MasterRpcServices extends RSRpcServices
}
}
@Override
public SplitTableRegionResponse splitRegion(
final RpcController controller,
final SplitTableRegionRequest request) throws ServiceException {
try {
long procId = master.splitRegion(
HRegionInfo.convert(request.getRegionInfo()),
request.getSplitRow().toByteArray(),
request.getNonceGroup(),
request.getNonce());
return SplitTableRegionResponse.newBuilder().setProcId(procId).build();
} catch (IOException ie) {
throw new ServiceException(ie);
}
}
@Override
public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller,
final ClientProtos.CoprocessorServiceRequest request) throws ServiceException {

View File

@ -264,6 +264,21 @@ public interface MasterServices extends Server {
final long nonce)
throws IOException;
/**
* Split a region.
* @param regionInfo region to split
* @param splitRow split point
* @param nonceGroup used to detect duplicate
* @param nonce used to detect duplicate
* @return procedure Id
* @throws IOException
*/
public long splitRegion(
final HRegionInfo regionInfo,
final byte [] splitRow,
final long nonceGroup,
final long nonce) throws IOException;
/**
* @return Return table descriptors implementation.
*/

View File

@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
@ -873,6 +874,17 @@ public class RegionStates {
return regions == null ? false : regions.contains(hri);
}
public void prepareAssignDaughters(HRegionInfo a, HRegionInfo b) {
synchronized (this) {
if (isRegionInState(a, State.SPLITTING_NEW)) {
updateRegionState(a, State.OFFLINE, null);
}
if (isRegionInState(b, State.SPLITTING_NEW)) {
updateRegionState(b, State.OFFLINE, null);
}
}
}
void splitRegion(HRegionInfo p,
HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
@ -1032,7 +1044,7 @@ public class RegionStates {
return result;
}
protected RegionState getRegionState(final HRegionInfo hri) {
public RegionState getRegionState(final HRegionInfo hri) {
return getRegionState(hri.getEncodedName());
}

View File

@ -80,7 +80,6 @@ import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
@ -828,6 +827,31 @@ public class ServerManager {
return sendRegionClose(server, region, null);
}
/**
* Sends an CLOSE RPC to the specified server to close the specified region for SPLIT.
* <p>
* A region server could reject the close request because it either does not
* have the specified region or the region is being split.
* @param server server to close a region
* @param regionToClose the info of the region to close
* @throws IOException
*/
public boolean sendRegionCloseForSplit(
final ServerName server,
final HRegionInfo regionToClose) throws IOException {
if (server == null) {
throw new NullPointerException("Passed server is null");
}
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
throw new IOException("Attempting to send CLOSE For Split RPC to server " +
server.toString() + " for region " + regionToClose.getRegionNameAsString() +
" failed because no RPC connection found to this server");
}
HBaseRpcController controller = newRpcController();
return ProtobufUtil.closeRegionForSplit(controller, admin, server, regionToClose);
}
/**
* Sends a WARMUP RPC to the specified server to warmup the specified region.
* <p>

View File

@ -0,0 +1,821 @@
/**
* 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.master.procedure;
import java.io.IOException;
import java.io.InputStream;
import java.io.InterruptedIOException;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import com.google.common.annotations.VisibleForTesting;
/**
* The procedure to split a region in a table.
*/
@InterfaceAudience.Private
public class SplitTableRegionProcedure
extends AbstractStateMachineTableProcedure<SplitTableRegionState> {
private static final Log LOG = LogFactory.getLog(SplitTableRegionProcedure.class);
private Boolean traceEnabled;
private User user;
private TableName tableName;
/*
* Region to split
*/
private HRegionInfo parentHRI;
private HRegionInfo daughter_1_HRI;
private HRegionInfo daughter_2_HRI;
/*
* Row to split around
*/
private byte [] splitRow;
public SplitTableRegionProcedure() {
this.traceEnabled = null;
}
public SplitTableRegionProcedure(
final MasterProcedureEnv env,
final TableName tableName,
final HRegionInfo parentHRI,
final byte [] splitRow) throws IOException {
this.traceEnabled = null;
this.tableName = tableName;
this.parentHRI = parentHRI;
this.splitRow = splitRow;
this.user = env.getRequestUser();
this.setOwner(this.user.getShortName());
}
@Override
protected Flow executeFromState(
final MasterProcedureEnv env,
final SplitTableRegionState state) throws InterruptedException {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case SPLIT_TABLE_REGION_PREPARE:
if (prepareSplitRegion(env)) {
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION);
break;
} else {
assert isFailed() : "split region should have an exception here";
return Flow.NO_MORE_STATE;
}
case SPLIT_TABLE_REGION_PRE_OPERATION:
preSplitRegion(env);
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE);
break;
case SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE:
setRegionStateToSplitting(env);
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CLOSED_PARENT_REGION);
break;
case SPLIT_TABLE_REGION_CLOSED_PARENT_REGION:
closeParentRegionForSplit(env);
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS);
break;
case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS:
createDaughterRegions(env);
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR);
break;
case SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR:
preSplitRegionBeforePONR(env);
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_UPDATE_META);
break;
case SPLIT_TABLE_REGION_UPDATE_META:
// This is the point of no return. Adding subsequent edits to .META. as we
// do below when we do the daughter opens adding each to .META. can fail in
// various interesting ways the most interesting of which is a timeout
// BUT the edits all go through (See HBASE-3872). IF we reach the PONR
// then subsequent failures need to crash out this region server; the
// server shutdown processing should be able to fix-up the incomplete split.
// The offlined parent will have the daughters as extra columns. If
// we leave the daughter regions in place and do not remove them when we
// crash out, then they will have their references to the parent in place
// still and the server shutdown fixup of .META. will point to these
// regions.
// We should add PONR JournalEntry before offlineParentInMeta,so even if
// OfflineParentInMeta timeout,this will cause regionserver exit,and then
// master ServerShutdownHandler will fix daughter & avoid data loss. (See
// HBase-4562).
updateMetaForDaughterRegions(env);
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR);
break;
case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR:
preSplitRegionAfterPONR(env);
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS);
break;
case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
openDaughterRegions(env);
setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_POST_OPERATION);
break;
case SPLIT_TABLE_REGION_POST_OPERATION:
postSplitRegion(env);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
String msg = "Error trying to split region " + parentHRI.getEncodedName() + " in the table "
+ tableName + " (in state=" + state + ")";
if (!isRollbackSupported(state)) {
// We reach a state that cannot be rolled back. We just need to keep retry.
LOG.warn(msg, e);
} else {
LOG.error(msg, e);
setFailure("master-split-region", e);
}
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final SplitTableRegionState state)
throws IOException, InterruptedException {
if (isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case SPLIT_TABLE_REGION_POST_OPERATION:
case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR:
case SPLIT_TABLE_REGION_UPDATE_META:
// PONR
throw new UnsupportedOperationException(this + " unhandled state=" + state);
case SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR:
break;
case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS:
// Doing nothing, as re-open parent region would clean up daughter region directories.
break;
case SPLIT_TABLE_REGION_CLOSED_PARENT_REGION:
openParentRegion(env);
break;
case SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE:
setRegionStateToRevertSplitting(env);
break;
case SPLIT_TABLE_REGION_PRE_OPERATION:
preSplitRegionRollback(env);
break;
case SPLIT_TABLE_REGION_PREPARE:
break; // nothing to do
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
// This will be retried. Unless there is a bug in the code,
// this should be just a "temporary error" (e.g. network down)
LOG.warn("Failed rollback attempt step " + state + " for splitting the region "
+ parentHRI.getEncodedName() + " in table " + tableName, e);
throw e;
}
}
/*
* Check whether we are in the state that can be rollback
*/
@Override
protected boolean isRollbackSupported(final SplitTableRegionState state) {
switch (state) {
case SPLIT_TABLE_REGION_POST_OPERATION:
case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR:
case SPLIT_TABLE_REGION_UPDATE_META:
// It is not safe to rollback if we reach to these states.
return false;
default:
break;
}
return true;
}
@Override
protected SplitTableRegionState getState(final int stateId) {
return SplitTableRegionState.valueOf(stateId);
}
@Override
protected int getStateId(final SplitTableRegionState state) {
return state.getNumber();
}
@Override
protected SplitTableRegionState getInitialState() {
return SplitTableRegionState.SPLIT_TABLE_REGION_PREPARE;
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.SplitTableRegionStateData.Builder splitTableRegionMsg =
MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setParentRegionInfo(HRegionInfo.convert(parentHRI));
if (splitRow != null) {
splitTableRegionMsg.setSplitRow(UnsafeByteOperations.unsafeWrap(splitRow));
}
if (daughter_1_HRI != null) {
splitTableRegionMsg.addChildRegionInfo(HRegionInfo.convert(daughter_1_HRI));
}
if (daughter_2_HRI != null) {
splitTableRegionMsg.addChildRegionInfo(HRegionInfo.convert(daughter_2_HRI));
}
splitTableRegionMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.SplitTableRegionStateData splitTableRegionsMsg =
MasterProcedureProtos.SplitTableRegionStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo());
tableName = ProtobufUtil.toTableName(splitTableRegionsMsg.getTableName());
parentHRI = HRegionInfo.convert(splitTableRegionsMsg.getParentRegionInfo());
if (splitTableRegionsMsg.hasSplitRow()) {
splitRow = splitTableRegionsMsg.getSplitRow().toByteArray();
} else {
splitRow = null;
}
if (splitTableRegionsMsg.getChildRegionInfoCount() == 0) {
daughter_1_HRI = daughter_2_HRI = null;
} else {
assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
daughter_1_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfoList().get(0));
daughter_2_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfoList().get(1));
}
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (table=");
sb.append(tableName);
sb.append(" parent region=");
sb.append(parentHRI);
if (daughter_1_HRI != null) {
sb.append(" first daughter region=");
sb.append(daughter_1_HRI);
}
if (daughter_2_HRI != null) {
sb.append(" and second daughter region=");
sb.append(daughter_2_HRI);
}
sb.append(")");
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
if (env.waitInitialized(this)) {
return false;
}
return !env.getProcedureQueue().waitRegions(this, getTableName(), parentHRI);
}
@Override
protected void releaseLock(final MasterProcedureEnv env) {
env.getProcedureQueue().wakeRegions(this, getTableName(), parentHRI);
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.SPLIT;
}
/**
* Prepare to Split region.
* @param env MasterProcedureEnv
* @throws IOException
*/
@VisibleForTesting
public Boolean prepareSplitRegion(final MasterProcedureEnv env) throws IOException {
// Check whether the region is splittable
RegionState state = getParentRegionState(env);
if (state.isClosing() ||
state.isClosed() ||
state.isSplittingOrSplitOnServer(state.getServerName())) {
setFailure(
"master-split-region",
new IOException("Split region " + parentHRI + " failed due to region is not splittable"));
return false;
}
// Split key can be null if this region is unsplittable; i.e. has refs.
if (this.splitRow == null || this.splitRow.length == 0) {
setFailure(
"master-split-region",
new IOException("Split region " + parentHRI + " failed due to invalid split point"));
return false;
}
// Check splitRow.
byte [] startKey = parentHRI.getStartKey();
byte [] endKey = parentHRI.getEndKey();
if (Bytes.equals(startKey, splitRow) ||
!this.parentHRI.containsRow(splitRow)) {
String msg = "Split row is not inside region key range or is equal to " +
"startkey: " + Bytes.toStringBinary(this.splitRow);
LOG.warn(msg);
setFailure("master-split-region", new IOException(msg));
return false;
}
long rid = getDaughterRegionIdTimestamp(parentHRI);
this.daughter_1_HRI = new HRegionInfo(tableName, startKey, this.splitRow, false, rid);
this.daughter_2_HRI = new HRegionInfo(tableName, this.splitRow, endKey, false, rid);
return true;
}
/**
* Action before splitting region in a table.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void preSplitRegion(final MasterProcedureEnv env)
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preSplitRegionAction(getTableName(), splitRow, user);
}
}
/**
* Action during rollback a pre split table region.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
*/
private void preSplitRegionRollback(final MasterProcedureEnv env) throws IOException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preRollBackSplitAction(user);
}
}
/**
* Set the parent region state to SPLITTING state
* @param env MasterProcedureEnv
* @throws IOException
*/
@VisibleForTesting
public void setRegionStateToSplitting(final MasterProcedureEnv env) throws IOException {
RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
transition.setTransitionCode(TransitionCode.READY_TO_SPLIT);
transition.addRegionInfo(HRegionInfo.convert(parentHRI));
transition.addRegionInfo(HRegionInfo.convert(daughter_1_HRI));
transition.addRegionInfo(HRegionInfo.convert(daughter_2_HRI));
if (env.getMasterServices().getAssignmentManager().onRegionTransition(
getParentRegionState(env).getServerName(), transition.build()) != null) {
throw new IOException("Failed to update region state to SPLITTING for "
+ parentHRI.getRegionNameAsString());
}
}
/**
* Rollback the region state change
* @param env MasterProcedureEnv
* @throws IOException
*/
private void setRegionStateToRevertSplitting(final MasterProcedureEnv env) throws IOException {
RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
transition.setTransitionCode(TransitionCode.SPLIT_REVERTED);
transition.addRegionInfo(HRegionInfo.convert(parentHRI));
transition.addRegionInfo(HRegionInfo.convert(daughter_1_HRI));
transition.addRegionInfo(HRegionInfo.convert(daughter_2_HRI));
if (env.getMasterServices().getAssignmentManager().onRegionTransition(
getParentRegionState(env).getServerName(), transition.build()) != null) {
throw new IOException("Failed to update region state for "
+ parentHRI.getRegionNameAsString() + " as part of operation for reverting split");
}
}
/**
* RPC to region server that host the parent region, ask for close the parent regions and
* creating daughter regions
* @param env MasterProcedureEnv
* @throws IOException
*/
@VisibleForTesting
public void closeParentRegionForSplit(final MasterProcedureEnv env) throws IOException {
Boolean success = env.getMasterServices().getServerManager().sendRegionCloseForSplit(
getParentRegionState(env).getServerName(), parentHRI);
if (!success) {
throw new IOException("Close parent region " + parentHRI + " for splitting failed."
+ " Check region server log for more details");
}
}
/**
* Rollback close parent region
* @param env MasterProcedureEnv
**/
private void openParentRegion(final MasterProcedureEnv env) throws IOException {
// Check whether the region is closed; if so, open it in the same server
RegionState state = getParentRegionState(env);
if (state.isClosing() || state.isClosed()) {
env.getMasterServices().getServerManager().sendRegionOpen(
getParentRegionState(env).getServerName(),
parentHRI,
ServerName.EMPTY_SERVER_LIST);
}
}
/**
* Create daughter regions
* @param env MasterProcedureEnv
* @throws IOException
*/
@VisibleForTesting
public void createDaughterRegions(final MasterProcedureEnv env) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), parentHRI.getTable());
final FileSystem fs = mfs.getFileSystem();
HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
env.getMasterConfiguration(), fs, tabledir, parentHRI, false);
regionFs.createSplitsDir();
Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
assertReferenceFileCount(
fs, expectedReferences.getFirst(), regionFs.getSplitsDir(daughter_1_HRI));
//Move the files from the temporary .splits to the final /table/region directory
regionFs.commitDaughterRegion(daughter_1_HRI);
assertReferenceFileCount(
fs,
expectedReferences.getFirst(),
new Path(tabledir, daughter_1_HRI.getEncodedName()));
assertReferenceFileCount(
fs, expectedReferences.getSecond(), regionFs.getSplitsDir(daughter_2_HRI));
regionFs.commitDaughterRegion(daughter_2_HRI);
assertReferenceFileCount(
fs,
expectedReferences.getSecond(),
new Path(tabledir, daughter_2_HRI.getEncodedName()));
}
/**
* Create Split directory
* @param env MasterProcedureEnv
* @throws IOException
*/
private Pair<Integer, Integer> splitStoreFiles(
final MasterProcedureEnv env,
final HRegionFileSystem regionFs) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final Configuration conf = env.getMasterConfiguration();
// The following code sets up a thread pool executor with as many slots as
// there's files to split. It then fires up everything, waits for
// completion and finally checks for any exception
//
// Note: splitStoreFiles creates daughter region dirs under the parent splits dir
// Nothing to unroll here if failure -- re-run createSplitsDir will
// clean this up.
int nbFiles = 0;
Collection<StoreFileInfo> storeFiles;
for (String family: regionFs.getFamilies()) {
storeFiles = regionFs.getStoreFiles(family);
if (storeFiles != null) {
nbFiles += storeFiles.size();
}
}
if (nbFiles == 0) {
// no file needs to be splitted.
return new Pair<Integer, Integer>(0,0);
}
// Default max #threads to use is the smaller of table's configured number of blocking store
// files or the available number of logical cores.
int defMaxThreads = Math.min(
conf.getInt(HStore.BLOCKING_STOREFILES_KEY, HStore.DEFAULT_BLOCKING_STOREFILE_COUNT),
Runtime.getRuntime().availableProcessors());
// Max #threads is the smaller of the number of storefiles or the default max determined above.
int maxThreads = Math.min(
conf.getInt(HConstants.REGION_SPLIT_THREADS_MAX, defMaxThreads), nbFiles);
LOG.info("Preparing to split " + nbFiles + " storefiles for region " + parentHRI +
" using " + maxThreads + " threads");
ThreadPoolExecutor threadPool = (ThreadPoolExecutor) Executors.newFixedThreadPool(
maxThreads, Threads.getNamedThreadFactory("StoreFileSplitter-%1$d"));
List<Future<Pair<Path,Path>>> futures = new ArrayList<Future<Pair<Path,Path>>> (nbFiles);
// Split each store file.
for (String family: regionFs.getFamilies()) {
HColumnDescriptor hcd =
env.getMasterServices().getTableDescriptors().get(tableName).getFamily(family.getBytes());
CacheConfig cacheConf = new CacheConfig(conf, hcd);
storeFiles = regionFs.getStoreFiles(family);
if (storeFiles != null) {
for (StoreFileInfo storeFileInfo: storeFiles) {
StoreFileSplitter sfs = new StoreFileSplitter(
regionFs,
family.getBytes(),
new StoreFile(
mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()));
futures.add(threadPool.submit(sfs));
}
}
}
// Shutdown the pool
threadPool.shutdown();
// Wait for all the tasks to finish
long fileSplitTimeout = conf.getLong("hbase.master.fileSplitTimeout", 30000);
try {
boolean stillRunning = !threadPool.awaitTermination(fileSplitTimeout, TimeUnit.MILLISECONDS);
if (stillRunning) {
threadPool.shutdownNow();
// wait for the thread to shutdown completely.
while (!threadPool.isTerminated()) {
Thread.sleep(50);
}
throw new IOException("Took too long to split the" +
" files and create the references, aborting split");
}
} catch (InterruptedException e) {
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
}
int daughterA = 0;
int daughterB = 0;
// Look for any exception
for (Future<Pair<Path, Path>> future : futures) {
try {
Pair<Path, Path> p = future.get();
daughterA += p.getFirst() != null ? 1 : 0;
daughterB += p.getSecond() != null ? 1 : 0;
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException().initCause(e);
} catch (ExecutionException e) {
throw new IOException(e);
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Split storefiles for region " + parentHRI + " Daughter A: " + daughterA
+ " storefiles, Daughter B: " + daughterB + " storefiles.");
}
return new Pair<Integer, Integer>(daughterA, daughterB);
}
private void assertReferenceFileCount(
final FileSystem fs,
final int expectedReferenceFileCount,
final Path dir)
throws IOException {
if (expectedReferenceFileCount != 0 &&
expectedReferenceFileCount != FSUtils.getRegionReferenceFileCount(fs, dir)) {
throw new IOException("Failing split. Expected reference file count isn't equal.");
}
}
private Pair<Path, Path> splitStoreFile(
final HRegionFileSystem regionFs,
final byte[] family,
final StoreFile sf)
throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Splitting started for store file: " + sf.getPath() + " for region: " + parentHRI);
}
String familyName = Bytes.toString(family);
Path path_first =
regionFs.splitStoreFile(this.daughter_1_HRI, familyName, sf, this.splitRow, false, null);
Path path_second =
regionFs.splitStoreFile(this.daughter_2_HRI, familyName, sf, this.splitRow, true, null);
if (LOG.isDebugEnabled()) {
LOG.debug("Splitting complete for store file: " + sf.getPath() + " for region: " + parentHRI);
}
return new Pair<Path,Path>(path_first, path_second);
}
/**
* Utility class used to do the file splitting / reference writing
* in parallel instead of sequentially.
*/
private class StoreFileSplitter implements Callable<Pair<Path,Path>> {
private final HRegionFileSystem regionFs;
private final byte[] family;
private final StoreFile sf;
/**
* Constructor that takes what it needs to split
* @param regionFs the file system
* @param family Family that contains the store file
* @param sf which file
*/
public StoreFileSplitter(
final HRegionFileSystem regionFs,
final byte[] family,
final StoreFile sf) {
this.regionFs = regionFs;
this.sf = sf;
this.family = family;
}
public Pair<Path,Path> call() throws IOException {
return splitStoreFile(regionFs, family, sf);
}
}
/**
* Post split region actions before the Point-of-No-Return step
* @param env MasterProcedureEnv
**/
private void preSplitRegionBeforePONR(final MasterProcedureEnv env)
throws IOException, InterruptedException {
final List<Mutation> metaEntries = new ArrayList<Mutation>();
boolean ret = false;
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
ret = cpHost.preSplitBeforePONRAction(splitRow, metaEntries, user);
if (ret) {
throw new IOException("Coprocessor bypassing region "
+ parentHRI.getRegionNameAsString() + " split.");
}
try {
for (Mutation p : metaEntries) {
HRegionInfo.parseRegionName(p.getRow());
}
} catch (IOException e) {
LOG.error("Row key of mutation from coprossor is not parsable as region name."
+ "Mutations from coprocessor should only for hbase:meta table.");
throw e;
}
}
}
/**
* Add daughter regions to META
* @param env MasterProcedureEnv
* @throws IOException
*/
private void updateMetaForDaughterRegions(final MasterProcedureEnv env) throws IOException {
RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
transition.setTransitionCode(TransitionCode.SPLIT_PONR);
transition.addRegionInfo(HRegionInfo.convert(parentHRI));
transition.addRegionInfo(HRegionInfo.convert(daughter_1_HRI));
transition.addRegionInfo(HRegionInfo.convert(daughter_2_HRI));
if (env.getMasterServices().getAssignmentManager().onRegionTransition(
getParentRegionState(env).getServerName(), transition.build()) != null) {
throw new IOException("Failed to update meta to add daughter regions in split region "
+ parentHRI.getRegionNameAsString());
}
}
/**
* Pre split region actions after the Point-of-No-Return step
* @param env MasterProcedureEnv
**/
private void preSplitRegionAfterPONR(final MasterProcedureEnv env)
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preSplitAfterPONRAction(user);
}
}
/**
* Assign daughter regions
* @param env MasterProcedureEnv
* @throws IOException
* @throws InterruptedException
**/
private void openDaughterRegions(
final MasterProcedureEnv env) throws IOException, InterruptedException {
env.getMasterServices().getAssignmentManager().assignDaughterRegions(
parentHRI, daughter_1_HRI, daughter_2_HRI);
}
/**
* Post split region actions
* @param env MasterProcedureEnv
**/
private void postSplitRegion(final MasterProcedureEnv env)
throws IOException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.postCompletedSplitRegionAction(daughter_1_HRI, daughter_2_HRI, user);
}
}
/**
* Calculate daughter regionid to use.
* @param hri Parent {@link HRegionInfo}
* @return Daughter region id (timestamp) to use.
*/
private long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
long rid = EnvironmentEdgeManager.currentTime();
// Regionid is timestamp. Can't be less than that of parent else will insert
// at wrong location in hbase:meta (See HBASE-710).
if (rid < hri.getRegionId()) {
LOG.warn("Clock skew: parent regions id is " + hri.getRegionId() +
" but current time here is " + rid);
rid = hri.getRegionId() + 1;
}
return rid;
}
/**
* Get parent region state
* @param env MasterProcedureEnv
* @return parent region state
*/
private RegionState getParentRegionState(final MasterProcedureEnv env) {
RegionStates regionStates = env.getMasterServices().getAssignmentManager().getRegionStates();
RegionState state = regionStates.getRegionState(parentHRI);
if (state == null) {
LOG.warn("Split but not in region states: " + parentHRI);
state = regionStates.createRegionState(parentHRI);
}
return state;
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @return traceEnabled
*/
private Boolean isTraceEnabled() {
if (traceEnabled == null) {
traceEnabled = LOG.isTraceEnabled();
}
return traceEnabled;
}
}

View File

@ -6649,7 +6649,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @param hri Spec. for daughter region to open.
* @throws IOException
*/
HRegion createDaughterRegionFromSplits(final HRegionInfo hri) throws IOException {
public HRegion createDaughterRegionFromSplits(final HRegionInfo hri) throws IOException {
// Move the files from the temporary .splits to the final /table/region directory
fs.commitDaughterRegion(hri);

View File

@ -480,7 +480,7 @@ public class HRegionFileSystem {
return new Path(getRegionDir(), REGION_SPLITS_DIR);
}
Path getSplitsDir(final HRegionInfo hri) {
public Path getSplitsDir(final HRegionInfo hri) {
return new Path(getSplitsDir(), hri.getEncodedName());
}
@ -539,7 +539,7 @@ public class HRegionFileSystem {
* @param regionInfo daughter {@link org.apache.hadoop.hbase.HRegionInfo}
* @throws IOException
*/
Path commitDaughterRegion(final HRegionInfo regionInfo)
public Path commitDaughterRegion(final HRegionInfo regionInfo)
throws IOException {
Path regionDir = new Path(this.tableDir, regionInfo.getEncodedName());
Path daughterTmpDir = this.getSplitsDir(regionInfo);
@ -563,7 +563,7 @@ public class HRegionFileSystem {
/**
* Create the region splits directory.
*/
void createSplitsDir() throws IOException {
public void createSplitsDir() throws IOException {
Path splitdir = getSplitsDir();
if (fs.exists(splitdir)) {
LOG.info("The " + splitdir + " directory exists. Hence deleting it to recreate it");
@ -590,12 +590,15 @@ public class HRegionFileSystem {
* @return Path to created reference.
* @throws IOException
*/
Path splitStoreFile(final HRegionInfo hri, final String familyName, final StoreFile f,
public Path splitStoreFile(final HRegionInfo hri, final String familyName, final StoreFile f,
final byte[] splitRow, final boolean top, RegionSplitPolicy splitPolicy)
throws IOException {
if (splitPolicy == null || !splitPolicy.skipStoreFileRangeCheck(familyName)) {
// Check whether the split row lies in the range of the store file
// If it is outside the range, return directly.
if (f.getReader() == null) {
f.createReader();
}
try {
if (top) {
//check if larger than last key.

View File

@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.NonceGenerator;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
import org.apache.hadoop.hbase.conf.ConfigurationManager;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
@ -142,6 +143,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringP
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@ -153,6 +156,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse;
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
@ -160,6 +165,7 @@ import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.JSONBean;
import org.apache.hadoop.hbase.util.JvmPauseMonitor;
@ -2099,6 +2105,81 @@ public class HRegionServer extends HasThread implements
return false;
}
@Override
public long requestRegionSplit(final HRegionInfo regionInfo, final byte[] splitRow) {
NonceGenerator ng = clusterConnection.getNonceGenerator();
final long nonceGroup = ng.getNonceGroup();
final long nonce = ng.newNonce();
long procId = -1;
SplitTableRegionRequest request =
RequestConverter.buildSplitTableRegionRequest(regionInfo, splitRow, nonceGroup, nonce);
while (keepLooping()) {
RegionServerStatusService.BlockingInterface rss = rssStub;
try {
if (rss == null) {
createRegionServerStatusStub();
continue;
}
SplitTableRegionResponse response = rss.splitRegion(null, request);
//TODO: should we limit the retry number before quitting?
if (response == null || (procId = response.getProcId()) == -1) {
LOG.warn("Failed to split " + regionInfo + " retrying...");
continue;
}
break;
} catch (ServiceException se) {
// TODO: retry or just fail
IOException ioe = ProtobufUtil.getRemoteException(se);
LOG.info("Failed to split region, will retry", ioe);
if (rssStub == rss) {
rssStub = null;
}
}
}
return procId;
}
@Override
public boolean isProcedureFinished(final long procId) throws IOException {
GetProcedureResultRequest request =
GetProcedureResultRequest.newBuilder().setProcId(procId).build();
while (keepLooping()) {
RegionServerStatusService.BlockingInterface rss = rssStub;
try {
if (rss == null) {
createRegionServerStatusStub();
continue;
}
// TODO: find a way to get proc result
GetProcedureResultResponse response = rss.getProcedureResult(null, request);
if (response == null) {
LOG.warn("Failed to get procedure (id=" + procId + ") status.");
return false;
} else if (response.getState() == GetProcedureResultResponse.State.RUNNING) {
return false;
} else if (response.hasException()) {
// Procedure failed.
throw ForeignExceptionUtil.toIOException(response.getException());
}
// Procedure completes successfully
break;
} catch (ServiceException se) {
// TODO: retry or just fail
IOException ioe = ProtobufUtil.getRemoteException(se);
LOG.warn("Failed to get split region procedure result. Retrying", ioe);
if (rssStub == rss) {
rssStub = null;
}
}
}
return true;
}
/**
* Trigger a flush in the primary region replica if this region is a secondary replica. Does not
* block this thread. See RegionReplicaFlushHandler for details.
@ -2946,6 +3027,41 @@ public class HRegionServer extends HasThread implements
return true;
}
/**
* Close and offline the region for split
*
* @param parentRegionEncodedName the name of the region to close
* @return True if closed the region successfully.
* @throws IOException
*/
protected boolean closeAndOfflineRegionForSplit(
final String parentRegionEncodedName) throws IOException {
Region parentRegion = this.getFromOnlineRegions(parentRegionEncodedName);
if (parentRegion != null) {
Map<byte[], List<StoreFile>> hstoreFilesToSplit = null;
Exception exceptionToThrow = null;
try{
hstoreFilesToSplit = ((HRegion)parentRegion).close(false);
} catch (Exception e) {
exceptionToThrow = e;
}
if (exceptionToThrow == null && hstoreFilesToSplit == null) {
// The region was closed by someone else
exceptionToThrow =
new IOException("Failed to close region: already closed by another thread");
}
if (exceptionToThrow != null) {
if (exceptionToThrow instanceof IOException) throw (IOException)exceptionToThrow;
throw new IOException(exceptionToThrow);
}
// Offline the region
this.removeFromOnlineRegions(parentRegion, null);
}
return true;
}
/**
* @param regionName
* @return HRegion for the passed binary <code>regionName</code> or null if

View File

@ -93,6 +93,8 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
@ -1361,6 +1363,33 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
@Override
@QosPriority(priority=HConstants.ADMIN_QOS)
public CloseRegionForSplitResponse closeRegionForSplit(
final RpcController controller,
final CloseRegionForSplitRequest request) throws ServiceException {
try {
checkOpen();
final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion());
// Can be null if we're calling close on a region that's not online
final Region parentRegion = regionServer.getFromOnlineRegions(encodedRegionName);
if ((parentRegion != null) && (parentRegion.getCoprocessorHost() != null)) {
parentRegion.getCoprocessorHost().preClose(false);
}
requestCount.increment();
LOG.info("Close and offline " + encodedRegionName + " and prepare for split.");
boolean closed = regionServer.closeAndOfflineRegionForSplit(encodedRegionName);
CloseRegionForSplitResponse.Builder builder =
CloseRegionForSplitResponse.newBuilder().setClosed(closed);
return builder.build();
} catch (IOException ie) {
throw new ServiceException(ie);
}
}
/**
* Compact a region on the region server.
*

View File

@ -707,7 +707,7 @@ public class RegionCoprocessorHost
* Invoked just before a split
* @throws IOException
*/
// TODO: Deprecate this
@Deprecated
public void preSplit(final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@Override
@ -721,7 +721,10 @@ public class RegionCoprocessorHost
/**
* Invoked just before a split
* @throws IOException
*
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
public void preSplit(final byte[] splitRow, final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@Override
@ -737,7 +740,10 @@ public class RegionCoprocessorHost
* @param l the new left-hand daughter region
* @param r the new right-hand daughter region
* @throws IOException
*
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
public void postSplit(final Region l, final Region r, final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@Override
@ -748,6 +754,10 @@ public class RegionCoprocessorHost
});
}
/**
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
public boolean preSplitBeforePONR(final byte[] splitKey,
final List<Mutation> metaEntries, final User user) throws IOException {
return execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@ -759,6 +769,10 @@ public class RegionCoprocessorHost
});
}
/**
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
public void preSplitAfterPONR(final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@Override
@ -772,7 +786,10 @@ public class RegionCoprocessorHost
/**
* Invoked just before the rollback of a failed split is started
* @throws IOException
*/
*
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
public void preRollBackSplit(final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@Override
@ -786,7 +803,10 @@ public class RegionCoprocessorHost
/**
* Invoked just after the rollback of a failed split is done
* @throws IOException
*/
*
* Note: the logic moves to Master; it is unused in RS
*/
@Deprecated
public void postRollBackSplit(final User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@Override

View File

@ -180,6 +180,16 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
@Deprecated
boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris);
/**
* Notify master that a region wants to be splitted.
*/
long requestRegionSplit(final HRegionInfo regionInfo, final byte[] splitRow);
/**
* Check with master whether a procedure is completed (either succeed or fail)
*/
boolean isProcedureFinished(final long procId) throws IOException;
/**
* Returns a reference to the region server's RPC server
*/

View File

@ -19,17 +19,15 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.security.PrivilegedAction;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import com.google.common.base.Preconditions;
@ -43,7 +41,6 @@ class SplitRequest implements Runnable {
private final byte[] midKey;
private final HRegionServer server;
private final User user;
private TableLock tableLock;
SplitRequest(Region region, byte[] midKey, HRegionServer hrs, User user) {
Preconditions.checkNotNull(hrs);
@ -58,63 +55,48 @@ class SplitRequest implements Runnable {
return "regionName=" + parent + ", midKey=" + Bytes.toStringBinary(midKey);
}
private void doSplitting(User user) {
private void doSplitting() {
boolean success = false;
server.metricsRegionServer.incrSplitRequest();
long startTime = EnvironmentEdgeManager.currentTime();
SplitTransactionImpl st = new SplitTransactionImpl(parent, midKey);
try {
//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.getRegionInfo().getRegionNameAsString());
try {
tableLock.acquire();
} catch (IOException ex) {
tableLock = null;
throw ex;
long procId;
if (user != null && user.getUGI() != null) {
procId = user.getUGI().doAs (new PrivilegedAction<Long>() {
@Override
public Long run() {
try {
return server.requestRegionSplit(parent.getRegionInfo(), midKey);
} catch (Exception e) {
LOG.error("Failed to complete region split ", e);
}
return (long)-1;
}
});
} else {
procId = server.requestRegionSplit(parent.getRegionInfo(), midKey);
}
// If prepare does not return true, for some reason -- logged inside in
// the prepare call -- we are not ready to split just now. Just return.
if (!st.prepare()) return;
try {
st.execute(this.server, this.server, user);
success = true;
} catch (Exception e) {
if (this.server.isStopping() || this.server.isStopped()) {
LOG.info(
"Skip rollback/cleanup of failed split of "
+ parent.getRegionInfo().getRegionNameAsString() + " because server is"
+ (this.server.isStopping() ? " stopping" : " stopped"), e);
return;
}
if (e instanceof DroppedSnapshotException) {
server.abort("Replay of WAL required. Forcing server shutdown", e);
return;
}
if (procId != -1) {
// wait for the split to complete or get interrupted. If the split completes successfully,
// the procedure will return true; if the split fails, the procedure would throw exception.
//
try {
LOG.info("Running rollback/cleanup of failed split of " +
parent.getRegionInfo().getRegionNameAsString() + "; " + e.getMessage(), e);
if (st.rollback(this.server, this.server)) {
LOG.info("Successful rollback of failed split of " +
parent.getRegionInfo().getRegionNameAsString());
} else {
this.server.abort("Abort; we got an error after point-of-no-return");
while (!(success = server.isProcedureFinished(procId))) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
LOG.warn("Split region " + parent + " is still in progress. Not waiting...");
break;
}
}
} catch (RuntimeException ee) {
String msg = "Failed rollback of failed split of " +
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());
} catch (IOException e) {
LOG.error("Split region " + parent + " failed.", e);
}
return;
} else {
LOG.error("Fail to split region " + parent);
}
} catch (IOException ex) {
ex = ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex;
LOG.error("Split failed " + this, ex);
server.checkFileSystem();
} finally {
if (this.parent.getCoprocessorHost() != null) {
try {
@ -124,24 +106,17 @@ class SplitRequest implements Runnable {
io instanceof RemoteException ? ((RemoteException) io).unwrapRemoteException() : io);
}
}
// Update regionserver metrics with the split transaction total running time
server.metricsRegionServer.updateSplitTime(EnvironmentEdgeManager.currentTime() - startTime);
if (parent.shouldForceSplit()) {
parent.clearSplit();
}
releaseTableLock();
long endTime = EnvironmentEdgeManager.currentTime();
// Update regionserver metrics with the split transaction total running time
server.metricsRegionServer.updateSplitTime(endTime - startTime);
if (success) {
server.metricsRegionServer.incrSplitSuccess();
// Log success
LOG.info("Region split, hbase:meta updated, and report to master. Parent="
+ parent.getRegionInfo().getRegionNameAsString() + ", new regions: "
+ st.getFirstDaughter().getRegionNameAsString() + ", "
+ st.getSecondDaughter().getRegionNameAsString() + ". Split took "
+ StringUtils.formatTimeDiff(EnvironmentEdgeManager.currentTime(), startTime));
}
// Always log the split transaction journal
LOG.info("Split transaction journal:\n\t" + StringUtils.join("\n\t", st.getJournal()));
}
}
@ -152,19 +127,7 @@ class SplitRequest implements Runnable {
this.server.isStopping() + " or stopped=" + this.server.isStopped());
return;
}
doSplitting(user);
}
protected void releaseTableLock() {
if (this.tableLock != null) {
try {
this.tableLock.release();
} catch (IOException ex) {
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.getRegionInfo().getRegionNameAsString());
}
}
doSplitting();
}
}

View File

@ -1446,6 +1446,14 @@ public class AccessController extends BaseMasterAndRegionObserver
Action.ADMIN, Action.CREATE);
}
@Override
public void preSplitRegion(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final byte[] splitRow) throws IOException {
requirePermission(getActiveUser(ctx), "split", tableName, null, null, Action.ADMIN);
}
/* ---- RegionObserver implementation ---- */
@Override
@ -1509,19 +1517,6 @@ public class AccessController extends BaseMasterAndRegionObserver
Action.ADMIN, Action.CREATE);
}
@Override
public void preSplit(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {
requirePermission(getActiveUser(c), "split", getTableName(c.getEnvironment()), null, null,
Action.ADMIN);
}
@Override
public void preSplit(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] splitRow) throws IOException {
requirePermission(getActiveUser(c), "split", getTableName(c.getEnvironment()), null, null,
Action.ADMIN);
}
@Override
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
final Store store, final InternalScanner scanner, final ScanType scanType)

View File

@ -306,6 +306,16 @@ public class MockRegionServerServices implements RegionServerServices {
return false;
}
@Override
public long requestRegionSplit(final HRegionInfo regionInfo, final byte[] splitRow) {
return -1;
}
@Override
public boolean isProcedureFinished(final long procId) {
return false;
}
@Override
public boolean registerService(Service service) {
// TODO Auto-generated method stub

View File

@ -163,9 +163,7 @@ public class TestCoprocessorInterface {
private boolean postCompactCalled;
private boolean preFlushCalled;
private boolean postFlushCalled;
private boolean preSplitCalled;
private boolean postSplitCalled;
private boolean preSplitWithSplitRowCalled;
private ConcurrentMap<String, Object> sharedData;
@Override
@ -218,16 +216,6 @@ public class TestCoprocessorInterface {
postFlushCalled = true;
}
@Override
public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) {
preSplitCalled = true;
}
@Override
public void preSplit(ObserverContext<RegionCoprocessorEnvironment> c,
byte[] splitRow) throws IOException {
preSplitWithSplitRowCalled = true;
}
@Override
public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, Region l, Region r) {
postSplitCalled = true;
}
@ -257,7 +245,7 @@ public class TestCoprocessorInterface {
return (preCompactCalled && postCompactCalled);
}
boolean wasSplit() {
return (preSplitCalled && postSplitCalled && preSplitWithSplitRowCalled);
return postSplitCalled;
}
Map<String, Object> getSharedData() {
return sharedData;

View File

@ -49,6 +49,7 @@ 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.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster;
@ -1466,6 +1467,44 @@ public class TestMasterObserver {
public void postBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
String groupName, boolean balancerRan) throws IOException {
}
@Override
public void preSplitRegion(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow) throws IOException {
}
@Override
public void preSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> c,
final TableName tableName,
final byte[] splitRow) throws IOException {
}
@Override
public void postCompletedSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> c,
final HRegionInfo regionInfoA,
final HRegionInfo regionInfoB) throws IOException {
}
@Override
public void preSplitRegionBeforePONRAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final byte[] splitKey,
final List<Mutation> metaEntries) throws IOException {
}
@Override
public void preSplitRegionAfterPONRAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
}
@Override
public void preRollBackSplitRegionAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
}
}
private static HBaseTestingUtility UTIL = new HBaseTestingUtility();

View File

@ -286,6 +286,15 @@ public class MockNoopMasterServices implements MasterServices, Server {
return -1;
}
@Override
public long splitRegion(
final HRegionInfo regionInfo,
final byte[] splitRow,
final long nonceGroup,
final long nonce) throws IOException {
return -1;
}
@Override
public TableLockManager getTableLockManager() {
return null;

View File

@ -51,6 +51,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
@ -382,9 +384,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
return builder.build();
}
@Override
public MutateResponse mutate(RpcController controller, MutateRequest request)
throws ServiceException {
@ -491,6 +490,13 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
return null;
}
@Override
public CloseRegionForSplitResponse closeRegionForSplit(
RpcController controller,
CloseRegionForSplitRequest request) throws ServiceException {
return null;
}
@Override
public FlushRegionResponse flushRegion(RpcController controller,
FlushRegionRequest request) throws ServiceException {
@ -498,10 +504,19 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
return null;
}
@Override
public long requestRegionSplit(HRegionInfo regionInfo, byte[] splitRow) {
return -1;
}
@Override
public boolean isProcedureFinished(final long procId) {
return false;
}
@Override
public SplitRegionResponse splitRegion(RpcController controller,
SplitRegionRequest request) throws ServiceException {
// TODO Auto-generated method stub
return null;
}

View File

@ -0,0 +1,480 @@
/**
* 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.master.procedure;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
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.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({MasterTests.class, MediumTests.class})
public class TestSplitTableRegionProcedure {
private static final Log LOG = LogFactory.getLog(TestSplitTableRegionProcedure.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
private static String ColumnFamilyName1 = "cf1";
private static String ColumnFamilyName2 = "cf2";
private static final int startRowNum = 11;
private static final int rowCount = 60;
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
}
@BeforeClass
public static void setupCluster() throws Exception {
setupConf(UTIL.getConfiguration());
UTIL.startMiniCluster(3);
}
@AfterClass
public static void cleanupTest() throws Exception {
try {
UTIL.shutdownMiniCluster();
} catch (Exception e) {
LOG.warn("failure shutting down cluster", e);
}
}
@Before
public void setup() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
// Turn off balancer so it doesn't cut in and mess up our placements.
UTIL.getHBaseAdmin().setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us.
UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
}
@After
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
}
@Test(timeout=60000)
public void testSplitTableRegion() throws Exception {
final TableName tableName = TableName.valueOf("testSplitTableRegion");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
int splitRowNum = startRowNum + rowCount / 2;
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
// Split region of the table
long procId = procExec.submitProcedure(
new SplitTableRegionProcedure(
procExec.getEnvironment(), tableName, regions[0], splitKey),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
verify(tableName, splitRowNum);
}
@Test(timeout=60000)
public void testSplitTableRegionNoStoreFile() throws Exception {
final TableName tableName = TableName.valueOf("testSplitTableRegionNoStoreFile");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
int splitRowNum = startRowNum + rowCount / 2;
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
// Split region of the table
long procId = procExec.submitProcedure(
new SplitTableRegionProcedure(
procExec.getEnvironment(), tableName, regions[0], splitKey),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
assertTrue(UTIL.getMiniHBaseCluster().getRegions(tableName).size() == 2);
assertTrue(UTIL.countRows(tableName) == 0);
}
@Test(timeout=60000)
public void testSplitTableRegionUnevenDaughter() throws Exception {
final TableName tableName = TableName.valueOf("testSplitTableRegionUnevenDaughter");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
// Split to two daughters with one of them only has 1 row
int splitRowNum = startRowNum + rowCount / 4;
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
// Split region of the table
long procId = procExec.submitProcedure(
new SplitTableRegionProcedure(
procExec.getEnvironment(), tableName, regions[0], splitKey),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
verify(tableName, splitRowNum);
}
@Test(timeout=60000)
public void testSplitTableRegionEmptyDaughter() throws Exception {
final TableName tableName = TableName.valueOf("testSplitTableRegionEmptyDaughter");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
// Split to two daughters with one of them only has 1 row
int splitRowNum = startRowNum + rowCount;
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
// Split region of the table
long procId = procExec.submitProcedure(
new SplitTableRegionProcedure(
procExec.getEnvironment(), tableName, regions[0], splitKey),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
// Make sure one daughter has 0 rows.
List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);
assertTrue(daughters.size() == 2);
assertTrue(UTIL.countRows(tableName) == rowCount);
assertTrue(UTIL.countRows(daughters.get(0)) == 0 || UTIL.countRows(daughters.get(1)) == 0);
}
@Test(timeout=60000)
public void testSplitTableRegionDeletedRowsDaughter() throws Exception {
final TableName tableName = TableName.valueOf("testSplitTableRegionDeletedRowsDaughter");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
// Split to two daughters with one of them only has 1 row
int splitRowNum = rowCount;
deleteData(tableName, splitRowNum);
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
// Split region of the table
long procId = procExec.submitProcedure(
new SplitTableRegionProcedure(
procExec.getEnvironment(), tableName, regions[0], splitKey),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
UTIL.getHBaseAdmin().majorCompact(tableName);
// waiting for the major compaction to complete
UTIL.waitFor(6000, new Waiter.Predicate<IOException>() {
@Override
public boolean evaluate() throws IOException {
return UTIL.getHBaseAdmin().getCompactionState(tableName) == CompactionState.NONE;
}
});
// Make sure one daughter has 0 rows.
List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);
assertTrue(daughters.size() == 2);
final int currentRowCount = splitRowNum - startRowNum;
assertTrue(UTIL.countRows(tableName) == currentRowCount);
assertTrue(UTIL.countRows(daughters.get(0)) == 0 || UTIL.countRows(daughters.get(1)) == 0);
}
@Test(timeout=60000)
public void testSplitTableRegionTwiceWithSameNonce() throws Exception {
final TableName tableName = TableName.valueOf("testSplitTableRegionTwiceWithSameNonce");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
int splitRowNum = startRowNum + rowCount / 2;
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
// Split region of the table
long procId1 = procExec.submitProcedure(
new SplitTableRegionProcedure(
procExec.getEnvironment(), tableName, regions[0], splitKey),
nonceGroup,
nonce);
// Split region of the table with the same nonce
long procId2 = procExec.submitProcedure(
new SplitTableRegionProcedure(
procExec.getEnvironment(), tableName, regions[0], splitKey),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
// The second proc should succeed too - because it is the same proc.
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
assertTrue(procId1 == procId2);
verify(tableName, splitRowNum);
}
@Test(timeout=60000)
public void testInvalidSplitKey() throws Exception {
final TableName tableName = TableName.valueOf("testInvalidSplitKey");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
int splitRowNum = startRowNum + rowCount / 2;
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
// Split region of the table with null split key
long procId1 = procExec.submitProcedure(
new SplitTableRegionProcedure(procExec.getEnvironment(), tableName, regions[0], null),
nonceGroup,
nonce);
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureInfo result = procExec.getResult(procId1);
assertTrue(result.isFailed());
LOG.debug("Split failed with exception: " + result.getExceptionFullMessage());
assertTrue(UTIL.getMiniHBaseCluster().getRegions(tableName).size() == 1);
}
@Test(timeout = 600000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
int splitRowNum = startRowNum + rowCount / 2;
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Split region of the table
long procId = procExec.submitProcedure(
new SplitTableRegionProcedure(procExec.getEnvironment(), tableName, regions[0], splitKey),
nonceGroup,
nonce);
// Failing before SPLIT_TABLE_REGION_UPDATE_META we should trigger the
// rollback
// NOTE: the 5 (number before SPLIT_TABLE_REGION_UPDATE_META step) is
// hardcoded, so you have to look at this test at least once when you add a new step.
int numberOfSteps = 5;
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
procExec,
procId,
numberOfSteps);
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
insertData(tableName);
int splitRowNum = startRowNum + rowCount / 2;
byte[] splitKey = Bytes.toBytes("" + splitRowNum);
assertTrue("not able to find a splittable region", regions != null);
assertTrue("not able to find a splittable region", regions.length == 1);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Split region of the table
long procId = procExec.submitProcedure(
new SplitTableRegionProcedure(procExec.getEnvironment(), tableName, regions[0], splitKey),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = SplitTableRegionState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
verify(tableName, splitRowNum);
}
private void insertData(final TableName tableName) throws IOException, InterruptedException {
Table t = UTIL.getConnection().getTable(tableName);
Put p;
for (int i= 0; i < rowCount / 2; i++) {
p = new Put(Bytes.toBytes("" + (startRowNum + i)));
p.addColumn(Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes("q1"), Bytes.toBytes(i));
p.addColumn(Bytes.toBytes(ColumnFamilyName2), Bytes.toBytes("q2"), Bytes.toBytes(i));
t.put(p);
p = new Put(Bytes.toBytes("" + (startRowNum + rowCount - i - 1)));
p.addColumn(Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes("q1"), Bytes.toBytes(i));
p.addColumn(Bytes.toBytes(ColumnFamilyName2), Bytes.toBytes("q2"), Bytes.toBytes(i));
t.put(p);
if (i % 5 == 0) {
UTIL.getHBaseAdmin().flush(tableName);
}
}
}
private void deleteData(
final TableName tableName,
final int startDeleteRowNum) throws IOException, InterruptedException {
Table t = UTIL.getConnection().getTable(tableName);
final int numRows = rowCount + startRowNum - startDeleteRowNum;
Delete d;
for (int i= startDeleteRowNum; i <= numRows + startDeleteRowNum; i++) {
d = new Delete(Bytes.toBytes("" + i));
t.delete(d);
if (i % 5 == 0) {
UTIL.getHBaseAdmin().flush(tableName);
}
}
}
private void verify(final TableName tableName, final int splitRowNum) throws IOException {
List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);
assertTrue(daughters.size() == 2);
LOG.info("Row Count = " + UTIL.countRows(tableName));
assertTrue(UTIL.countRows(tableName) == rowCount);
int startRow;
int numRows;
for (int i = 0; i < daughters.size(); i++) {
if (Bytes.compareTo(
daughters.get(i).getRegionInfo().getStartKey(), HConstants.EMPTY_BYTE_ARRAY) == 0) {
startRow = startRowNum; // first region
numRows = splitRowNum - startRowNum;
} else {
startRow = splitRowNum;
numRows = rowCount + startRowNum - splitRowNum;
}
verifyData(
daughters.get(i),
startRow,
numRows,
ColumnFamilyName1.getBytes(),
ColumnFamilyName2.getBytes());
}
}
private void verifyData(
final HRegion newReg,
final int startRow,
final int numRows,
final byte[]... families)
throws IOException {
for (int i = startRow; i < startRow + numRows; i++) {
byte[] row = Bytes.toBytes("" + i);
Get get = new Get(row);
Result result = newReg.get(get);
Cell[] raw = result.rawCells();
assertEquals(families.length, result.size());
for (int j = 0; j < families.length; j++) {
assertTrue(CellUtil.matchingRow(raw[j], row));
assertTrue(CellUtil.matchingFamily(raw[j], families[j]));
}
}
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
}

View File

@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.Waiter;
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.client.Mutation;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
@ -495,7 +494,6 @@ public class TestNamespaceAuditor {
// Make sure no regions have been added.
List<HRegionInfo> hris = ADMIN.getTableRegions(tableOne);
assertEquals(2, hris.size());
assertTrue("split completed", observer.preSplitBeforePONR.getCount() == 1);
htable.close();
}
@ -570,7 +568,6 @@ public class TestNamespaceAuditor {
public static class CustomObserver extends BaseRegionObserver{
volatile CountDownLatch postSplit;
volatile CountDownLatch preSplitBeforePONR;
volatile CountDownLatch postCompact;
@Override
@ -585,17 +582,9 @@ public class TestNamespaceAuditor {
postCompact.countDown();
}
@Override
public void preSplitBeforePONR(ObserverContext<RegionCoprocessorEnvironment> ctx,
byte[] splitKey, List<Mutation> metaEntries) throws IOException {
preSplitBeforePONR.countDown();
}
@Override
public void start(CoprocessorEnvironment e) throws IOException {
postSplit = new CountDownLatch(1);
preSplitBeforePONR = new CountDownLatch(1);
postCompact = new CountDownLatch(1);
}
}

View File

@ -89,92 +89,6 @@ public class TestEndToEndSplitTransaction {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testMasterOpsWhileSplitting() throws Exception {
TableName tableName = TableName.valueOf("TestSplit");
byte[] familyName = Bytes.toBytes("fam");
try (Table ht = TEST_UTIL.createTable(tableName, familyName)) {
TEST_UTIL.loadTable(ht, familyName, false);
}
HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(0);
byte[] firstRow = Bytes.toBytes("aaa");
byte[] splitRow = Bytes.toBytes("lll");
byte[] lastRow = Bytes.toBytes("zzz");
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
// this will also cache the region
byte[] regionName = conn.getRegionLocator(tableName).getRegionLocation(splitRow)
.getRegionInfo().getRegionName();
Region region = server.getRegion(regionName);
SplitTransactionImpl split = new SplitTransactionImpl((HRegion) region, splitRow);
split.prepare();
// 1. phase I
PairOfSameType<Region> regions = split.createDaughters(server, server, null);
assertFalse(test(conn, tableName, firstRow, server));
assertFalse(test(conn, tableName, lastRow, server));
// passing null as services prevents final step
// 2, most of phase II
split.openDaughters(server, null, regions.getFirst(), regions.getSecond());
assertFalse(test(conn, tableName, firstRow, server));
assertFalse(test(conn, tableName, lastRow, server));
// 3. finish phase II
// note that this replicates some code from SplitTransaction
// 2nd daughter first
server.reportRegionStateTransition(
RegionServerStatusProtos.RegionStateTransition.TransitionCode.SPLIT,
region.getRegionInfo(), regions.getFirst().getRegionInfo(), regions.getSecond()
.getRegionInfo());
// Add to online regions
server.addToOnlineRegions(regions.getSecond());
// THIS is the crucial point:
// the 2nd daughter was added, so querying before the split key should fail.
assertFalse(test(conn, tableName, firstRow, server));
// past splitkey is ok.
assertTrue(test(conn, tableName, lastRow, server));
// Add to online regions
server.addToOnlineRegions(regions.getFirst());
assertTrue(test(conn, tableName, firstRow, server));
assertTrue(test(conn, tableName, lastRow, server));
assertTrue(test(conn, tableName, firstRow, server));
assertTrue(test(conn, tableName, lastRow, server));
}
}
/**
* attempt to locate the region and perform a get and scan
* @return True if successful, False otherwise.
*/
private boolean test(Connection conn, TableName tableName, byte[] row,
HRegionServer server) {
// not using HTable to avoid timeouts and retries
try {
byte[] regionName = conn.getRegionLocator(tableName).getRegionLocation(row, true)
.getRegionInfo().getRegionName();
// get and scan should now succeed without exception
ClientProtos.GetRequest request =
RequestConverter.buildGetRequest(regionName, new Get(row));
server.getRSRpcServices().get(null, request);
ScanRequest scanRequest = RequestConverter.buildScanRequest(
regionName, new Scan(row), 1, true);
try {
server.getRSRpcServices().scan(
new HBaseRpcControllerImpl(), scanRequest);
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException e) {
throw ProtobufUtil.handleRemoteException(e);
}
} catch (IOException e) {
return false;
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException e1) {
return false;
}
return true;
}
/**
* Tests that the client sees meta table changes as atomic during splits
*/

View File

@ -31,7 +31,6 @@ import java.io.InterruptedIOException;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
@ -52,14 +51,11 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
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.client.Consistency;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@ -70,10 +66,9 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TestReplicasClient.SlowMeCopro;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterRpcServices;
@ -95,7 +90,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -105,13 +99,12 @@ import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Like TestSplitTransaction in that we're testing {@link SplitTransactionImpl}
* only the below tests are against a running cluster where TestSplitTransaction
* is tests against a bare {@link HRegion}.
* The below tests are testing split region against a running cluster
*/
@Category({RegionServerTests.class, LargeTests.class})
@SuppressWarnings("deprecation")
@ -121,8 +114,6 @@ public class TestSplitTransactionOnCluster {
private Admin admin = null;
private MiniHBaseCluster cluster = null;
private static final int NB_SERVERS = 3;
private static CountDownLatch latch = new CountDownLatch(1);
private static volatile boolean secondSplit = false;
static final HBaseTestingUtility TESTING_UTIL =
new HBaseTestingUtility();
@ -145,21 +136,48 @@ public class TestSplitTransactionOnCluster {
@After
public void tearDown() throws Exception {
this.admin.close();
for (HTableDescriptor htd: this.admin.listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
TESTING_UTIL.deleteTable(htd.getTableName());
}
}
private HRegionInfo getAndCheckSingleTableRegion(final List<HRegion> regions)
throws IOException, InterruptedException {
assertEquals(1, regions.size());
HRegionInfo hri = regions.get(0).getRegionInfo();
TESTING_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager()
cluster.getMaster().getAssignmentManager()
.waitOnRegionToClearRegionsInTransition(hri, 600000);
return hri;
}
private void requestSplitRegion(
final HRegionServer rsServer,
final Region region,
final byte[] midKey) throws IOException {
long procId = cluster.getMaster().splitRegion(region.getRegionInfo(), midKey, 0, 0);
// wait
if (procId != -1) {
// wait for the split to complete or get interrupted. If the split completes successfully,
// the procedure will return true; if the split fails, the procedure would throw exception.
//
while (!rsServer.isProcedureFinished(procId)) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
throw new IOException("Split region interrupted.");
}
}
} else {
throw new IOException ("Request split region failed.");
}
}
@Test(timeout = 60000)
public void testRITStateForRollback() throws Exception {
final TableName tableName =
TableName.valueOf("testRITStateForRollback");
final HMaster master = cluster.getMaster();
try {
// Create table then get the single region for our new table.
Table t = createTableAndWait(tableName, Bytes.toBytes("cf"));
@ -171,22 +189,25 @@ public class TestSplitTransactionOnCluster {
// Turn off balancer so it doesn't cut in and mess up our placements.
this.admin.setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us.
cluster.getMaster().setCatalogJanitorEnabled(false);
master.setCatalogJanitorEnabled(false);
// find a splittable region
final HRegion region = findSplittableRegion(regions);
assertTrue("not able to find a splittable region", region != null);
// install region co-processor to fail splits
region.getCoprocessorHost().load(FailingSplitRegionObserver.class,
Coprocessor.PRIORITY_USER, region.getBaseConf());
// install master co-processor to fail splits
master.getMasterCoprocessorHost().load(
FailingSplitMasterObserver.class,
Coprocessor.PRIORITY_USER,
master.getConfiguration());
// split async
this.admin.splitRegion(region.getRegionInfo().getRegionName(), new byte[] {42});
// we have to wait until the SPLITTING state is seen by the master
FailingSplitRegionObserver observer = (FailingSplitRegionObserver) region
.getCoprocessorHost().findCoprocessor(FailingSplitRegionObserver.class.getName());
FailingSplitMasterObserver observer =
(FailingSplitMasterObserver) master.getMasterCoprocessorHost().findCoprocessor(
FailingSplitMasterObserver.class.getName());
assertNotNull(observer);
observer.latch.await();
@ -194,10 +215,12 @@ public class TestSplitTransactionOnCluster {
cluster.getMaster().getAssignmentManager().waitOnRegionToClearRegionsInTransition(hri, 60000);
} finally {
admin.setBalancerRunning(true, false);
cluster.getMaster().setCatalogJanitorEnabled(true);
master.setCatalogJanitorEnabled(true);
abortAndWaitForMaster();
TESTING_UTIL.deleteTable(tableName);
}
}
@Test(timeout = 60000)
public void testSplitFailedCompactionAndSplit() throws Exception {
final TableName tableName = TableName.valueOf("testSplitFailedCompactionAndSplit");
@ -238,80 +261,28 @@ public class TestSplitTransactionOnCluster {
assertTrue(fileNum > store.getStorefiles().size());
// 3, Split
SplitTransactionImpl st = new SplitTransactionImpl(region, Bytes.toBytes("row3"));
assertTrue(st.prepare());
st.execute(regionServer, regionServer);
requestSplitRegion(regionServer, region, Bytes.toBytes("row3"));
assertEquals(2, cluster.getRegions(tableName).size());
}
public static class FailingSplitRegionObserver extends BaseRegionObserver {
public static class FailingSplitMasterObserver extends BaseMasterObserver {
volatile CountDownLatch latch;
@Override
public void start(CoprocessorEnvironment e) throws IOException {
latch = new CountDownLatch(1);
}
@Override
public void preSplitBeforePONR(ObserverContext<RegionCoprocessorEnvironment> ctx,
byte[] splitKey, List<Mutation> metaEntries) throws IOException {
public void preSplitRegionBeforePONRAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final byte[] splitKey,
final List<Mutation> metaEntries) throws IOException {
latch.countDown();
throw new IOException("Causing rollback of region split");
}
}
/**
* A test that intentionally has master fail the processing of the split message.
* Tests that after we process server shutdown, the daughters are up on line.
* @throws IOException
* @throws InterruptedException
* @throws ServiceException
*/
@Test (timeout = 300000) public void testRSSplitDaughtersAreOnlinedAfterShutdownHandling()
throws IOException, InterruptedException, ServiceException {
final TableName tableName =
TableName.valueOf("testRSSplitDaughtersAreOnlinedAfterShutdownHandling");
// Create table then get the single region for our new table.
Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions);
int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
// Turn off balancer so it doesn't cut in and mess up our placements.
this.admin.setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us.
cluster.getMaster().setCatalogJanitorEnabled(false);
try {
// Add a bit of load up into the table so splittable.
TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY, false);
// Get region pre-split.
HRegionServer server = cluster.getRegionServer(tableRegionIndex);
printOutRegions(server, "Initial regions: ");
int regionCount = ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size();
// Now, before we split, set special flag in master, a flag that has
// it FAIL the processing of split.
AssignmentManager.TEST_SKIP_SPLIT_HANDLING = true;
try {
// Now try splitting and it should work.
split(hri, server, regionCount);
} catch (RegionServerStoppedException rsse) {
// Expected. The regionserver should crash
}
waitUntilRegionServerDead();
awaitDaughters(tableName, 2);
} finally {
// Set this flag back.
AssignmentManager.TEST_SKIP_SPLIT_HANDLING = false;
admin.setBalancerRunning(true, false);
cluster.getMaster().setCatalogJanitorEnabled(true);
cluster.startRegionServer();
t.close();
}
}
@Test (timeout = 300000) public void testExistingZnodeBlocksSplitAndWeRollback()
throws IOException, InterruptedException, NodeExistsException, KeeperException, ServiceException {
@Test (timeout = 300000)
public void testExistingZnodeBlocksSplitAndWeRollback() throws IOException, InterruptedException {
final TableName tableName =
TableName.valueOf("testExistingZnodeBlocksSplitAndWeRollback");
@ -368,8 +339,9 @@ public class TestSplitTransactionOnCluster {
* @throws IOException
* @throws InterruptedException
*/
@Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit()
throws IOException, InterruptedException {
@Ignore // TODO: revisit this test when the new AM and SSH is implement
@Test (timeout=300000)
public void testShutdownFixupWhenDaughterHasSplit()throws IOException, InterruptedException {
final TableName tableName =
TableName.valueOf("testShutdownFixupWhenDaughterHasSplit");
@ -469,8 +441,8 @@ public class TestSplitTransactionOnCluster {
admin.flush(userTableName);
}
admin.majorCompact(userTableName);
List<HRegionInfo> regionsOfTable = TESTING_UTIL.getMiniHBaseCluster()
.getMaster().getAssignmentManager().getRegionStates()
List<HRegionInfo> regionsOfTable =
cluster.getMaster().getAssignmentManager().getRegionStates()
.getRegionsOfTable(userTableName);
HRegionInfo hRegionInfo = regionsOfTable.get(0);
Put p = new Put("row6".getBytes());
@ -484,17 +456,18 @@ public class TestSplitTransactionOnCluster {
table.put(p);
admin.flush(userTableName);
admin.splitRegion(hRegionInfo.getRegionName(), "row7".getBytes());
regionsOfTable = TESTING_UTIL.getMiniHBaseCluster().getMaster()
regionsOfTable = cluster.getMaster()
.getAssignmentManager().getRegionStates()
.getRegionsOfTable(userTableName);
while (regionsOfTable.size() != 2) {
Thread.sleep(2000);
regionsOfTable = TESTING_UTIL.getMiniHBaseCluster().getMaster()
regionsOfTable = cluster.getMaster()
.getAssignmentManager().getRegionStates()
.getRegionsOfTable(userTableName);
}
Assert.assertEquals(2, regionsOfTable.size());
Scan s = new Scan();
ResultScanner scanner = table.getScanner(s);
int mainTableCount = 0;
@ -583,71 +556,6 @@ public class TestSplitTransactionOnCluster {
}
}
/**
*
* While transitioning node from RS_ZK_REGION_SPLITTING to
* RS_ZK_REGION_SPLITTING during region split,if zookeper went down split always
* fails for the region. HBASE-6088 fixes this scenario.
* This test case is to test the znode is deleted(if created) or not in roll back.
*
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
@Test(timeout = 60000)
public void testSplitBeforeSettingSplittingInZK() throws Exception,
InterruptedException, KeeperException {
testSplitBeforeSettingSplittingInZKInternals();
}
@Test(timeout = 60000)
public void testTableExistsIfTheSpecifiedTableRegionIsSplitParent() throws Exception {
final TableName tableName =
TableName.valueOf("testTableExistsIfTheSpecifiedTableRegionIsSplitParent");
// Create table then get the single region for our new table.
Table t = createTableAndWait(tableName, Bytes.toBytes("cf"));
List<HRegion> regions = null;
try {
regions = cluster.getRegions(tableName);
int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionInfo()
.getRegionName());
HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
insertData(tableName, admin, t);
// Turn off balancer so it doesn't cut in and mess up our placements.
admin.setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us.
cluster.getMaster().setCatalogJanitorEnabled(false);
boolean tableExists = MetaTableAccessor.tableExists(regionServer.getConnection(),
tableName);
assertEquals("The specified table should present.", true, tableExists);
final HRegion region = findSplittableRegion(regions);
assertTrue("not able to find a splittable region", region != null);
SplitTransactionImpl st = new SplitTransactionImpl(region, Bytes.toBytes("row2"));
try {
st.prepare();
st.createDaughters(regionServer, regionServer, null);
} catch (IOException e) {
}
tableExists = MetaTableAccessor.tableExists(regionServer.getConnection(),
tableName);
assertEquals("The specified table should present.", true, tableExists);
Set<RegionState> rit = cluster.getMaster().getAssignmentManager().getRegionStates()
.getRegionsInTransition();
assertTrue(rit.size() == 3);
cluster.getMaster().getAssignmentManager().regionOffline(st.getFirstDaughter());
cluster.getMaster().getAssignmentManager().regionOffline(st.getSecondDaughter());
cluster.getMaster().getAssignmentManager().regionOffline(region.getRegionInfo());
rit = cluster.getMaster().getAssignmentManager().getRegionStates().getRegionsInTransition();
assertTrue(rit.size() == 0);
} finally {
admin.setBalancerRunning(true, false);
cluster.getMaster().setCatalogJanitorEnabled(true);
t.close();
TESTING_UTIL.deleteTable(tableName);
}
}
@Test
public void testSplitWithRegionReplicas() throws Exception {
final TableName tableName =
@ -679,10 +587,8 @@ public class TestSplitTransactionOnCluster {
regionServerIndex = cluster.getServerWith(region.getRegionInfo().getRegionName());
regionServer = cluster.getRegionServer(regionServerIndex);
assertTrue("not able to find a splittable region", region != null);
SplitTransactionImpl st = new SplitTransactionImpl(region, Bytes.toBytes("row2"));
try {
st.prepare();
st.execute(regionServer, regionServer);
requestSplitRegion(regionServer, region, Bytes.toBytes("row2"));
} catch (IOException e) {
e.printStackTrace();
fail("Split execution should have succeeded with no exceptions thrown " + e);
@ -779,10 +685,8 @@ public class TestSplitTransactionOnCluster {
assertTrue("not able to find a splittable region", region != null);
// Now split.
SplitTransactionImpl st = new MockedSplitTransaction(region, Bytes.toBytes("row2"));
try {
st.prepare();
st.execute(regionServer, regionServer);
requestSplitRegion(regionServer, region, Bytes.toBytes("row2"));
} catch (IOException e) {
fail("Split execution should have succeeded with no exceptions thrown");
}
@ -826,195 +730,6 @@ public class TestSplitTransactionOnCluster {
}
}
/**
* Not really restarting the master. Simulate it by clear of new region
* state since it is not persisted, will be lost after master restarts.
*/
@Test(timeout = 180000)
public void testSplitAndRestartingMaster() throws Exception {
LOG.info("Starting testSplitAndRestartingMaster");
final TableName tableName = TableName.valueOf("testSplitAndRestartingMaster");
// Create table then get the single region for our new table.
createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions);
ensureTableRegionNotOnSameServerAsMeta(admin, hri);
int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionInfo()
.getRegionName());
HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
// Turn off balancer so it doesn't cut in and mess up our placements.
this.admin.setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us.
cluster.getMaster().setCatalogJanitorEnabled(false);
try {
MyMasterRpcServices.enabled.set(true);
// find a splittable region. Refresh the regions list
regions = cluster.getRegions(tableName);
final HRegion region = findSplittableRegion(regions);
assertTrue("not able to find a splittable region", region != null);
// Now split.
SplitTransactionImpl st = new SplitTransactionImpl(region, Bytes.toBytes("row2"));
try {
st.prepare();
st.execute(regionServer, regionServer);
} catch (IOException e) {
fail("Split execution should have succeeded with no exceptions thrown");
}
// Postcondition
List<HRegion> daughters = cluster.getRegions(tableName);
LOG.info("xxx " + regions.size() + AssignmentManager.TEST_SKIP_SPLIT_HANDLING);
assertTrue(daughters.size() == 2);
} finally {
MyMasterRpcServices.enabled.set(false);
admin.setBalancerRunning(true, false);
cluster.getMaster().setCatalogJanitorEnabled(true);
}
}
@Test(timeout = 180000)
public void testSplitHooksBeforeAndAfterPONR() throws Exception {
TableName firstTable = TableName.valueOf("testSplitHooksBeforeAndAfterPONR_1");
TableName secondTable = TableName.valueOf("testSplitHooksBeforeAndAfterPONR_2");
HColumnDescriptor hcd = new HColumnDescriptor("cf");
HTableDescriptor desc = new HTableDescriptor(firstTable);
desc.addCoprocessor(MockedRegionObserver.class.getName());
desc.addFamily(hcd);
admin.createTable(desc);
TESTING_UTIL.waitUntilAllRegionsAssigned(firstTable);
desc = new HTableDescriptor(secondTable);
desc.addFamily(hcd);
admin.createTable(desc);
TESTING_UTIL.waitUntilAllRegionsAssigned(secondTable);
List<HRegion> firstTableRegions = cluster.getRegions(firstTable);
List<HRegion> secondTableRegions = cluster.getRegions(secondTable);
// Check that both tables actually have regions.
if (firstTableRegions.size() == 0 || secondTableRegions.size() == 0) {
fail("Each table should have at least one region.");
}
ServerName serverName = cluster.getServerHoldingRegion(firstTable,
firstTableRegions.get(0).getRegionInfo().getRegionName());
admin.move(secondTableRegions.get(0).getRegionInfo().getEncodedNameAsBytes(),
Bytes.toBytes(serverName.getServerName()));
Table table1 = null;
Table table2 = null;
try {
table1 = TESTING_UTIL.getConnection().getTable(firstTable);
table2 = TESTING_UTIL.getConnection().getTable(firstTable);
insertData(firstTable, admin, table1);
insertData(secondTable, admin, table2);
admin.split(firstTable, "row2".getBytes());
firstTableRegions = cluster.getRegions(firstTable);
while (firstTableRegions.size() != 2) {
Thread.sleep(1000);
firstTableRegions = cluster.getRegions(firstTable);
}
assertEquals("Number of regions after split should be 2.", 2, firstTableRegions.size());
secondTableRegions = cluster.getRegions(secondTable);
assertEquals("Number of regions after split should be 2.", 2, secondTableRegions.size());
} finally {
if (table1 != null) {
table1.close();
}
if (table2 != null) {
table2.close();
}
TESTING_UTIL.deleteTable(firstTable);
TESTING_UTIL.deleteTable(secondTable);
}
}
@Test (timeout=300000)
public void testSSHCleanupDaugtherRegionsOfAbortedSplit() throws Exception {
TableName table = TableName.valueOf("testSSHCleanupDaugtherRegionsOfAbortedSplit");
try {
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
admin.createTable(desc);
Connection connection = ConnectionFactory.createConnection(cluster.getConfiguration());
Table hTable = connection.getTable(desc.getTableName());
for(int i = 1; i < 5; i++) {
Put p1 = new Put(("r"+i).getBytes());
p1.addColumn(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
hTable.put(p1);
}
admin.flush(desc.getTableName());
List<HRegion> regions = cluster.getRegions(desc.getTableName());
int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
HRegionServer regionServer = cluster.getRegionServer(serverWith);
SplitTransactionImpl st = new SplitTransactionImpl(regions.get(0), Bytes.toBytes("r3"));
st.prepare();
st.stepsBeforePONR(regionServer, regionServer, false);
Path tableDir =
FSUtils.getTableDir(cluster.getMaster().getMasterFileSystem().getRootDir(),
desc.getTableName());
List<Path> regionDirs =
FSUtils.getRegionDirs(tableDir.getFileSystem(cluster.getConfiguration()), tableDir);
assertEquals(3,regionDirs.size());
regionServer.kill();
// Before we check deadServerInProgress, we should ensure server is dead at master side.
while (!cluster.getMaster().getServerManager().
getDeadServers().isDeadServer(regionServer.serverName)) {
Thread.sleep(10);
}
// Wait until finish processing of shutdown
while (cluster.getMaster().getServerManager().areDeadServersInProgress()) {
Thread.sleep(10);
}
AssignmentManager am = cluster.getMaster().getAssignmentManager();
assertEquals(am.getRegionStates().getRegionsInTransition().toString(), 0, am
.getRegionStates().getRegionsInTransition().size());
regionDirs =
FSUtils.getRegionDirs(tableDir.getFileSystem(cluster.getConfiguration()), tableDir);
assertEquals(1,regionDirs.size());
} finally {
TESTING_UTIL.deleteTable(table);
}
}
private void testSplitBeforeSettingSplittingInZKInternals() throws Exception {
final TableName tableName = TableName.valueOf("testSplitBeforeSettingSplittingInZK");
try {
// Create table then get the single region for our new table.
createTableAndWait(tableName, Bytes.toBytes("cf"));
List<HRegion> regions = awaitTableRegions(tableName);
assertTrue("Table not online", cluster.getRegions(tableName).size() != 0);
int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionInfo()
.getRegionName());
HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
final HRegion region = findSplittableRegion(regions);
assertTrue("not able to find a splittable region", region != null);
SplitTransactionImpl st = new MockedSplitTransaction(region, Bytes.toBytes("row2")) {
@Override
public PairOfSameType<Region> stepsBeforePONR(final Server server,
final RegionServerServices services, boolean testing) throws IOException {
throw new SplittingNodeCreationFailedException ();
}
};
try {
st.prepare();
st.execute(regionServer, regionServer);
} catch (IOException e) {
// check for the specific instance in case the Split failed due to the
// existence of the znode in OPENED state.
// This will at least make the test to fail;
assertTrue("Should be instance of CreateSplittingNodeFailedException",
e instanceof SplittingNodeCreationFailedException );
assertTrue(st.rollback(regionServer, regionServer));
}
} finally {
TESTING_UTIL.deleteTable(tableName);
}
}
@Test
public void testStoreFileReferenceCreationWhenSplitPolicySaysToSkipRangeCheck()
throws Exception {
@ -1052,35 +767,6 @@ public class TestSplitTransactionOnCluster {
}
}
public static class MockedCoordinatedStateManager extends ZkCoordinatedStateManager {
public void initialize(Server server, HRegion region) {
this.server = server;
this.watcher = server.getZooKeeper();
}
}
public static class MockedSplitTransaction extends SplitTransactionImpl {
private HRegion currentRegion;
public MockedSplitTransaction(HRegion region, byte[] splitrow) {
super(region, splitrow);
this.currentRegion = region;
}
@Override
public boolean rollback(Server server, RegionServerServices services) throws IOException {
if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
if(secondSplit){
super.rollback(server, services);
latch.countDown();
return true;
}
}
return super.rollback(server, services);
}
}
private HRegion findSplittableRegion(final List<HRegion> regions) throws InterruptedException {
for (int i = 0; i < 5; ++i) {
for (HRegion r: regions) {
@ -1118,14 +804,13 @@ public class TestSplitTransactionOnCluster {
private void split(final HRegionInfo hri, final HRegionServer server, final int regionCount)
throws IOException, InterruptedException {
this.admin.splitRegion(hri.getRegionName());
for (int i = 0; ProtobufUtil.getOnlineRegions(
server.getRSRpcServices()).size() <= regionCount && i < 300; i++) {
for (int i = 0; this.cluster.getRegions(hri.getTable()).size() <= regionCount && i < 60; i++) {
LOG.debug("Waiting on region to split");
Thread.sleep(100);
Thread.sleep(2000);
}
assertFalse("Waited too long for split",
ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size() <= regionCount);
this.cluster.getRegions(hri.getTable()).size() <= regionCount);
}
/**
@ -1248,14 +933,6 @@ public class TestSplitTransactionOnCluster {
return t;
}
private static class SplittingNodeCreationFailedException extends IOException {
private static final long serialVersionUID = 1652404976265623004L;
public SplittingNodeCreationFailedException () {
super();
}
}
// Make it public so that JVMClusterUtil can access it.
public static class MyMaster extends HMaster {
public MyMaster(Configuration conf, CoordinatedStateManager cp)
@ -1297,61 +974,6 @@ public class TestSplitTransactionOnCluster {
}
}
public static class MockedRegionObserver extends BaseRegionObserver {
private SplitTransactionImpl st = null;
private PairOfSameType<Region> daughterRegions = null;
@Override
public void preSplitBeforePONR(ObserverContext<RegionCoprocessorEnvironment> ctx,
byte[] splitKey, List<Mutation> metaEntries) throws IOException {
RegionCoprocessorEnvironment environment = ctx.getEnvironment();
HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
List<Region> onlineRegions =
rs.getOnlineRegions(TableName.valueOf("testSplitHooksBeforeAndAfterPONR_2"));
Region region = onlineRegions.get(0);
for (Region r : onlineRegions) {
if (r.getRegionInfo().containsRow(splitKey)) {
region = r;
break;
}
}
st = new SplitTransactionImpl((HRegion) region, splitKey);
if (!st.prepare()) {
LOG.error("Prepare for the table " + region.getTableDesc().getNameAsString()
+ " failed. So returning null. ");
ctx.bypass();
return;
}
((HRegion)region).forceSplit(splitKey);
daughterRegions = st.stepsBeforePONR(rs, rs, false);
HRegionInfo copyOfParent = new HRegionInfo(region.getRegionInfo());
copyOfParent.setOffline(true);
copyOfParent.setSplit(true);
// Put for parent
Put putParent = MetaTableAccessor.makePutFromRegionInfo(copyOfParent);
MetaTableAccessor.addDaughtersToPut(putParent, daughterRegions.getFirst().getRegionInfo(),
daughterRegions.getSecond().getRegionInfo());
metaEntries.add(putParent);
// Puts for daughters
Put putA = MetaTableAccessor.makePutFromRegionInfo(
daughterRegions.getFirst().getRegionInfo());
Put putB = MetaTableAccessor.makePutFromRegionInfo(
daughterRegions.getSecond().getRegionInfo());
st.addLocation(putA, rs.getServerName(), 1);
st.addLocation(putB, rs.getServerName(), 1);
metaEntries.add(putA);
metaEntries.add(putB);
}
@Override
public void preSplitAfterPONR(ObserverContext<RegionCoprocessorEnvironment> ctx)
throws IOException {
RegionCoprocessorEnvironment environment = ctx.getEnvironment();
HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
st.stepsAfterPONR(rs, rs, daughterRegions, null);
}
}
static class CustomSplitPolicy extends RegionSplitPolicy {
@Override

View File

@ -815,28 +815,16 @@ public class TestAccessController extends SecureTestUtil {
verifyDenied(action, USER_NONE, USER_RO, USER_GROUP_ADMIN, USER_GROUP_READ, USER_GROUP_CREATE);
}
@Test (timeout=180000)
public void testSplit() throws Exception {
AccessTestAction action = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preSplit(ObserverContext.createAndPrepare(RCP_ENV, null));
return null;
}
};
verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
USER_GROUP_WRITE, USER_GROUP_CREATE);
}
@Test (timeout=180000)
public void testSplitWithSplitRow() throws Exception {
final TableName tname = TableName.valueOf("testSplitWithSplitRow");
createTestTable(tname);
AccessTestAction action = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preSplit(
ObserverContext.createAndPrepare(RCP_ENV, null),
ACCESS_CONTROLLER.preSplitRegion(
ObserverContext.createAndPrepare(CP_ENV, null),
tname,
TEST_ROW);
return null;
}

View File

@ -777,6 +777,18 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
}
}, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
// preSplit
verifyAllowed(new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preSplitRegion(
ObserverContext.createAndPrepare(CP_ENV, null),
TEST_TABLE.getTableName(),
Bytes.toBytes("ss"));
return null;
}
}, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
// preSetUserQuota
verifyAllowed(new AccessTestAction() {
@Override
@ -873,15 +885,6 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
}
}, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
// preSplit
verifyAllowed(new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preSplit(ObserverContext.createAndPrepare(RCP_ENV, null));
return null;
}
}, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
// preGetOp
verifyAllowed(new AccessTestAction() {
@Override

View File

@ -49,10 +49,10 @@ import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.SplitTableRegionProcedure;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl;
import org.apache.hadoop.hbase.regionserver.SplitTransactionFactory;
import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@ -205,7 +205,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
HConstants.EMPTY_END_ROW, false, false, true);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS });
doFsck(conf, true);
@ -236,7 +237,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
//to report error if .tableinfo is missing.
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NO_TABLEINFO_FILE });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NO_TABLEINFO_FILE });
// fix OrphanTable with default .tableinfo (htd not yet cached on master)
hbck = doFsck(conf, true);
@ -311,7 +313,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck,
new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.ORPHAN_HDFS_REGION, HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.ORPHAN_HDFS_REGION,
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
// fix the problem.
@ -349,7 +353,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.OVERLAP_IN_REGION_CHAIN,
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.OVERLAP_IN_REGION_CHAIN,
HBaseFsck.ErrorReporter.ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
assertEquals(3, hbck.getOverlapGroups(table).size());
assertEquals(ROWKEYS.length, countRows());
@ -386,7 +391,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
admin.enableTable(table);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
// holes are separate from overlap groups
assertEquals(0, hbck.getOverlapGroups(table).size());
@ -449,7 +455,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
hrs.addToOnlineRegions(r);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.SHOULD_NOT_BE_DEPLOYED });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.SHOULD_NOT_BE_DEPLOYED });
// fix this fault
doFsck(conf, true);
@ -678,9 +685,12 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
deleteTableDir(table);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS,
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS, HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS,
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS, HBaseFsck.ErrorReporter.ERROR_CODE.ORPHAN_TABLE_STATE, });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS,
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS,
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS,
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS,
HBaseFsck.ErrorReporter.ERROR_CODE.ORPHAN_TABLE_STATE, });
// holes are separate from overlap groups
assertEquals(0, hbck.getOverlapGroups(table).size());
@ -705,7 +715,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// test
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NO_VERSION_FILE });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NO_VERSION_FILE });
// fix hbase.version missing
doFsck(conf, true);
@ -727,7 +738,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// test
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NO_TABLE_STATE });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NO_TABLE_STATE });
// fix table state missing
doFsck(conf, true);
@ -763,14 +775,17 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
false, true); // don't rm meta
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS, HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS,
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS });
// fix hole in table 1
doFsck(conf, true, table1);
// check that hole in table 1 fixed
assertNoErrors(doFsck(conf, false, table1));
// check that hole in table 2 still there
assertErrors(doFsck(conf, false, table2), new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS });
assertErrors(doFsck(conf, false, table2), new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS });
// fix hole in table 2
doFsck(conf, true, table2);
@ -824,16 +839,19 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.LINGERING_SPLIT_PARENT, HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN});
HBaseFsck.ErrorReporter.ERROR_CODE.LINGERING_SPLIT_PARENT,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN});
// regular repair cannot fix lingering split parent
hbck = doFsck(conf, true);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.LINGERING_SPLIT_PARENT, HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
HBaseFsck.ErrorReporter.ERROR_CODE.LINGERING_SPLIT_PARENT,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
assertFalse(hbck.shouldRerun());
hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.LINGERING_SPLIT_PARENT, HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN});
HBaseFsck.ErrorReporter.ERROR_CODE.LINGERING_SPLIT_PARENT,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN});
// fix lingering split parent
hbck = new HBaseFsck(conf, hbfsckExecutorService);
@ -897,7 +915,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// overlapping regions
HBaseFsck hbck = doFsck(conf, true, true, false, false, false, true, true, true, false,
false, false, null);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {}); //no LINGERING_SPLIT_PARENT reported
// no LINGERING_SPLIT_PARENT reported
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {});
// assert that the split hbase:meta entry is still there.
Get get = new Get(hri.getRegionName());
@ -908,8 +927,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
assertEquals(ROWKEYS.length, countRows());
// assert that we still have the split regions
assertEquals(rl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions
// pre-split.
//SPLITS + 1 is # regions pre-split.
assertEquals(rl.getStartKeys().length, SPLITS.length + 1 + 1);
assertNoErrors(doFsck(conf, false));
}
} finally {
@ -1023,7 +1042,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
admin.enableTable(table);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY });
// fix hole
doFsck(conf, true);
// check that hole fixed
@ -1051,7 +1071,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
false, true);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS });
// fix hole
doFsck(conf, true);
// check that hole fixed
@ -1080,7 +1101,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
admin.enableTable(table);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY });
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY });
// fix hole
doFsck(conf, true);
// check that hole fixed
@ -1108,7 +1130,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// verify there is no other errors
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck,
new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NOT_DEPLOYED, HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_DEPLOYED,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
// verify that noHdfsChecking report the same errors
HBaseFsck fsck = new HBaseFsck(conf, hbfsckExecutorService);
@ -1118,7 +1142,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
fsck.setCheckHdfs(false);
fsck.onlineHbck();
assertErrors(fsck,
new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NOT_DEPLOYED, HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_DEPLOYED,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
fsck.close();
// verify that fixAssignments works fine with noHdfsChecking
@ -1161,7 +1187,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
// verify there is no other errors
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck,
new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META, HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
// verify that noHdfsChecking report the same errors
HBaseFsck fsck = new HBaseFsck(conf, hbfsckExecutorService);
@ -1171,7 +1199,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
fsck.setCheckHdfs(false);
fsck.onlineHbck();
assertErrors(fsck,
new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META, HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
fsck.close();
// verify that fixMeta doesn't work with noHdfsChecking
@ -1185,7 +1215,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
fsck.onlineHbck();
assertFalse(fsck.shouldRerun());
assertErrors(fsck,
new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META, HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
fsck.close();
// fix the cluster so other tests won't be impacted
@ -1225,7 +1257,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.ORPHAN_HDFS_REGION, HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
HBaseFsck.ErrorReporter.ERROR_CODE.ORPHAN_HDFS_REGION,
HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN});
// verify that noHdfsChecking can't detect ORPHAN_HDFS_REGION
@ -1235,7 +1268,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
fsck.setTimeLag(0);
fsck.setCheckHdfs(false);
fsck.onlineHbck();
assertErrors(fsck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
assertErrors(fsck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
fsck.close();
// verify that fixHdfsHoles doesn't work with noHdfsChecking
@ -1249,7 +1283,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
fsck.setFixHdfsOrphans(true);
fsck.onlineHbck();
assertFalse(fsck.shouldRerun());
assertErrors(fsck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
assertErrors(fsck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
fsck.close();
} finally {
if (admin.isTableDisabled(table)) {
@ -1349,7 +1384,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
TEST_UTIL.assertRegionOnServer(hriDupe, server, REGION_ONLINE_TIMEOUT);
HBaseFsck hbck = doFsck(conf,false);
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] { HBaseFsck.ErrorReporter.ERROR_CODE.DEGENERATE_REGION, HBaseFsck.ErrorReporter.ERROR_CODE.DUPE_STARTKEYS,
assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
HBaseFsck.ErrorReporter.ERROR_CODE.DEGENERATE_REGION,
HBaseFsck.ErrorReporter.ERROR_CODE.DUPE_STARTKEYS,
HBaseFsck.ErrorReporter.ERROR_CODE.DUPE_STARTKEYS });
assertEquals(2, hbck.getOverlapGroups(table).size());
assertEquals(ROWKEYS.length, countRows());
@ -1405,13 +1442,15 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
meta.close();
HBaseFsck hbck = doFsck(conf, false);
assertTrue(hbck.getErrors().getErrorList().contains(HBaseFsck.ErrorReporter.ERROR_CODE.EMPTY_META_CELL));
assertTrue(hbck.getErrors().getErrorList().contains(
HBaseFsck.ErrorReporter.ERROR_CODE.EMPTY_META_CELL));
// fix reference file
hbck = doFsck(conf, true);
// check that reference file fixed
assertFalse(hbck.getErrors().getErrorList().contains(HBaseFsck.ErrorReporter.ERROR_CODE.EMPTY_META_CELL));
assertFalse(hbck.getErrors().getErrorList().contains(
HBaseFsck.ErrorReporter.ERROR_CODE.EMPTY_META_CELL));
} finally {
cleanupTable(table);
}
@ -1668,16 +1707,22 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
List<HRegion> regions = cluster.getRegions(desc.getTableName());
int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
HRegionServer regionServer = cluster.getRegionServer(serverWith);
cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
SplitTransactionImpl st = (SplitTransactionImpl)
new SplitTransactionFactory(TEST_UTIL.getConfiguration())
.create(regions.get(0), Bytes.toBytes("r3"));
st.prepare();
st.stepsBeforePONR(regionServer, regionServer, false);
byte[] parentRegionName = regions.get(0).getRegionInfo().getRegionName();
cluster.getServerWith(parentRegionName);
// Create daughters without adding to META table
MasterProcedureEnv env = cluster.getMaster().getMasterProcedureExecutor().getEnvironment();
SplitTableRegionProcedure splitR = new SplitTableRegionProcedure(
env, desc.getTableName(), regions.get(0).getRegionInfo(), Bytes.toBytes("r3"));
splitR.prepareSplitRegion(env);
splitR.setRegionStateToSplitting(env);
splitR.closeParentRegionForSplit(env);
splitR.createDaughterRegions(env);
AssignmentManager am = cluster.getMaster().getAssignmentManager();
for (RegionState state : am.getRegionStates().getRegionsInTransition()) {
am.regionOffline(state.getRegion());
}
Map<HRegionInfo, ServerName> regionsMap = new HashMap<HRegionInfo, ServerName>();
regionsMap.put(regions.get(0).getRegionInfo(), regionServer.getServerName());
am.assign(regionsMap);