HBASE-27250 MasterRpcService#setRegionStateInMeta does not support replica region encodedNames or region names
- Added sanity check to make sure input region encoded name or region name is valid - Assignment improvements pertaining to read replica regions - make several MetaTableAccessor methods more precise in their handling of replica regions - hbck2 setRegionStateInMeta and HBCKServerCrashProcedure handle read replicas - separate AM helper methods -- loading RegionInfo from cache vs. refreshing cache from meta - AM helper method support loading RegionInfo from cache via either region name and encoded region name (both caches are maintained, and under lock) - consolidate, extend tests to cover read replica regions Co-authored-by: Huaxiang Sun <huaxiangsun@apache.com> Co-authored-by: Nick Dimiduk <ndimiduk@apache.org> Signed-off-by: Peter Somogyi <psomogyi@apache.org>
This commit is contained in:
parent
1ac6f1689f
commit
9a69a69648
|
@ -289,7 +289,8 @@ public class MetaTableAccessor {
|
||||||
RegionLocations locations = getRegionLocations(r);
|
RegionLocations locations = getRegionLocations(r);
|
||||||
return locations == null
|
return locations == null
|
||||||
? null
|
? null
|
||||||
: locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
|
: locations.getRegionLocation(
|
||||||
|
parsedInfo == null ? RegionInfo.DEFAULT_REPLICA_ID : parsedInfo.getReplicaId());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -333,12 +334,12 @@ public class MetaTableAccessor {
|
||||||
/**
|
/**
|
||||||
* Gets the result in hbase:meta for the specified region.
|
* Gets the result in hbase:meta for the specified region.
|
||||||
* @param connection connection we're using
|
* @param connection connection we're using
|
||||||
* @param regionName region we're looking for
|
* @param regionInfo region we're looking for
|
||||||
* @return result of the specified region
|
* @return result of the specified region
|
||||||
*/
|
*/
|
||||||
public static Result getRegionResult(Connection connection, byte[] regionName)
|
public static Result getRegionResult(Connection connection, RegionInfo regionInfo)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Get get = new Get(regionName);
|
Get get = new Get(getMetaKeyForRegion(regionInfo));
|
||||||
get.addFamily(HConstants.CATALOG_FAMILY);
|
get.addFamily(HConstants.CATALOG_FAMILY);
|
||||||
return get(getMetaHTable(connection), get);
|
return get(getMetaHTable(connection), get);
|
||||||
}
|
}
|
||||||
|
@ -364,20 +365,20 @@ public class MetaTableAccessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns Return all regioninfos listed in the 'info:merge*' columns of the
|
* Returns Return all regioninfos listed in the 'info:merge*' columns of the {@code regionInfo}
|
||||||
* <code>regionName</code> row.
|
* row.
|
||||||
*/
|
*/
|
||||||
@Nullable
|
@Nullable
|
||||||
public static List<RegionInfo> getMergeRegions(Connection connection, byte[] regionName)
|
public static List<RegionInfo> getMergeRegions(Connection connection, RegionInfo regionInfo)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return getMergeRegions(getRegionResult(connection, regionName).rawCells());
|
return getMergeRegions(getRegionResult(connection, regionInfo).rawCells());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check whether the given {@code regionName} has any 'info:merge*' columns.
|
* Check whether the given {@code regionInfo} has any 'info:merge*' columns.
|
||||||
*/
|
*/
|
||||||
public static boolean hasMergeRegions(Connection conn, byte[] regionName) throws IOException {
|
public static boolean hasMergeRegions(Connection conn, RegionInfo regionInfo) throws IOException {
|
||||||
return hasMergeRegions(getRegionResult(conn, regionName).rawCells());
|
return hasMergeRegions(getRegionResult(conn, regionInfo).rawCells());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1294,11 +1295,21 @@ public class MetaTableAccessor {
|
||||||
////////////////////////
|
////////////////////////
|
||||||
// Editing operations //
|
// Editing operations //
|
||||||
////////////////////////
|
////////////////////////
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Generates and returns a Put containing the region into for the catalog table
|
* Generates and returns a {@link Put} containing the {@link RegionInfo} for the catalog table.
|
||||||
|
* @throws IllegalArgumentException when the provided RegionInfo is not the default replica.
|
||||||
|
*/
|
||||||
|
public static Put makePutFromRegionInfo(RegionInfo regionInfo) throws IOException {
|
||||||
|
return makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates and returns a {@link Put} containing the {@link RegionInfo} for the catalog table.
|
||||||
|
* @throws IllegalArgumentException when the provided RegionInfo is not the default replica.
|
||||||
*/
|
*/
|
||||||
public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts) throws IOException {
|
public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts) throws IOException {
|
||||||
return addRegionInfo(new Put(regionInfo.getRegionName(), ts), regionInfo);
|
return addRegionInfo(new Put(getMetaKeyForRegion(regionInfo), ts), regionInfo);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1308,7 +1319,11 @@ public class MetaTableAccessor {
|
||||||
if (regionInfo == null) {
|
if (regionInfo == null) {
|
||||||
throw new IllegalArgumentException("Can't make a delete for null region");
|
throw new IllegalArgumentException("Can't make a delete for null region");
|
||||||
}
|
}
|
||||||
Delete delete = new Delete(regionInfo.getRegionName());
|
if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"Can't make delete for a replica region. Operate on the primary");
|
||||||
|
}
|
||||||
|
Delete delete = new Delete(getMetaKeyForRegion(regionInfo));
|
||||||
delete.addFamily(getCatalogFamily(), ts);
|
delete.addFamily(getCatalogFamily(), ts);
|
||||||
return delete;
|
return delete;
|
||||||
}
|
}
|
||||||
|
@ -1399,9 +1414,14 @@ public class MetaTableAccessor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Put addRegionStateToPut(Put put, RegionState.State state) throws IOException {
|
/**
|
||||||
|
* Set the column value corresponding to this {@code replicaId}'s {@link RegionState} to the
|
||||||
|
* provided {@code state}. Mutates the provided {@link Put}.
|
||||||
|
*/
|
||||||
|
private static Put addRegionStateToPut(Put put, int replicaId, RegionState.State state)
|
||||||
|
throws IOException {
|
||||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||||
.setFamily(HConstants.CATALOG_FAMILY).setQualifier(getRegionStateColumn())
|
.setFamily(HConstants.CATALOG_FAMILY).setQualifier(getRegionStateColumn(replicaId))
|
||||||
.setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(state.name()))
|
.setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(state.name()))
|
||||||
.build());
|
.build());
|
||||||
return put;
|
return put;
|
||||||
|
@ -1412,9 +1432,9 @@ public class MetaTableAccessor {
|
||||||
*/
|
*/
|
||||||
public static void updateRegionState(Connection connection, RegionInfo ri,
|
public static void updateRegionState(Connection connection, RegionInfo ri,
|
||||||
RegionState.State state) throws IOException {
|
RegionState.State state) throws IOException {
|
||||||
Put put = new Put(RegionReplicaUtil.getRegionInfoForDefaultReplica(ri).getRegionName());
|
final Put put = makePutFromRegionInfo(ri);
|
||||||
MetaTableAccessor.putsToMetaTable(connection,
|
addRegionStateToPut(put, ri.getReplicaId(), state);
|
||||||
Collections.singletonList(addRegionStateToPut(put, state)));
|
putsToMetaTable(connection, Collections.singletonList(put));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1432,7 +1452,7 @@ public class MetaTableAccessor {
|
||||||
public static void addSplitsToParent(Connection connection, RegionInfo regionInfo,
|
public static void addSplitsToParent(Connection connection, RegionInfo regionInfo,
|
||||||
RegionInfo splitA, RegionInfo splitB) throws IOException {
|
RegionInfo splitA, RegionInfo splitB) throws IOException {
|
||||||
try (Table meta = getMetaHTable(connection)) {
|
try (Table meta = getMetaHTable(connection)) {
|
||||||
Put put = makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
|
Put put = makePutFromRegionInfo(regionInfo);
|
||||||
addDaughtersToPut(put, splitA, splitB);
|
addDaughtersToPut(put, splitA, splitB);
|
||||||
meta.put(put);
|
meta.put(put);
|
||||||
debugLogMutation(put);
|
debugLogMutation(put);
|
||||||
|
@ -1483,7 +1503,7 @@ public class MetaTableAccessor {
|
||||||
if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
|
if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
|
||||||
Put put = makePutFromRegionInfo(regionInfo, ts);
|
Put put = makePutFromRegionInfo(regionInfo, ts);
|
||||||
// New regions are added with initial state of CLOSED.
|
// New regions are added with initial state of CLOSED.
|
||||||
addRegionStateToPut(put, RegionState.State.CLOSED);
|
addRegionStateToPut(put, regionInfo.getReplicaId(), RegionState.State.CLOSED);
|
||||||
// Add empty locations for region replicas so that number of replicas can be cached
|
// Add empty locations for region replicas so that number of replicas can be cached
|
||||||
// whenever the primary region is looked up from meta
|
// whenever the primary region is looked up from meta
|
||||||
for (int i = 1; i < regionReplication; i++) {
|
for (int i = 1; i < regionReplication; i++) {
|
||||||
|
@ -1548,7 +1568,7 @@ public class MetaTableAccessor {
|
||||||
// default OFFLINE state. If Master gets restarted after this step, start up sequence of
|
// default OFFLINE state. If Master gets restarted after this step, start up sequence of
|
||||||
// master tries to assign this offline region. This is followed by re-assignments of the
|
// master tries to assign this offline region. This is followed by re-assignments of the
|
||||||
// merged region from resumed {@link MergeTableRegionsProcedure}
|
// merged region from resumed {@link MergeTableRegionsProcedure}
|
||||||
addRegionStateToPut(putOfMerged, RegionState.State.CLOSED);
|
addRegionStateToPut(putOfMerged, RegionInfo.DEFAULT_REPLICA_ID, RegionState.State.CLOSED);
|
||||||
mutations.add(putOfMerged);
|
mutations.add(putOfMerged);
|
||||||
// The merged is a new region, openSeqNum = 1 is fine. ServerName may be null
|
// The merged is a new region, openSeqNum = 1 is fine. ServerName may be null
|
||||||
// if crash after merge happened but before we got to here.. means in-memory
|
// if crash after merge happened but before we got to here.. means in-memory
|
||||||
|
@ -1606,8 +1626,8 @@ public class MetaTableAccessor {
|
||||||
// default OFFLINE state. If Master gets restarted after this step, start up sequence of
|
// default OFFLINE state. If Master gets restarted after this step, start up sequence of
|
||||||
// master tries to assign these offline regions. This is followed by re-assignments of the
|
// master tries to assign these offline regions. This is followed by re-assignments of the
|
||||||
// daughter regions from resumed {@link SplitTableRegionProcedure}
|
// daughter regions from resumed {@link SplitTableRegionProcedure}
|
||||||
addRegionStateToPut(putA, RegionState.State.CLOSED);
|
addRegionStateToPut(putA, RegionInfo.DEFAULT_REPLICA_ID, RegionState.State.CLOSED);
|
||||||
addRegionStateToPut(putB, RegionState.State.CLOSED);
|
addRegionStateToPut(putB, RegionInfo.DEFAULT_REPLICA_ID, RegionState.State.CLOSED);
|
||||||
|
|
||||||
addSequenceNum(putA, 1, splitA.getReplicaId()); // new regions, openSeqNum = 1 is fine.
|
addSequenceNum(putA, 1, splitA.getReplicaId()); // new regions, openSeqNum = 1 is fine.
|
||||||
addSequenceNum(putB, 1, splitB.getReplicaId());
|
addSequenceNum(putB, 1, splitB.getReplicaId());
|
||||||
|
@ -1820,7 +1840,7 @@ public class MetaTableAccessor {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Delete delete = new Delete(mergeRegion.getRegionName());
|
Delete delete = new Delete(mergeRegion.getRegionName());
|
||||||
// NOTE: We are doing a new hbase:meta read here.
|
// NOTE: We are doing a new hbase:meta read here.
|
||||||
Cell[] cells = getRegionResult(connection, mergeRegion.getRegionName()).rawCells();
|
Cell[] cells = getRegionResult(connection, mergeRegion).rawCells();
|
||||||
if (cells == null || cells.length == 0) {
|
if (cells == null || cells.length == 0) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,6 +33,7 @@ import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
@ -53,7 +54,6 @@ import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.UnknownRegionException;
|
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||||
import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;
|
import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
|
@ -74,6 +74,7 @@ import org.apache.hadoop.hbase.ipc.RpcServerFactory;
|
||||||
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
||||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||||
|
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||||
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
|
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
|
||||||
import org.apache.hadoop.hbase.master.assignment.RegionStates;
|
import org.apache.hadoop.hbase.master.assignment.RegionStates;
|
||||||
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
|
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
|
||||||
|
@ -1867,15 +1868,10 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
@QosPriority(priority = HConstants.ADMIN_QOS)
|
@QosPriority(priority = HConstants.ADMIN_QOS)
|
||||||
public GetRegionInfoResponse getRegionInfo(final RpcController controller,
|
public GetRegionInfoResponse getRegionInfo(final RpcController controller,
|
||||||
final GetRegionInfoRequest request) throws ServiceException {
|
final GetRegionInfoRequest request) throws ServiceException {
|
||||||
RegionInfo ri = null;
|
final GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
|
||||||
try {
|
final RegionInfo info = getRegionInfo(request.getRegion());
|
||||||
ri = getRegionInfo(request.getRegion());
|
if (info != null) {
|
||||||
} catch (UnknownRegionException ure) {
|
builder.setRegionInfo(ProtobufUtil.toRegionInfo(info));
|
||||||
throw new ServiceException(ure);
|
|
||||||
}
|
|
||||||
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
|
|
||||||
if (ri != null) {
|
|
||||||
builder.setRegionInfo(ProtobufUtil.toRegionInfo(ri));
|
|
||||||
} else {
|
} else {
|
||||||
// Is it a MOB name? These work differently.
|
// Is it a MOB name? These work differently.
|
||||||
byte[] regionName = request.getRegion().getValue().toByteArray();
|
byte[] regionName = request.getRegion().getValue().toByteArray();
|
||||||
|
@ -2529,64 +2525,69 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
SetRegionStateInMetaRequest request) throws ServiceException {
|
SetRegionStateInMetaRequest request) throws ServiceException {
|
||||||
rpcPreCheck("setRegionStateInMeta");
|
rpcPreCheck("setRegionStateInMeta");
|
||||||
SetRegionStateInMetaResponse.Builder builder = SetRegionStateInMetaResponse.newBuilder();
|
SetRegionStateInMetaResponse.Builder builder = SetRegionStateInMetaResponse.newBuilder();
|
||||||
|
final AssignmentManager am = master.getAssignmentManager();
|
||||||
try {
|
try {
|
||||||
for (RegionSpecifierAndState s : request.getStatesList()) {
|
for (RegionSpecifierAndState s : request.getStatesList()) {
|
||||||
RegionSpecifier spec = s.getRegionSpecifier();
|
final RegionSpecifier spec = s.getRegionSpecifier();
|
||||||
String encodedName;
|
final RegionInfo targetRegionInfo = getRegionInfo(spec);
|
||||||
if (spec.getType() == RegionSpecifierType.ENCODED_REGION_NAME) {
|
final RegionState.State targetState = RegionState.State.convert(s.getState());
|
||||||
encodedName = spec.getValue().toStringUtf8();
|
final RegionState.State currentState = Optional.ofNullable(targetRegionInfo)
|
||||||
} else {
|
.map(info -> am.getRegionStates().getRegionState(info)).map(RegionState::getState)
|
||||||
// TODO: actually, a full region name can save a lot on meta scan, improve later.
|
.orElseThrow(
|
||||||
encodedName = RegionInfo.encodeRegionName(spec.getValue().toByteArray());
|
() -> new ServiceException("No existing state known for region '" + spec + "'."));
|
||||||
|
LOG.info("{} set region={} state from {} to {}", master.getClientIdAuditPrefix(),
|
||||||
|
targetRegionInfo, currentState, targetState);
|
||||||
|
if (currentState == targetState) {
|
||||||
|
LOG.debug("Proposed state matches current state. {}, {}", targetRegionInfo, currentState);
|
||||||
|
continue;
|
||||||
}
|
}
|
||||||
RegionInfo info = this.master.getAssignmentManager().loadRegionFromMeta(encodedName);
|
MetaTableAccessor.updateRegionState(master.getConnection(), targetRegionInfo, targetState);
|
||||||
LOG.trace("region info loaded from meta table: {}", info);
|
|
||||||
RegionState prevState =
|
|
||||||
this.master.getAssignmentManager().getRegionStates().getRegionState(info);
|
|
||||||
RegionState.State newState = RegionState.State.convert(s.getState());
|
|
||||||
LOG.info("{} set region={} state from {} to {}", master.getClientIdAuditPrefix(), info,
|
|
||||||
prevState.getState(), newState);
|
|
||||||
Put metaPut =
|
|
||||||
MetaTableAccessor.makePutFromRegionInfo(info, EnvironmentEdgeManager.currentTime());
|
|
||||||
metaPut.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
|
|
||||||
Bytes.toBytes(newState.name()));
|
|
||||||
List<Put> putList = new ArrayList<>();
|
|
||||||
putList.add(metaPut);
|
|
||||||
MetaTableAccessor.putsToMetaTable(this.master.getConnection(), putList);
|
|
||||||
// Loads from meta again to refresh AM cache with the new region state
|
// Loads from meta again to refresh AM cache with the new region state
|
||||||
this.master.getAssignmentManager().loadRegionFromMeta(encodedName);
|
am.populateRegionStatesFromMeta(targetRegionInfo);
|
||||||
builder.addStates(RegionSpecifierAndState.newBuilder().setRegionSpecifier(spec)
|
builder.addStates(RegionSpecifierAndState.newBuilder().setRegionSpecifier(spec)
|
||||||
.setState(prevState.getState().convert()));
|
.setState(currentState.convert()));
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (IOException e) {
|
||||||
throw new ServiceException(e);
|
throw new ServiceException(e);
|
||||||
}
|
}
|
||||||
return builder.build();
|
return builder.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get RegionInfo from Master using content of RegionSpecifier as key.
|
* Get {@link RegionInfo} from Master using content of {@link RegionSpecifier} as key.
|
||||||
* @return RegionInfo found by decoding <code>rs</code> or null if none found
|
* @return {@link RegionInfo} found by decoding {@code rs} or {@code null} if {@code rs} is
|
||||||
|
* unknown to the master.
|
||||||
|
* @throws ServiceException If some error occurs while querying META or parsing results.
|
||||||
*/
|
*/
|
||||||
private RegionInfo getRegionInfo(HBaseProtos.RegionSpecifier rs) throws UnknownRegionException {
|
private RegionInfo getRegionInfo(HBaseProtos.RegionSpecifier rs) throws ServiceException {
|
||||||
RegionInfo ri = null;
|
// TODO: this doesn't handle MOB regions. Should it? See the public method #getRegionInfo
|
||||||
|
final AssignmentManager am = master.getAssignmentManager();
|
||||||
|
final String encodedRegionName;
|
||||||
|
final RegionInfo info;
|
||||||
|
// first try resolving from the AM's caches.
|
||||||
switch (rs.getType()) {
|
switch (rs.getType()) {
|
||||||
case REGION_NAME:
|
case REGION_NAME:
|
||||||
final byte[] regionName = rs.getValue().toByteArray();
|
final byte[] regionName = rs.getValue().toByteArray();
|
||||||
ri = this.master.getAssignmentManager().getRegionInfo(regionName);
|
encodedRegionName = RegionInfo.encodeRegionName(regionName);
|
||||||
|
info = am.getRegionInfo(regionName);
|
||||||
break;
|
break;
|
||||||
case ENCODED_REGION_NAME:
|
case ENCODED_REGION_NAME:
|
||||||
String encodedRegionName = Bytes.toString(rs.getValue().toByteArray());
|
encodedRegionName = rs.getValue().toStringUtf8();
|
||||||
RegionState regionState =
|
info = am.getRegionInfo(encodedRegionName);
|
||||||
this.master.getAssignmentManager().getRegionStates().getRegionState(encodedRegionName);
|
|
||||||
ri = regionState == null
|
|
||||||
? this.master.getAssignmentManager().loadRegionFromMeta(encodedRegionName)
|
|
||||||
: regionState.getRegion();
|
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
break;
|
throw new IllegalArgumentException("Unrecognized RegionSpecifierType " + rs.getType());
|
||||||
}
|
}
|
||||||
return ri;
|
if (info != null) {
|
||||||
|
return info;
|
||||||
|
}
|
||||||
|
// fall back to a meta scan and check the cache again.
|
||||||
|
try {
|
||||||
|
am.populateRegionStatesFromMeta(encodedRegionName);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new ServiceException(e);
|
||||||
|
}
|
||||||
|
return am.getRegionInfo(encodedRegionName);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2606,28 +2607,22 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
public MasterProtos.AssignsResponse assigns(RpcController controller,
|
public MasterProtos.AssignsResponse assigns(RpcController controller,
|
||||||
MasterProtos.AssignsRequest request) throws ServiceException {
|
MasterProtos.AssignsRequest request) throws ServiceException {
|
||||||
checkMasterProcedureExecutor();
|
checkMasterProcedureExecutor();
|
||||||
|
final ProcedureExecutor<MasterProcedureEnv> pe = master.getMasterProcedureExecutor();
|
||||||
|
final AssignmentManager am = master.getAssignmentManager();
|
||||||
MasterProtos.AssignsResponse.Builder responseBuilder =
|
MasterProtos.AssignsResponse.Builder responseBuilder =
|
||||||
MasterProtos.AssignsResponse.newBuilder();
|
MasterProtos.AssignsResponse.newBuilder();
|
||||||
try {
|
final boolean override = request.getOverride();
|
||||||
boolean override = request.getOverride();
|
|
||||||
LOG.info("{} assigns, override={}", master.getClientIdAuditPrefix(), override);
|
LOG.info("{} assigns, override={}", master.getClientIdAuditPrefix(), override);
|
||||||
for (HBaseProtos.RegionSpecifier rs : request.getRegionList()) {
|
for (HBaseProtos.RegionSpecifier rs : request.getRegionList()) {
|
||||||
long pid = Procedure.NO_PROC_ID;
|
final RegionInfo info = getRegionInfo(rs);
|
||||||
RegionInfo ri = getRegionInfo(rs);
|
if (info == null) {
|
||||||
if (ri == null) {
|
LOG.info("Unknown region {}", rs);
|
||||||
LOG.info("Unknown={}", rs);
|
continue;
|
||||||
} else {
|
|
||||||
Procedure p = this.master.getAssignmentManager().createOneAssignProcedure(ri, override);
|
|
||||||
if (p != null) {
|
|
||||||
pid = this.master.getMasterProcedureExecutor().submitProcedure(p);
|
|
||||||
}
|
}
|
||||||
}
|
responseBuilder.addPid(Optional.ofNullable(am.createOneAssignProcedure(info, override))
|
||||||
responseBuilder.addPid(pid);
|
.map(pe::submitProcedure).orElse(Procedure.NO_PROC_ID));
|
||||||
}
|
}
|
||||||
return responseBuilder.build();
|
return responseBuilder.build();
|
||||||
} catch (IOException ioe) {
|
|
||||||
throw new ServiceException(ioe);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2639,35 +2634,29 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
public MasterProtos.UnassignsResponse unassigns(RpcController controller,
|
public MasterProtos.UnassignsResponse unassigns(RpcController controller,
|
||||||
MasterProtos.UnassignsRequest request) throws ServiceException {
|
MasterProtos.UnassignsRequest request) throws ServiceException {
|
||||||
checkMasterProcedureExecutor();
|
checkMasterProcedureExecutor();
|
||||||
|
final ProcedureExecutor<MasterProcedureEnv> pe = master.getMasterProcedureExecutor();
|
||||||
|
final AssignmentManager am = master.getAssignmentManager();
|
||||||
MasterProtos.UnassignsResponse.Builder responseBuilder =
|
MasterProtos.UnassignsResponse.Builder responseBuilder =
|
||||||
MasterProtos.UnassignsResponse.newBuilder();
|
MasterProtos.UnassignsResponse.newBuilder();
|
||||||
try {
|
final boolean override = request.getOverride();
|
||||||
boolean override = request.getOverride();
|
|
||||||
LOG.info("{} unassigns, override={}", master.getClientIdAuditPrefix(), override);
|
LOG.info("{} unassigns, override={}", master.getClientIdAuditPrefix(), override);
|
||||||
for (HBaseProtos.RegionSpecifier rs : request.getRegionList()) {
|
for (HBaseProtos.RegionSpecifier rs : request.getRegionList()) {
|
||||||
long pid = Procedure.NO_PROC_ID;
|
final RegionInfo info = getRegionInfo(rs);
|
||||||
RegionInfo ri = getRegionInfo(rs);
|
if (info == null) {
|
||||||
if (ri == null) {
|
LOG.info("Unknown region {}", rs);
|
||||||
LOG.info("Unknown={}", rs);
|
continue;
|
||||||
} else {
|
|
||||||
Procedure p = this.master.getAssignmentManager().createOneUnassignProcedure(ri, override);
|
|
||||||
if (p != null) {
|
|
||||||
pid = this.master.getMasterProcedureExecutor().submitProcedure(p);
|
|
||||||
}
|
}
|
||||||
}
|
responseBuilder.addPid(Optional.ofNullable(am.createOneUnassignProcedure(info, override))
|
||||||
responseBuilder.addPid(pid);
|
.map(pe::submitProcedure).orElse(Procedure.NO_PROC_ID));
|
||||||
}
|
}
|
||||||
return responseBuilder.build();
|
return responseBuilder.build();
|
||||||
} catch (IOException ioe) {
|
|
||||||
throw new ServiceException(ioe);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Bypass specified procedure to completion. Procedure is marked completed but no actual work is
|
* Bypass specified procedure to completion. Procedure is marked completed but no actual work is
|
||||||
* done from the current state/ step onwards. Parents of the procedure are also marked for bypass.
|
* done from the current state/ step onwards. Parents of the procedure are also marked for bypass.
|
||||||
* NOTE: this is a dangerous operation and may be used to unstuck buggy procedures. This may leave
|
* NOTE: this is a dangerous operation and may be used to unstuck buggy procedures. This may leave
|
||||||
* system in inconherent state. This may need to be followed by some cleanup steps/ actions by
|
* system in incoherent state. This may need to be followed by some cleanup steps/ actions by
|
||||||
* operator.
|
* operator.
|
||||||
* @return BypassProcedureToCompletionResponse indicating success or failure
|
* @return BypassProcedureToCompletionResponse indicating success or failure
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.master.assignment;
|
package org.apache.hadoop.hbase.master.assignment;
|
||||||
|
|
||||||
|
import edu.umd.cs.findbugs.annotations.NonNull;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
@ -1737,26 +1738,29 @@ public class AssignmentManager {
|
||||||
};
|
};
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Query META if the given <code>RegionInfo</code> exists, adding to
|
* Attempt to load {@code regionInfo} from META, adding any results to the
|
||||||
* <code>AssignmentManager.regionStateStore</code> cache if the region is found in META.
|
* {@link #regionStateStore} Is NOT aware of replica regions.
|
||||||
* @param regionEncodedName encoded name for the region to be loaded from META into
|
* @param regionInfo the region to be loaded from META.
|
||||||
* <code>AssignmentManager.regionStateStore</code> cache
|
* @throws IOException If some error occurs while querying META or parsing results.
|
||||||
* @return <code>RegionInfo</code> instance for the given region if it is present in META and got
|
|
||||||
* successfully loaded into <code>AssignmentManager.regionStateStore</code> cache,
|
|
||||||
* <b>null</b> otherwise.
|
|
||||||
* @throws UnknownRegionException if any errors occur while querying meta.
|
|
||||||
*/
|
*/
|
||||||
public RegionInfo loadRegionFromMeta(String regionEncodedName) throws UnknownRegionException {
|
public void populateRegionStatesFromMeta(@NonNull final RegionInfo regionInfo)
|
||||||
try {
|
throws IOException {
|
||||||
RegionMetaLoadingVisitor visitor = new RegionMetaLoadingVisitor();
|
final String regionEncodedName = RegionInfo.DEFAULT_REPLICA_ID == regionInfo.getReplicaId()
|
||||||
regionStateStore.visitMetaForRegion(regionEncodedName, visitor);
|
? regionInfo.getEncodedName()
|
||||||
return regionStates.getRegionState(regionEncodedName) == null
|
: RegionInfoBuilder.newBuilder(regionInfo).setReplicaId(RegionInfo.DEFAULT_REPLICA_ID).build()
|
||||||
? null
|
.getEncodedName();
|
||||||
: regionStates.getRegionState(regionEncodedName).getRegion();
|
populateRegionStatesFromMeta(regionEncodedName);
|
||||||
} catch (IOException e) {
|
|
||||||
throw new UnknownRegionException(
|
|
||||||
"Error trying to load region " + regionEncodedName + " from META", e);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Attempt to load {@code regionEncodedName} from META, adding any results to the
|
||||||
|
* {@link #regionStateStore} Is NOT aware of replica regions.
|
||||||
|
* @param regionEncodedName encoded name for the region to be loaded from META.
|
||||||
|
* @throws IOException If some error occurs while querying META or parsing results.
|
||||||
|
*/
|
||||||
|
public void populateRegionStatesFromMeta(@NonNull String regionEncodedName) throws IOException {
|
||||||
|
final RegionMetaLoadingVisitor visitor = new RegionMetaLoadingVisitor();
|
||||||
|
regionStateStore.visitMetaForRegion(regionEncodedName, visitor);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void loadMeta() throws IOException {
|
private void loadMeta() throws IOException {
|
||||||
|
@ -1910,11 +1914,23 @@ public class AssignmentManager {
|
||||||
return regionStates.getAssignedRegions();
|
return regionStates.getAssignedRegions();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Resolve a cached {@link RegionInfo} from the region name as a {@code byte[]}.
|
||||||
|
*/
|
||||||
public RegionInfo getRegionInfo(final byte[] regionName) {
|
public RegionInfo getRegionInfo(final byte[] regionName) {
|
||||||
final RegionStateNode regionState = regionStates.getRegionStateNodeFromName(regionName);
|
final RegionStateNode regionState = regionStates.getRegionStateNodeFromName(regionName);
|
||||||
return regionState != null ? regionState.getRegionInfo() : null;
|
return regionState != null ? regionState.getRegionInfo() : null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Resolve a cached {@link RegionInfo} from the encoded region name as a {@code String}.
|
||||||
|
*/
|
||||||
|
public RegionInfo getRegionInfo(final String encodedRegionName) {
|
||||||
|
final RegionStateNode regionState =
|
||||||
|
regionStates.getRegionStateNodeFromEncodedRegionName(encodedRegionName);
|
||||||
|
return regionState != null ? regionState.getRegionInfo() : null;
|
||||||
|
}
|
||||||
|
|
||||||
// ============================================================================================
|
// ============================================================================================
|
||||||
// Expected states on region state transition.
|
// Expected states on region state transition.
|
||||||
// Notice that there is expected states for transiting to OPENING state, this is because SCP.
|
// Notice that there is expected states for transiting to OPENING state, this is because SCP.
|
||||||
|
|
|
@ -100,8 +100,8 @@ public class GCMultipleMergedRegionsProcedure
|
||||||
// If GCMultipleMergedRegionsProcedure processing is slower than the CatalogJanitor's scan
|
// If GCMultipleMergedRegionsProcedure processing is slower than the CatalogJanitor's scan
|
||||||
// interval, it will end resubmitting GCMultipleMergedRegionsProcedure for the same
|
// interval, it will end resubmitting GCMultipleMergedRegionsProcedure for the same
|
||||||
// region. We can skip duplicate GCMultipleMergedRegionsProcedure while previous finished
|
// region. We can skip duplicate GCMultipleMergedRegionsProcedure while previous finished
|
||||||
List<RegionInfo> parents = MetaTableAccessor
|
List<RegionInfo> parents =
|
||||||
.getMergeRegions(env.getMasterServices().getConnection(), mergedChild.getRegionName());
|
MetaTableAccessor.getMergeRegions(env.getMasterServices().getConnection(), mergedChild);
|
||||||
if (parents == null || parents.isEmpty()) {
|
if (parents == null || parents.isEmpty()) {
|
||||||
LOG.info("{} mergeXXX qualifiers have ALL been deleted",
|
LOG.info("{} mergeXXX qualifiers have ALL been deleted",
|
||||||
mergedChild.getShortNameToLog());
|
mergedChild.getShortNameToLog());
|
||||||
|
|
|
@ -469,10 +469,7 @@ public class MergeTableRegionsProcedure
|
||||||
|
|
||||||
RegionStates regionStates = env.getAssignmentManager().getRegionStates();
|
RegionStates regionStates = env.getAssignmentManager().getRegionStates();
|
||||||
for (RegionInfo ri : this.regionsToMerge) {
|
for (RegionInfo ri : this.regionsToMerge) {
|
||||||
if (
|
if (MetaTableAccessor.hasMergeRegions(env.getMasterServices().getConnection(), ri)) {
|
||||||
MetaTableAccessor.hasMergeRegions(env.getMasterServices().getConnection(),
|
|
||||||
ri.getRegionName())
|
|
||||||
) {
|
|
||||||
String msg = "Skip merging " + RegionInfo.getShortNameToLog(regionsToMerge)
|
String msg = "Skip merging " + RegionInfo.getShortNameToLog(regionsToMerge)
|
||||||
+ ", because a parent, " + RegionInfo.getShortNameToLog(ri) + ", has a merge qualifier "
|
+ ", because a parent, " + RegionInfo.getShortNameToLog(ri) + ", has a merge qualifier "
|
||||||
+ "(if a 'merge column' in parent, it was recently merged but still has outstanding "
|
+ "(if a 'merge column' in parent, it was recently merged but still has outstanding "
|
||||||
|
|
|
@ -110,9 +110,9 @@ public class RegionStateStore {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Queries META table for the passed region encoded name, delegating action upon results to the
|
* Queries META table for the passed region encoded name, delegating action upon results to the
|
||||||
* <code>RegionStateVisitor</code> passed as second parameter.
|
* {@code RegionStateVisitor} passed as second parameter.
|
||||||
* @param regionEncodedName encoded name for the Region we want to query META for.
|
* @param regionEncodedName encoded name for the Region we want to query META for.
|
||||||
* @param visitor The <code>RegionStateVisitor</code> instance to react over the query
|
* @param visitor The {@code RegionStateVisitor} instance to react over the query
|
||||||
* results.
|
* results.
|
||||||
* @throws IOException If some error occurs while querying META or parsing results.
|
* @throws IOException If some error occurs while querying META or parsing results.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -72,7 +72,7 @@ public class RegionStates {
|
||||||
|
|
||||||
// TODO: Replace the ConcurrentSkipListMaps
|
// TODO: Replace the ConcurrentSkipListMaps
|
||||||
/**
|
/**
|
||||||
* RegionName -- i.e. RegionInfo.getRegionName() -- as bytes to {@link RegionStateNode}
|
* A Map from {@link RegionInfo#getRegionName()} to {@link RegionStateNode}
|
||||||
*/
|
*/
|
||||||
private final ConcurrentSkipListMap<byte[], RegionStateNode> regionsMap =
|
private final ConcurrentSkipListMap<byte[], RegionStateNode> regionsMap =
|
||||||
new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
|
new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
|
||||||
|
@ -144,6 +144,10 @@ public class RegionStates {
|
||||||
return regionsMap.get(regionName);
|
return regionsMap.get(regionName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public RegionStateNode getRegionStateNodeFromEncodedRegionName(final String encodedRegionName) {
|
||||||
|
return encodedRegionsMap.get(encodedRegionName);
|
||||||
|
}
|
||||||
|
|
||||||
public RegionStateNode getRegionStateNode(RegionInfo regionInfo) {
|
public RegionStateNode getRegionStateNode(RegionInfo regionInfo) {
|
||||||
return getRegionStateNodeFromName(regionInfo.getRegionName());
|
return getRegionStateNodeFromName(regionInfo.getRegionName());
|
||||||
}
|
}
|
||||||
|
|
|
@ -279,8 +279,8 @@ public class DeleteTableProcedure extends AbstractStateMachineTableProcedure<Del
|
||||||
for (RegionInfo region : regions) {
|
for (RegionInfo region : regions) {
|
||||||
if (RegionReplicaUtil.isDefaultReplica(region)) {
|
if (RegionReplicaUtil.isDefaultReplica(region)) {
|
||||||
regionDirList.add(FSUtils.getRegionDirFromTableDir(tableDir, region));
|
regionDirList.add(FSUtils.getRegionDirFromTableDir(tableDir, region));
|
||||||
List<RegionInfo> mergeRegions = MetaTableAccessor
|
List<RegionInfo> mergeRegions =
|
||||||
.getMergeRegions(env.getMasterServices().getConnection(), region.getRegionName());
|
MetaTableAccessor.getMergeRegions(env.getMasterServices().getConnection(), region);
|
||||||
if (!CollectionUtils.isEmpty(mergeRegions)) {
|
if (!CollectionUtils.isEmpty(mergeRegions)) {
|
||||||
mergeRegions.stream()
|
mergeRegions.stream()
|
||||||
.forEach(r -> regionDirList.add(FSUtils.getRegionDirFromTableDir(tableDir, r)));
|
.forEach(r -> regionDirList.add(FSUtils.getRegionDirFromTableDir(tableDir, r)));
|
||||||
|
|
|
@ -1918,7 +1918,7 @@ public class HBaseFsck extends Configured implements Closeable {
|
||||||
|
|
||||||
RegionInfo hri =
|
RegionInfo hri =
|
||||||
RegionInfoBuilder.newBuilder(hi.getMetaEntry()).setOffline(false).setSplit(false).build();
|
RegionInfoBuilder.newBuilder(hi.getMetaEntry()).setOffline(false).setSplit(false).build();
|
||||||
Put p = MetaTableAccessor.makePutFromRegionInfo(hri, EnvironmentEdgeManager.currentTime());
|
Put p = MetaTableAccessor.makePutFromRegionInfo(hri);
|
||||||
mutations.add(p);
|
mutations.add(p);
|
||||||
|
|
||||||
meta.mutateRow(mutations);
|
meta.mutateRow(mutations);
|
||||||
|
|
|
@ -146,7 +146,7 @@ public class HBaseFsckRepair {
|
||||||
Collection<ServerName> servers, int numReplicas) throws IOException {
|
Collection<ServerName> servers, int numReplicas) throws IOException {
|
||||||
Connection conn = ConnectionFactory.createConnection(conf);
|
Connection conn = ConnectionFactory.createConnection(conf);
|
||||||
Table meta = conn.getTable(TableName.META_TABLE_NAME);
|
Table meta = conn.getTable(TableName.META_TABLE_NAME);
|
||||||
Put put = MetaTableAccessor.makePutFromRegionInfo(hri, EnvironmentEdgeManager.currentTime());
|
Put put = MetaTableAccessor.makePutFromRegionInfo(hri);
|
||||||
if (numReplicas > 1) {
|
if (numReplicas > 1) {
|
||||||
Random rand = ThreadLocalRandom.current();
|
Random rand = ThreadLocalRandom.current();
|
||||||
ServerName[] serversArr = servers.toArray(new ServerName[servers.size()]);
|
ServerName[] serversArr = servers.toArray(new ServerName[servers.size()]);
|
||||||
|
|
|
@ -133,20 +133,19 @@ public class TestMetaTableAccessor {
|
||||||
RegionInfo mergedRegion0 = mergedRegions.get(0);
|
RegionInfo mergedRegion0 = mergedRegions.get(0);
|
||||||
RegionInfo mergedRegion1 = mergedRegions.get(1);
|
RegionInfo mergedRegion1 = mergedRegions.get(1);
|
||||||
|
|
||||||
List<RegionInfo> mergeParents =
|
List<RegionInfo> mergeParents = MetaTableAccessor.getMergeRegions(connection, mergedRegion0);
|
||||||
MetaTableAccessor.getMergeRegions(connection, mergedRegion0.getRegionName());
|
|
||||||
assertTrue(mergeParents.contains(regions.get(0)));
|
assertTrue(mergeParents.contains(regions.get(0)));
|
||||||
assertTrue(mergeParents.contains(regions.get(1)));
|
assertTrue(mergeParents.contains(regions.get(1)));
|
||||||
mergeParents = MetaTableAccessor.getMergeRegions(connection, mergedRegion1.getRegionName());
|
mergeParents = MetaTableAccessor.getMergeRegions(connection, mergedRegion1);
|
||||||
assertTrue(mergeParents.contains(regions.get(2)));
|
assertTrue(mergeParents.contains(regions.get(2)));
|
||||||
assertTrue(mergeParents.contains(regions.get(3)));
|
assertTrue(mergeParents.contains(regions.get(3)));
|
||||||
|
|
||||||
// Delete merge qualifiers for mergedRegion0, then cannot getMergeRegions again
|
// Delete merge qualifiers for mergedRegion0, then cannot getMergeRegions again
|
||||||
MetaTableAccessor.deleteMergeQualifiers(connection, mergedRegion0);
|
MetaTableAccessor.deleteMergeQualifiers(connection, mergedRegion0);
|
||||||
mergeParents = MetaTableAccessor.getMergeRegions(connection, mergedRegion0.getRegionName());
|
mergeParents = MetaTableAccessor.getMergeRegions(connection, mergedRegion0);
|
||||||
assertNull(mergeParents);
|
assertNull(mergeParents);
|
||||||
|
|
||||||
mergeParents = MetaTableAccessor.getMergeRegions(connection, mergedRegion1.getRegionName());
|
mergeParents = MetaTableAccessor.getMergeRegions(connection, mergedRegion1);
|
||||||
assertTrue(mergeParents.contains(regions.get(2)));
|
assertTrue(mergeParents.contains(regions.get(2)));
|
||||||
assertTrue(mergeParents.contains(regions.get(3)));
|
assertTrue(mergeParents.contains(regions.get(3)));
|
||||||
}
|
}
|
||||||
|
|
|
@ -142,7 +142,7 @@ public class TestSplitMerge {
|
||||||
RegionInfo mergedRegion = mergedRegions.get(0);
|
RegionInfo mergedRegion = mergedRegions.get(0);
|
||||||
|
|
||||||
List<RegionInfo> mergeParentRegions =
|
List<RegionInfo> mergeParentRegions =
|
||||||
MetaTableAccessor.getMergeRegions(UTIL.getConnection(), mergedRegion.getRegionName());
|
MetaTableAccessor.getMergeRegions(UTIL.getConnection(), mergedRegion);
|
||||||
|
|
||||||
assertEquals(mergeParentRegions.size(), regionCount);
|
assertEquals(mergeParentRegions.size(), regionCount);
|
||||||
|
|
||||||
|
|
|
@ -29,6 +29,7 @@ import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
@ -57,7 +58,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
|
@ -119,7 +119,7 @@ public class TestHbck {
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setUpBeforeClass() throws Exception {
|
public static void setUpBeforeClass() throws Exception {
|
||||||
TEST_UTIL.startMiniCluster(3);
|
TEST_UTIL.startMiniCluster(3);
|
||||||
TEST_UTIL.createMultiRegionTable(TABLE_NAME, Bytes.toBytes("family1"), 5);
|
TEST_UTIL.createMultiRegionTable(TABLE_NAME, 3, new byte[][] { Bytes.toBytes("family1") });
|
||||||
procExec = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
|
procExec = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
|
||||||
ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
||||||
TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessorHost().load(
|
TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessorHost().load(
|
||||||
|
@ -189,7 +189,7 @@ public class TestHbck {
|
||||||
// will be DISABLED
|
// will be DISABLED
|
||||||
TableState prevState =
|
TableState prevState =
|
||||||
hbck.setTableStateInMeta(new TableState(TABLE_NAME, TableState.State.ENABLED));
|
hbck.setTableStateInMeta(new TableState(TABLE_NAME, TableState.State.ENABLED));
|
||||||
assertTrue("Incorrect previous state! expeced=DISABLED, found=" + prevState.getState(),
|
assertTrue("Incorrect previous state! expected=DISABLED, found=" + prevState.getState(),
|
||||||
prevState.isDisabled());
|
prevState.isDisabled());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -197,48 +197,57 @@ public class TestHbck {
|
||||||
public void testSetRegionStateInMeta() throws Exception {
|
public void testSetRegionStateInMeta() throws Exception {
|
||||||
Hbck hbck = getHbck();
|
Hbck hbck = getHbck();
|
||||||
Admin admin = TEST_UTIL.getAdmin();
|
Admin admin = TEST_UTIL.getAdmin();
|
||||||
|
TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME);
|
||||||
final List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
|
final List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
|
||||||
final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
|
final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
|
||||||
Map<String, RegionState.State> prevStates = new HashMap<>();
|
final Map<String, RegionState.State> beforeStates = new HashMap<>();
|
||||||
Map<String, RegionState.State> newStates = new HashMap<>();
|
final Map<String, RegionState.State> requestStates = new HashMap<>();
|
||||||
final Map<String, Pair<RegionState.State, RegionState.State>> regionsMap = new HashMap<>();
|
|
||||||
regions.forEach(r -> {
|
regions.forEach(r -> {
|
||||||
RegionState prevState = am.getRegionStates().getRegionState(r);
|
RegionState beforeState = am.getRegionStates().getRegionState(r);
|
||||||
prevStates.put(r.getEncodedName(), prevState.getState());
|
beforeStates.put(r.getEncodedName(), beforeState.getState());
|
||||||
newStates.put(r.getEncodedName(), RegionState.State.CLOSED);
|
LOG.debug("Before test: {} ; {}", r, beforeState.getState());
|
||||||
regionsMap.put(r.getEncodedName(),
|
requestStates.put(r.getEncodedName(), RegionState.State.CLOSED);
|
||||||
new Pair<>(prevState.getState(), RegionState.State.CLOSED));
|
|
||||||
});
|
});
|
||||||
final Map<String, RegionState.State> result = hbck.setRegionStateInMeta(newStates);
|
final Callable<Void> doTest = () -> {
|
||||||
|
// run the entire test with the ProcedureExecution environment paused. This prevents
|
||||||
|
// background operations from modifying AM internal state between the assertions this test
|
||||||
|
// relies upon.
|
||||||
|
Map<String, RegionState.State> result = hbck.setRegionStateInMeta(requestStates);
|
||||||
result.forEach((k, v) -> {
|
result.forEach((k, v) -> {
|
||||||
RegionState.State prevState = regionsMap.get(k).getFirst();
|
RegionState.State beforeState = beforeStates.get(k);
|
||||||
assertEquals(prevState, v);
|
assertEquals("response state should match before state; " + k, beforeState, v);
|
||||||
});
|
});
|
||||||
regions.forEach(r -> {
|
regions.forEach(r -> {
|
||||||
RegionState cachedState = am.getRegionStates().getRegionState(r.getEncodedName());
|
RegionState afterState = am.getRegionStates().getRegionState(r.getEncodedName());
|
||||||
RegionState.State newState = regionsMap.get(r.getEncodedName()).getSecond();
|
RegionState.State expectedState = requestStates.get(r.getEncodedName());
|
||||||
assertEquals(newState, cachedState.getState());
|
LOG.debug("After test: {}, {}", r, afterState);
|
||||||
|
assertEquals("state in AM should match requested state ; " + r, expectedState,
|
||||||
|
afterState.getState());
|
||||||
});
|
});
|
||||||
hbck.setRegionStateInMeta(prevStates);
|
return null;
|
||||||
|
};
|
||||||
|
ProcedureTestingUtility.restart(procExec, true, true, null, doTest, null, false, true);
|
||||||
|
// restore the table as we found it -- fragile?
|
||||||
|
hbck.setRegionStateInMeta(beforeStates);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAssigns() throws Exception {
|
public void testAssigns() throws Exception {
|
||||||
Hbck hbck = getHbck();
|
Hbck hbck = getHbck();
|
||||||
|
final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
|
||||||
try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
|
try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
|
||||||
List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
|
List<RegionInfo> regions = admin.getRegions(TABLE_NAME).stream()
|
||||||
for (RegionInfo ri : regions) {
|
.filter(ri -> ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID).peek(ri -> {
|
||||||
RegionState rs = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
|
final RegionState rs = am.getRegionStates().getRegionState(ri.getEncodedName());
|
||||||
.getRegionStates().getRegionState(ri.getEncodedName());
|
LOG.info("RS: {}", rs);
|
||||||
LOG.info("RS: {}", rs.toString());
|
}).collect(Collectors.toList());
|
||||||
}
|
List<Long> pids = hbck
|
||||||
List<Long> pids =
|
.unassigns(regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()));
|
||||||
hbck.unassigns(regions.stream().map(r -> r.getEncodedName()).collect(Collectors.toList()));
|
|
||||||
waitOnPids(pids);
|
waitOnPids(pids);
|
||||||
// Rerun the unassign. Should fail for all Regions since they already unassigned; failed
|
// Rerun the unassign. Should fail for all Regions since they already unassigned; failed
|
||||||
// unassign will manifest as all pids being -1 (ever since HBASE-24885).
|
// unassign will manifest as all pids being -1 (ever since HBASE-24885).
|
||||||
pids =
|
pids = hbck
|
||||||
hbck.unassigns(regions.stream().map(r -> r.getEncodedName()).collect(Collectors.toList()));
|
.unassigns(regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()));
|
||||||
waitOnPids(pids);
|
waitOnPids(pids);
|
||||||
for (long pid : pids) {
|
for (long pid : pids) {
|
||||||
assertEquals(Procedure.NO_PROC_ID, pid);
|
assertEquals(Procedure.NO_PROC_ID, pid);
|
||||||
|
@ -247,7 +256,7 @@ public class TestHbck {
|
||||||
// unassigned.... makes for a mess but operator might want to do this at an extreme when
|
// unassigned.... makes for a mess but operator might want to do this at an extreme when
|
||||||
// doing fixup of broke cluster.
|
// doing fixup of broke cluster.
|
||||||
pids = hbck.unassigns(
|
pids = hbck.unassigns(
|
||||||
regions.stream().map(r -> r.getEncodedName()).collect(Collectors.toList()), true);
|
regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()), true);
|
||||||
waitOnPids(pids);
|
waitOnPids(pids);
|
||||||
for (long pid : pids) {
|
for (long pid : pids) {
|
||||||
assertNotEquals(Procedure.NO_PROC_ID, pid);
|
assertNotEquals(Procedure.NO_PROC_ID, pid);
|
||||||
|
@ -261,12 +270,12 @@ public class TestHbck {
|
||||||
assertTrue(rs.toString(), rs.isClosed());
|
assertTrue(rs.toString(), rs.isClosed());
|
||||||
}
|
}
|
||||||
pids =
|
pids =
|
||||||
hbck.assigns(regions.stream().map(r -> r.getEncodedName()).collect(Collectors.toList()));
|
hbck.assigns(regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()));
|
||||||
waitOnPids(pids);
|
waitOnPids(pids);
|
||||||
// Rerun the assign. Should fail for all Regions since they already assigned; failed
|
// Rerun the assign. Should fail for all Regions since they already assigned; failed
|
||||||
// assign will manifest as all pids being -1 (ever since HBASE-24885).
|
// assign will manifest as all pids being -1 (ever since HBASE-24885).
|
||||||
pids =
|
pids =
|
||||||
hbck.assigns(regions.stream().map(r -> r.getEncodedName()).collect(Collectors.toList()));
|
hbck.assigns(regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.toList()));
|
||||||
for (long pid : pids) {
|
for (long pid : pids) {
|
||||||
assertEquals(Procedure.NO_PROC_ID, pid);
|
assertEquals(Procedure.NO_PROC_ID, pid);
|
||||||
}
|
}
|
||||||
|
@ -288,19 +297,21 @@ public class TestHbck {
|
||||||
@Test
|
@Test
|
||||||
public void testScheduleSCP() throws Exception {
|
public void testScheduleSCP() throws Exception {
|
||||||
HRegionServer testRs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
|
HRegionServer testRs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
|
||||||
TEST_UTIL.loadTable(TEST_UTIL.getConnection().getTable(TABLE_NAME), Bytes.toBytes("family1"),
|
try (final Table t = TEST_UTIL.getConnection().getTable(TABLE_NAME)) {
|
||||||
true);
|
TEST_UTIL.loadTable(t, Bytes.toBytes("family1"), true);
|
||||||
|
}
|
||||||
ServerName serverName = testRs.getServerName();
|
ServerName serverName = testRs.getServerName();
|
||||||
Hbck hbck = getHbck();
|
Hbck hbck = getHbck();
|
||||||
List<Long> pids =
|
List<Long> pids =
|
||||||
hbck.scheduleServerCrashProcedure(Arrays.asList(ProtobufUtil.toServerName(serverName)));
|
hbck.scheduleServerCrashProcedure(Arrays.asList(ProtobufUtil.toServerName(serverName)));
|
||||||
assertTrue(pids.get(0) > 0);
|
assertEquals(1, pids.size());
|
||||||
LOG.info("pid is {}", pids.get(0));
|
assertNotEquals((Long) Procedure.NO_PROC_ID, pids.get(0));
|
||||||
|
LOG.debug("SCP pid is {}", pids.get(0));
|
||||||
|
|
||||||
List<Long> newPids =
|
List<Long> newPids =
|
||||||
hbck.scheduleServerCrashProcedure(Arrays.asList(ProtobufUtil.toServerName(serverName)));
|
hbck.scheduleServerCrashProcedure(Arrays.asList(ProtobufUtil.toServerName(serverName)));
|
||||||
assertTrue(newPids.get(0) < 0);
|
assertEquals(1, pids.size());
|
||||||
LOG.info("pid is {}", newPids.get(0));
|
assertEquals((Long) Procedure.NO_PROC_ID, newPids.get(0));
|
||||||
waitOnPids(pids);
|
waitOnPids(pids);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -140,7 +140,7 @@ public class TestRegionsRecoveryChore {
|
||||||
|
|
||||||
// Verify that we need to reopen total 3 regions that have refCount > 300
|
// Verify that we need to reopen total 3 regions that have refCount > 300
|
||||||
Mockito.verify(hMaster, Mockito.times(3)).getAssignmentManager();
|
Mockito.verify(hMaster, Mockito.times(3)).getAssignmentManager();
|
||||||
Mockito.verify(assignmentManager, Mockito.times(3)).getRegionInfo(Mockito.any());
|
Mockito.verify(assignmentManager, Mockito.times(3)).getRegionInfo(Mockito.any(byte[].class));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -175,7 +175,7 @@ public class TestRegionsRecoveryChore {
|
||||||
|
|
||||||
// Verify that we need to reopen only 1 region with refCount > 400
|
// Verify that we need to reopen only 1 region with refCount > 400
|
||||||
Mockito.verify(hMaster, Mockito.times(1)).getAssignmentManager();
|
Mockito.verify(hMaster, Mockito.times(1)).getAssignmentManager();
|
||||||
Mockito.verify(assignmentManager, Mockito.times(1)).getRegionInfo(Mockito.any());
|
Mockito.verify(assignmentManager, Mockito.times(1)).getRegionInfo(Mockito.any(byte[].class));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -210,7 +210,7 @@ public class TestRegionsRecoveryChore {
|
||||||
|
|
||||||
// default maxCompactedStoreFileRefCount is -1 (no regions to be reopened using AM)
|
// default maxCompactedStoreFileRefCount is -1 (no regions to be reopened using AM)
|
||||||
Mockito.verify(hMaster, Mockito.times(0)).getAssignmentManager();
|
Mockito.verify(hMaster, Mockito.times(0)).getAssignmentManager();
|
||||||
Mockito.verify(assignmentManager, Mockito.times(0)).getRegionInfo(Mockito.any());
|
Mockito.verify(assignmentManager, Mockito.times(0)).getRegionInfo(Mockito.any(byte[].class));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static ClusterMetrics getClusterMetrics(int noOfLiveServer) {
|
private static ClusterMetrics getClusterMetrics(int noOfLiveServer) {
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
package org.apache.hadoop.hbase.master.assignment;
|
package org.apache.hadoop.hbase.master.assignment;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
import static org.junit.Assert.assertNull;
|
import static org.junit.Assert.assertNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
@ -303,12 +304,13 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
|
||||||
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
|
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
|
||||||
MetaTableAccessor.addRegionsToMeta(this.util.getConnection(), Collections.singletonList(hri),
|
MetaTableAccessor.addRegionsToMeta(this.util.getConnection(), Collections.singletonList(hri),
|
||||||
1);
|
1);
|
||||||
assertNull(
|
// TODO: is there a race here -- no other thread else will refresh the table states behind
|
||||||
"RegionInfo was manually added in META, but " + "shouldn't be in AM regionStates yet.",
|
// the scenes?
|
||||||
|
assertNull("RegionInfo was manually added in META, but shouldn't be in AM regionStates yet.",
|
||||||
am.getRegionStates().getRegionState(hri));
|
am.getRegionStates().getRegionState(hri));
|
||||||
hri = am.loadRegionFromMeta(hri.getEncodedName());
|
am.populateRegionStatesFromMeta(hri.getEncodedName());
|
||||||
assertEquals(hri.getEncodedName(),
|
assertNotNull(am.getRegionInfo(hri.getRegionName()));
|
||||||
am.getRegionStates().getRegionState(hri).getRegion().getEncodedName());
|
assertNotNull(am.getRegionInfo(hri.getEncodedName()));
|
||||||
} finally {
|
} finally {
|
||||||
this.util.killMiniHBaseCluster();
|
this.util.killMiniHBaseCluster();
|
||||||
}
|
}
|
||||||
|
@ -322,10 +324,10 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
|
||||||
final TableName tableName = TableName.valueOf("testLoadRegionFromMetaRegionNotInMeta");
|
final TableName tableName = TableName.valueOf("testLoadRegionFromMetaRegionNotInMeta");
|
||||||
this.util.createTable(tableName, "f");
|
this.util.createTable(tableName, "f");
|
||||||
final RegionInfo hri = createRegionInfo(tableName, 1);
|
final RegionInfo hri = createRegionInfo(tableName, 1);
|
||||||
assertNull("RegionInfo was just instantiated by the test, but "
|
assertNull("Bogus RegionInfo discovered in RegionStates.",
|
||||||
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
|
am.getRegionStates().getRegionState(hri));
|
||||||
assertNull("RegionInfo was never added in META, should had returned null.",
|
am.populateRegionStatesFromMeta(hri.getEncodedName());
|
||||||
am.loadRegionFromMeta(hri.getEncodedName()));
|
assertNull("RegionInfo was never added in META", am.getRegionStates().getRegionState(hri));
|
||||||
} finally {
|
} finally {
|
||||||
this.util.killMiniHBaseCluster();
|
this.util.killMiniHBaseCluster();
|
||||||
}
|
}
|
||||||
|
|
|
@ -205,10 +205,10 @@ public class TestMergeTableRegionsProcedure {
|
||||||
// the merged regions cleanup.
|
// the merged regions cleanup.
|
||||||
UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true);
|
UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true);
|
||||||
UTIL.getHBaseCluster().getMaster().getCatalogJanitor().triggerNow();
|
UTIL.getHBaseCluster().getMaster().getCatalogJanitor().triggerNow();
|
||||||
byte[] mergedRegion = proc.getMergedRegion().getRegionName();
|
RegionInfo mergedRegion = proc.getMergedRegion();
|
||||||
while (ris != null && ris.get(0) != null && ris.get(1) != null) {
|
while (ris != null && ris.get(0) != null && ris.get(1) != null) {
|
||||||
ris = MetaTableAccessor.getMergeRegions(UTIL.getConnection(), mergedRegion);
|
ris = MetaTableAccessor.getMergeRegions(UTIL.getConnection(), mergedRegion);
|
||||||
LOG.info("{} {}", Bytes.toStringBinary(mergedRegion), ris);
|
LOG.info("{} {}", Bytes.toStringBinary(mergedRegion.getRegionName()), ris);
|
||||||
Threads.sleep(1000);
|
Threads.sleep(1000);
|
||||||
}
|
}
|
||||||
assertEquals(countOfRowsLoaded, UTIL.countRows(tableName));
|
assertEquals(countOfRowsLoaded, UTIL.countRows(tableName));
|
||||||
|
|
|
@ -328,10 +328,10 @@ public class TestMetaFixer {
|
||||||
&& regionStates.getRegionState(pair.getSecond()).isOpened())
|
&& regionStates.getRegionState(pair.getSecond()).isOpened())
|
||||||
) {
|
) {
|
||||||
// Make sure GC is done.
|
// Make sure GC is done.
|
||||||
List<RegionInfo> firstParents = MetaTableAccessor
|
List<RegionInfo> firstParents =
|
||||||
.getMergeRegions(services.getConnection(), pair.getFirst().getRegionName());
|
MetaTableAccessor.getMergeRegions(services.getConnection(), pair.getFirst());
|
||||||
List<RegionInfo> secondParents = MetaTableAccessor
|
List<RegionInfo> secondParents =
|
||||||
.getMergeRegions(services.getConnection(), pair.getSecond().getRegionName());
|
MetaTableAccessor.getMergeRegions(services.getConnection(), pair.getSecond());
|
||||||
|
|
||||||
return (firstParents == null || firstParents.isEmpty())
|
return (firstParents == null || firstParents.isEmpty())
|
||||||
&& (secondParents == null || secondParents.isEmpty());
|
&& (secondParents == null || secondParents.isEmpty());
|
||||||
|
|
|
@ -25,6 +25,9 @@ import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.NoSuchElementException;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
@ -32,23 +35,26 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.TableNameTestRule;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.master.HMaster;
|
import org.apache.hadoop.hbase.master.HMaster;
|
||||||
import org.apache.hadoop.hbase.master.RegionState;
|
import org.apache.hadoop.hbase.master.RegionState;
|
||||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||||
|
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
|
||||||
import org.junit.ClassRule;
|
import org.junit.ClassRule;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
import org.junit.rules.TestName;
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -62,6 +68,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
|
||||||
* Regions that were on the server-to-process rather than consult Master in-memory-state.
|
* Regions that were on the server-to-process rather than consult Master in-memory-state.
|
||||||
*/
|
*/
|
||||||
@Category({ MasterTests.class, LargeTests.class })
|
@Category({ MasterTests.class, LargeTests.class })
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
public class TestHBCKSCP extends TestSCPBase {
|
public class TestHBCKSCP extends TestSCPBase {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(TestHBCKSCP.class);
|
private static final Logger LOG = LoggerFactory.getLogger(TestHBCKSCP.class);
|
||||||
|
|
||||||
|
@ -69,7 +76,27 @@ public class TestHBCKSCP extends TestSCPBase {
|
||||||
public static final HBaseClassTestRule CLASS_RULE =
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
HBaseClassTestRule.forClass(TestHBCKSCP.class);
|
HBaseClassTestRule.forClass(TestHBCKSCP.class);
|
||||||
@Rule
|
@Rule
|
||||||
public TestName name = new TestName();
|
public TableNameTestRule tableNameTestRule = new TableNameTestRule();
|
||||||
|
|
||||||
|
private final int replicas;
|
||||||
|
private final HBCKSCPScheduler hbckscpScheduler;
|
||||||
|
private final RegionSelector regionSelector;
|
||||||
|
|
||||||
|
public TestHBCKSCP(final int replicas, final HBCKSCPScheduler hbckscpScheduler,
|
||||||
|
final RegionSelector regionSelector) {
|
||||||
|
this.replicas = replicas;
|
||||||
|
this.hbckscpScheduler = hbckscpScheduler;
|
||||||
|
this.regionSelector = regionSelector;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Parameterized.Parameters(name = "replicas:{0} scheduler:{1} selector:{2}")
|
||||||
|
public static Object[][] params() {
|
||||||
|
return new Object[][] {
|
||||||
|
{ 1, new ScheduleServerCrashProcedure(), new PrimaryNotMetaRegionSelector() },
|
||||||
|
{ 3, new ScheduleServerCrashProcedure(), new ReplicaNonMetaRegionSelector() },
|
||||||
|
{ 1, new ScheduleSCPsForUnknownServers(), new PrimaryNotMetaRegionSelector() },
|
||||||
|
{ 3, new ScheduleSCPsForUnknownServers(), new ReplicaNonMetaRegionSelector() } };
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void test() throws Exception {
|
public void test() throws Exception {
|
||||||
|
@ -80,7 +107,7 @@ public class TestHBCKSCP extends TestSCPBase {
|
||||||
assertEquals(RS_COUNT, cluster.getLiveRegionServerThreads().size());
|
assertEquals(RS_COUNT, cluster.getLiveRegionServerThreads().size());
|
||||||
|
|
||||||
int count;
|
int count;
|
||||||
try (Table table = createTable(TableName.valueOf(this.name.getMethodName()))) {
|
try (Table table = createTable(tableNameTestRule.getTableName())) {
|
||||||
// Load the table with a bit of data so some logs to split and some edits in each region.
|
// Load the table with a bit of data so some logs to split and some edits in each region.
|
||||||
this.util.loadTable(table, HBaseTestingUtility.COLUMNS[0]);
|
this.util.loadTable(table, HBaseTestingUtility.COLUMNS[0]);
|
||||||
count = util.countRows(table);
|
count = util.countRows(table);
|
||||||
|
@ -91,17 +118,24 @@ public class TestHBCKSCP extends TestSCPBase {
|
||||||
// Find another RS. Purge it from Master memory w/o running SCP (if
|
// Find another RS. Purge it from Master memory w/o running SCP (if
|
||||||
// SCP runs, it will clear entries from hbase:meta which frustrates
|
// SCP runs, it will clear entries from hbase:meta which frustrates
|
||||||
// our attempt at manufacturing 'Unknown Servers' condition).
|
// our attempt at manufacturing 'Unknown Servers' condition).
|
||||||
int metaIndex = this.util.getMiniHBaseCluster().getServerWithMeta();
|
final ServerName metaServer = util.getMiniHBaseCluster().getServerHoldingMeta();
|
||||||
int rsIndex = (metaIndex + 1) % RS_COUNT;
|
final ServerName rsServerName = cluster.getRegionServerThreads().stream()
|
||||||
ServerName rsServerName = cluster.getRegionServer(rsIndex).getServerName();
|
.map(JVMClusterUtil.RegionServerThread::getRegionServer).map(HRegionServer::getServerName)
|
||||||
|
.filter(sn -> !sn.equals(metaServer)).findAny().orElseThrow(() -> new NoSuchElementException(
|
||||||
|
"Cannot locate a region server that is not hosting meta."));
|
||||||
HMaster master = cluster.getMaster();
|
HMaster master = cluster.getMaster();
|
||||||
// Get a Region that is on the server.
|
// Get a Region that is on the server.
|
||||||
RegionInfo rsRI = master.getAssignmentManager().getRegionsOnServer(rsServerName).get(0);
|
final List<RegionInfo> regions = master.getAssignmentManager().getRegionsOnServer(rsServerName);
|
||||||
Result r = MetaTableAccessor.getRegionResult(master.getConnection(), rsRI.getRegionName());
|
LOG.debug("{} is holding {} regions.", rsServerName, regions.size());
|
||||||
|
final RegionInfo rsRI =
|
||||||
|
regions.stream().peek(info -> LOG.debug("{}", info)).filter(regionSelector::regionFilter)
|
||||||
|
.findAny().orElseThrow(regionSelector::regionFilterFailure);
|
||||||
|
final int replicaId = rsRI.getReplicaId();
|
||||||
|
Result r = MetaTableAccessor.getRegionResult(master.getConnection(), rsRI);
|
||||||
// Assert region is OPEN.
|
// Assert region is OPEN.
|
||||||
assertEquals(RegionState.State.OPEN.toString(),
|
assertEquals(RegionState.State.OPEN.toString(), Bytes.toString(
|
||||||
Bytes.toString(r.getValue(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER)));
|
r.getValue(HConstants.CATALOG_FAMILY, MetaTableAccessor.getRegionStateColumn(replicaId))));
|
||||||
ServerName serverName = MetaTableAccessor.getServerName(r, 0);
|
ServerName serverName = MetaTableAccessor.getServerName(r, replicaId);
|
||||||
assertEquals(rsServerName, serverName);
|
assertEquals(rsServerName, serverName);
|
||||||
// moveFrom adds to dead servers and adds it to processing list only we will
|
// moveFrom adds to dead servers and adds it to processing list only we will
|
||||||
// not be processing this server 'normally'. Remove it from processing by
|
// not be processing this server 'normally'. Remove it from processing by
|
||||||
|
@ -117,18 +151,16 @@ public class TestHBCKSCP extends TestSCPBase {
|
||||||
// Kill the server. Nothing should happen since an 'Unknown Server' as far
|
// Kill the server. Nothing should happen since an 'Unknown Server' as far
|
||||||
// as the Master is concerned; i.e. no SCP.
|
// as the Master is concerned; i.e. no SCP.
|
||||||
HRegionServer hrs = cluster.getRegionServer(rsServerName);
|
HRegionServer hrs = cluster.getRegionServer(rsServerName);
|
||||||
while (!hrs.isStopped()) {
|
util.waitFor(TimeUnit.MINUTES.toMillis(1), hrs::isStopped);
|
||||||
Threads.sleep(10);
|
|
||||||
}
|
|
||||||
LOG.info("Dead {}", rsServerName);
|
LOG.info("Dead {}", rsServerName);
|
||||||
// Now assert still references in hbase:meta to the 'dead' server -- they haven't been
|
// Now assert still references in hbase:meta to the 'dead' server -- they haven't been
|
||||||
// cleaned up by an SCP or by anything else.
|
// cleaned up by an SCP or by anything else.
|
||||||
assertTrue(searchMeta(master, rsServerName));
|
assertTrue(searchMeta(master, rsServerName));
|
||||||
// Assert region is OPEN on dead server still.
|
// Assert region is OPEN on dead server still.
|
||||||
r = MetaTableAccessor.getRegionResult(master.getConnection(), rsRI.getRegionName());
|
r = MetaTableAccessor.getRegionResult(master.getConnection(), rsRI);
|
||||||
assertEquals(RegionState.State.OPEN.toString(),
|
assertEquals(RegionState.State.OPEN.toString(), Bytes.toString(
|
||||||
Bytes.toString(r.getValue(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER)));
|
r.getValue(HConstants.CATALOG_FAMILY, MetaTableAccessor.getRegionStateColumn(replicaId))));
|
||||||
serverName = MetaTableAccessor.getServerName(r, 0);
|
serverName = MetaTableAccessor.getServerName(r, replicaId);
|
||||||
assertNotNull(cluster.getRegionServer(serverName));
|
assertNotNull(cluster.getRegionServer(serverName));
|
||||||
assertEquals(rsServerName, serverName);
|
assertEquals(rsServerName, serverName);
|
||||||
|
|
||||||
|
@ -136,13 +168,11 @@ public class TestHBCKSCP extends TestSCPBase {
|
||||||
// with no corresponding SCP. Queue one.
|
// with no corresponding SCP. Queue one.
|
||||||
long pid = scheduleHBCKSCP(rsServerName, master);
|
long pid = scheduleHBCKSCP(rsServerName, master);
|
||||||
assertNotEquals(Procedure.NO_PROC_ID, pid);
|
assertNotEquals(Procedure.NO_PROC_ID, pid);
|
||||||
while (master.getMasterProcedureExecutor().getActiveProcIds().contains(pid)) {
|
ProcedureTestingUtility.waitProcedure(master.getMasterProcedureExecutor(), pid);
|
||||||
Threads.sleep(10);
|
|
||||||
}
|
|
||||||
// After SCP, assert region is OPEN on new server.
|
// After SCP, assert region is OPEN on new server.
|
||||||
r = MetaTableAccessor.getRegionResult(master.getConnection(), rsRI.getRegionName());
|
r = MetaTableAccessor.getRegionResult(master.getConnection(), rsRI);
|
||||||
assertEquals(RegionState.State.OPEN.toString(),
|
assertEquals(RegionState.State.OPEN.toString(), Bytes.toString(
|
||||||
Bytes.toString(r.getValue(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER)));
|
r.getValue(HConstants.CATALOG_FAMILY, MetaTableAccessor.getRegionStateColumn(replicaId))));
|
||||||
serverName = MetaTableAccessor.getServerName(r, 0);
|
serverName = MetaTableAccessor.getServerName(r, 0);
|
||||||
assertNotNull(cluster.getRegionServer(serverName));
|
assertNotNull(cluster.getRegionServer(serverName));
|
||||||
assertNotEquals(rsServerName, serverName);
|
assertNotEquals(rsServerName, serverName);
|
||||||
|
@ -151,12 +181,12 @@ public class TestHBCKSCP extends TestSCPBase {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected long scheduleHBCKSCP(ServerName rsServerName, HMaster master) throws ServiceException {
|
protected long scheduleHBCKSCP(ServerName rsServerName, HMaster master) throws ServiceException {
|
||||||
MasterProtos.ScheduleServerCrashProcedureResponse response = master.getMasterRpcServices()
|
return hbckscpScheduler.scheduleHBCKSCP(rsServerName, master);
|
||||||
.scheduleServerCrashProcedure(null, MasterProtos.ScheduleServerCrashProcedureRequest
|
}
|
||||||
.newBuilder().addServerName(ProtobufUtil.toServerName(rsServerName)).build());
|
|
||||||
assertEquals(1, response.getPidCount());
|
@Override
|
||||||
long pid = response.getPid(0);
|
protected int getRegionReplication() {
|
||||||
return pid;
|
return replicas;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns True if we find reference to <code>sn</code> in meta table. */
|
/** Returns True if we find reference to <code>sn</code> in meta table. */
|
||||||
|
@ -170,4 +200,90 @@ public class TestHBCKSCP extends TestSCPBase {
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Encapsulates the choice of which HBCK2 method to call.
|
||||||
|
*/
|
||||||
|
private abstract static class HBCKSCPScheduler {
|
||||||
|
abstract long scheduleHBCKSCP(ServerName rsServerName, HMaster master) throws ServiceException;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return this.getClass().getSimpleName();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Invokes {@code MasterRpcServices#scheduleServerCrashProcedure}.
|
||||||
|
*/
|
||||||
|
private static class ScheduleServerCrashProcedure extends HBCKSCPScheduler {
|
||||||
|
@Override
|
||||||
|
public long scheduleHBCKSCP(ServerName rsServerName, HMaster master) throws ServiceException {
|
||||||
|
MasterProtos.ScheduleServerCrashProcedureResponse response = master.getMasterRpcServices()
|
||||||
|
.scheduleServerCrashProcedure(null, MasterProtos.ScheduleServerCrashProcedureRequest
|
||||||
|
.newBuilder().addServerName(ProtobufUtil.toServerName(rsServerName)).build());
|
||||||
|
assertEquals(1, response.getPidCount());
|
||||||
|
return response.getPid(0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Invokes {@code MasterRpcServices#scheduleSCPsForUnknownServers}.
|
||||||
|
*/
|
||||||
|
private static class ScheduleSCPsForUnknownServers extends HBCKSCPScheduler {
|
||||||
|
@Override
|
||||||
|
long scheduleHBCKSCP(ServerName rsServerName, HMaster master) throws ServiceException {
|
||||||
|
MasterProtos.ScheduleSCPsForUnknownServersResponse response =
|
||||||
|
master.getMasterRpcServices().scheduleSCPsForUnknownServers(null,
|
||||||
|
MasterProtos.ScheduleSCPsForUnknownServersRequest.newBuilder().build());
|
||||||
|
assertEquals(1, response.getPidCount());
|
||||||
|
return response.getPid(0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Encapsulates how the target region is selected.
|
||||||
|
*/
|
||||||
|
private static abstract class RegionSelector {
|
||||||
|
abstract boolean regionFilter(RegionInfo info);
|
||||||
|
|
||||||
|
abstract Exception regionFilterFailure();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return this.getClass().getSimpleName();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Selects a non-meta region that is also a primary region.
|
||||||
|
*/
|
||||||
|
private static class PrimaryNotMetaRegionSelector extends RegionSelector {
|
||||||
|
@Override
|
||||||
|
boolean regionFilter(final RegionInfo info) {
|
||||||
|
return !Objects.equals(TableName.META_TABLE_NAME, info.getTable())
|
||||||
|
&& Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
Exception regionFilterFailure() {
|
||||||
|
return new NoSuchElementException("Cannot locate a primary, non-meta region.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Selects a non-meta region that is also a replica region.
|
||||||
|
*/
|
||||||
|
private static class ReplicaNonMetaRegionSelector extends RegionSelector {
|
||||||
|
@Override
|
||||||
|
boolean regionFilter(RegionInfo info) {
|
||||||
|
return !Objects.equals(TableName.META_TABLE_NAME, info.getTable())
|
||||||
|
&& !Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
Exception regionFilterFailure() {
|
||||||
|
return new NoSuchElementException("Cannot locate a replica, non-meta region.");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,54 +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 static org.junit.Assert.assertEquals;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
|
||||||
import org.apache.hadoop.hbase.master.HMaster;
|
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
|
||||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
|
||||||
import org.junit.ClassRule;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
|
|
||||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test of the HBCK-version of SCP. The HBCKSCP is an SCP only it reads hbase:meta for list of
|
|
||||||
* Regions that were on the server-to-process rather than consult Master in-memory-state.
|
|
||||||
*/
|
|
||||||
@Category({ MasterTests.class, LargeTests.class })
|
|
||||||
public class TestHBCKSCPUnknown extends TestHBCKSCP {
|
|
||||||
|
|
||||||
@ClassRule
|
|
||||||
public static final HBaseClassTestRule CLASS_RULE =
|
|
||||||
HBaseClassTestRule.forClass(TestHBCKSCPUnknown.class);
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected long scheduleHBCKSCP(ServerName rsServerName, HMaster master) throws ServiceException {
|
|
||||||
MasterProtos.ScheduleSCPsForUnknownServersResponse response =
|
|
||||||
master.getMasterRpcServices().scheduleSCPsForUnknownServers(null,
|
|
||||||
MasterProtos.ScheduleSCPsForUnknownServersRequest.newBuilder().build());
|
|
||||||
assertEquals(1, response.getPidCount());
|
|
||||||
long pid = response.getPid(0);
|
|
||||||
return pid;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -224,7 +224,7 @@ public class TestRegionMergeTransactionOnCluster {
|
||||||
RegionInfo mergedRegionInfo = tableRegions.get(0).getFirst();
|
RegionInfo mergedRegionInfo = tableRegions.get(0).getFirst();
|
||||||
TableDescriptor tableDescriptor = MASTER.getTableDescriptors().get(tableName);
|
TableDescriptor tableDescriptor = MASTER.getTableDescriptors().get(tableName);
|
||||||
Result mergedRegionResult =
|
Result mergedRegionResult =
|
||||||
MetaTableAccessor.getRegionResult(MASTER.getConnection(), mergedRegionInfo.getRegionName());
|
MetaTableAccessor.getRegionResult(MASTER.getConnection(), mergedRegionInfo);
|
||||||
|
|
||||||
// contains merge reference in META
|
// contains merge reference in META
|
||||||
assertTrue(MetaTableAccessor.hasMergeRegions(mergedRegionResult.rawCells()));
|
assertTrue(MetaTableAccessor.hasMergeRegions(mergedRegionResult.rawCells()));
|
||||||
|
@ -299,8 +299,8 @@ public class TestRegionMergeTransactionOnCluster {
|
||||||
|
|
||||||
// Wait around a bit to give stuff a chance to complete.
|
// Wait around a bit to give stuff a chance to complete.
|
||||||
while (true) {
|
while (true) {
|
||||||
mergedRegionResult = MetaTableAccessor.getRegionResult(TEST_UTIL.getConnection(),
|
mergedRegionResult =
|
||||||
mergedRegionInfo.getRegionName());
|
MetaTableAccessor.getRegionResult(TEST_UTIL.getConnection(), mergedRegionInfo);
|
||||||
if (MetaTableAccessor.hasMergeRegions(mergedRegionResult.rawCells())) {
|
if (MetaTableAccessor.hasMergeRegions(mergedRegionResult.rawCells())) {
|
||||||
LOG.info("Waiting on cleanup of merge columns {}",
|
LOG.info("Waiting on cleanup of merge columns {}",
|
||||||
Arrays.asList(mergedRegionResult.rawCells()).stream().map(c -> c.toString())
|
Arrays.asList(mergedRegionResult.rawCells()).stream().map(c -> c.toString())
|
||||||
|
|
Loading…
Reference in New Issue