HBASE-17470 Remove merge region code from region server (Stephen Yuan Jiang)
This commit is contained in:
parent
287f95a579
commit
805d39fca6
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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 });
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue