HBASE-17470 Remove merge region code from region server (Stephen Yuan Jiang)

This commit is contained in:
Stephen Yuan Jiang 2017-01-17 15:39:51 -08:00
parent 287f95a579
commit 805d39fca6
35 changed files with 1210 additions and 9919 deletions

View File

@ -1311,13 +1311,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return stub.moveRegion(controller, request);
}
@Override
public MasterProtos.DispatchMergingRegionsResponse dispatchMergingRegions(
RpcController controller, MasterProtos.DispatchMergingRegionsRequest request)
throws ServiceException {
return stub.dispatchMergingRegions(controller, request);
}
@Override
public MasterProtos.MergeTableRegionsResponse mergeTableRegions(
RpcController controller, MasterProtos.MergeTableRegionsRequest request)

View File

@ -121,7 +121,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerIn
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.SplitRegionRequest;
@ -1950,46 +1949,6 @@ public final class ProtobufUtil {
}
}
/**
* A helper to merge regions using admin protocol. Send request to
* regionserver.
* @param admin
* @param region_a
* @param region_b
* @param forcible true if do a compulsory merge, otherwise we will only merge
* two adjacent regions
* @param user effective user
* @throws IOException
*/
public static void mergeRegions(final RpcController controller,
final AdminService.BlockingInterface admin,
final HRegionInfo region_a, final HRegionInfo region_b,
final boolean forcible, final User user) throws IOException {
final MergeRegionsRequest request = ProtobufUtil.buildMergeRegionsRequest(
region_a.getRegionName(), region_b.getRegionName(),forcible);
if (user != null) {
try {
user.runAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
admin.mergeRegions(controller, request);
return null;
}
});
} catch (InterruptedException ie) {
InterruptedIOException iioe = new InterruptedIOException();
iioe.initCause(ie);
throw iioe;
}
} else {
try {
admin.mergeRegions(controller, request);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}
// End helpers for Admin
/*
@ -3224,28 +3183,6 @@ public final class ProtobufUtil {
return builder.build();
}
/**
* Create a MergeRegionsRequest for the given regions
* @param regionA name of region a
* @param regionB name of region b
* @param forcible true if it is a compulsory merge
* @return a MergeRegionsRequest
*/
public static MergeRegionsRequest buildMergeRegionsRequest(
final byte[] regionA, final byte[] regionB, final boolean forcible) {
MergeRegionsRequest.Builder builder = MergeRegionsRequest.newBuilder();
RegionSpecifier regionASpecifier = RequestConverter.buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionA);
RegionSpecifier regionBSpecifier = RequestConverter.buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionB);
builder.setRegionA(regionASpecifier);
builder.setRegionB(regionBSpecifier);
builder.setForcible(forcible);
// send the master's wall clock time as well, so that the RS can refer to it
builder.setMasterSystemTime(EnvironmentEdgeManager.currentTime());
return builder.build();
}
/**
* Get a ServerName from the passed in data bytes.
* @param data Data with a serialize server name in it; can handle the old style

View File

@ -84,7 +84,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTabl
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
@ -1089,23 +1088,6 @@ public final class RequestConverter {
return builder.build();
}
public static DispatchMergingRegionsRequest buildDispatchMergingRegionsRequest(
final byte[] encodedNameOfRegionA,
final byte[] encodedNameOfRegionB,
final boolean forcible,
final long nonceGroup,
final long nonce) throws DeserializationException {
DispatchMergingRegionsRequest.Builder builder = DispatchMergingRegionsRequest.newBuilder();
builder.setRegionA(buildRegionSpecifier(
RegionSpecifierType.ENCODED_REGION_NAME, encodedNameOfRegionA));
builder.setRegionB(buildRegionSpecifier(
RegionSpecifierType.ENCODED_REGION_NAME, encodedNameOfRegionB));
builder.setForcible(forcible);
builder.setNonceGroup(nonceGroup);
builder.setNonce(nonce);
return builder.build();
}
public static MergeTableRegionsRequest buildMergeTableRegionsRequest(
final byte[][] encodedNameOfdaughaterRegions,
final boolean forcible,

View File

@ -189,22 +189,6 @@ message UpdateFavoredNodesResponse {
optional uint32 response = 1;
}
/**
* Merges the specified regions.
* <p>
* This method currently closes the regions and then merges them
*/
message MergeRegionsRequest {
required RegionSpecifier region_a = 1;
required RegionSpecifier region_b = 2;
optional bool forcible = 3 [default = false];
// wall clock time from master
optional uint64 master_system_time = 4;
}
message MergeRegionsResponse {
}
// Protocol buffer version of WAL for replication
message WALEntry {
required WALKey key = 1;
@ -307,9 +291,6 @@ service AdminService {
rpc CompactRegion(CompactRegionRequest)
returns(CompactRegionResponse);
rpc MergeRegions(MergeRegionsRequest)
returns(MergeRegionsResponse);
rpc ReplicateWALEntry(ReplicateWALEntryRequest)
returns(ReplicateWALEntryResponse);

View File

@ -79,21 +79,6 @@ message MoveRegionRequest {
message MoveRegionResponse {
}
/**
* Dispatch merging the specified regions.
*/
message DispatchMergingRegionsRequest {
required RegionSpecifier region_a = 1;
required RegionSpecifier region_b = 2;
optional bool forcible = 3 [default = false];
optional uint64 nonce_group = 4 [default = 0];
optional uint64 nonce = 5 [default = 0];
}
message DispatchMergingRegionsResponse {
optional uint64 proc_id = 1;
}
/**
* Merging the specified regions in a table.
*/
@ -625,10 +610,6 @@ service MasterService {
rpc MoveRegion(MoveRegionRequest)
returns(MoveRegionResponse);
/** Master dispatch merging the regions */
rpc DispatchMergingRegions(DispatchMergingRegionsRequest)
returns(DispatchMergingRegionsResponse);
/** Master merge the regions */
rpc MergeTableRegions(MergeTableRegionsRequest)
returns(MergeTableRegionsResponse);

View File

@ -262,21 +262,6 @@ message RestoreSnapshotStateData {
repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 7;
}
enum DispatchMergingRegionsState {
DISPATCH_MERGING_REGIONS_PREPARE = 1;
DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;
DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;
DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;
DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;
}
message DispatchMergingRegionsStateData {
required UserInformation user_info = 1;
required TableName table_name = 2;
repeated RegionInfo region_info = 3;
optional bool forcible = 4;
}
enum MergeTableRegionsState {
MERGE_TABLE_REGIONS_PREPARE = 1;
MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;

View File

@ -59,11 +59,23 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver
HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
* Use {@link #preMergeRegions(ObserverContext, HRegionInfo[])}
*/
@Deprecated
@Override
public void preDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionA, HRegionInfo regionB) throws IOException {
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
* Use {@link #postMergeRegions(ObserverContext, HRegionInfo[])}
*/
@Deprecated
@Override
public void postDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionA, HRegionInfo regionB) throws IOException {

View File

@ -111,11 +111,23 @@ public class BaseMasterObserver implements MasterObserver {
final HRegionInfo[] regions) throws IOException {
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
* Use {@link #preMergeRegions(ObserverContext, HRegionInfo[])}
*/
@Deprecated
@Override
public void preDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionA, HRegionInfo regionB) throws IOException {
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
* Use {@link #postMergeRegions(ObserverContext, HRegionInfo[])}
*/
@Deprecated
@Override
public void postDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionA, HRegionInfo regionB) throws IOException {

View File

@ -1699,7 +1699,11 @@ public interface MasterObserver extends Coprocessor {
* @param regionA first region to be merged
* @param regionB second region to be merged
* @throws IOException if an error occurred on the coprocessor
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
* Use {@link #preMergeRegions(ObserverContext, HRegionInfo[])}
*/
@Deprecated
void preDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> ctx,
HRegionInfo regionA, HRegionInfo regionB) throws IOException;
@ -1709,7 +1713,11 @@ public interface MasterObserver extends Coprocessor {
* @param regionA first region to be merged
* @param regionB second region to be merged
* @throws IOException if an error occurred on the coprocessor
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
* Use {@link #postMergeRegions(ObserverContext, HRegionInfo[])}
*/
@Deprecated
void postDispatchMerge(final ObserverContext<MasterCoprocessorEnvironment> c,
final HRegionInfo regionA, final HRegionInfo regionB) throws IOException;

View File

@ -109,7 +109,6 @@ import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DispatchMergingRegionsProcedure;
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@ -1418,55 +1417,6 @@ public class HMaster extends HRegionServer implements MasterServices {
this.catalogJanitorChore.setEnabled(b);
}
@Override
public long dispatchMergingRegions(
final HRegionInfo regionInfoA,
final HRegionInfo regionInfoB,
final boolean forcible,
final long nonceGroup,
final long nonce) throws IOException {
checkInitialized();
TableName tableName = regionInfoA.getTable();
if (tableName == null || regionInfoB.getTable() == null) {
throw new UnknownRegionException ("Can't merge regions without table associated");
}
if (!tableName.equals(regionInfoB.getTable())) {
throw new IOException ("Cannot merge regions from two different tables");
}
if (regionInfoA.compareTo(regionInfoB) == 0) {
throw new MergeRegionException(
"Cannot merge a region to itself " + regionInfoA + ", " + regionInfoB);
}
HRegionInfo [] regionsToMerge = new HRegionInfo[2];
regionsToMerge [0] = regionInfoA;
regionsToMerge [1] = regionInfoB;
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
getMaster().getMasterCoprocessorHost().preDispatchMerge(regionInfoA, regionInfoB);
LOG.info(getClientIdAuditPrefix() + " Merge regions "
+ regionInfoA.getEncodedName() + " and " + regionInfoB.getEncodedName());
submitProcedure(new DispatchMergingRegionsProcedure(procedureExecutor.getEnvironment(),
tableName, regionsToMerge, forcible));
getMaster().getMasterCoprocessorHost().postDispatchMerge(regionInfoA, regionInfoB);
}
@Override
protected String getDescription() {
return "DisableTableProcedure";
}
});
}
@Override
public long mergeRegions(
final HRegionInfo[] regionsToMerge,

View File

@ -773,28 +773,6 @@ public class MasterCoprocessorHost
});
}
public void preDispatchMerge(final HRegionInfo regionInfoA, final HRegionInfo regionInfoB)
throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preDispatchMerge(ctx, regionInfoA, regionInfoB);
}
});
}
public void postDispatchMerge(final HRegionInfo regionInfoA, final HRegionInfo regionInfoB)
throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.postDispatchMerge(ctx, regionInfoA, regionInfoB);
}
});
}
public void preMergeRegions(final HRegionInfo[] regionsToMerge)
throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {

View File

@ -500,52 +500,6 @@ public class MasterRpcServices extends RSRpcServices
}
}
@Override
public DispatchMergingRegionsResponse dispatchMergingRegions(RpcController c,
DispatchMergingRegionsRequest request) throws ServiceException {
try {
master.checkInitialized();
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
final byte[] encodedNameOfRegionA = request.getRegionA().getValue()
.toByteArray();
final byte[] encodedNameOfRegionB = request.getRegionB().getValue()
.toByteArray();
if (request.getRegionA().getType() != RegionSpecifierType.ENCODED_REGION_NAME
|| request.getRegionB().getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
LOG.warn("mergeRegions specifier type: expected: "
+ RegionSpecifierType.ENCODED_REGION_NAME + " actual: region_a="
+ request.getRegionA().getType() + ", region_b="
+ request.getRegionB().getType());
}
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
RegionState regionStateA = regionStates.getRegionState(Bytes.toString(encodedNameOfRegionA));
RegionState regionStateB = regionStates.getRegionState(Bytes.toString(encodedNameOfRegionB));
if (regionStateA == null || regionStateB == null) {
throw new ServiceException(new UnknownRegionException(
Bytes.toStringBinary(regionStateA == null ? encodedNameOfRegionA
: encodedNameOfRegionB)));
}
final HRegionInfo regionInfoA = regionStateA.getRegion();
final HRegionInfo regionInfoB = regionStateB.getRegion();
try {
long procId = master.dispatchMergingRegions(
regionInfoA,
regionInfoB,
request.getForcible(),
request.getNonceGroup(),
request.getNonce());
return DispatchMergingRegionsResponse.newBuilder().setProcId(procId).build();
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
}
@Override
public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c,
EnableCatalogJanitorRequest req) throws ServiceException {

View File

@ -325,23 +325,6 @@ public interface MasterServices extends Server {
*/
boolean registerService(Service instance);
/**
* Merge two regions. The real implementation is on the regionserver, master
* just move the regions together and send MERGE RPC to regionserver
* @param region_a region to merge
* @param region_b region to merge
* @param forcible true if do a compulsory merge, otherwise we will only merge
* two adjacent regions
* @return procedure Id
* @throws IOException
*/
long dispatchMergingRegions(
final HRegionInfo region_a,
final HRegionInfo region_b,
final boolean forcible,
final long nonceGroup,
final long nonce) throws IOException;
/**
* @return true if master is the active one
*/

View File

@ -895,37 +895,6 @@ public class ServerManager {
+ " timeout " + timeout);
}
/**
* Sends an MERGE REGIONS RPC to the specified server to merge the specified
* regions.
* <p>
* A region server could reject the close request because it either does not
* have the specified region.
* @param server server to merge regions
* @param region_a region to merge
* @param region_b region to merge
* @param forcible true if do a compulsory merge, otherwise we will only merge
* two adjacent regions
* @throws IOException
*/
public void sendRegionsMerge(ServerName server, HRegionInfo region_a,
HRegionInfo region_b, boolean forcible, final User user) throws IOException {
if (server == null)
throw new NullPointerException("Passed server is null");
if (region_a == null || region_b == null)
throw new NullPointerException("Passed region is null");
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
throw new IOException("Attempting to send MERGE REGIONS RPC to server "
+ server.toString() + " for region "
+ region_a.getRegionNameAsString() + ","
+ region_b.getRegionNameAsString()
+ " failed because no RPC connection found to this server");
}
HBaseRpcController controller = newRpcController();
ProtobufUtil.mergeRegions(controller, admin, region_a, region_b, forcible, user);
}
/**
* Check if a region server is reachable and has the expected start code
*/

View File

@ -1,579 +0,0 @@
/**
* 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.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.CatalogJanitor;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.ServerManager;
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.DispatchMergingRegionsState;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* The procedure to Merge a region in a table.
*/
@InterfaceAudience.Private
public class DispatchMergingRegionsProcedure
extends AbstractStateMachineTableProcedure<DispatchMergingRegionsState> {
private static final Log LOG = LogFactory.getLog(DispatchMergingRegionsProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
private Boolean traceEnabled;
private AssignmentManager assignmentManager;
private int timeout;
private ServerName regionLocation;
private String regionsToMergeListFullName;
private String regionsToMergeListEncodedName;
private TableName tableName;
private HRegionInfo [] regionsToMerge;
private boolean forcible;
public DispatchMergingRegionsProcedure() {
this.traceEnabled = isTraceEnabled();
this.assignmentManager = null;
this.timeout = -1;
this.regionLocation = null;
this.regionsToMergeListFullName = null;
this.regionsToMergeListEncodedName = null;
}
public DispatchMergingRegionsProcedure(
final MasterProcedureEnv env,
final TableName tableName,
final HRegionInfo [] regionsToMerge,
final boolean forcible) {
super(env);
this.traceEnabled = isTraceEnabled();
this.assignmentManager = getAssignmentManager(env);
this.tableName = tableName;
// For now, we only merge 2 regions. It could be extended to more than 2 regions in
// the future.
assert(regionsToMerge.length == 2);
this.regionsToMerge = regionsToMerge;
this.forcible = forcible;
this.timeout = -1;
this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
this.regionsToMergeListEncodedName = getRegionsToMergeListEncodedNameString();
}
@Override
protected Flow executeFromState(
final MasterProcedureEnv env,
final DispatchMergingRegionsState state) throws InterruptedException {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case DISPATCH_MERGING_REGIONS_PREPARE:
prepareMergeRegion(env);
setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_PRE_OPERATION);
break;
case DISPATCH_MERGING_REGIONS_PRE_OPERATION:
//Unused for now - reserve to add preMerge coprocessor in the future
setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS);
break;
case DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS:
if (MoveRegionsToSameRS(env)) {
setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS);
} else {
LOG.info("Cancel merging regions " + getRegionsToMergeListFullNameString()
+ ", because can't move them to the same RS");
setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_POST_OPERATION);
}
break;
case DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS:
doMergeInRS(env);
setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_POST_OPERATION);
break;
case DISPATCH_MERGING_REGIONS_POST_OPERATION:
//Unused for now - reserve to add postCompletedMerge coprocessor in the future
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
LOG.warn("Error trying to merge regions " + getRegionsToMergeListFullNameString() +
" in the table " + tableName + " (in state=" + state + ")", e);
setFailure("master-merge-regions", e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(
final MasterProcedureEnv env,
final DispatchMergingRegionsState state) throws IOException, InterruptedException {
if (isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case DISPATCH_MERGING_REGIONS_POST_OPERATION:
case DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS:
String msg = this + " We are in the " + state + " state."
+ " It is complicated to rollback the merge operation that region server is working on."
+ " Rollback is not supported and we should let the merge operation to complete";
LOG.warn(msg);
// PONR
throw new UnsupportedOperationException(this + " unhandled state=" + state);
case DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS:
break; // nothing to rollback
case DISPATCH_MERGING_REGIONS_PRE_OPERATION:
break; // nothing to rollback
case DISPATCH_MERGING_REGIONS_PREPARE:
break; // nothing to rollback
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (Exception 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 merging the regions "
+ getRegionsToMergeListFullNameString() + " in table " + tableName, e);
throw e;
}
}
@Override
protected DispatchMergingRegionsState getState(final int stateId) {
return DispatchMergingRegionsState.valueOf(stateId);
}
@Override
protected int getStateId(final DispatchMergingRegionsState state) {
return state.getNumber();
}
@Override
protected DispatchMergingRegionsState getInitialState() {
return DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_PREPARE;
}
/*
* Check whether we are in the state that can be rollback
*/
@Override
protected boolean isRollbackSupported(final DispatchMergingRegionsState state) {
switch (state) {
case DISPATCH_MERGING_REGIONS_POST_OPERATION:
case DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS:
// It is not safe to rollback if we reach to these states.
return false;
default:
break;
}
return true;
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.DispatchMergingRegionsStateData.Builder dispatchMergingRegionsMsg =
MasterProcedureProtos.DispatchMergingRegionsStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setForcible(forcible);
for (HRegionInfo hri: regionsToMerge) {
dispatchMergingRegionsMsg.addRegionInfo(HRegionInfo.convert(hri));
}
dispatchMergingRegionsMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.DispatchMergingRegionsStateData dispatchMergingRegionsMsg =
MasterProcedureProtos.DispatchMergingRegionsStateData.parseDelimitedFrom(stream);
setUser(MasterProcedureUtil.toUserInfo(dispatchMergingRegionsMsg.getUserInfo()));
tableName = ProtobufUtil.toTableName(dispatchMergingRegionsMsg.getTableName());
assert(dispatchMergingRegionsMsg.getRegionInfoCount() == 2);
regionsToMerge = new HRegionInfo[dispatchMergingRegionsMsg.getRegionInfoCount()];
for (int i = 0; i < regionsToMerge.length; i++) {
regionsToMerge[i] = HRegionInfo.convert(dispatchMergingRegionsMsg.getRegionInfo(i));
}
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (table=");
sb.append(tableName);
sb.append(" regions=");
sb.append(getRegionsToMergeListFullNameString());
sb.append(" forcible=");
sb.append(forcible);
sb.append(")");
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
return !env.getProcedureQueue().waitRegions(
this, getTableName(), regionsToMerge[0], regionsToMerge[1]);
}
@Override
protected void releaseLock(final MasterProcedureEnv env) {
env.getProcedureQueue().wakeRegions(this, getTableName(), regionsToMerge[0], regionsToMerge[1]);
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.MERGE;
}
/**
* Prepare merge and do some check
* @param env MasterProcedureEnv
* @throws IOException
*/
private void prepareMergeRegion(final MasterProcedureEnv env) throws IOException {
// Note: the following logic assumes that we only have 2 regions to merge. In the future,
// if we want to extend to more than 2 regions, the code needs to modify a little bit.
//
CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor();
boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]);
if (regionAHasMergeQualifier
|| !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) {
String msg = "Skip merging regions " + regionsToMerge[0].getRegionNameAsString()
+ ", " + regionsToMerge[1].getRegionNameAsString() + ", because region "
+ (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
.getEncodedName()) + " has merge qualifier";
LOG.info(msg);
throw new MergeRegionException(msg);
}
RegionStates regionStates = getAssignmentManager(env).getRegionStates();
RegionState regionStateA = regionStates.getRegionState(regionsToMerge[0].getEncodedName());
RegionState regionStateB = regionStates.getRegionState(regionsToMerge[1].getEncodedName());
if (regionStateA == null || regionStateB == null) {
throw new UnknownRegionException(
regionStateA == null ?
regionsToMerge[0].getEncodedName() : regionsToMerge[1].getEncodedName());
}
if (!regionStateA.isOpened() || !regionStateB.isOpened()) {
throw new MergeRegionException(
"Unable to merge regions not online " + regionStateA + ", " + regionStateB);
}
if (regionsToMerge[0].getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
regionsToMerge[1].getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
throw new MergeRegionException("Can't merge non-default replicas");
}
if (!forcible && !HRegionInfo.areAdjacent(regionsToMerge[0], regionsToMerge[1])) {
throw new MergeRegionException(
"Unable to merge not adjacent regions "
+ regionsToMerge[0].getRegionNameAsString() + ", "
+ regionsToMerge[1].getRegionNameAsString()
+ " where forcible = " + forcible);
}
}
/**
* Move all regions to the same region server
* @param env MasterProcedureEnv
* @return whether target regions hosted by the same RS
* @throws IOException
*/
private boolean MoveRegionsToSameRS(final MasterProcedureEnv env) throws IOException {
// Make sure regions are on the same regionserver before send merge
// regions request to region server.
//
boolean onSameRS = isRegionsOnTheSameServer(env);
if (!onSameRS) {
// Note: the following logic assumes that we only have 2 regions to merge. In the future,
// if we want to extend to more than 2 regions, the code needs to modify a little bit.
//
RegionStates regionStates = getAssignmentManager(env).getRegionStates();
ServerName regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[1]);
RegionLoad loadOfRegionA = getRegionLoad(env, regionLocation, regionsToMerge[0]);
RegionLoad loadOfRegionB = getRegionLoad(env, regionLocation2, regionsToMerge[1]);
if (loadOfRegionA != null && loadOfRegionB != null
&& loadOfRegionA.getRequestsCount() < loadOfRegionB.getRequestsCount()) {
// switch regionsToMerge[0] and regionsToMerge[1]
HRegionInfo tmpRegion = this.regionsToMerge[0];
this.regionsToMerge[0] = this.regionsToMerge[1];
this.regionsToMerge[1] = tmpRegion;
ServerName tmpLocation = regionLocation;
regionLocation = regionLocation2;
regionLocation2 = tmpLocation;
}
long startTime = EnvironmentEdgeManager.currentTime();
RegionPlan regionPlan = new RegionPlan(regionsToMerge[1], regionLocation2, regionLocation);
LOG.info("Moving regions to same server for merge: " + regionPlan.toString());
getAssignmentManager(env).balance(regionPlan);
do {
try {
Thread.sleep(20);
// Make sure check RIT first, then get region location, otherwise
// we would make a wrong result if region is online between getting
// region location and checking RIT
boolean isRIT = regionStates.isRegionInTransition(regionsToMerge[1]);
regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[1]);
onSameRS = regionLocation.equals(regionLocation2);
if (onSameRS || !isRIT) {
// Regions are on the same RS, or regionsToMerge[1] is not in
// RegionInTransition any more
break;
}
} catch (InterruptedException e) {
InterruptedIOException iioe = new InterruptedIOException();
iioe.initCause(e);
throw iioe;
}
} while ((EnvironmentEdgeManager.currentTime() - startTime) <= getTimeout(env));
}
return onSameRS;
}
/**
* Do the real merge operation in the region server that hosts regions
* @param env MasterProcedureEnv
* @throws IOException
*/
private void doMergeInRS(final MasterProcedureEnv env) throws IOException {
long duration = 0;
long startTime = EnvironmentEdgeManager.currentTime();
do {
try {
if (getServerName(env) == null) {
// The merge probably already happen. Check
RegionState regionState = getAssignmentManager(env).getRegionStates().getRegionState(
regionsToMerge[0].getEncodedName());
if (regionState.isMerging() || regionState.isMerged()) {
LOG.info("Merge regions " + getRegionsToMergeListEncodedNameString() +
" is in progress or completed. No need to send a new request.");
} else {
LOG.warn("Cannot sending merge to hosting server of the regions " +
getRegionsToMergeListEncodedNameString() + " as the server is unknown");
}
return;
}
// TODO: the following RPC call is not idempotent. Multiple calls (eg. after master
// failover, re-execute this step) could result in some exception thrown that does not
// paint the correct picture. This behavior is on-par with old releases. Improvement
// could happen in the future.
env.getMasterServices().getServerManager().sendRegionsMerge(
getServerName(env),
regionsToMerge[0],
regionsToMerge[1],
forcible,
getUser());
LOG.info("Sent merge to server " + getServerName(env) + " for region " +
getRegionsToMergeListEncodedNameString() + ", forcible=" + forcible);
return;
} catch (RegionOpeningException roe) {
// Do a retry since region should be online on RS immediately
LOG.warn("Failed mergering regions in " + getServerName(env) + ", retrying...", roe);
} catch (Exception ie) {
LOG.warn("Failed sending merge to " + getServerName(env) + " for regions " +
getRegionsToMergeListEncodedNameString() + ", forcible=" + forcible, ie);
return;
}
} while ((duration = EnvironmentEdgeManager.currentTime() - startTime) <= getTimeout(env));
// If we reaches here, it means that we get timed out.
String msg = "Failed sending merge to " + getServerName(env) + " after " + duration + "ms";
LOG.warn(msg);
throw new IOException(msg);
}
private RegionLoad getRegionLoad(
final MasterProcedureEnv env,
final ServerName sn,
final HRegionInfo hri) {
ServerManager serverManager = env.getMasterServices().getServerManager();
ServerLoad load = serverManager.getLoad(sn);
if (load != null) {
Map<byte[], RegionLoad> regionsLoad = load.getRegionsLoad();
if (regionsLoad != null) {
return regionsLoad.get(hri.getRegionName());
}
}
return null;
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @param env MasterProcedureEnv
* @return whether target regions hosted by the same RS
*/
private boolean isRegionsOnTheSameServer(final MasterProcedureEnv env) throws IOException{
Boolean onSameRS = true;
int i = 0;
RegionStates regionStates = getAssignmentManager(env).getRegionStates();
regionLocation = regionStates.getRegionServerOfRegion(regionsToMerge[i]);
if (regionLocation != null) {
for(i = 1; i < regionsToMerge.length; i++) {
ServerName regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[i]);
if (regionLocation2 != null) {
if (onSameRS) {
onSameRS = regionLocation.equals(regionLocation2);
}
} else {
// At least one region is not online, merge will fail, no need to continue.
break;
}
}
if (i == regionsToMerge.length) {
// Finish checking all regions, return the result;
return onSameRS;
}
}
// If reaching here, at least one region is not online.
String msg = "Skip merging regions " + getRegionsToMergeListFullNameString() +
", because region " + regionsToMerge[i].getEncodedName() + " is not online now.";
LOG.warn(msg);
throw new IOException(msg);
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @param env MasterProcedureEnv
* @return assignmentManager
*/
private AssignmentManager getAssignmentManager(final MasterProcedureEnv env) {
if (assignmentManager == null) {
assignmentManager = env.getMasterServices().getAssignmentManager();
}
return assignmentManager;
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @param env MasterProcedureEnv
* @return timeout value
*/
private int getTimeout(final MasterProcedureEnv env) {
if (timeout == -1) {
timeout = env.getMasterConfiguration().getInt(
"hbase.master.regionmerge.timeout", regionsToMerge.length * 60 * 1000);
}
return timeout;
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @param env MasterProcedureEnv
* @return serverName
*/
private ServerName getServerName(final MasterProcedureEnv env) {
if (regionLocation == null) {
regionLocation =
getAssignmentManager(env).getRegionStates().getRegionServerOfRegion(regionsToMerge[0]);
}
return regionLocation;
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @param fullName whether return only encoded name
* @return region names in a list
*/
private String getRegionsToMergeListFullNameString() {
if (regionsToMergeListFullName == null) {
StringBuilder sb = new StringBuilder("[");
int i = 0;
while(i < regionsToMerge.length - 1) {
sb.append(regionsToMerge[i].getRegionNameAsString() + ", ");
i++;
}
sb.append(regionsToMerge[i].getRegionNameAsString() + " ]");
regionsToMergeListFullName = sb.toString();
}
return regionsToMergeListFullName;
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @return encoded region names
*/
private String getRegionsToMergeListEncodedNameString() {
if (regionsToMergeListEncodedName == null) {
StringBuilder sb = new StringBuilder("[");
int i = 0;
while(i < regionsToMerge.length - 1) {
sb.append(regionsToMerge[i].getEncodedName() + ", ");
i++;
}
sb.append(regionsToMerge[i].getEncodedName() + " ]");
regionsToMergeListEncodedName = sb.toString();
}
return regionsToMergeListEncodedName;
}
/**
* 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

@ -217,20 +217,6 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
return queueLists.toString();
}
public synchronized void requestRegionsMerge(final Region a,
final Region b, final boolean forcible, long masterSystemTime, User user) {
try {
mergePool.execute(new RegionMergeRequest(a, b, this.server, forcible, masterSystemTime,user));
if (LOG.isDebugEnabled()) {
LOG.debug("Region merge requested for " + a + "," + b + ", forcible="
+ forcible + ". " + this);
}
} catch (RejectedExecutionException ree) {
LOG.warn("Could not execute merge for " + a + "," + b + ", forcible="
+ forcible, ree);
}
}
public synchronized boolean requestSplit(final Region r) {
// don't split regions that are blocking
if (shouldSplitRegion() && ((HRegion)r).getCompactPriority() >= Store.PRIORITY_USER) {

View File

@ -6881,112 +6881,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
(Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
}
/**
* Merge two HRegions. The regions must be adjacent and must not overlap.
*
* @return new merged HRegion
* @throws IOException
*/
public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
throws IOException {
HRegion a = srcA;
HRegion b = srcB;
// Make sure that srcA comes first; important for key-ordering during
// write of the merged file.
if (srcA.getRegionInfo().getStartKey() == null) {
if (srcB.getRegionInfo().getStartKey() == null) {
throw new IOException("Cannot merge two regions with null start key");
}
// A's start key is null but B's isn't. Assume A comes before B
} else if ((srcB.getRegionInfo().getStartKey() == null) ||
(Bytes.compareTo(srcA.getRegionInfo().getStartKey(),
srcB.getRegionInfo().getStartKey()) > 0)) {
a = srcB;
b = srcA;
}
if (!(Bytes.compareTo(a.getRegionInfo().getEndKey(),
b.getRegionInfo().getStartKey()) == 0)) {
throw new IOException("Cannot merge non-adjacent regions");
}
return merge(a, b);
}
/**
* Merge two regions whether they are adjacent or not.
*
* @param a region a
* @param b region b
* @return new merged region
* @throws IOException
*/
public static HRegion merge(final HRegion a, final HRegion b) throws IOException {
if (!a.getRegionInfo().getTable().equals(b.getRegionInfo().getTable())) {
throw new IOException("Regions do not belong to the same table");
}
FileSystem fs = a.getRegionFileSystem().getFileSystem();
// Make sure each region's cache is empty
a.flush(true);
b.flush(true);
// Compact each region so we only have one store file per family
a.compact(true);
if (LOG.isDebugEnabled()) {
LOG.debug("Files for region: " + a);
a.getRegionFileSystem().logFileSystemState(LOG);
}
b.compact(true);
if (LOG.isDebugEnabled()) {
LOG.debug("Files for region: " + b);
b.getRegionFileSystem().logFileSystemState(LOG);
}
RegionMergeTransactionImpl rmt = new RegionMergeTransactionImpl(a, b, true);
if (!rmt.prepare(null)) {
throw new IOException("Unable to merge regions " + a + " and " + b);
}
HRegionInfo mergedRegionInfo = rmt.getMergedRegionInfo();
LOG.info("starting merge of regions: " + a + " and " + b
+ " into new region " + mergedRegionInfo.getRegionNameAsString()
+ " with start key <"
+ Bytes.toStringBinary(mergedRegionInfo.getStartKey())
+ "> and end key <"
+ Bytes.toStringBinary(mergedRegionInfo.getEndKey()) + ">");
HRegion dstRegion;
try {
dstRegion = (HRegion)rmt.execute(null, null);
} catch (IOException ioe) {
rmt.rollback(null, null);
throw new IOException("Failed merging region " + a + " and " + b
+ ", and successfully rolled back");
}
dstRegion.compact(true);
if (LOG.isDebugEnabled()) {
LOG.debug("Files for new region");
dstRegion.getRegionFileSystem().logFileSystemState(LOG);
}
// clear the compacted files if any
for (Store s : dstRegion.getStores()) {
s.closeAndArchiveCompactedFiles();
}
if (dstRegion.getRegionFileSystem().hasReferences(dstRegion.getTableDesc())) {
throw new IOException("Merged region " + dstRegion
+ " still has references after the compaction, is compaction canceled?");
}
// Archiving the 'A' region
HFileArchiver.archiveRegion(a.getBaseConf(), fs, a.getRegionInfo());
// Archiving the 'B' region
HFileArchiver.archiveRegion(b.getBaseConf(), fs, b.getRegionInfo());
LOG.info("merge completed. New region is " + dstRegion);
return dstRegion;
}
@Override
public Result get(final Get get) throws IOException {
prepareGet(get);

View File

@ -115,8 +115,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerIn
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
@ -1641,46 +1639,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
/**
* Merge regions on the region server.
*
* @param controller the RPC controller
* @param request the request
* @return merge regions response
* @throws ServiceException
*/
@Override
@QosPriority(priority = HConstants.ADMIN_QOS)
public MergeRegionsResponse mergeRegions(final RpcController controller,
final MergeRegionsRequest request) throws ServiceException {
try {
checkOpen();
requestCount.increment();
Region regionA = getRegion(request.getRegionA());
Region regionB = getRegion(request.getRegionB());
boolean forcible = request.getForcible();
long masterSystemTime = request.hasMasterSystemTime() ? request.getMasterSystemTime() : -1;
regionA.startRegionOperation(Operation.MERGE_REGION);
regionB.startRegionOperation(Operation.MERGE_REGION);
if (regionA.getRegionInfo().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
regionB.getRegionInfo().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
throw new ServiceException(new MergeRegionException("Can't merge non-default replicas"));
}
LOG.info("Receiving merging request for " + regionA + ", " + regionB
+ ",forcible=" + forcible);
regionA.flush(true);
regionB.flush(true);
regionServer.compactSplitThread.requestRegionsMerge(regionA, regionB, forcible,
masterSystemTime, RpcServer.getRequestUser());
return MergeRegionsResponse.newBuilder().build();
} catch (DroppedSnapshotException ex) {
regionServer.abort("Replay of WAL required. Forcing server shutdown", ex);
throw new ServiceException(ex);
} catch (IOException ie) {
throw new ServiceException(ie);
}
}
/**
* Open asynchronously a region or a set of regions on the region server.
*

View File

@ -1,153 +0,0 @@
/**
* Copyright The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import 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.EnvironmentEdgeManager;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import com.google.common.base.Preconditions;
/**
* Handles processing region merges. Put in a queue, owned by HRegionServer.
*/
@InterfaceAudience.Private
class RegionMergeRequest implements Runnable {
private static final Log LOG = LogFactory.getLog(RegionMergeRequest.class);
private final HRegion region_a;
private final HRegion region_b;
private final HRegionServer server;
private final boolean forcible;
private TableLock tableLock;
private final long masterSystemTime;
private final User user;
RegionMergeRequest(Region a, Region b, HRegionServer hrs, boolean forcible,
long masterSystemTime, User user) {
Preconditions.checkNotNull(hrs);
this.region_a = (HRegion)a;
this.region_b = (HRegion)b;
this.server = hrs;
this.forcible = forcible;
this.masterSystemTime = masterSystemTime;
this.user = user;
}
@Override
public String toString() {
return "MergeRequest,regions:" + region_a + ", " + region_b + ", forcible="
+ forcible;
}
@Override
public void run() {
if (this.server.isStopping() || this.server.isStopped()) {
LOG.debug("Skipping merge because server is stopping="
+ this.server.isStopping() + " or stopped=" + this.server.isStopped());
return;
}
try {
final long startTime = EnvironmentEdgeManager.currentTime();
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a,
region_b, forcible, masterSystemTime);
//acquire a shared read lock on the table, so that table schema modifications
//do not happen concurrently
tableLock = server.getTableLockManager().readLock(region_a.getTableDesc().getTableName()
, "MERGE_REGIONS:" + region_a.getRegionInfo().getRegionNameAsString() + ", " +
region_b.getRegionInfo().getRegionNameAsString());
try {
tableLock.acquire();
} catch (IOException ex) {
tableLock = null;
throw ex;
}
// If prepare does not return true, for some reason -- logged inside in
// the prepare call -- we are not ready to merge just now. Just return.
if (!mt.prepare(this.server)) return;
try {
mt.execute(this.server, this.server, this.user);
} catch (Exception e) {
if (this.server.isStopping() || this.server.isStopped()) {
LOG.info(
"Skip rollback/cleanup of failed merge of " + region_a + " and "
+ region_b + " 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;
}
try {
LOG.warn("Running rollback/cleanup of failed merge of "
+ region_a +" and "+ region_b + "; " + e.getMessage(), e);
if (mt.rollback(this.server, this.server)) {
LOG.info("Successful rollback of failed merge of "
+ region_a +" and "+ region_b);
} else {
this.server.abort("Abort; we got an error after point-of-no-return"
+ "when merging " + region_a + " and " + region_b);
}
} catch (RuntimeException ee) {
String msg = "Failed rollback of failed merge of "
+ region_a +" and "+ region_b + " -- aborting server";
// If failed rollback, kill this server to avoid having a hole in
// table.
LOG.info(msg, ee);
this.server.abort(msg);
}
return;
}
LOG.info("Regions merged, hbase:meta updated, and report to master. region_a="
+ region_a + ", region_b=" + region_b + ",merged region="
+ mt.getMergedRegionInfo().getRegionNameAsString()
+ ". Region merge took "
+ StringUtils.formatTimeDiff(EnvironmentEdgeManager.currentTime(), startTime));
} catch (IOException ex) {
ex = ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex;
LOG.error("Merge failed " + this, ex);
server.checkFileSystem();
} finally {
releaseTableLock();
}
}
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 " + region_a.getRegionInfo().getRegionNameAsString());
}
}
}
}

View File

@ -1,76 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.ReflectionUtils;
/**
* A factory for creating RegionMergeTransactions, which execute region split as a "transaction".
* See {@link RegionMergeTransactionImpl}
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public class RegionMergeTransactionFactory implements Configurable {
public static final String MERGE_TRANSACTION_IMPL_KEY =
"hbase.regionserver.merge.transaction.impl";
private Configuration conf;
public RegionMergeTransactionFactory(Configuration conf) {
this.conf = conf;
}
@Override
public Configuration getConf() {
return conf;
}
@Override
public void setConf(Configuration conf) {
this.conf = conf;
}
/**
* Create a merge transaction
* @param a region a to merge
* @param b region b to merge
* @param forcible if false, we will only merge adjacent regions
* @return transaction instance
*/
public RegionMergeTransactionImpl create(final Region a, final Region b,
final boolean forcible) {
// The implementation class must extend RegionMergeTransactionImpl, not only
// implement the RegionMergeTransaction interface like you might expect,
// because various places such as AssignmentManager use static methods
// from RegionMergeTransactionImpl. Whatever we use for implementation must
// be compatible, so it's safest to require ? extends RegionMergeTransactionImpl.
// If not compatible we will throw a runtime exception from here.
return ReflectionUtils.instantiateWithCustomCtor(
conf.getClass(MERGE_TRANSACTION_IMPL_KEY, RegionMergeTransactionImpl.class,
RegionMergeTransactionImpl.class).getName(),
new Class[] { Region.class, Region.class, boolean.class },
new Object[] { a, b, forcible });
}
}

View File

@ -1,742 +0,0 @@
/**
* Copyright The Apache Software Foundation
*
* 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 limitationsME
* under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl.LoggingProgressable;
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.Pair;
import com.google.common.annotations.VisibleForTesting;
@InterfaceAudience.Private
public class RegionMergeTransactionImpl implements RegionMergeTransaction {
private static final Log LOG = LogFactory.getLog(RegionMergeTransactionImpl.class);
// Merged region info
private HRegionInfo mergedRegionInfo;
// region_a sorts before region_b
private final HRegion region_a;
private final HRegion region_b;
// merges dir is under region_a
private final Path mergesdir;
// We only merge adjacent regions if forcible is false
private final boolean forcible;
private final long masterSystemTime;
/*
* Transaction state for listener, only valid during execute and
* rollback
*/
private RegionMergeTransactionPhase currentPhase = RegionMergeTransactionPhase.STARTED;
private Server server;
private RegionServerServices rsServices;
public static class JournalEntryImpl implements JournalEntry {
private RegionMergeTransactionPhase type;
private long timestamp;
public JournalEntryImpl(RegionMergeTransactionPhase type) {
this(type, EnvironmentEdgeManager.currentTime());
}
public JournalEntryImpl(RegionMergeTransactionPhase type, long timestamp) {
this.type = type;
this.timestamp = timestamp;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(type);
sb.append(" at ");
sb.append(timestamp);
return sb.toString();
}
@Override
public RegionMergeTransactionPhase getPhase() {
return type;
}
@Override
public long getTimeStamp() {
return timestamp;
}
}
/*
* Journal of how far the merge transaction has progressed.
*/
private final List<JournalEntry> journal = new ArrayList<JournalEntry>();
/**
* Listeners
*/
private final ArrayList<TransactionListener> listeners = new ArrayList<TransactionListener>();
private static IOException closedByOtherException = new IOException(
"Failed to close region: already closed by another thread");
private RegionServerCoprocessorHost rsCoprocessorHost = null;
/**
* Constructor
* @param a region a to merge
* @param b region b to merge
* @param forcible if false, we will only merge adjacent regions
*/
public RegionMergeTransactionImpl(final Region a, final Region b,
final boolean forcible) {
this(a, b, forcible, EnvironmentEdgeManager.currentTime());
}
/**
* Constructor
* @param a region a to merge
* @param b region b to merge
* @param forcible if false, we will only merge adjacent regions
* @param masterSystemTime the time at the master side
*/
public RegionMergeTransactionImpl(final Region a, final Region b,
final boolean forcible, long masterSystemTime) {
if (a.getRegionInfo().compareTo(b.getRegionInfo()) <= 0) {
this.region_a = (HRegion)a;
this.region_b = (HRegion)b;
} else {
this.region_a = (HRegion)b;
this.region_b = (HRegion)a;
}
this.forcible = forcible;
this.masterSystemTime = masterSystemTime;
this.mergesdir = region_a.getRegionFileSystem().getMergesDir();
}
private void transition(RegionMergeTransactionPhase nextPhase) throws IOException {
transition(nextPhase, false);
}
private void transition(RegionMergeTransactionPhase nextPhase, boolean isRollback)
throws IOException {
if (!isRollback) {
// Add to the journal first, because if the listener throws an exception
// we need to roll back starting at 'nextPhase'
this.journal.add(new JournalEntryImpl(nextPhase));
}
for (int i = 0; i < listeners.size(); i++) {
TransactionListener listener = listeners.get(i);
if (!isRollback) {
listener.transition(this, currentPhase, nextPhase);
} else {
listener.rollback(this, currentPhase, nextPhase);
}
}
currentPhase = nextPhase;
}
@Override
public boolean prepare(final RegionServerServices services) throws IOException {
if (!region_a.getTableDesc().getTableName()
.equals(region_b.getTableDesc().getTableName())) {
LOG.info("Can't merge regions " + region_a + "," + region_b
+ " because they do not belong to the same table");
return false;
}
if (region_a.getRegionInfo().equals(region_b.getRegionInfo())) {
LOG.info("Can't merge the same region " + region_a);
return false;
}
if (!forcible && !HRegionInfo.areAdjacent(region_a.getRegionInfo(),
region_b.getRegionInfo())) {
String msg = "Skip merging " + region_a.getRegionInfo().getRegionNameAsString()
+ " and " + region_b.getRegionInfo().getRegionNameAsString()
+ ", because they are not adjacent.";
LOG.info(msg);
return false;
}
if (!this.region_a.isMergeable() || !this.region_b.isMergeable()) {
return false;
}
try {
boolean regionAHasMergeQualifier = hasMergeQualifierInMeta(services,
region_a.getRegionInfo().getRegionName());
if (regionAHasMergeQualifier ||
hasMergeQualifierInMeta(services, region_b.getRegionInfo().getRegionName())) {
LOG.debug("Region " + (regionAHasMergeQualifier ?
region_a.getRegionInfo().getRegionNameAsString()
: region_b.getRegionInfo().getRegionNameAsString())
+ " is not mergeable because it has merge qualifier in META");
return false;
}
} catch (IOException e) {
LOG.warn("Failed judging whether merge transaction is available for "
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
+ region_b.getRegionInfo().getRegionNameAsString(), e);
return false;
}
// WARN: make sure there is no parent region of the two merging regions in
// hbase:meta If exists, fixing up daughters would cause daughter regions(we
// have merged one) online again when we restart master, so we should clear
// the parent region to prevent the above case
// Since HBASE-7721, we don't need fix up daughters any more. so here do
// nothing
this.mergedRegionInfo = getMergedRegionInfo(region_a.getRegionInfo(),
region_b.getRegionInfo());
transition(RegionMergeTransactionPhase.PREPARED);
return true;
}
@Override
public Region execute(final Server server, final RegionServerServices services)
throws IOException {
if (User.isHBaseSecurityEnabled(region_a.getBaseConf())) {
LOG.warn("Should use execute(Server, RegionServerServices, User)");
}
return execute(server, services, null);
}
@Override
public Region execute(final Server server, final RegionServerServices services, User user)
throws IOException {
this.server = server;
this.rsServices = services;
if (rsCoprocessorHost == null) {
rsCoprocessorHost = server != null ?
((HRegionServer) server).getRegionServerCoprocessorHost() : null;
}
final HRegion mergedRegion = createMergedRegion(server, services, user);
if (rsCoprocessorHost != null) {
rsCoprocessorHost.postMergeCommit(this.region_a, this.region_b, mergedRegion, user);
}
stepsAfterPONR(server, services, mergedRegion, user);
transition(RegionMergeTransactionPhase.COMPLETED);
return mergedRegion;
}
@VisibleForTesting
public void stepsAfterPONR(final Server server, final RegionServerServices services,
final HRegion mergedRegion, User user) throws IOException {
openMergedRegion(server, services, mergedRegion);
if (rsCoprocessorHost != null) {
rsCoprocessorHost.postMerge(region_a, region_b, mergedRegion, user);
}
}
/**
* Prepare the merged region and region files.
* @param server Hosting server instance. Can be null when testing
* @param services Used to online/offline regions.
* @return merged region
* @throws IOException If thrown, transaction failed. Call
* {@link #rollback(Server, RegionServerServices)}
*/
private HRegion createMergedRegion(final Server server, final RegionServerServices services,
User user) throws IOException {
LOG.info("Starting merge of " + region_a + " and "
+ region_b.getRegionInfo().getRegionNameAsString() + ", forcible=" + forcible);
if ((server != null && server.isStopped())
|| (services != null && services.isStopping())) {
throw new IOException("Server is stopped or stopping");
}
if (rsCoprocessorHost != null) {
boolean ret = rsCoprocessorHost.preMerge(region_a, region_b, user);
if (ret) {
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
+ this.region_b + " merge.");
}
}
// If true, no cluster to write meta edits to or to use coordination.
boolean testing = server == null ? true : server.getConfiguration()
.getBoolean("hbase.testing.nocluster", false);
HRegion mergedRegion = stepsBeforePONR(server, services, testing);
@MetaMutationAnnotation
final List<Mutation> metaEntries = new ArrayList<Mutation>();
if (rsCoprocessorHost != null) {
boolean ret = rsCoprocessorHost.preMergeCommit(region_a, region_b, metaEntries, user);
if (ret) {
throw new IOException("Coprocessor bypassing regions " + this.region_a + " "
+ this.region_b + " merge.");
}
try {
for (Mutation p : metaEntries) {
HRegionInfo.parseRegionName(p.getRow());
}
} catch (IOException e) {
LOG.error("Row key of mutation from coprocessor is not parsable as region name."
+ "Mutations from coprocessor should only be for hbase:meta table.", e);
throw e;
}
}
// This is the point of no return. Similar with SplitTransaction.
// IF we reach the PONR then subsequent failures need to crash out this
// regionserver
transition(RegionMergeTransactionPhase.PONR);
// Add merged region and delete region_a and region_b
// as an atomic update. See HBASE-7721. This update to hbase:meta makes the region
// will determine whether the region is merged or not in case of failures.
// If it is successful, master will roll-forward, if not, master will
// rollback
if (services != null && !services.reportRegionStateTransition(TransitionCode.MERGE_PONR,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
// Passed PONR, let SSH clean it up
throw new IOException("Failed to notify master that merge passed PONR: "
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
+ region_b.getRegionInfo().getRegionNameAsString());
}
return mergedRegion;
}
@VisibleForTesting
public void prepareMutationsForMerge(HRegionInfo mergedRegion, HRegionInfo regionA,
HRegionInfo regionB, ServerName serverName, List<Mutation> mutations) throws IOException {
HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
// use the maximum of what master passed us vs local time.
long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
// Put for parent
Put putOfMerged = MetaTableAccessor.makePutFromRegionInfo(copyOfMerged, time);
putOfMerged.addColumn(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
regionA.toByteArray());
putOfMerged.addColumn(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
regionB.toByteArray());
mutations.add(putOfMerged);
// Deletes for merging regions
Delete deleteA = MetaTableAccessor.makeDeleteFromRegionInfo(regionA, time);
Delete deleteB = MetaTableAccessor.makeDeleteFromRegionInfo(regionB, time);
mutations.add(deleteA);
mutations.add(deleteB);
// The merged is a new region, openSeqNum = 1 is fine.
addLocation(putOfMerged, serverName, 1);
}
@VisibleForTesting
Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
p.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, Bytes
.toBytes(sn.getHostAndPort()));
p.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER, Bytes.toBytes(sn
.getStartcode()));
p.addColumn(HConstants.CATALOG_FAMILY, HConstants.SEQNUM_QUALIFIER, Bytes.toBytes(openSeqNum));
return p;
}
@VisibleForTesting
public HRegion stepsBeforePONR(final Server server, final RegionServerServices services,
boolean testing) throws IOException {
if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_MERGE,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
throw new IOException("Failed to get ok from master to merge "
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
+ region_b.getRegionInfo().getRegionNameAsString());
}
transition(RegionMergeTransactionPhase.SET_MERGING);
this.region_a.getRegionFileSystem().createMergesDir();
transition(RegionMergeTransactionPhase.CREATED_MERGE_DIR);
Map<byte[], List<StoreFile>> hstoreFilesOfRegionA = closeAndOfflineRegion(
services, this.region_a, true, testing);
Map<byte[], List<StoreFile>> hstoreFilesOfRegionB = closeAndOfflineRegion(
services, this.region_b, false, testing);
assert hstoreFilesOfRegionA != null && hstoreFilesOfRegionB != null;
// mergeStoreFiles creates merged region dirs under the region_a merges dir
// Nothing to unroll here if failure -- clean up of CREATE_MERGE_DIR will
// clean this up.
mergeStoreFiles(hstoreFilesOfRegionA, hstoreFilesOfRegionB);
// Log to the journal that we are creating merged region. We could fail
// halfway through. If we do, we could have left
// stuff in fs that needs cleanup -- a storefile or two. Thats why we
// add entry to journal BEFORE rather than AFTER the change.
transition(RegionMergeTransactionPhase.STARTED_MERGED_REGION_CREATION);
HRegion mergedRegion = createMergedRegionFromMerges(this.region_a,
this.region_b, this.mergedRegionInfo);
return mergedRegion;
}
/**
* Create a merged region from the merges directory under region a. In order
* to mock it for tests, place it with a new method.
* @param a hri of region a
* @param b hri of region b
* @param mergedRegion hri of merged region
* @return merged HRegion.
* @throws IOException
*/
@VisibleForTesting
HRegion createMergedRegionFromMerges(final HRegion a, final HRegion b,
final HRegionInfo mergedRegion) throws IOException {
return a.createMergedRegionFromMerges(mergedRegion, b);
}
/**
* Close the merging region and offline it in regionserver
* @param services
* @param region
* @param isRegionA true if it is merging region a, false if it is region b
* @param testing true if it is testing
* @return a map of family name to list of store files
* @throws IOException
*/
private Map<byte[], List<StoreFile>> closeAndOfflineRegion(
final RegionServerServices services, final HRegion region,
final boolean isRegionA, final boolean testing) throws IOException {
Map<byte[], List<StoreFile>> hstoreFilesToMerge = null;
Exception exceptionToThrow = null;
try {
hstoreFilesToMerge = region.close(false);
} catch (Exception e) {
exceptionToThrow = e;
}
if (exceptionToThrow == null && hstoreFilesToMerge == null) {
// The region was closed by a concurrent thread. We can't continue
// with the merge, instead we must just abandon the merge. If we
// reopen or merge this could cause problems because the region has
// probably already been moved to a different server, or is in the
// process of moving to a different server.
exceptionToThrow = closedByOtherException;
}
if (exceptionToThrow != closedByOtherException) {
transition(isRegionA ? RegionMergeTransactionPhase.CLOSED_REGION_A
: RegionMergeTransactionPhase.CLOSED_REGION_B);
}
if (exceptionToThrow != null) {
if (exceptionToThrow instanceof IOException)
throw (IOException) exceptionToThrow;
throw new IOException(exceptionToThrow);
}
if (!testing) {
services.removeFromOnlineRegions(region, null);
}
transition(isRegionA ? RegionMergeTransactionPhase.OFFLINED_REGION_A
: RegionMergeTransactionPhase.OFFLINED_REGION_B);
return hstoreFilesToMerge;
}
/**
* Get merged region info through the specified two regions
* @param a merging region A
* @param b merging region B
* @return the merged region info
*/
@VisibleForTesting
static HRegionInfo getMergedRegionInfo(final HRegionInfo a, final HRegionInfo b) {
long rid = EnvironmentEdgeManager.currentTime();
// Regionid is timestamp. Merged region's id can't be less than that of
// merging regions else will insert at wrong location in hbase:meta
if (rid < a.getRegionId() || rid < b.getRegionId()) {
LOG.warn("Clock skew; merging regions id are " + a.getRegionId()
+ " and " + b.getRegionId() + ", but current time here is " + rid);
rid = Math.max(a.getRegionId(), b.getRegionId()) + 1;
}
byte[] startKey = null;
byte[] endKey = null;
// Choose the smaller as start key
if (a.compareTo(b) <= 0) {
startKey = a.getStartKey();
} else {
startKey = b.getStartKey();
}
// Choose the bigger as end key
if (Bytes.equals(a.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
|| (!Bytes.equals(b.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
&& Bytes.compareTo(a.getEndKey(), b.getEndKey()) > 0)) {
endKey = a.getEndKey();
} else {
endKey = b.getEndKey();
}
// Merged region is sorted between two merging regions in META
HRegionInfo mergedRegionInfo = new HRegionInfo(a.getTable(), startKey,
endKey, false, rid);
return mergedRegionInfo;
}
/**
* Perform time consuming opening of the merged region.
* @param server Hosting server instance. Can be null when testing
* @param services Used to online/offline regions.
* @param merged the merged region
* @throws IOException If thrown, transaction failed. Call
* {@link #rollback(Server, RegionServerServices)}
*/
@VisibleForTesting
void openMergedRegion(final Server server, final RegionServerServices services,
HRegion merged) throws IOException {
boolean stopped = server != null && server.isStopped();
boolean stopping = services != null && services.isStopping();
if (stopped || stopping) {
LOG.info("Not opening merged region " + merged.getRegionInfo().getRegionNameAsString()
+ " because stopping=" + stopping + ", stopped=" + stopped);
return;
}
HRegionInfo hri = merged.getRegionInfo();
LoggingProgressable reporter = server == null ? null
: new LoggingProgressable(hri, server.getConfiguration().getLong(
"hbase.regionserver.regionmerge.open.log.interval", 10000));
merged.openHRegion(reporter);
if (services != null) {
if (!services.reportRegionStateTransition(TransitionCode.MERGED,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
throw new IOException("Failed to report merged region to master: "
+ mergedRegionInfo.getShortNameToLog());
}
services.addToOnlineRegions(merged);
}
}
/**
* Create reference file(s) of merging regions under the region_a merges dir
* @param hstoreFilesOfRegionA
* @param hstoreFilesOfRegionB
* @throws IOException
*/
private void mergeStoreFiles(
Map<byte[], List<StoreFile>> hstoreFilesOfRegionA,
Map<byte[], List<StoreFile>> hstoreFilesOfRegionB)
throws IOException {
// Create reference file(s) of region A in mergdir
HRegionFileSystem fs_a = this.region_a.getRegionFileSystem();
for (Map.Entry<byte[], List<StoreFile>> entry : hstoreFilesOfRegionA
.entrySet()) {
String familyName = Bytes.toString(entry.getKey());
for (StoreFile storeFile : entry.getValue()) {
fs_a.mergeStoreFile(this.mergedRegionInfo, familyName, storeFile,
this.mergesdir);
}
}
// Create reference file(s) of region B in mergedir
HRegionFileSystem fs_b = this.region_b.getRegionFileSystem();
for (Map.Entry<byte[], List<StoreFile>> entry : hstoreFilesOfRegionB
.entrySet()) {
String familyName = Bytes.toString(entry.getKey());
for (StoreFile storeFile : entry.getValue()) {
fs_b.mergeStoreFile(this.mergedRegionInfo, familyName, storeFile,
this.mergesdir);
}
}
}
@Override
public boolean rollback(final Server server,
final RegionServerServices services) throws IOException {
if (User.isHBaseSecurityEnabled(region_a.getBaseConf())) {
LOG.warn("Should use execute(Server, RegionServerServices, User)");
}
return rollback(server, services, null);
}
@Override
public boolean rollback(final Server server,
final RegionServerServices services, User user) throws IOException {
assert this.mergedRegionInfo != null;
this.server = server;
this.rsServices = services;
// Coprocessor callback
if (rsCoprocessorHost != null) {
rsCoprocessorHost.preRollBackMerge(region_a, region_b, user);
}
boolean result = true;
ListIterator<JournalEntry> iterator = this.journal
.listIterator(this.journal.size());
// Iterate in reverse.
while (iterator.hasPrevious()) {
JournalEntry je = iterator.previous();
transition(je.getPhase(), true);
switch (je.getPhase()) {
case SET_MERGING:
if (services != null
&& !services.reportRegionStateTransition(TransitionCode.MERGE_REVERTED,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
return false;
}
break;
case CREATED_MERGE_DIR:
this.region_a.writestate.writesEnabled = true;
this.region_b.writestate.writesEnabled = true;
this.region_a.getRegionFileSystem().cleanupMergesDir();
break;
case CLOSED_REGION_A:
try {
// So, this returns a seqid but if we just closed and then reopened,
// we should be ok. On close, we flushed using sequenceid obtained
// from hosting regionserver so no need to propagate the sequenceid
// returned out of initialize below up into regionserver as we
// normally do.
this.region_a.initialize();
} catch (IOException e) {
LOG.error("Failed rollbacking CLOSED_REGION_A of region "
+ region_a.getRegionInfo().getRegionNameAsString(), e);
throw new RuntimeException(e);
}
break;
case OFFLINED_REGION_A:
if (services != null)
services.addToOnlineRegions(this.region_a);
break;
case CLOSED_REGION_B:
try {
this.region_b.initialize();
} catch (IOException e) {
LOG.error("Failed rollbacking CLOSED_REGION_A of region "
+ region_b.getRegionInfo().getRegionNameAsString(), e);
throw new RuntimeException(e);
}
break;
case OFFLINED_REGION_B:
if (services != null)
services.addToOnlineRegions(this.region_b);
break;
case STARTED_MERGED_REGION_CREATION:
this.region_a.getRegionFileSystem().cleanupMergedRegion(
this.mergedRegionInfo);
break;
case PONR:
// We got to the point-of-no-return so we need to just abort. Return
// immediately. Do not clean up created merged regions.
return false;
// Informational states only
case STARTED:
case PREPARED:
case COMPLETED:
break;
default:
throw new RuntimeException("Unhandled journal entry: " + je);
}
}
// Coprocessor callback
if (rsCoprocessorHost != null) {
rsCoprocessorHost.postRollBackMerge(region_a, region_b, user);
}
return result;
}
@Override
public HRegionInfo getMergedRegionInfo() {
return this.mergedRegionInfo;
}
@VisibleForTesting
Path getMergesDir() {
return this.mergesdir;
}
/**
* Checks if the given region has merge qualifier in hbase:meta
* @param services
* @param regionName name of specified region
* @return true if the given region has merge qualifier in META.(It will be
* cleaned by CatalogJanitor)
* @throws IOException
*/
@VisibleForTesting
boolean hasMergeQualifierInMeta(final RegionServerServices services, final byte[] regionName)
throws IOException {
if (services == null) return false;
// Get merge regions if it is a merged region and already has merge
// qualifier
Pair<HRegionInfo, HRegionInfo> mergeRegions = MetaTableAccessor
.getRegionsFromMergeQualifier(services.getConnection(), regionName);
if (mergeRegions != null &&
(mergeRegions.getFirst() != null || mergeRegions.getSecond() != null)) {
// It has merge qualifier
return true;
}
return false;
}
@Override
public List<JournalEntry> getJournal() {
return journal;
}
@Override
public RegionMergeTransaction registerTransactionListener(TransactionListener listener) {
listeners.add(listener);
return this;
}
@Override
public Server getServer() {
return server;
}
@Override
public RegionServerServices getRegionServerServices() {
return rsServices;
}
}

View File

@ -1,348 +0,0 @@
/**
*
* 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.util;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Random;
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.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.ipc.RemoteException;
/**
* A non-instantiable class that has a static method capable of compacting
* a table by merging adjacent regions.
*/
@InterfaceAudience.Private
class HMerge {
// TODO: Where is this class used? How does it relate to Merge in same package?
private static final Log LOG = LogFactory.getLog(HMerge.class);
static final Random rand = new Random();
/*
* Not instantiable
*/
private HMerge() {
super();
}
/**
* Scans the table and merges two adjacent regions if they are small. This
* only happens when a lot of rows are deleted.
*
* When merging the hbase:meta region, the HBase instance must be offline.
* When merging a normal table, the HBase instance must be online, but the
* table must be disabled.
*
* @param conf - configuration object for HBase
* @param fs - FileSystem where regions reside
* @param tableName - Table to be compacted
* @throws IOException
*/
public static void merge(Configuration conf, FileSystem fs,
final TableName tableName)
throws IOException {
merge(conf, fs, tableName, true);
}
/**
* Scans the table and merges two adjacent regions if they are small. This
* only happens when a lot of rows are deleted.
*
* When merging the hbase:meta region, the HBase instance must be offline.
* When merging a normal table, the HBase instance must be online, but the
* table must be disabled.
*
* @param conf - configuration object for HBase
* @param fs - FileSystem where regions reside
* @param tableName - Table to be compacted
* @param testMasterRunning True if we are to verify master is down before
* running merge
* @throws IOException
*/
public static void merge(Configuration conf, FileSystem fs,
final TableName tableName, final boolean testMasterRunning)
throws IOException {
boolean masterIsRunning = false;
ClusterConnection hConnection = null;
if (testMasterRunning) {
try {
hConnection = (ClusterConnection) ConnectionFactory.createConnection(conf);
masterIsRunning = hConnection.isMasterRunning();
} finally {
if (hConnection != null) {
hConnection.close();
}
}
}
if (tableName.equals(TableName.META_TABLE_NAME)) {
if (masterIsRunning) {
throw new IllegalStateException(
"Can not compact hbase:meta table if instance is on-line");
}
// TODO reenable new OfflineMerger(conf, fs).process();
} else {
if(!masterIsRunning) {
throw new IllegalStateException(
"HBase instance must be running to merge a normal table");
}
try (Connection conn = ConnectionFactory.createConnection(conf);
Admin admin = conn.getAdmin()) {
if (!admin.isTableDisabled(tableName)) {
throw new TableNotDisabledException(tableName);
}
}
new OnlineMerger(conf, fs, tableName).process();
}
}
private static abstract class Merger {
protected final Configuration conf;
protected final FileSystem fs;
protected final Path rootDir;
protected final HTableDescriptor htd;
protected final WALFactory walFactory;
private final long maxFilesize;
protected Merger(Configuration conf, FileSystem fs, final TableName tableName)
throws IOException {
this.conf = conf;
this.fs = fs;
this.maxFilesize = conf.getLong(HConstants.HREGION_MAX_FILESIZE,
HConstants.DEFAULT_MAX_FILE_SIZE);
this.rootDir = FSUtils.getRootDir(conf);
Path tabledir = FSUtils.getTableDir(this.rootDir, tableName);
this.htd = FSTableDescriptors.getTableDescriptorFromFs(this.fs, tabledir);
String logname = "merge_" + System.currentTimeMillis() + HConstants.HREGION_LOGDIR_NAME;
final Configuration walConf = new Configuration(conf);
FSUtils.setRootDir(walConf, tabledir);
this.walFactory = new WALFactory(walConf, null, logname);
}
void process() throws IOException {
try {
for (HRegionInfo[] regionsToMerge = next();
regionsToMerge != null;
regionsToMerge = next()) {
if (!merge(regionsToMerge)) {
return;
}
}
} finally {
try {
walFactory.close();
} catch(IOException e) {
LOG.error(e);
}
}
}
protected boolean merge(final HRegionInfo[] info) throws IOException {
if (info.length < 2) {
LOG.info("only one region - nothing to merge");
return false;
}
HRegion currentRegion = null;
long currentSize = 0;
HRegion nextRegion = null;
long nextSize = 0;
for (int i = 0; i < info.length - 1; i++) {
if (currentRegion == null) {
currentRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i], this.htd,
walFactory.getWAL(info[i].getEncodedNameAsBytes(),
info[i].getTable().getNamespace()));
currentSize = currentRegion.getLargestHStoreSize();
}
nextRegion = HRegion.openHRegion(conf, fs, this.rootDir, info[i + 1], this.htd,
walFactory.getWAL(info[i + 1].getEncodedNameAsBytes(),
info[i + 1].getTable().getNamespace()));
nextSize = nextRegion.getLargestHStoreSize();
if ((currentSize + nextSize) <= (maxFilesize / 2)) {
// We merge two adjacent regions if their total size is less than
// one half of the desired maximum size
LOG.info("Merging regions " + currentRegion.getRegionInfo().getRegionNameAsString() +
" and " + nextRegion.getRegionInfo().getRegionNameAsString());
HRegion mergedRegion =
HRegion.mergeAdjacent(currentRegion, nextRegion);
updateMeta(currentRegion.getRegionInfo().getRegionName(),
nextRegion.getRegionInfo().getRegionName(), mergedRegion);
break;
}
LOG.info("not merging regions " +
Bytes.toStringBinary(currentRegion.getRegionInfo().getRegionName()) +
" and " + Bytes.toStringBinary(nextRegion.getRegionInfo().getRegionName()));
currentRegion.close();
currentRegion = nextRegion;
currentSize = nextSize;
}
if(currentRegion != null) {
currentRegion.close();
}
return true;
}
protected abstract HRegionInfo[] next() throws IOException;
protected abstract void updateMeta(final byte [] oldRegion1,
final byte [] oldRegion2, HRegion newRegion)
throws IOException;
}
/** Instantiated to compact a normal user table */
private static class OnlineMerger extends Merger {
private final TableName tableName;
private final Table table;
private final ResultScanner metaScanner;
private HRegionInfo latestRegion;
OnlineMerger(Configuration conf, FileSystem fs,
final TableName tableName)
throws IOException {
super(conf, fs, tableName);
this.tableName = tableName;
Connection connection = ConnectionFactory.createConnection(conf);
this.table = connection.getTable(TableName.META_TABLE_NAME);
this.metaScanner = table.getScanner(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
this.latestRegion = null;
}
private HRegionInfo nextRegion() throws IOException {
try {
Result results = getMetaRow();
if (results == null) {
return null;
}
HRegionInfo region = MetaTableAccessor.getHRegionInfo(results);
if (region == null) {
throw new NoSuchElementException("meta region entry missing " +
Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
Bytes.toString(HConstants.REGIONINFO_QUALIFIER));
}
if (!region.getTable().equals(this.tableName)) {
return null;
}
return region;
} catch (IOException e) {
e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
LOG.error("meta scanner error", e);
metaScanner.close();
throw e;
}
}
/*
* Check current row has a HRegionInfo. Skip to next row if HRI is empty.
* @return A Map of the row content else null if we are off the end.
* @throws IOException
*/
private Result getMetaRow() throws IOException {
Result currentRow = metaScanner.next();
boolean foundResult = false;
while (currentRow != null) {
LOG.info("Row: <" + Bytes.toStringBinary(currentRow.getRow()) + ">");
byte[] regionInfoValue = currentRow.getValue(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
if (regionInfoValue == null || regionInfoValue.length == 0) {
currentRow = metaScanner.next();
continue;
}
HRegionInfo region = MetaTableAccessor.getHRegionInfo(currentRow);
if (!region.getTable().equals(this.tableName)) {
currentRow = metaScanner.next();
continue;
}
foundResult = true;
break;
}
return foundResult ? currentRow : null;
}
@Override
protected HRegionInfo[] next() throws IOException {
List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
if(latestRegion == null) {
latestRegion = nextRegion();
}
if(latestRegion != null) {
regions.add(latestRegion);
}
latestRegion = nextRegion();
if(latestRegion != null) {
regions.add(latestRegion);
}
return regions.toArray(new HRegionInfo[regions.size()]);
}
@Override
protected void updateMeta(final byte [] oldRegion1,
final byte [] oldRegion2,
HRegion newRegion)
throws IOException {
byte[][] regionsToDelete = {oldRegion1, oldRegion2};
for (int r = 0; r < regionsToDelete.length; r++) {
if(Bytes.equals(regionsToDelete[r], latestRegion.getRegionName())) {
latestRegion = null;
}
Delete delete = new Delete(regionsToDelete[r]);
table.delete(delete);
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: " + Bytes.toStringBinary(regionsToDelete[r]));
}
}
newRegion.getRegionInfo().setOffline(true);
MetaTableAccessor.addRegionToMeta(table, newRegion.getRegionInfo());
if(LOG.isDebugEnabled()) {
LOG.debug("updated columns in row: "
+ Bytes.toStringBinary(newRegion.getRegionInfo().getRegionName()));
}
}
}
}

View File

@ -1,264 +0,0 @@
/**
*
* 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.util;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import com.google.common.base.Preconditions;
/**
* Utility that can merge any two regions in the same table: adjacent,
* overlapping or disjoint.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
public class Merge extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(Merge.class);
private Path rootdir;
private volatile MetaUtils utils;
private TableName tableName; // Name of table
private volatile byte [] region1; // Name of region 1
private volatile byte [] region2; // Name of region 2
private volatile HRegionInfo mergeInfo = null;
@Override
public int run(String[] args) throws Exception {
if (parseArgs(args) != 0) {
return -1;
}
// Verify file system is up.
FileSystem fs = FileSystem.get(getConf()); // get DFS handle
LOG.info("Verifying that file system is available...");
try {
FSUtils.checkFileSystemAvailable(fs);
} catch (IOException e) {
LOG.fatal("File system is not available", e);
return -1;
}
// Verify HBase is down
LOG.info("Verifying that HBase is not running...");
try {
HBaseAdmin.available(getConf());
LOG.fatal("HBase cluster must be off-line, and is not. Aborting.");
return -1;
} catch (ZooKeeperConnectionException zkce) {
// If no zk, presume no master.
}
// Initialize MetaUtils and and get the root of the HBase installation
this.utils = new MetaUtils(getConf());
this.rootdir = FSUtils.getRootDir(getConf());
try {
mergeTwoRegions();
return 0;
} catch (IOException e) {
LOG.fatal("Merge failed", e);
return -1;
} finally {
if (this.utils != null) {
this.utils.shutdown();
}
}
}
/** @return HRegionInfo for merge result */
HRegionInfo getMergedHRegionInfo() {
return this.mergeInfo;
}
/*
* Merges two regions from a user table.
*/
private void mergeTwoRegions() throws IOException {
LOG.info("Merging regions " + Bytes.toStringBinary(this.region1) + " and " +
Bytes.toStringBinary(this.region2) + " in table " + this.tableName);
HRegion meta = this.utils.getMetaRegion();
Get get = new Get(region1);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result result1 = meta.get(get);
Preconditions.checkState(!result1.isEmpty(),
"First region cells can not be null");
HRegionInfo info1 = MetaTableAccessor.getHRegionInfo(result1);
if (info1 == null) {
throw new NullPointerException("info1 is null using key " +
Bytes.toStringBinary(region1) + " in " + meta);
}
get = new Get(region2);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result result2 = meta.get(get);
Preconditions.checkState(!result2.isEmpty(),
"Second region cells can not be null");
HRegionInfo info2 = MetaTableAccessor.getHRegionInfo(result2);
if (info2 == null) {
throw new NullPointerException("info2 is null using key " + meta);
}
HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(FileSystem.get(getConf()),
this.rootdir, this.tableName);
HRegion merged = merge(htd, meta, info1, info2);
LOG.info("Adding " + merged.getRegionInfo() + " to " +
meta.getRegionInfo());
HRegion.addRegionToMETA(meta, merged);
merged.close();
}
/*
* Actually merge two regions and update their info in the meta region(s)
* Returns HRegion object for newly merged region
*/
private HRegion merge(final HTableDescriptor htd, HRegion meta,
HRegionInfo info1, HRegionInfo info2)
throws IOException {
if (info1 == null) {
throw new IOException("Could not find " + Bytes.toStringBinary(region1) + " in " +
Bytes.toStringBinary(meta.getRegionInfo().getRegionName()));
}
if (info2 == null) {
throw new IOException("Could not find " + Bytes.toStringBinary(region2) + " in " +
Bytes.toStringBinary(meta.getRegionInfo().getRegionName()));
}
HRegion merged = null;
HRegion r1 = HRegion.openHRegion(info1, htd, utils.getLog(info1), getConf());
try {
HRegion r2 = HRegion.openHRegion(info2, htd, utils.getLog(info2), getConf());
try {
merged = HRegion.merge(r1, r2);
} finally {
if (!r2.isClosed()) {
r2.close();
}
}
} finally {
if (!r1.isClosed()) {
r1.close();
}
}
// Remove the old regions from meta.
// HRegion.merge has already deleted their files
removeRegionFromMeta(meta, info1);
removeRegionFromMeta(meta, info2);
this.mergeInfo = merged.getRegionInfo();
return merged;
}
/*
* Removes a region's meta information from the passed <code>meta</code>
* region.
*
* @param meta hbase:meta HRegion to be updated
* @param regioninfo HRegionInfo of region to remove from <code>meta</code>
*
* @throws IOException
*/
private void removeRegionFromMeta(HRegion meta, HRegionInfo regioninfo)
throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Removing region: " + regioninfo + " from " + meta);
}
Delete delete = new Delete(regioninfo.getRegionName(),
System.currentTimeMillis());
meta.delete(delete);
}
/**
* Parse given arguments and assign table name and regions names.
* (generic args are handled by ToolRunner.)
*
* @param args the arguments to parse
*
* @throws IOException
*/
private int parseArgs(String[] args) throws IOException {
if (args.length != 3) {
usage();
return -1;
}
tableName = TableName.valueOf(args[0]);
region1 = Bytes.toBytesBinary(args[1]);
region2 = Bytes.toBytesBinary(args[2]);
int status = 0;
if (notInTable(tableName, region1) || notInTable(tableName, region2)) {
status = -1;
} else if (Bytes.equals(region1, region2)) {
LOG.error("Can't merge a region with itself");
status = -1;
}
return status;
}
private boolean notInTable(final TableName tn, final byte [] rn) {
if (WritableComparator.compareBytes(tn.getName(), 0, tn.getName().length,
rn, 0, tn.getName().length) != 0) {
LOG.error("Region " + Bytes.toStringBinary(rn) + " does not belong to table " +
tn);
return true;
}
return false;
}
private void usage() {
System.err
.println("For hadoop 0.21+, Usage: hbase org.apache.hadoop.hbase.util.Merge "
+ "[-Dfs.defaultFS=hdfs://nn:port] <table-name> <region-1> <region-2>\n");
}
public static void main(String[] args) {
int status;
try {
status = ToolRunner.run(HBaseConfiguration.create(), new Merge(), args);
} catch (Exception e) {
LOG.error("exiting due to error", e);
status = -1;
}
System.exit(status);
}
}

View File

@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
@ -1189,15 +1189,17 @@ public class TestAdmin1 {
assertTrue(gotException);
// Try going to the master directly (that will skip the check in admin)
try {
DispatchMergingRegionsRequest request = RequestConverter
.buildDispatchMergingRegionsRequest(
regions.get(1).getFirst().getEncodedNameAsBytes(),
regions.get(2).getFirst().getEncodedNameAsBytes(),
byte[][] nameofRegionsToMerge = new byte[2][];
nameofRegionsToMerge[0] = regions.get(1).getFirst().getEncodedNameAsBytes();
nameofRegionsToMerge[1] = regions.get(2).getFirst().getEncodedNameAsBytes();
MergeTableRegionsRequest request = RequestConverter
.buildMergeTableRegionsRequest(
nameofRegionsToMerge,
true,
HConstants.NO_NONCE,
HConstants.NO_NONCE);
HConstants.NO_NONCE);
((ClusterConnection) TEST_UTIL.getAdmin().getConnection()).getMaster()
.dispatchMergingRegions(null, request);
.mergeTableRegions(null, request);
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException m) {
Throwable t = m.getCause();
do {
@ -1209,40 +1211,6 @@ public class TestAdmin1 {
} while (t != null);
}
assertTrue(gotException);
gotException = false;
// Try going to the regionservers directly
// first move the region to the same regionserver
if (!regions.get(2).getSecond().equals(regions.get(1).getSecond())) {
moveRegionAndWait(regions.get(2).getFirst(), regions.get(1).getSecond());
}
try {
AdminService.BlockingInterface admin = ((ClusterConnection) TEST_UTIL.getAdmin()
.getConnection()).getAdmin(regions.get(1).getSecond());
ProtobufUtil.mergeRegions(null, admin, regions.get(1).getFirst(), regions.get(2).getFirst(),
true, null);
} catch (MergeRegionException mm) {
gotException = true;
}
assertTrue(gotException);
}
private void moveRegionAndWait(HRegionInfo destRegion, ServerName destServer)
throws InterruptedException, MasterNotRunningException,
ZooKeeperConnectionException, IOException {
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
TEST_UTIL.getAdmin().move(
destRegion.getEncodedNameAsBytes(),
Bytes.toBytes(destServer.getServerName()));
while (true) {
ServerName serverName = master.getAssignmentManager()
.getRegionStates().getRegionServerOfRegion(destRegion);
if (serverName != null && serverName.equals(destServer)) {
TEST_UTIL.assertRegionOnServer(
destRegion, serverName, 200);
break;
}
Thread.sleep(10);
}
}
@Test (expected=IllegalArgumentException.class, timeout=300000)

View File

@ -1,223 +0,0 @@
/*
* 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.coprocessor;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
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.Coprocessor;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionMergeTransactionFactory;
import org.apache.hadoop.hbase.regionserver.RegionMergeTransactionImpl;
import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests invocation of the {@link org.apache.hadoop.hbase.coprocessor.RegionServerObserver}
* interface hooks at all appropriate times during normal HMaster operations.
*/
@Category({CoprocessorTests.class, MediumTests.class})
public class TestRegionServerObserver {
private static final Log LOG = LogFactory.getLog(TestRegionServerObserver.class);
/**
* Test verifies the hooks in regions merge.
* @throws Exception
*/
@Ignore
@Test
public void testCoprocessorHooksInRegionsMerge() throws Exception {
final int NUM_MASTERS = 1;
final int NUM_RS = 1;
final String TABLENAME = "testRegionServerObserver";
final String TABLENAME2 = "testRegionServerObserver_2";
final byte[] FAM = Bytes.toBytes("fam");
// Create config to use for this cluster
Configuration conf = HBaseConfiguration.create();
conf.setClass("hbase.coprocessor.regionserver.classes", CPRegionServerObserver.class,
RegionServerObserver.class);
// Start the cluster
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
Admin admin = TEST_UTIL.getHBaseAdmin();
try {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HRegionServer regionServer = cluster.getRegionServer(0);
RegionServerCoprocessorHost cpHost = regionServer.getRegionServerCoprocessorHost();
Coprocessor coprocessor = cpHost.findCoprocessor(CPRegionServerObserver.class.getName());
CPRegionServerObserver regionServerObserver = (CPRegionServerObserver) coprocessor;
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(TABLENAME));
desc.addFamily(new HColumnDescriptor(FAM));
admin.createTable(desc, new byte[][] { Bytes.toBytes("row") });
desc = new HTableDescriptor(TableName.valueOf(TABLENAME2));
desc.addFamily(new HColumnDescriptor(FAM));
admin.createTable(desc, new byte[][] { Bytes.toBytes("row") });
assertFalse(regionServerObserver.wasRegionMergeCalled());
List<Region> regions = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME));
admin.mergeRegionsAsync(
regions.get(0).getRegionInfo().getEncodedNameAsBytes(),
regions.get(1).getRegionInfo().getEncodedNameAsBytes(),
true);
int regionsCount = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)).size();
while (regionsCount != 1) {
regionsCount = regionServer.getOnlineRegions(TableName.valueOf(TABLENAME)).size();
Thread.sleep(1000);
}
assertTrue(regionServerObserver.wasRegionMergeCalled());
assertTrue(regionServerObserver.wasPreMergeCommit());
assertTrue(regionServerObserver.wasPostMergeCommit());
assertEquals(regionsCount, 1);
assertEquals(regionServer.getOnlineRegions(TableName.valueOf(TABLENAME2)).size(), 1);
} finally {
if (admin != null) admin.close();
TEST_UTIL.shutdownMiniCluster();
}
}
public static class CPRegionServerObserver extends BaseRegionServerObserver {
private RegionMergeTransactionImpl rmt = null;
private HRegion mergedRegion = null;
private boolean preMergeCalled;
private boolean preMergeBeforePONRCalled;
private boolean preMergeAfterPONRCalled;
private boolean preRollBackMergeCalled;
private boolean postRollBackMergeCalled;
private boolean postMergeCalled;
public void resetStates() {
preMergeCalled = false;
preMergeBeforePONRCalled = false;
preMergeAfterPONRCalled = false;
preRollBackMergeCalled = false;
postRollBackMergeCalled = false;
postMergeCalled = false;
}
@Override
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, Region regionA,
Region regionB) throws IOException {
preMergeCalled = true;
}
@Override
public void preMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
Region regionA, Region regionB, List<Mutation> metaEntries) throws IOException {
preMergeBeforePONRCalled = true;
RegionServerCoprocessorEnvironment environment = ctx.getEnvironment();
HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
List<Region> onlineRegions =
rs.getOnlineRegions(TableName.valueOf("testRegionServerObserver_2"));
rmt = (RegionMergeTransactionImpl) new RegionMergeTransactionFactory(rs.getConfiguration())
.create(onlineRegions.get(0), onlineRegions.get(1), true);
if (!rmt.prepare(rs)) {
LOG.error("Prepare for the region merge of table "
+ onlineRegions.get(0).getTableDesc().getNameAsString()
+ " failed. So returning null. ");
ctx.bypass();
return;
}
mergedRegion = rmt.stepsBeforePONR(rs, rs, false);
rmt.prepareMutationsForMerge(mergedRegion.getRegionInfo(), regionA.getRegionInfo(),
regionB.getRegionInfo(), rs.getServerName(), metaEntries);
MetaTableAccessor.mutateMetaTable(rs.getConnection(), metaEntries);
}
@Override
public void postMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
Region regionA, Region regionB, Region mr) throws IOException {
preMergeAfterPONRCalled = true;
RegionServerCoprocessorEnvironment environment = ctx.getEnvironment();
HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
rmt.stepsAfterPONR(rs, rs, this.mergedRegion, null);
}
@Override
public void preRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
Region regionA, Region regionB) throws IOException {
preRollBackMergeCalled = true;
}
@Override
public void postRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
Region regionA, Region regionB) throws IOException {
postRollBackMergeCalled = true;
}
@Override
public void postMerge(ObserverContext<RegionServerCoprocessorEnvironment> c, Region regionA,
Region regionB, Region mergedRegion) throws IOException {
postMergeCalled = true;
}
public boolean wasPreMergeCalled() {
return this.preMergeCalled;
}
public boolean wasPostMergeCalled() {
return this.postMergeCalled;
}
public boolean wasPreMergeCommit() {
return this.preMergeBeforePONRCalled;
}
public boolean wasPostMergeCommit() {
return this.preMergeAfterPONRCalled;
}
public boolean wasPreRollBackMerge() {
return this.preRollBackMergeCalled;
}
public boolean wasPostRollBackMerge() {
return this.postRollBackMergeCalled;
}
public boolean wasRegionMergeCalled() {
return this.preMergeCalled && this.postMergeCalled;
}
}
}

View File

@ -319,16 +319,6 @@ public class MockNoopMasterServices implements MasterServices, Server {
return null;
}
@Override
public long dispatchMergingRegions(
final HRegionInfo region_a,
final HRegionInfo region_b,
final boolean forcible,
final long nonceGroup,
final long nonce) throws IOException {
return -1;
}
@Override
public boolean isActiveMaster() {
return true;

View File

@ -68,8 +68,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerIn
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
@ -530,13 +528,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
return null;
}
@Override
public MergeRegionsResponse mergeRegions(RpcController controller,
MergeRegionsRequest request) throws ServiceException {
// TODO Auto-generated method stub
return null;
}
@Override
public CompactRegionResponse compactRegion(RpcController controller,
CompactRegionRequest request) throws ServiceException {

View File

@ -1,264 +0,0 @@
/**
* 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.util.List;
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.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
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;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@Category({MasterTests.class, MediumTests.class})
public class TestDispatchMergingRegionsProcedure {
private static final Log LOG = LogFactory.getLog(TestDispatchMergingRegionsProcedure.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
final static Configuration conf = UTIL.getConfiguration();
private static Admin admin;
private static void setupConf(Configuration conf) {
// Reduce the maximum attempts to speed up the test
conf.setInt("hbase.assignment.maximum.attempts", 3);
conf.setInt("hbase.master.maximum.ping.server.attempts", 3);
conf.setInt("hbase.master.ping.server.retry.sleep.interval", 1);
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@BeforeClass
public static void setupCluster() throws Exception {
setupConf(conf);
UTIL.startMiniCluster(1);
admin = UTIL.getHBaseAdmin();
}
@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 {
resetProcExecutorTestingKillFlag();
// 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);
resetProcExecutorTestingKillFlag();
}
@After
public void tearDown() throws Exception {
resetProcExecutorTestingKillFlag();
for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
}
private void resetProcExecutorTestingKillFlag() {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
assertTrue("expected executor to be running", procExec.isRunning());
}
/**
* This tests two region merges
*/
@Test(timeout=60000)
public void testMergeTwoRegions() throws Exception {
final TableName tableName = TableName.valueOf("testMergeTwoRegions");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
List<HRegionInfo> tableRegions = createTable(tableName, 3);
HRegionInfo[] regionsToMerge = new HRegionInfo[2];
regionsToMerge[0] = tableRegions.get(0);
regionsToMerge[1] = tableRegions.get(1);
final int initCompletedTaskCount = countOfCompletedMergeTaskCount();
long procId = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
procExec.getEnvironment(), tableName, regionsToMerge, true));
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
assertRegionCount(tableName, 2, 1, initCompletedTaskCount);
}
/**
* This tests two concurrent region merges
*/
@Test(timeout=60000)
public void testMergeRegionsConcurrently() throws Exception {
final TableName tableName = TableName.valueOf("testMergeRegionsConcurrently");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
List<HRegionInfo> tableRegions = createTable(tableName, 4);
HRegionInfo[] regionsToMerge1 = new HRegionInfo[2];
HRegionInfo[] regionsToMerge2 = new HRegionInfo[2];
regionsToMerge1[0] = tableRegions.get(0);
regionsToMerge1[1] = tableRegions.get(1);
regionsToMerge2[0] = tableRegions.get(2);
regionsToMerge2[1] = tableRegions.get(3);
final int initCompletedTaskCount = countOfCompletedMergeTaskCount();
long procId1 = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
procExec.getEnvironment(), tableName, regionsToMerge1, true));
long procId2 = procExec.submitProcedure(new DispatchMergingRegionsProcedure(
procExec.getEnvironment(), tableName, regionsToMerge2, true));
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
assertRegionCount(tableName, 2, 2, initCompletedTaskCount);
}
private void waitForCompletedMergeTask(int expectedTaskCount, int initCompletedTaskCount)
throws IOException, InterruptedException {
while (true) {
long currentCompletedTaskCount = countOfCompletedMergeTaskCount() - initCompletedTaskCount;
if (currentCompletedTaskCount == expectedTaskCount) {
return;
}
LOG.info("There are " + (expectedTaskCount - currentCompletedTaskCount) +
" merge requests are not completed, wait 100 ms");
TimeUnit.MILLISECONDS.sleep(100);
}
}
private static int countOfCompletedMergeTaskCount() {
int completedTaskCount = 0;
for (RegionServerThread server : UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
completedTaskCount += server.getRegionServer().getCompactSplitThread().getCompletedMergeTaskCount();
}
return completedTaskCount;
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
List<HRegionInfo> tableRegions = createTable(tableName, 3);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
HRegionInfo[] regionsToMerge = new HRegionInfo[2];
regionsToMerge[0] = tableRegions.get(0);
regionsToMerge[1] = tableRegions.get(1);
final int initCompletedTaskCount = countOfCompletedMergeTaskCount();
long procId = procExec.submitProcedure(
new DispatchMergingRegionsProcedure(
procExec.getEnvironment(), tableName, regionsToMerge, true));
// Restart the executor and execute the step twice
int numberOfSteps = DispatchMergingRegionsState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
assertRegionCount(tableName, 2, 1, initCompletedTaskCount);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
List<HRegionInfo> tableRegions = createTable(tableName, 3);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
HRegionInfo[] regionsToMerge = new HRegionInfo[2];
regionsToMerge[0] = tableRegions.get(0);
regionsToMerge[1] = tableRegions.get(1);
long procId = procExec.submitProcedure(
new DispatchMergingRegionsProcedure(
procExec.getEnvironment(), tableName, regionsToMerge, true));
int numberOfSteps = DispatchMergingRegionsState.values().length - 3;
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
}
private List<HRegionInfo> createTable(final TableName tableName, final int nregions)
throws Exception {
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(FAMILY));
byte[][] splitRows = new byte[nregions - 1][];
for (int i = 0; i < splitRows.length; ++i) {
splitRows[i] = Bytes.toBytes(String.format("%d", i));
}
admin.createTable(desc, splitRows);
return assertRegionCount(tableName, nregions);
}
public List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions)
throws Exception {
UTIL.waitUntilNoRegionsInTransition();
List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
assertEquals(nregions, tableRegions.size());
return tableRegions;
}
public List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions,
int expectedTaskCount, int initCompletedTaskCount) throws Exception {
waitForCompletedMergeTask(expectedTaskCount, initCompletedTaskCount);
return assertRegionCount(tableName, nregions);
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
}

View File

@ -2646,55 +2646,6 @@ public class TestHRegion {
}
}
// ////////////////////////////////////////////////////////////////////////////
// Merge test
// ////////////////////////////////////////////////////////////////////////////
@Test
public void testMerge() throws IOException {
byte[][] families = { fam1, fam2, fam3 };
Configuration hc = initSplit();
// Setting up region
this.region = initHRegion(tableName, method, hc, families);
try {
LOG.info("" + HBaseTestCase.addContent(region, fam3));
region.flush(true);
region.compactStores();
byte[] splitRow = region.checkSplit();
assertNotNull(splitRow);
LOG.info("SplitRow: " + Bytes.toString(splitRow));
HRegion[] subregions = splitRegion(region, splitRow);
try {
// Need to open the regions.
for (int i = 0; i < subregions.length; i++) {
HRegion.openHRegion(subregions[i], null);
subregions[i].compactStores();
}
Path oldRegionPath = region.getRegionFileSystem().getRegionDir();
Path oldRegion1 = subregions[0].getRegionFileSystem().getRegionDir();
Path oldRegion2 = subregions[1].getRegionFileSystem().getRegionDir();
long startTime = System.currentTimeMillis();
region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
LOG.info("Merge regions elapsed time: "
+ ((System.currentTimeMillis() - startTime) / 1000.0));
FILESYSTEM.delete(oldRegion1, true);
FILESYSTEM.delete(oldRegion2, true);
FILESYSTEM.delete(oldRegionPath, true);
LOG.info("splitAndMerge completed.");
} finally {
for (int i = 0; i < subregions.length; i++) {
try {
HBaseTestingUtility.closeRegionAndWAL(subregions[i]);
} catch (IOException e) {
// Ignore.
}
}
}
} finally {
HBaseTestingUtility.closeRegionAndWAL(this.region);
this.region = null;
}
}
/**
* @param parent
* Region to split.

View File

@ -1,485 +0,0 @@
/**
* Copyright The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.*;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.common.collect.ImmutableList;
/**
* Test the {@link RegionMergeTransactionImpl} class against two HRegions (as
* opposed to running cluster).
*/
@Category({RegionServerTests.class, SmallTests.class})
public class TestRegionMergeTransaction {
private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final Path testdir = TEST_UTIL.getDataTestDir(this.getClass()
.getName());
private HRegion region_a;
private HRegion region_b;
private HRegion region_c;
private WALFactory wals;
private FileSystem fs;
// Start rows of region_a,region_b,region_c
private static final byte[] STARTROW_A = new byte[] { 'a', 'a', 'a' };
private static final byte[] STARTROW_B = new byte[] { 'g', 'g', 'g' };
private static final byte[] STARTROW_C = new byte[] { 'w', 'w', 'w' };
private static final byte[] ENDROW = new byte[] { '{', '{', '{' };
private static final byte[] CF = HConstants.CATALOG_FAMILY;
@Before
public void setup() throws IOException {
this.fs = FileSystem.get(TEST_UTIL.getConfiguration());
this.fs.delete(this.testdir, true);
final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration());
FSUtils.setRootDir(walConf, this.testdir);
this.wals = new WALFactory(walConf, null, TestRegionMergeTransaction.class.getName());
this.region_a = createRegion(this.testdir, this.wals, STARTROW_A, STARTROW_B);
this.region_b = createRegion(this.testdir, this.wals, STARTROW_B, STARTROW_C);
this.region_c = createRegion(this.testdir, this.wals, STARTROW_C, ENDROW);
assert region_a != null && region_b != null && region_c != null;
TEST_UTIL.getConfiguration().setBoolean("hbase.testing.nocluster", true);
}
@After
public void teardown() throws IOException {
for (HRegion region : new HRegion[] { region_a, region_b, region_c }) {
if (region != null && !region.isClosed()) region.close();
if (this.fs.exists(region.getRegionFileSystem().getRegionDir())
&& !this.fs.delete(region.getRegionFileSystem().getRegionDir(), true)) {
throw new IOException("Failed deleting of "
+ region.getRegionFileSystem().getRegionDir());
}
}
if (this.wals != null) {
this.wals.close();
}
this.fs.delete(this.testdir, true);
}
/**
* Test straight prepare works. Tries to merge on {@link #region_a} and
* {@link #region_b}
* @throws IOException
*/
@Test
public void testPrepare() throws IOException {
prepareOnGoodRegions();
}
private RegionMergeTransactionImpl prepareOnGoodRegions() throws IOException {
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, region_b,
false);
RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
region_a.getRegionInfo().getRegionName());
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
region_b.getRegionInfo().getRegionName());
assertTrue(spyMT.prepare(null));
return spyMT;
}
/**
* Test merging the same region
*/
@Test
public void testPrepareWithSameRegion() throws IOException {
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a,
this.region_a, true);
assertFalse("should not merge the same region even if it is forcible ",
mt.prepare(null));
}
/**
* Test merging two not adjacent regions under a common merge
*/
@Test
public void testPrepareWithRegionsNotAdjacent() throws IOException {
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a,
this.region_c, false);
assertFalse("should not merge two regions if they are adjacent except it is forcible",
mt.prepare(null));
}
/**
* Test merging two not adjacent regions under a compulsory merge
*/
@Test
public void testPrepareWithRegionsNotAdjacentUnderCompulsory()
throws IOException {
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, region_c,
true);
RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
region_a.getRegionInfo().getRegionName());
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
region_c.getRegionInfo().getRegionName());
assertTrue("Since focible is true, should merge two regions even if they are not adjacent",
spyMT.prepare(null));
}
/**
* Pass a reference store
*/
@Test
public void testPrepareWithRegionsWithReference() throws IOException {
HStore storeMock = Mockito.mock(HStore.class);
when(storeMock.hasReferences()).thenReturn(true);
when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf"));
when(storeMock.getSizeToFlush()).thenReturn(new MemstoreSize());
when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
this.region_a.stores.put(Bytes.toBytes(""), storeMock);
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a,
this.region_b, false);
assertFalse(
"a region should not be mergeable if it has instances of store file references",
mt.prepare(null));
}
@Test
public void testPrepareWithClosedRegion() throws IOException {
this.region_a.close();
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a,
this.region_b, false);
assertFalse(mt.prepare(null));
}
/**
* Test merging regions which are merged regions and has reference in hbase:meta all
* the same
*/
@Test
public void testPrepareWithRegionsWithMergeReference() throws IOException {
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, region_b,
false);
RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
doReturn(true).when(spyMT).hasMergeQualifierInMeta(null,
region_a.getRegionInfo().getRegionName());
doReturn(true).when(spyMT).hasMergeQualifierInMeta(null,
region_b.getRegionInfo().getRegionName());
assertFalse(spyMT.prepare(null));
}
/**
* Test RegionMergeTransactionListener
*/
@Test public void testRegionMergeTransactionListener() throws IOException {
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, region_b,
false);
RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
region_a.getRegionInfo().getRegionName());
doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
region_b.getRegionInfo().getRegionName());
RegionMergeTransaction.TransactionListener listener =
Mockito.mock(RegionMergeTransaction.TransactionListener.class);
mt.registerTransactionListener(listener);
mt.prepare(null);
TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
TEST_UTIL.getConfiguration());
Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
mt.execute(mockServer, null);
verify(listener).transition(mt,
RegionMergeTransaction.RegionMergeTransactionPhase.STARTED,
RegionMergeTransaction.RegionMergeTransactionPhase.PREPARED);
verify(listener, times(10)).transition(any(RegionMergeTransaction.class),
any(RegionMergeTransaction.RegionMergeTransactionPhase.class),
any(RegionMergeTransaction.RegionMergeTransactionPhase.class));
verifyNoMoreInteractions(listener);
}
@Test
public void testWholesomeMerge() throws IOException, InterruptedException {
final int rowCountOfRegionA = loadRegion(this.region_a, CF, true);
final int rowCountOfRegionB = loadRegion(this.region_b, CF, true);
assertTrue(rowCountOfRegionA > 0 && rowCountOfRegionB > 0);
assertEquals(rowCountOfRegionA, TEST_UTIL.countRows(this.region_a));
assertEquals(rowCountOfRegionB, TEST_UTIL.countRows(this.region_b));
// Start transaction.
RegionMergeTransactionImpl mt = prepareOnGoodRegions();
// Run the execute. Look at what it returns.
TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
TEST_UTIL.getConfiguration());
Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
HRegion mergedRegion = (HRegion)mt.execute(mockServer, null);
// Do some assertions about execution.
assertTrue(this.fs.exists(mt.getMergesDir()));
// Assert region_a and region_b is closed.
assertTrue(region_a.isClosed());
assertTrue(region_b.isClosed());
// Assert mergedir is empty -- because its content will have been moved out
// to be under the merged region dirs.
assertEquals(0, this.fs.listStatus(mt.getMergesDir()).length);
// Check merged region have correct key span.
assertTrue(Bytes.equals(this.region_a.getRegionInfo().getStartKey(),
mergedRegion.getRegionInfo().getStartKey()));
assertTrue(Bytes.equals(this.region_b.getRegionInfo().getEndKey(),
mergedRegion.getRegionInfo().getEndKey()));
// Count rows. merged region are already open
try {
int mergedRegionRowCount = TEST_UTIL.countRows(mergedRegion);
assertEquals((rowCountOfRegionA + rowCountOfRegionB),
mergedRegionRowCount);
} finally {
HBaseTestingUtility.closeRegionAndWAL(mergedRegion);
}
// Assert the write lock is no longer held on region_a and region_b
assertTrue(!this.region_a.lock.writeLock().isHeldByCurrentThread());
assertTrue(!this.region_b.lock.writeLock().isHeldByCurrentThread());
}
@Test
public void testRollback() throws IOException, InterruptedException {
final int rowCountOfRegionA = loadRegion(this.region_a, CF, true);
final int rowCountOfRegionB = loadRegion(this.region_b, CF, true);
assertTrue(rowCountOfRegionA > 0 && rowCountOfRegionB > 0);
assertEquals(rowCountOfRegionA, TEST_UTIL.countRows(this.region_a));
assertEquals(rowCountOfRegionB, TEST_UTIL.countRows(this.region_b));
// Start transaction.
RegionMergeTransactionImpl mt = prepareOnGoodRegions();
when(mt.createMergedRegionFromMerges(region_a, region_b,
mt.getMergedRegionInfo())).thenThrow(
new MockedFailedMergedRegionCreation());
// Run the execute. Look at what it returns.
boolean expectedException = false;
TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
TEST_UTIL.getConfiguration());
Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
try {
mt.execute(mockServer, null);
} catch (MockedFailedMergedRegionCreation e) {
expectedException = true;
}
assertTrue(expectedException);
// Run rollback
assertTrue(mt.rollback(null, null));
// Assert I can scan region_a and region_b.
int rowCountOfRegionA2 = TEST_UTIL.countRows(this.region_a);
assertEquals(rowCountOfRegionA, rowCountOfRegionA2);
int rowCountOfRegionB2 = TEST_UTIL.countRows(this.region_b);
assertEquals(rowCountOfRegionB, rowCountOfRegionB2);
// Assert rollback cleaned up stuff in fs
assertTrue(!this.fs.exists(HRegion.getRegionDir(this.testdir,
mt.getMergedRegionInfo())));
assertTrue(!this.region_a.lock.writeLock().isHeldByCurrentThread());
assertTrue(!this.region_b.lock.writeLock().isHeldByCurrentThread());
// Now retry the merge but do not throw an exception this time.
assertTrue(mt.prepare(null));
HRegion mergedRegion = (HRegion)mt.execute(mockServer, null);
// Count rows. daughters are already open
// Count rows. merged region are already open
try {
int mergedRegionRowCount = TEST_UTIL.countRows(mergedRegion);
assertEquals((rowCountOfRegionA + rowCountOfRegionB),
mergedRegionRowCount);
} finally {
HBaseTestingUtility.closeRegionAndWAL(mergedRegion);
}
// Assert the write lock is no longer held on region_a and region_b
assertTrue(!this.region_a.lock.writeLock().isHeldByCurrentThread());
assertTrue(!this.region_b.lock.writeLock().isHeldByCurrentThread());
}
@Test
public void testFailAfterPONR() throws IOException, KeeperException, InterruptedException {
final int rowCountOfRegionA = loadRegion(this.region_a, CF, true);
final int rowCountOfRegionB = loadRegion(this.region_b, CF, true);
assertTrue(rowCountOfRegionA > 0 && rowCountOfRegionB > 0);
assertEquals(rowCountOfRegionA, TEST_UTIL.countRows(this.region_a));
assertEquals(rowCountOfRegionB, TEST_UTIL.countRows(this.region_b));
// Start transaction.
RegionMergeTransactionImpl mt = prepareOnGoodRegions();
Mockito.doThrow(new MockedFailedMergedRegionOpen())
.when(mt)
.openMergedRegion((Server) Mockito.anyObject(),
(RegionServerServices) Mockito.anyObject(),
(HRegion) Mockito.anyObject());
// Run the execute. Look at what it returns.
boolean expectedException = false;
TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
TEST_UTIL.getConfiguration());
Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
try {
mt.execute(mockServer, null);
} catch (MockedFailedMergedRegionOpen e) {
expectedException = true;
}
assertTrue(expectedException);
// Run rollback returns false that we should restart.
assertFalse(mt.rollback(null, null));
// Make sure that merged region is still in the filesystem, that
// they have not been removed; this is supposed to be the case if we go
// past point of no return.
Path tableDir = this.region_a.getRegionFileSystem().getRegionDir()
.getParent();
Path mergedRegionDir = new Path(tableDir, mt.getMergedRegionInfo()
.getEncodedName());
assertTrue(TEST_UTIL.getTestFileSystem().exists(mergedRegionDir));
}
@Test
public void testMergedRegionBoundary() {
TableName tableName =
TableName.valueOf("testMergedRegionBoundary");
byte[] a = Bytes.toBytes("a");
byte[] b = Bytes.toBytes("b");
byte[] z = Bytes.toBytes("z");
HRegionInfo r1 = new HRegionInfo(tableName);
HRegionInfo r2 = new HRegionInfo(tableName, a, z);
HRegionInfo m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
&& Bytes.equals(m.getEndKey(), r1.getEndKey()));
r1 = new HRegionInfo(tableName, null, a);
r2 = new HRegionInfo(tableName, a, z);
m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
r1 = new HRegionInfo(tableName, null, a);
r2 = new HRegionInfo(tableName, z, null);
m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
r1 = new HRegionInfo(tableName, a, z);
r2 = new HRegionInfo(tableName, z, null);
m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
r1 = new HRegionInfo(tableName, a, b);
r2 = new HRegionInfo(tableName, b, z);
m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
&& Bytes.equals(m.getEndKey(), r2.getEndKey()));
}
/**
* Exception used in this class only.
*/
@SuppressWarnings("serial")
private class MockedFailedMergedRegionCreation extends IOException {
}
@SuppressWarnings("serial")
private class MockedFailedMergedRegionOpen extends IOException {
}
private HRegion createRegion(final Path testdir, final WALFactory wals,
final byte[] startrow, final byte[] endrow)
throws IOException {
// Make a region with start and end keys.
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("table"));
HColumnDescriptor hcd = new HColumnDescriptor(CF);
htd.addFamily(hcd);
HRegionInfo hri = new HRegionInfo(htd.getTableName(), startrow, endrow);
HRegion a = HBaseTestingUtility.createRegionAndWAL(hri, testdir,
TEST_UTIL.getConfiguration(), htd);
HBaseTestingUtility.closeRegionAndWAL(a);
return HRegion.openHRegion(testdir, hri, htd,
wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()),
TEST_UTIL.getConfiguration());
}
/**
* Load region with rows from 'aaa' to 'zzz', skip the rows which are out of
* range of the region
* @param r Region
* @param f Family
* @param flush flush the cache if true
* @return Count of rows loaded.
* @throws IOException
*/
private int loadRegion(final HRegion r, final byte[] f, final boolean flush)
throws IOException {
byte[] k = new byte[3];
int rowCount = 0;
for (byte b1 = 'a'; b1 <= 'z'; b1++) {
for (byte b2 = 'a'; b2 <= 'z'; b2++) {
for (byte b3 = 'a'; b3 <= 'z'; b3++) {
k[0] = b1;
k[1] = b2;
k[2] = b3;
if (!HRegion.rowIsInRange(r.getRegionInfo(), k)) {
continue;
}
Put put = new Put(k);
put.addColumn(f, null, k);
if (r.getWAL() == null)
put.setDurability(Durability.SKIP_WAL);
r.put(put);
rowCount++;
}
}
if (flush) {
r.flush(true);
}
}
return rowCount;
}
}

View File

@ -1,172 +0,0 @@
/**
*
* 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.util;
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.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests merging a normal table's regions
*/
@Category({MiscTests.class, MediumTests.class})
public class TestMergeTable {
private static final Log LOG = LogFactory.getLog(TestMergeTable.class);
private final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final byte [] COLUMN_NAME = Bytes.toBytes("contents");
private static final byte [] VALUE;
static {
// We will use the same value for the rows as that is not really important here
String partialValue = String.valueOf(System.currentTimeMillis());
StringBuilder val = new StringBuilder();
while (val.length() < 1024) {
val.append(partialValue);
}
VALUE = Bytes.toBytes(val.toString());
}
/**
* Test merge.
* Hand-makes regions of a mergeable size and adds the hand-made regions to
* hand-made meta. The hand-made regions are created offline. We then start
* up mini cluster, disables the hand-made table and starts in on merging.
* @throws Exception
*/
@Test (timeout=300000) public void testMergeTable() throws Exception {
// Table we are manually creating offline.
HTableDescriptor desc = new HTableDescriptor(org.apache.hadoop.hbase.TableName.valueOf(Bytes.toBytes("test")));
desc.addFamily(new HColumnDescriptor(COLUMN_NAME));
// Set maximum regionsize down.
UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE, 64L * 1024L * 1024L);
// Make it so we don't split.
UTIL.getConfiguration().setInt("hbase.regionserver.regionSplitLimit", 0);
// Startup hdfs. Its in here we'll be putting our manually made regions.
UTIL.startMiniDFSCluster(1);
// Create hdfs hbase rootdir.
Path rootdir = UTIL.createRootDir();
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
if (fs.exists(rootdir)) {
if (fs.delete(rootdir, true)) {
LOG.info("Cleaned up existing " + rootdir);
}
}
// Now create three data regions: The first is too large to merge since it
// will be > 64 MB in size. The second two will be smaller and will be
// selected for merging.
// To ensure that the first region is larger than 64MB we need to write at
// least 65536 rows. We will make certain by writing 70000
byte [] row_70001 = Bytes.toBytes("row_70001");
byte [] row_80001 = Bytes.toBytes("row_80001");
// Create regions and populate them at same time. Create the tabledir
// for them first.
new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir).createTableDescriptor(desc);
HRegion [] regions = {
createRegion(desc, null, row_70001, 1, 70000, rootdir),
createRegion(desc, row_70001, row_80001, 70001, 10000, rootdir),
createRegion(desc, row_80001, null, 80001, 11000, rootdir)
};
// Now create the root and meta regions and insert the data regions
// created above into hbase:meta
setupMeta(rootdir, regions);
try {
LOG.info("Starting mini zk cluster");
UTIL.startMiniZKCluster();
LOG.info("Starting mini hbase cluster");
UTIL.startMiniHBaseCluster(1, 1);
Configuration c = new Configuration(UTIL.getConfiguration());
Connection connection = UTIL.getConnection();
List<HRegionInfo> originalTableRegions =
MetaTableAccessor.getTableRegions(connection, desc.getTableName());
LOG.info("originalTableRegions size=" + originalTableRegions.size() +
"; " + originalTableRegions);
Admin admin = connection.getAdmin();
admin.disableTable(desc.getTableName());
admin.close();
HMerge.merge(c, FileSystem.get(c), desc.getTableName());
List<HRegionInfo> postMergeTableRegions =
MetaTableAccessor.getTableRegions(connection, desc.getTableName());
LOG.info("postMergeTableRegions size=" + postMergeTableRegions.size() +
"; " + postMergeTableRegions);
assertTrue("originalTableRegions=" + originalTableRegions.size() +
", postMergeTableRegions=" + postMergeTableRegions.size(),
postMergeTableRegions.size() < originalTableRegions.size());
LOG.info("Done with merge");
} finally {
UTIL.shutdownMiniCluster();
LOG.info("After cluster shutdown");
}
}
private HRegion createRegion(final HTableDescriptor desc,
byte [] startKey, byte [] endKey, int firstRow, int nrows, Path rootdir)
throws IOException {
HRegionInfo hri = new HRegionInfo(desc.getTableName(), startKey, endKey);
HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, rootdir, UTIL.getConfiguration(),
desc);
LOG.info("Created region " + region.getRegionInfo().getRegionNameAsString());
for(int i = firstRow; i < firstRow + nrows; i++) {
Put put = new Put(Bytes.toBytes("row_" + String.format("%1$05d", i)));
put.setDurability(Durability.SKIP_WAL);
put.addColumn(COLUMN_NAME, null, VALUE);
region.put(put);
if (i % 10000 == 0) {
LOG.info("Flushing write #" + i);
region.flush(true);
}
}
HBaseTestingUtility.closeRegionAndWAL(region);
return region;
}
protected void setupMeta(Path rootdir, final HRegion [] regions)
throws IOException {
HRegion meta =
HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO, rootdir,
UTIL.getConfiguration(), UTIL.getMetaTableDescriptor());
for (HRegion r: regions) {
HRegion.addRegionToMETA(meta, r);
}
HBaseTestingUtility.closeRegionAndWAL(meta);
}
}

View File

@ -1,301 +0,0 @@
/**
*
* 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.util;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
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.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.util.ToolRunner;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/** Test stand alone merge tool that can merge arbitrary regions */
@Category({MiscTests.class, LargeTests.class})
public class TestMergeTool extends HBaseTestCase {
private static final Log LOG = LogFactory.getLog(TestMergeTool.class);
HBaseTestingUtility TEST_UTIL;
// static final byte [] COLUMN_NAME = Bytes.toBytes("contents:");
static final byte [] FAMILY = Bytes.toBytes("contents");
static final byte [] QUALIFIER = Bytes.toBytes("dc");
private final HRegionInfo[] sourceRegions = new HRegionInfo[5];
private final HRegion[] regions = new HRegion[5];
private HTableDescriptor desc;
private byte [][][] rows;
private MiniDFSCluster dfsCluster = null;
private WALFactory wals;
@Override
@Before
public void setUp() throws Exception {
// Set the timeout down else this test will take a while to complete.
this.conf.setLong("hbase.zookeeper.recoverable.waittime", 10);
// Make it so we try and connect to a zk that is not there (else we might
// find a zk ensemble put up by another concurrent test and this will
// mess up this test. Choose unlikely port. Default test port is 21818.
// Default zk port is 2181.
this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, 10001);
this.conf.set("hbase.hstore.compactionThreshold", "2");
// Create table description
this.desc = new HTableDescriptor(org.apache.hadoop.hbase.TableName.valueOf("TestMergeTool"));
this.desc.addFamily(new HColumnDescriptor(FAMILY));
/*
* Create the HRegionInfos for the regions.
*/
// Region 0 will contain the key range [row_0200,row_0300)
sourceRegions[0] = new HRegionInfo(this.desc.getTableName(),
Bytes.toBytes("row_0200"),
Bytes.toBytes("row_0300"));
// Region 1 will contain the key range [row_0250,row_0400) and overlaps
// with Region 0
sourceRegions[1] =
new HRegionInfo(this.desc.getTableName(),
Bytes.toBytes("row_0250"),
Bytes.toBytes("row_0400"));
// Region 2 will contain the key range [row_0100,row_0200) and is adjacent
// to Region 0 or the region resulting from the merge of Regions 0 and 1
sourceRegions[2] =
new HRegionInfo(this.desc.getTableName(),
Bytes.toBytes("row_0100"),
Bytes.toBytes("row_0200"));
// Region 3 will contain the key range [row_0500,row_0600) and is not
// adjacent to any of Regions 0, 1, 2 or the merged result of any or all
// of those regions
sourceRegions[3] =
new HRegionInfo(this.desc.getTableName(),
Bytes.toBytes("row_0500"),
Bytes.toBytes("row_0600"));
// Region 4 will have empty start and end keys and overlaps all regions.
sourceRegions[4] =
new HRegionInfo(this.desc.getTableName(),
HConstants.EMPTY_BYTE_ARRAY,
HConstants.EMPTY_BYTE_ARRAY);
/*
* Now create some row keys
*/
this.rows = new byte [5][][];
this.rows[0] = Bytes.toByteArrays(new String[] { "row_0210", "row_0280" });
this.rows[1] = Bytes.toByteArrays(new String[] { "row_0260", "row_0350",
"row_035" });
this.rows[2] = Bytes.toByteArrays(new String[] { "row_0110", "row_0175",
"row_0175", "row_0175"});
this.rows[3] = Bytes.toByteArrays(new String[] { "row_0525", "row_0560",
"row_0560", "row_0560", "row_0560"});
this.rows[4] = Bytes.toByteArrays(new String[] { "row_0050", "row_1000",
"row_1000", "row_1000", "row_1000", "row_1000" });
// Start up dfs
TEST_UTIL = new HBaseTestingUtility(conf);
this.dfsCluster = TEST_UTIL.startMiniDFSCluster(2);
this.fs = this.dfsCluster.getFileSystem();
System.out.println("fs=" + this.fs);
FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
TEST_UTIL.createRootDir();
// Note: we must call super.setUp after starting the mini cluster or
// we will end up with a local file system
super.setUp();
wals = new WALFactory(conf, null, "TestMergeTool");
try {
// Create meta region
createMetaRegion();
new FSTableDescriptors(this.conf, this.fs, testDir).createTableDescriptor(this.desc);
/*
* Create the regions we will merge
*/
for (int i = 0; i < sourceRegions.length; i++) {
regions[i] =
HBaseTestingUtility.createRegionAndWAL(this.sourceRegions[i], testDir, this.conf,
this.desc);
/*
* Insert data
*/
for (int j = 0; j < rows[i].length; j++) {
byte [] row = rows[i][j];
Put put = new Put(row);
put.addColumn(FAMILY, QUALIFIER, row);
regions[i].put(put);
}
HRegion.addRegionToMETA(meta, regions[i]);
}
// Close root and meta regions
closeRootAndMeta();
} catch (Exception e) {
TEST_UTIL.shutdownMiniCluster();
throw e;
}
}
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
for (int i = 0; i < sourceRegions.length; i++) {
HRegion r = regions[i];
if (r != null) {
HBaseTestingUtility.closeRegionAndWAL(r);
}
}
wals.close();
TEST_UTIL.shutdownMiniCluster();
}
/*
* @param msg Message that describes this merge
* @param regionName1
* @param regionName2
* @param log Log to use merging.
* @param upperbound Verifying, how high up in this.rows to go.
* @return Merged region.
* @throws Exception
*/
private HRegion mergeAndVerify(final String msg, final String regionName1,
final String regionName2, final WAL log, final int upperbound)
throws Exception {
Merge merger = new Merge();
LOG.info(msg);
LOG.info("fs2=" + this.conf.get("fs.defaultFS"));
int errCode = ToolRunner.run(this.conf, merger,
new String[] {this.desc.getTableName().getNameAsString(), regionName1, regionName2}
);
assertTrue("'" + msg + "' failed with errCode " + errCode, errCode == 0);
HRegionInfo mergedInfo = merger.getMergedHRegionInfo();
// Now verify that we can read all the rows from regions 0, 1
// in the new merged region.
HRegion merged = HRegion.openHRegion(mergedInfo, this.desc, log, this.conf);
verifyMerge(merged, upperbound);
merged.close();
LOG.info("Verified " + msg);
return merged;
}
private void verifyMerge(final HRegion merged, final int upperbound)
throws IOException {
//Test
Scan scan = new Scan();
scan.addFamily(FAMILY);
InternalScanner scanner = merged.getScanner(scan);
try {
List<Cell> testRes = null;
while (true) {
testRes = new ArrayList<Cell>();
boolean hasNext = scanner.next(testRes);
if (!hasNext) {
break;
}
}
} finally {
scanner.close();
}
//!Test
for (int i = 0; i < upperbound; i++) {
for (int j = 0; j < rows[i].length; j++) {
Get get = new Get(rows[i][j]);
get.addFamily(FAMILY);
Result result = merged.get(get);
assertEquals(1, result.size());
byte [] bytes = CellUtil.cloneValue(result.rawCells()[0]);
assertNotNull(Bytes.toStringBinary(rows[i][j]), bytes);
assertTrue(Bytes.equals(bytes, rows[i][j]));
}
}
}
/**
* Test merge tool.
* @throws Exception
*/
@Test
public void testMergeTool() throws Exception {
// First verify we can read the rows from the source regions and that they
// contain the right data.
for (int i = 0; i < regions.length; i++) {
for (int j = 0; j < rows[i].length; j++) {
Get get = new Get(rows[i][j]);
get.addFamily(FAMILY);
Result result = regions[i].get(get);
byte [] bytes = CellUtil.cloneValue(result.rawCells()[0]);
assertNotNull(bytes);
assertTrue(Bytes.equals(bytes, rows[i][j]));
}
// Close the region and delete the log
HBaseTestingUtility.closeRegionAndWAL(regions[i]);
}
WAL log = wals.getWAL(new byte[]{}, null);
// Merge Region 0 and Region 1
HRegion merged = mergeAndVerify("merging regions 0 and 1 ",
this.sourceRegions[0].getRegionNameAsString(),
this.sourceRegions[1].getRegionNameAsString(), log, 2);
// Merge the result of merging regions 0 and 1 with region 2
merged = mergeAndVerify("merging regions 0+1 and 2",
merged.getRegionInfo().getRegionNameAsString(),
this.sourceRegions[2].getRegionNameAsString(), log, 3);
// Merge the result of merging regions 0, 1 and 2 with region 3
merged = mergeAndVerify("merging regions 0+1+2 and 3",
merged.getRegionInfo().getRegionNameAsString(),
this.sourceRegions[3].getRegionNameAsString(), log, 4);
// Merge the result of merging regions 0, 1, 2 and 3 with region 4
merged = mergeAndVerify("merging regions 0+1+2+3 and 4",
merged.getRegionInfo().getRegionNameAsString(),
this.sourceRegions[4].getRegionNameAsString(), log, rows.length);
}
}