Revert "HBASE-21965 Fix failed split and merge transactions that have failed to roll back"
This reverts commit 7e3ca7063c
.
This commit is contained in:
parent
c144c814b0
commit
386703811d
|
@ -19,12 +19,10 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
|
@ -174,18 +172,4 @@ public class HBaseHbck implements Hbck {
|
|||
throw new IOException(se);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, MasterProtos.REGION_ERROR_TYPE>
|
||||
getFailedSplitMergeLegacyRegions(List<TableName> tableNames) throws IOException {
|
||||
try {
|
||||
MasterProtos.GetFailedSplitMergeLegacyRegionsResponse response =
|
||||
this.hbck.getFailedSplitMergeLegacyRegions(rpcControllerFactory.newController(),
|
||||
RequestConverter.toGetFailedSplitMergeLegacyRegionsRequest(tableNames));
|
||||
return response.getErrorsMap();
|
||||
} catch (ServiceException se) {
|
||||
LOG.debug("get failed split/merge legacy regions failed", se);
|
||||
throw new IOException(se);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,15 +20,12 @@ package org.apache.hadoop.hbase.client;
|
|||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
|
||||
|
||||
/**
|
||||
* Hbck fixup tool APIs. Obtain an instance from {@link ClusterConnection#getHbck()} and call
|
||||
|
@ -109,16 +106,4 @@ public interface Hbck extends Abortable, Closeable {
|
|||
|
||||
List<Long> scheduleServerCrashProcedure(List<HBaseProtos.ServerName> serverNames)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* This method is to get the regions which left by failed split/merge procedures for a certain
|
||||
* table. There are two kinds of region this method will return. One is orphan regions left on FS,
|
||||
* which left because split/merge procedure crashed before updating meta. And the other one is
|
||||
* unassigned split daughter region or merged region, which left because split/merge procedure
|
||||
* crashed before assignment.
|
||||
* @param tableName table to check
|
||||
* @return Map of problematic regions
|
||||
*/
|
||||
Map<String, MasterProtos.REGION_ERROR_TYPE>
|
||||
getFailedSplitMergeLegacyRegions(List<TableName> tableName) throws IOException;
|
||||
}
|
||||
|
|
|
@ -1913,15 +1913,6 @@ public final class RequestConverter {
|
|||
return b.addAllServerName(serverNames).build();
|
||||
}
|
||||
|
||||
public static MasterProtos.GetFailedSplitMergeLegacyRegionsRequest
|
||||
toGetFailedSplitMergeLegacyRegionsRequest(List<TableName> tableNames) {
|
||||
MasterProtos.GetFailedSplitMergeLegacyRegionsRequest.Builder b =
|
||||
MasterProtos.GetFailedSplitMergeLegacyRegionsRequest.newBuilder();
|
||||
List<HBaseProtos.TableName> protoTableNames = tableNames.stream()
|
||||
.map(tableName -> ProtobufUtil.toProtoTableName(tableName)).collect(Collectors.toList());
|
||||
return b.addAllTable(protoTableNames).build();
|
||||
}
|
||||
|
||||
private static List<RegionSpecifier> toEncodedRegionNameRegionSpecifiers(
|
||||
List<String> encodedRegionNames) {
|
||||
return encodedRegionNames.stream().
|
||||
|
|
|
@ -1093,19 +1093,6 @@ message ScheduleServerCrashProcedureResponse {
|
|||
repeated uint64 pid = 1;
|
||||
}
|
||||
|
||||
message GetFailedSplitMergeLegacyRegionsRequest {
|
||||
repeated TableName table = 1;
|
||||
}
|
||||
|
||||
enum REGION_ERROR_TYPE {
|
||||
daughter_merged_region_not_online = 0;
|
||||
orphan_region_on_fs = 1;
|
||||
}
|
||||
|
||||
message GetFailedSplitMergeLegacyRegionsResponse {
|
||||
map<string, REGION_ERROR_TYPE> errors = 1;
|
||||
}
|
||||
|
||||
service HbckService {
|
||||
/** Update state of the table in meta only*/
|
||||
rpc SetTableStateInMeta(SetTableStateInMetaRequest)
|
||||
|
@ -1136,7 +1123,4 @@ service HbckService {
|
|||
/** Schedule a ServerCrashProcedure to help recover a crash server */
|
||||
rpc ScheduleServerCrashProcedure(ScheduleServerCrashProcedureRequest)
|
||||
returns(ScheduleServerCrashProcedureResponse);
|
||||
|
||||
rpc getFailedSplitMergeLegacyRegions(GetFailedSplitMergeLegacyRegionsRequest)
|
||||
returns(GetFailedSplitMergeLegacyRegionsResponse);
|
||||
}
|
||||
|
|
|
@ -27,14 +27,12 @@ import java.net.InetSocketAddress;
|
|||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
|
@ -51,7 +49,6 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
|
@ -69,10 +66,7 @@ import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
|
|||
import org.apache.hadoop.hbase.ipc.RpcServerFactory;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStates;
|
||||
import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
|
||||
import org.apache.hadoop.hbase.master.locking.LockProcedure;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
|
||||
|
@ -92,7 +86,6 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
|
|||
import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaUtil;
|
||||
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
||||
import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationException;
|
||||
|
@ -110,10 +103,8 @@ import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
|||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.PairOfSameType;
|
||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -2503,164 +2494,6 @@ public class MasterRpcServices extends RSRpcServices
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public MasterProtos.GetFailedSplitMergeLegacyRegionsResponse getFailedSplitMergeLegacyRegions(
|
||||
RpcController controller, MasterProtos.GetFailedSplitMergeLegacyRegionsRequest request)
|
||||
throws ServiceException {
|
||||
List<HBaseProtos.TableName> tables = request.getTableList();
|
||||
|
||||
Map<String, MasterProtos.REGION_ERROR_TYPE> errorRegions = new HashMap<>();
|
||||
try {
|
||||
for (HBaseProtos.TableName tableName : tables) {
|
||||
errorRegions.putAll(getFailedSplitMergeLegacyRegions(ProtobufUtil.toTableName(tableName)));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
return MasterProtos.GetFailedSplitMergeLegacyRegionsResponse.newBuilder()
|
||||
.putAllErrors(errorRegions).build();
|
||||
}
|
||||
|
||||
private Map<String, MasterProtos.REGION_ERROR_TYPE>
|
||||
getFailedSplitMergeLegacyRegions(TableName tableName) throws IOException {
|
||||
if (!MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
|
||||
throw new IOException("table " + tableName.getNameAsString() + " doesn't exist");
|
||||
}
|
||||
if (!MetaTableAccessor.getTableState(master.getConnection(), tableName).isEnabled()) {
|
||||
throw new IOException(
|
||||
"table " + tableName.getNameAsString() + " is not enabled yet");
|
||||
}
|
||||
final Map<String, MasterProtos.REGION_ERROR_TYPE> problemRegions = new HashMap<>();
|
||||
|
||||
// Case 1. find orphan region on fs
|
||||
// orphan regions may due to a failed split region procedure, which daughter regions are created
|
||||
// then the procedure is aborted. Or merged region is created then the procedure is aborted.
|
||||
List<String> orphanRegions = findOrphanRegionOnFS(tableName);
|
||||
orphanRegions.stream().forEach(
|
||||
region -> problemRegions.put(region, MasterProtos.REGION_ERROR_TYPE.orphan_region_on_fs));
|
||||
|
||||
// Case 2. find unassigned daughter regions or merged regions
|
||||
List<String> unassignedDaughterOrMergedRegions =
|
||||
findUnassignedDaughterOrMergedRegions(tableName);
|
||||
unassignedDaughterOrMergedRegions.stream().forEach(region -> problemRegions.put(region,
|
||||
MasterProtos.REGION_ERROR_TYPE.daughter_merged_region_not_online));
|
||||
|
||||
// if these regions in problemRegions are currently handled by SplitTableRegionProcedure or
|
||||
// MergeTableRegionsProcedure, we should remove them from this map
|
||||
master.getProcedures().stream().filter(p -> !(p.isFinished() || p.isBypass())).forEach(p -> {
|
||||
if (p instanceof SplitTableRegionProcedure) {
|
||||
problemRegions
|
||||
.remove(((SplitTableRegionProcedure) p).getDaughterOneRI().getRegionNameAsString());
|
||||
problemRegions
|
||||
.remove(((SplitTableRegionProcedure) p).getDaughterTwoRI().getRegionNameAsString());
|
||||
} else if (p instanceof MergeTableRegionsProcedure) {
|
||||
problemRegions
|
||||
.remove(((MergeTableRegionsProcedure) p).getMergedRegion().getRegionNameAsString());
|
||||
}
|
||||
});
|
||||
|
||||
// check if regions are still problematic now
|
||||
checkRegionStillProblematic(problemRegions, tableName);
|
||||
return problemRegions;
|
||||
}
|
||||
|
||||
|
||||
private void checkRegionStillProblematic(
|
||||
Map<String, MasterProtos.REGION_ERROR_TYPE> problemRegions, TableName tableName)
|
||||
throws IOException {
|
||||
Iterator<Map.Entry<String, MasterProtos.REGION_ERROR_TYPE>> iterator =
|
||||
problemRegions.entrySet().iterator();
|
||||
while (iterator.hasNext()) {
|
||||
Map.Entry<String, MasterProtos.REGION_ERROR_TYPE> entry = iterator.next();
|
||||
Result r = MetaTableAccessor.getRegionResult(master.getConnection(),
|
||||
Bytes.toBytesBinary(entry.getKey()));
|
||||
RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(r);
|
||||
switch (entry.getValue()) {
|
||||
case orphan_region_on_fs:
|
||||
// region is build for this directory, it is not a problematic region any more
|
||||
if (r != null) {
|
||||
problemRegions.remove(regionInfo.getRegionNameAsString());
|
||||
}
|
||||
break;
|
||||
case daughter_merged_region_not_online:
|
||||
RegionState.State state = RegionStateStore.getRegionState(r, 0);
|
||||
if (!state.matches(RegionState.State.CLOSED, RegionState.State.SPLITTING_NEW,
|
||||
RegionState.State.MERGED)) {
|
||||
problemRegions.remove(regionInfo.getRegionNameAsString());
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new IOException("there should be no problematic region of this type");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private List<String> findUnassignedDaughterOrMergedRegions(TableName tableName)
|
||||
throws IOException {
|
||||
Set<String> checkRegions = new HashSet<>();
|
||||
Map<String, RegionState.State> regionStates = new HashMap<>();
|
||||
Map<String, RegionInfo> regionInfos = new HashMap<>();
|
||||
|
||||
MetaTableAccessor.scanMeta(master.getConnection(), tableName,
|
||||
MetaTableAccessor.QueryType.REGION, Integer.MAX_VALUE, r -> {
|
||||
RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(r);
|
||||
regionInfos.put(regionInfo.getRegionNameAsString(), regionInfo);
|
||||
RegionState.State state = RegionStateStore.getRegionState(r, 0);
|
||||
regionStates.put(regionInfo.getEncodedName(), state);
|
||||
if (regionInfo.isSplitParent()) {
|
||||
PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(r);
|
||||
checkRegions.add(daughters.getFirst().getRegionNameAsString());
|
||||
checkRegions.add(daughters.getSecond().getRegionNameAsString());
|
||||
} else if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null
|
||||
|| r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER) != null) {
|
||||
checkRegions.add(regionInfo.getRegionNameAsString());
|
||||
}
|
||||
return true;
|
||||
});
|
||||
|
||||
// find unassigned merged or split daughter region
|
||||
return checkRegions.stream().map(regionName -> regionInfos.get(regionName))
|
||||
.filter(regionInfo -> !regionInfo.isSplitParent())
|
||||
.filter(regionInfo -> !regionStates.get(regionInfo.getEncodedName())
|
||||
.matches(RegionState.State.OPEN))
|
||||
.map(regionInfo -> regionInfo.getRegionNameAsString()).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private List<String> findOrphanRegionOnFS(TableName tableName) throws IOException {
|
||||
// get available regions from meta, merged region should be consider available
|
||||
HashSet<String> regionsInMeta = new HashSet<>();
|
||||
MetaTableAccessor.scanMeta(master.getConnection(), tableName,
|
||||
MetaTableAccessor.QueryType.REGION, Integer.MAX_VALUE, r -> {
|
||||
RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(r);
|
||||
regionsInMeta.add(regionInfo.getEncodedName());
|
||||
if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null
|
||||
|| r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER) != null) {
|
||||
PairOfSameType<RegionInfo> mergedRegions = MetaTableAccessor.getMergeRegions(r);
|
||||
regionsInMeta.add(mergedRegions.getFirst().getEncodedName());
|
||||
regionsInMeta.add(mergedRegions.getSecond().getEncodedName());
|
||||
}
|
||||
return true;
|
||||
});
|
||||
// get regionInfo from fs
|
||||
Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
|
||||
FileStatus[] regions =
|
||||
master.getFileSystem().listStatus(tableDir, path -> !path.getName().startsWith("."));
|
||||
HashMap<String, String> regionNames = new HashMap<>();
|
||||
for (FileStatus region : regions) {
|
||||
RegionInfo regionInfo =
|
||||
HRegionFileSystem.loadRegionInfoFileContent(master.getFileSystem(), region.getPath());
|
||||
regionNames.put(regionInfo.getEncodedName(), regionInfo.getRegionNameAsString());
|
||||
}
|
||||
Iterator<Map.Entry<String, String>> regionIterator = regionNames.entrySet().iterator();
|
||||
while (regionIterator.hasNext()) {
|
||||
Map.Entry<String, String> region = regionIterator.next();
|
||||
if (regionsInMeta.contains(region.getKey())) {
|
||||
regionIterator.remove();
|
||||
}
|
||||
}
|
||||
return new ArrayList<>(regionNames.values());
|
||||
}
|
||||
|
||||
@Override
|
||||
public SwitchRpcThrottleResponse switchRpcThrottle(RpcController controller,
|
||||
SwitchRpcThrottleRequest request) throws ServiceException {
|
||||
|
|
|
@ -93,8 +93,8 @@ public class SplitTableRegionProcedure
|
|||
extends AbstractStateMachineRegionProcedure<SplitTableRegionState> {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(SplitTableRegionProcedure.class);
|
||||
private Boolean traceEnabled = null;
|
||||
private RegionInfo daughterOneRI;
|
||||
private RegionInfo daughterTwoRI;
|
||||
private RegionInfo daughter_1_RI;
|
||||
private RegionInfo daughter_2_RI;
|
||||
private byte[] bestSplitRow;
|
||||
private RegionSplitPolicy splitPolicy;
|
||||
|
||||
|
@ -113,13 +113,13 @@ public class SplitTableRegionProcedure
|
|||
checkSplittable(env, regionToSplit, bestSplitRow);
|
||||
final TableName table = regionToSplit.getTable();
|
||||
final long rid = getDaughterRegionIdTimestamp(regionToSplit);
|
||||
this.daughterOneRI = RegionInfoBuilder.newBuilder(table)
|
||||
this.daughter_1_RI = RegionInfoBuilder.newBuilder(table)
|
||||
.setStartKey(regionToSplit.getStartKey())
|
||||
.setEndKey(bestSplitRow)
|
||||
.setSplit(false)
|
||||
.setRegionId(rid)
|
||||
.build();
|
||||
this.daughterTwoRI = RegionInfoBuilder.newBuilder(table)
|
||||
this.daughter_2_RI = RegionInfoBuilder.newBuilder(table)
|
||||
.setStartKey(bestSplitRow)
|
||||
.setEndKey(regionToSplit.getEndKey())
|
||||
.setSplit(false)
|
||||
|
@ -140,7 +140,7 @@ public class SplitTableRegionProcedure
|
|||
@Override
|
||||
protected LockState acquireLock(final MasterProcedureEnv env) {
|
||||
if (env.getProcedureScheduler().waitRegions(this, getTableName(), getParentRegion(),
|
||||
daughterOneRI, daughterTwoRI)) {
|
||||
daughter_1_RI, daughter_2_RI)) {
|
||||
try {
|
||||
LOG.debug(LockState.LOCK_EVENT_WAIT + " " + env.getProcedureScheduler().dumpLocks());
|
||||
} catch (IOException e) {
|
||||
|
@ -153,18 +153,8 @@ public class SplitTableRegionProcedure
|
|||
|
||||
@Override
|
||||
protected void releaseLock(final MasterProcedureEnv env) {
|
||||
env.getProcedureScheduler().wakeRegions(this, getTableName(), getParentRegion(), daughterOneRI,
|
||||
daughterTwoRI);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public RegionInfo getDaughterOneRI() {
|
||||
return daughterOneRI;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public RegionInfo getDaughterTwoRI() {
|
||||
return daughterTwoRI;
|
||||
env.getProcedureScheduler().wakeRegions(this, getTableName(), getParentRegion(), daughter_1_RI,
|
||||
daughter_2_RI);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -426,8 +416,8 @@ public class SplitTableRegionProcedure
|
|||
MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
|
||||
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
|
||||
.setParentRegionInfo(ProtobufUtil.toRegionInfo(getRegion()))
|
||||
.addChildRegionInfo(ProtobufUtil.toRegionInfo(daughterOneRI))
|
||||
.addChildRegionInfo(ProtobufUtil.toRegionInfo(daughterTwoRI));
|
||||
.addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_1_RI))
|
||||
.addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_2_RI));
|
||||
serializer.serialize(splitTableRegionMsg.build());
|
||||
}
|
||||
|
||||
|
@ -441,8 +431,8 @@ public class SplitTableRegionProcedure
|
|||
setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo()));
|
||||
setRegion(ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getParentRegionInfo()));
|
||||
assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
|
||||
daughterOneRI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(0));
|
||||
daughterTwoRI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(1));
|
||||
daughter_1_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(0));
|
||||
daughter_2_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(1));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -453,9 +443,9 @@ public class SplitTableRegionProcedure
|
|||
sb.append(", parent=");
|
||||
sb.append(getParentRegion().getShortNameToLog());
|
||||
sb.append(", daughterA=");
|
||||
sb.append(daughterOneRI.getShortNameToLog());
|
||||
sb.append(daughter_1_RI.getShortNameToLog());
|
||||
sb.append(", daughterB=");
|
||||
sb.append(daughterTwoRI.getShortNameToLog());
|
||||
sb.append(daughter_2_RI.getShortNameToLog());
|
||||
}
|
||||
|
||||
private RegionInfo getParentRegion() {
|
||||
|
@ -473,7 +463,7 @@ public class SplitTableRegionProcedure
|
|||
}
|
||||
|
||||
private byte[] getSplitRow() {
|
||||
return daughterTwoRI.getStartKey();
|
||||
return daughter_2_RI.getStartKey();
|
||||
}
|
||||
|
||||
private static final State[] EXPECTED_SPLIT_STATES = new State[] { State.OPEN, State.CLOSED };
|
||||
|
@ -605,17 +595,17 @@ public class SplitTableRegionProcedure
|
|||
Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
|
||||
|
||||
assertReferenceFileCount(fs, expectedReferences.getFirst(),
|
||||
regionFs.getSplitsDir(daughterOneRI));
|
||||
regionFs.getSplitsDir(daughter_1_RI));
|
||||
//Move the files from the temporary .splits to the final /table/region directory
|
||||
regionFs.commitDaughterRegion(daughterOneRI);
|
||||
regionFs.commitDaughterRegion(daughter_1_RI);
|
||||
assertReferenceFileCount(fs, expectedReferences.getFirst(),
|
||||
new Path(tabledir, daughterOneRI.getEncodedName()));
|
||||
new Path(tabledir, daughter_1_RI.getEncodedName()));
|
||||
|
||||
assertReferenceFileCount(fs, expectedReferences.getSecond(),
|
||||
regionFs.getSplitsDir(daughterTwoRI));
|
||||
regionFs.commitDaughterRegion(daughterTwoRI);
|
||||
regionFs.getSplitsDir(daughter_2_RI));
|
||||
regionFs.commitDaughterRegion(daughter_2_RI);
|
||||
assertReferenceFileCount(fs, expectedReferences.getSecond(),
|
||||
new Path(tabledir, daughterTwoRI.getEncodedName()));
|
||||
new Path(tabledir, daughter_2_RI.getEncodedName()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -755,9 +745,9 @@ public class SplitTableRegionProcedure
|
|||
|
||||
final byte[] splitRow = getSplitRow();
|
||||
final String familyName = Bytes.toString(family);
|
||||
final Path path_first = regionFs.splitStoreFile(this.daughterOneRI, familyName, sf, splitRow,
|
||||
final Path path_first = regionFs.splitStoreFile(this.daughter_1_RI, familyName, sf, splitRow,
|
||||
false, splitPolicy);
|
||||
final Path path_second = regionFs.splitStoreFile(this.daughterTwoRI, familyName, sf, splitRow,
|
||||
final Path path_second = regionFs.splitStoreFile(this.daughter_2_RI, familyName, sf, splitRow,
|
||||
true, splitPolicy);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("pid=" + getProcId() + " splitting complete for store file: " +
|
||||
|
@ -822,7 +812,7 @@ public class SplitTableRegionProcedure
|
|||
*/
|
||||
private void updateMeta(final MasterProcedureEnv env) throws IOException {
|
||||
env.getAssignmentManager().markRegionAsSplit(getParentRegion(), getParentRegionServerName(env),
|
||||
daughterOneRI, daughterTwoRI);
|
||||
daughter_1_RI, daughter_2_RI);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -844,7 +834,7 @@ public class SplitTableRegionProcedure
|
|||
private void postSplitRegion(final MasterProcedureEnv env) throws IOException {
|
||||
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
|
||||
if (cpHost != null) {
|
||||
cpHost.postCompletedSplitRegionAction(daughterOneRI, daughterTwoRI, getUser());
|
||||
cpHost.postCompletedSplitRegionAction(daughter_1_RI, daughter_2_RI, getUser());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -862,8 +852,8 @@ public class SplitTableRegionProcedure
|
|||
private TransitRegionStateProcedure[] createAssignProcedures(MasterProcedureEnv env)
|
||||
throws IOException {
|
||||
List<RegionInfo> hris = new ArrayList<RegionInfo>(2);
|
||||
hris.add(daughterOneRI);
|
||||
hris.add(daughterTwoRI);
|
||||
hris.add(daughter_1_RI);
|
||||
hris.add(daughter_2_RI);
|
||||
return AssignmentManagerUtil.createAssignProceduresForOpeningNewRegions(env, hris,
|
||||
getRegionReplication(env), getParentRegionServerName(env));
|
||||
}
|
||||
|
@ -878,9 +868,9 @@ public class SplitTableRegionProcedure
|
|||
long maxSequenceId =
|
||||
WALSplitter.getMaxRegionSequenceId(walFS, getWALRegionDir(env, getParentRegion()));
|
||||
if (maxSequenceId > 0) {
|
||||
WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughterOneRI),
|
||||
WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughter_1_RI),
|
||||
maxSequenceId);
|
||||
WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughterTwoRI),
|
||||
WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughter_2_RI),
|
||||
maxSequenceId);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -807,12 +807,6 @@ public class HRegionFileSystem {
|
|||
Path mergedRegionTmpDir = this.getMergesDir(mergedRegionInfo);
|
||||
// Move the tmp dir in the expected location
|
||||
if (mergedRegionTmpDir != null && fs.exists(mergedRegionTmpDir)) {
|
||||
|
||||
// Write HRI to a file in case we need to recover hbase:meta
|
||||
Path regionInfoFile = new Path(mergedRegionTmpDir, REGION_INFO_FILE);
|
||||
byte[] regionInfoContent = getRegionInfoFileContent(regionInfo);
|
||||
writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
|
||||
|
||||
if (!fs.rename(mergedRegionTmpDir, regionDir)) {
|
||||
throw new IOException("Unable to rename " + mergedRegionTmpDir + " to "
|
||||
+ regionDir);
|
||||
|
|
|
@ -18,32 +18,17 @@
|
|||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
|
||||
import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
|
@ -55,7 +40,6 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
|
|||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
|
@ -73,7 +57,6 @@ import org.slf4j.LoggerFactory;
|
|||
import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
|
||||
|
||||
/**
|
||||
* Class to test HBaseHbck. Spins up the minicluster once at test start and then takes it down
|
||||
|
@ -119,12 +102,6 @@ public class TestHbck {
|
|||
TEST_UTIL.createMultiRegionTable(TABLE_NAME, Bytes.toBytes("family1"), 5);
|
||||
procExec = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
|
||||
ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
|
||||
TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessorHost().load(
|
||||
FailingMergeAfterMetaUpdatedMasterObserver.class, Coprocessor.PRIORITY_USER,
|
||||
TEST_UTIL.getHBaseCluster().getMaster().getConfiguration());
|
||||
TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessorHost().load(
|
||||
FailingSplitAfterMetaUpdatedMasterObserver.class, Coprocessor.PRIORITY_USER,
|
||||
TEST_UTIL.getHBaseCluster().getMaster().getConfiguration());
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -227,101 +204,6 @@ public class TestHbck {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecoverMergeAfterMetaUpdated() throws Exception {
|
||||
String testTable = async ? "mergeTestAsync" : "mergeTestSync";
|
||||
TEST_UTIL.createMultiRegionTable(TableName.valueOf(testTable), Bytes.toBytes("family1"), 5);
|
||||
TEST_UTIL.loadTable(TEST_UTIL.getConnection().getTable(TableName.valueOf(testTable)),
|
||||
Bytes.toBytes("family1"), true);
|
||||
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
|
||||
Hbck hbck = getHbck();
|
||||
FailingMergeAfterMetaUpdatedMasterObserver observer = master.getMasterCoprocessorHost()
|
||||
.findCoprocessor(FailingMergeAfterMetaUpdatedMasterObserver.class);
|
||||
try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
|
||||
List<RegionInfo> regions = admin.getRegions(TableName.valueOf(testTable));
|
||||
admin.mergeRegionsAsync(regions.get(0).getRegionName(), regions.get(1).getRegionName(), true);
|
||||
assertNotNull(observer);
|
||||
observer.latch.await(5000, TimeUnit.MILLISECONDS);
|
||||
Map<String, MasterProtos.REGION_ERROR_TYPE> result =
|
||||
hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
|
||||
Assert.assertEquals(0, result.size());
|
||||
Optional<Procedure<?>> procedure = TEST_UTIL.getHBaseCluster().getMaster().getProcedures()
|
||||
.stream().filter(p -> p instanceof MergeTableRegionsProcedure).findAny();
|
||||
Assert.assertTrue(procedure.isPresent());
|
||||
hbck.bypassProcedure(Arrays.asList(procedure.get().getProcId()), 5, true, false);
|
||||
result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
|
||||
Assert.assertEquals(1, result.size());
|
||||
hbck.assigns(Arrays.asList(result.keySet().toArray(new String[0])).stream()
|
||||
.map(regionName -> regionName.split("\\.")[1]).collect(Collectors.toList()));
|
||||
ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
|
||||
// now the state should be fixed
|
||||
result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
|
||||
Assert.assertEquals(0, result.size());
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException(ie);
|
||||
} finally {
|
||||
observer.resetLatch();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecoverSplitAfterMetaUpdated() throws Exception {
|
||||
String testTable = async ? "splitTestAsync" : "splitTestSync";
|
||||
TEST_UTIL.createMultiRegionTable(TableName.valueOf(testTable), Bytes.toBytes("family1"), 5);
|
||||
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
|
||||
Hbck hbck = getHbck();
|
||||
FailingSplitAfterMetaUpdatedMasterObserver observer = master.getMasterCoprocessorHost()
|
||||
.findCoprocessor(FailingSplitAfterMetaUpdatedMasterObserver.class);
|
||||
assertNotNull(observer);
|
||||
try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
|
||||
byte[] splitKey = Bytes.toBytes("bcd");
|
||||
admin.split(TableName.valueOf(testTable), splitKey);
|
||||
observer.latch.await(5000, TimeUnit.MILLISECONDS);
|
||||
Map<String, MasterProtos.REGION_ERROR_TYPE> result =
|
||||
hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
|
||||
// since there is a split procedure work on the region, thus this check should return a empty
|
||||
// map.
|
||||
Assert.assertEquals(0, result.size());
|
||||
Optional<Procedure<?>> procedure = TEST_UTIL.getHBaseCluster().getMaster().getProcedures()
|
||||
.stream().filter(p -> p instanceof SplitTableRegionProcedure).findAny();
|
||||
Assert.assertTrue(procedure.isPresent());
|
||||
hbck.bypassProcedure(Arrays.asList(procedure.get().getProcId()), 5, true, false);
|
||||
result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
|
||||
Assert.assertEquals(2, result.size());
|
||||
hbck.assigns(Arrays.asList(result.keySet().toArray(new String[0])).stream()
|
||||
.map(regionName -> regionName.split("\\.")[1]).collect(Collectors.toList()));
|
||||
ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
|
||||
// now the state should be fixed
|
||||
result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
|
||||
Assert.assertEquals(0, result.size());
|
||||
|
||||
//split one of the daughter region again
|
||||
observer.resetLatch();
|
||||
byte[] splitKey2 = Bytes.toBytes("bcde");
|
||||
|
||||
admin.split(TableName.valueOf(testTable), splitKey2);
|
||||
observer.latch.await(5000, TimeUnit.MILLISECONDS);
|
||||
|
||||
procedure = TEST_UTIL.getHBaseCluster().getMaster().getProcedures()
|
||||
.stream().filter(p -> p instanceof SplitTableRegionProcedure).findAny();
|
||||
Assert.assertTrue(procedure.isPresent());
|
||||
hbck.bypassProcedure(Arrays.asList(procedure.get().getProcId()), 5, true, false);
|
||||
result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
|
||||
Assert.assertEquals(2, result.size());
|
||||
hbck.assigns(Arrays.asList(result.keySet().toArray(new String[0])).stream()
|
||||
.map(regionName -> regionName.split("\\.")[1]).collect(Collectors.toList()));
|
||||
ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
|
||||
// now the state should be fixed
|
||||
result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
|
||||
Assert.assertEquals(0, result.size());
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException(ie);
|
||||
} finally {
|
||||
observer.resetLatch();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testScheduleSCP() throws Exception {
|
||||
HRegionServer testRs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
|
||||
|
@ -341,60 +223,6 @@ public class TestHbck {
|
|||
waitOnPids(pids);
|
||||
}
|
||||
|
||||
public static class FailingSplitAfterMetaUpdatedMasterObserver
|
||||
implements MasterCoprocessor, MasterObserver {
|
||||
public volatile CountDownLatch latch;
|
||||
|
||||
@Override
|
||||
public void start(CoprocessorEnvironment e) throws IOException {
|
||||
resetLatch();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<MasterObserver> getMasterObserver() {
|
||||
return Optional.of(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preSplitRegionAfterMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
throws IOException {
|
||||
LOG.info("I'm here");
|
||||
latch.countDown();
|
||||
throw new IOException("this procedure will fail at here forever");
|
||||
}
|
||||
|
||||
public void resetLatch() {
|
||||
this.latch = new CountDownLatch(1);
|
||||
}
|
||||
}
|
||||
|
||||
public static class FailingMergeAfterMetaUpdatedMasterObserver
|
||||
implements MasterCoprocessor, MasterObserver {
|
||||
public volatile CountDownLatch latch;
|
||||
|
||||
@Override
|
||||
public void start(CoprocessorEnvironment e) throws IOException {
|
||||
resetLatch();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<MasterObserver> getMasterObserver() {
|
||||
return Optional.of(this);
|
||||
}
|
||||
|
||||
public void resetLatch() {
|
||||
this.latch = new CountDownLatch(1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postMergeRegionsCommitAction(
|
||||
final ObserverContext<MasterCoprocessorEnvironment> ctx, final RegionInfo[] regionsToMerge,
|
||||
final RegionInfo mergedRegion) throws IOException {
|
||||
latch.countDown();
|
||||
throw new IOException("this procedure will fail at here forever");
|
||||
}
|
||||
}
|
||||
|
||||
private void waitOnPids(List<Long> pids) {
|
||||
TEST_UTIL.waitFor(60000, () -> pids.stream().allMatch(procExec::isFinished));
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue