HBASE-26193 Do not store meta region location as permanent state on zookeeper (#3583)
Signed-off-by: stack <stack@apache.org> Signed-off-by: Zach York <zyork@apache.org>
This commit is contained in:
parent
13c5cfa361
commit
d99ca37307
|
@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS
|
|||
import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;
|
||||
import static org.apache.hadoop.hbase.util.DNS.MASTER_HOSTNAME_KEY;
|
||||
|
||||
import com.google.errorprone.annotations.RestrictedApi;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.IOException;
|
||||
|
@ -57,6 +58,9 @@ import org.apache.commons.lang3.StringUtils;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellBuilderFactory;
|
||||
import org.apache.hadoop.hbase.CellBuilderType;
|
||||
import org.apache.hadoop.hbase.ClusterId;
|
||||
import org.apache.hadoop.hbase.ClusterMetrics;
|
||||
import org.apache.hadoop.hbase.ClusterMetrics.Option;
|
||||
|
@ -67,7 +71,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
|
||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||
import org.apache.hadoop.hbase.MetaRegionLocationCache;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.PleaseHoldException;
|
||||
import org.apache.hadoop.hbase.PleaseRestartMasterException;
|
||||
|
@ -83,9 +87,12 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|||
import org.apache.hadoop.hbase.client.CompactionState;
|
||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
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.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.client.RegionStatesCount;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
|
@ -103,6 +110,7 @@ import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
|
|||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStates;
|
||||
import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
|
||||
import org.apache.hadoop.hbase.master.balancer.BalancerChore;
|
||||
|
@ -210,6 +218,7 @@ import org.apache.hadoop.hbase.util.Threads;
|
|||
import org.apache.hadoop.hbase.util.VersionInfo;
|
||||
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.SnapshotCleanupTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
|
@ -380,7 +389,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
|
||||
private ProcedureStore procedureStore;
|
||||
|
||||
// the master local storage to store procedure data, etc.
|
||||
// the master local storage to store procedure data, meta region locations, etc.
|
||||
private MasterRegion masterRegion;
|
||||
|
||||
// handle table states
|
||||
|
@ -737,8 +746,49 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
|
||||
// Will be overriden in test to inject customized AssignmentManager
|
||||
@InterfaceAudience.Private
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManager(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManager(master, masterRegion);
|
||||
}
|
||||
|
||||
private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperException {
|
||||
// try migrate data from zookeeper
|
||||
try (ResultScanner scanner =
|
||||
masterRegion.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {
|
||||
if (scanner.next() != null) {
|
||||
// notice that all replicas for a region are in the same row, so the migration can be
|
||||
// done with in a one row put, which means if we have data in catalog family then we can
|
||||
// make sure that the migration is done.
|
||||
LOG.info("The {} family in master local region already has data in it, skip migrating...",
|
||||
HConstants.CATALOG_FAMILY);
|
||||
return;
|
||||
}
|
||||
}
|
||||
// start migrating
|
||||
byte[] row = MetaTableAccessor.getMetaKeyForRegion(RegionInfoBuilder.FIRST_META_REGIONINFO);
|
||||
Put put = new Put(row);
|
||||
List<String> metaReplicaNodes = zooKeeper.getMetaReplicaNodes();
|
||||
StringBuilder info = new StringBuilder("Migrating meta locations:");
|
||||
for (String metaReplicaNode : metaReplicaNodes) {
|
||||
int replicaId = zooKeeper.getZNodePaths().getMetaReplicaIdFromZNode(metaReplicaNode);
|
||||
RegionState state = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);
|
||||
info.append(" ").append(state);
|
||||
put.setTimestamp(state.getStamp());
|
||||
MetaTableAccessor.addRegionInfo(put, state.getRegion());
|
||||
if (state.getServerName() != null) {
|
||||
MetaTableAccessor.addLocation(put, state.getServerName(), HConstants.NO_SEQNUM, replicaId);
|
||||
}
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||
.setFamily(HConstants.CATALOG_FAMILY)
|
||||
.setQualifier(RegionStateStore.getStateColumn(replicaId)).setTimestamp(put.getTimestamp())
|
||||
.setType(Cell.Type.Put).setValue(Bytes.toBytes(state.getState().name())).build());
|
||||
}
|
||||
if (!put.isEmpty()) {
|
||||
LOG.info(info.toString());
|
||||
masterRegion.update(r -> r.put(put));
|
||||
} else {
|
||||
LOG.info("No meta location available on zookeeper, skip migrating...");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -754,6 +804,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
* region server tracker
|
||||
* <ol type='i'>
|
||||
* <li>Create server manager</li>
|
||||
* <li>Create master local region</li>
|
||||
* <li>Create procedure executor, load the procedures, but do not start workers. We will start it
|
||||
* later after we finish scheduling SCPs to avoid scheduling duplicated SCPs for the same
|
||||
* server</li>
|
||||
|
@ -832,13 +883,16 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
|
||||
// initialize master local region
|
||||
masterRegion = MasterRegionFactory.create(this);
|
||||
|
||||
tryMigrateMetaLocationsFromZooKeeper();
|
||||
|
||||
createProcedureExecutor();
|
||||
Map<Class<?>, List<Procedure<MasterProcedureEnv>>> procsByType =
|
||||
procedureExecutor.getActiveProceduresNoCopy().stream()
|
||||
.collect(Collectors.groupingBy(p -> p.getClass()));
|
||||
|
||||
// Create Assignment Manager
|
||||
this.assignmentManager = createAssignmentManager(this);
|
||||
this.assignmentManager = createAssignmentManager(this, masterRegion);
|
||||
this.assignmentManager.start();
|
||||
// TODO: TRSP can perform as the sub procedure for other procedures, so even if it is marked as
|
||||
// completed, it could still be in the procedure list. This is a bit strange but is another
|
||||
|
@ -3836,4 +3890,10 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
public MetaLocationSyncer getMetaLocationSyncer() {
|
||||
return metaLocationSyncer;
|
||||
}
|
||||
|
||||
@RestrictedApi(explanation = "Should only be called in tests", link = "",
|
||||
allowedOnPath = ".*/src/test/.*")
|
||||
MasterRegion getMasterRegion() {
|
||||
return masterRegion;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,6 +48,9 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
|||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.RegionStatesCount;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
|
||||
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
|
||||
|
@ -66,6 +69,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
|||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
|
||||
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
|
||||
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
|
@ -194,18 +198,21 @@ public class AssignmentManager {
|
|||
private final int assignMaxAttempts;
|
||||
private final int assignRetryImmediatelyMaxAttempts;
|
||||
|
||||
private final MasterRegion masterRegion;
|
||||
|
||||
private final Object checkIfShouldMoveSystemRegionLock = new Object();
|
||||
|
||||
private Thread assignThread;
|
||||
|
||||
public AssignmentManager(final MasterServices master) {
|
||||
this(master, new RegionStateStore(master));
|
||||
public AssignmentManager(MasterServices master, MasterRegion masterRegion) {
|
||||
this(master, masterRegion, new RegionStateStore(master, masterRegion));
|
||||
}
|
||||
|
||||
AssignmentManager(final MasterServices master, final RegionStateStore stateStore) {
|
||||
AssignmentManager(MasterServices master, MasterRegion masterRegion, RegionStateStore stateStore) {
|
||||
this.master = master;
|
||||
this.regionStateStore = stateStore;
|
||||
this.metrics = new MetricsAssignmentManager();
|
||||
this.masterRegion = masterRegion;
|
||||
|
||||
final Configuration conf = master.getConfiguration();
|
||||
|
||||
|
@ -238,6 +245,30 @@ public class AssignmentManager {
|
|||
DEFAULT_MIN_VERSION_MOVE_SYS_TABLES_CONFIG);
|
||||
}
|
||||
|
||||
private void mirrorMetaLocations() throws IOException, KeeperException {
|
||||
// For compatibility, mirror the meta region state to zookeeper
|
||||
// And we still need to use zookeeper to publish the meta region locations to region
|
||||
// server, so they can serve as ClientMetaService
|
||||
ZKWatcher zk = master.getZooKeeper();
|
||||
if (zk == null || !zk.getRecoverableZooKeeper().getState().isAlive()) {
|
||||
// this is possible in tests, we do not provide a zk watcher or the zk watcher has been closed
|
||||
return;
|
||||
}
|
||||
Collection<RegionStateNode> metaStates = regionStates.getRegionStateNodes();
|
||||
for (RegionStateNode metaState : metaStates) {
|
||||
MetaTableLocator.setMetaLocation(zk, metaState.getRegionLocation(),
|
||||
metaState.getRegionInfo().getReplicaId(), metaState.getState());
|
||||
}
|
||||
int replicaCount = metaStates.size();
|
||||
// remove extra mirror locations
|
||||
for (String znode : zk.getMetaReplicaNodes()) {
|
||||
int replicaId = zk.getZNodePaths().getMetaReplicaIdFromZNode(znode);
|
||||
if (replicaId >= replicaCount) {
|
||||
MetaTableLocator.deleteMetaLocation(zk, replicaId);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void start() throws IOException, KeeperException {
|
||||
if (!running.compareAndSet(false, true)) {
|
||||
return;
|
||||
|
@ -247,36 +278,39 @@ public class AssignmentManager {
|
|||
|
||||
// Start the Assignment Thread
|
||||
startAssignmentThread();
|
||||
|
||||
// load meta region state
|
||||
ZKWatcher zkw = master.getZooKeeper();
|
||||
// it could be null in some tests
|
||||
if (zkw == null) {
|
||||
return;
|
||||
}
|
||||
List<String> metaZNodes = zkw.getMetaReplicaNodes();
|
||||
LOG.debug("hbase:meta replica znodes: {}", metaZNodes);
|
||||
for (String metaZNode : metaZNodes) {
|
||||
int replicaId = zkw.getZNodePaths().getMetaReplicaIdFromZNode(metaZNode);
|
||||
// load meta region states.
|
||||
// here we are still in the early steps of active master startup. There is only one thread(us)
|
||||
// can access AssignmentManager and create region node, so here we do not need to lock the
|
||||
// region node.
|
||||
RegionState regionState = MetaTableLocator.getMetaRegionState(zkw, replicaId);
|
||||
RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionState.getRegion());
|
||||
regionNode.setRegionLocation(regionState.getServerName());
|
||||
regionNode.setState(regionState.getState());
|
||||
try (ResultScanner scanner =
|
||||
masterRegion.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {
|
||||
for (;;) {
|
||||
Result result = scanner.next();
|
||||
if (result == null) {
|
||||
break;
|
||||
}
|
||||
RegionStateStore
|
||||
.visitMetaEntry((r, regionInfo, state, regionLocation, lastHost, openSeqNum) -> {
|
||||
RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
|
||||
regionNode.setState(state);
|
||||
regionNode.setLastHost(lastHost);
|
||||
regionNode.setRegionLocation(regionLocation);
|
||||
regionNode.setOpenSeqNum(openSeqNum);
|
||||
if (regionNode.getProcedure() != null) {
|
||||
regionNode.getProcedure().stateLoaded(this, regionNode);
|
||||
}
|
||||
if (regionState.getServerName() != null) {
|
||||
if (regionLocation != null) {
|
||||
regionStates.addRegionToServer(regionNode);
|
||||
}
|
||||
if (RegionReplicaUtil.isDefaultReplica(replicaId)) {
|
||||
setMetaAssigned(regionState.getRegion(), regionState.getState() == State.OPEN);
|
||||
if (RegionReplicaUtil.isDefaultReplica(regionInfo.getReplicaId())) {
|
||||
setMetaAssigned(regionInfo, state == State.OPEN);
|
||||
}
|
||||
LOG.debug("Loaded hbase:meta {}", regionNode);
|
||||
}, result);
|
||||
}
|
||||
}
|
||||
mirrorMetaLocations();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create RegionStateNode based on the TRSP list, and attach the TRSP to the RegionStateNode.
|
||||
|
|
|
@ -44,14 +44,13 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
|
|||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -72,10 +71,14 @@ public class RegionStateStore {
|
|||
|
||||
private final MasterServices master;
|
||||
|
||||
public RegionStateStore(final MasterServices master) {
|
||||
private final MasterRegion masterRegion;
|
||||
|
||||
public RegionStateStore(MasterServices master, MasterRegion masterRegion) {
|
||||
this.master = master;
|
||||
this.masterRegion = masterRegion;
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
public interface RegionStateVisitor {
|
||||
void visitRegionState(Result result, RegionInfo regionInfo, State state,
|
||||
ServerName regionLocation, ServerName lastHost, long openSeqNum);
|
||||
|
@ -121,7 +124,7 @@ public class RegionStateStore {
|
|||
}
|
||||
}
|
||||
|
||||
private void visitMetaEntry(final RegionStateVisitor visitor, final Result result)
|
||||
public static void visitMetaEntry(final RegionStateVisitor visitor, final Result result)
|
||||
throws IOException {
|
||||
final RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
|
||||
if (rl == null) return;
|
||||
|
@ -152,33 +155,14 @@ public class RegionStateStore {
|
|||
}
|
||||
|
||||
void updateRegionLocation(RegionStateNode regionStateNode) throws IOException {
|
||||
if (regionStateNode.getRegionInfo().isMetaRegion()) {
|
||||
updateMetaLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(),
|
||||
regionStateNode.getState());
|
||||
} else {
|
||||
long time = EnvironmentEdgeManager.currentTime();
|
||||
long openSeqNum = regionStateNode.getState() == State.OPEN ? regionStateNode.getOpenSeqNum() :
|
||||
HConstants.NO_SEQNUM;
|
||||
updateUserRegionLocation(regionStateNode.getRegionInfo(), regionStateNode.getState(),
|
||||
regionStateNode.getRegionLocation(), openSeqNum,
|
||||
// The regionStateNode may have no procedure in a test scenario; allow for this.
|
||||
regionStateNode.getProcedure() != null ? regionStateNode.getProcedure().getProcId() :
|
||||
Procedure.NO_PROC_ID);
|
||||
}
|
||||
}
|
||||
|
||||
private void updateMetaLocation(RegionInfo regionInfo, ServerName serverName, State state)
|
||||
throws IOException {
|
||||
try {
|
||||
MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName, regionInfo.getReplicaId(),
|
||||
state);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private void updateUserRegionLocation(RegionInfo regionInfo, State state,
|
||||
ServerName regionLocation, long openSeqNum, long pid) throws IOException {
|
||||
long time = EnvironmentEdgeManager.currentTime();
|
||||
RegionInfo regionInfo = regionStateNode.getRegionInfo();
|
||||
State state = regionStateNode.getState();
|
||||
ServerName regionLocation = regionStateNode.getRegionLocation();
|
||||
TransitRegionStateProcedure rit = regionStateNode.getProcedure();
|
||||
long pid = rit != null ? rit.getProcId() : Procedure.NO_PROC_ID;
|
||||
final int replicaId = regionInfo.getReplicaId();
|
||||
final Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(regionInfo), time);
|
||||
MetaTableAccessor.addRegionInfo(put, regionInfo);
|
||||
|
@ -213,13 +197,43 @@ public class RegionStateStore {
|
|||
.build());
|
||||
LOG.info(info.toString());
|
||||
updateRegionLocation(regionInfo, state, put);
|
||||
if (regionInfo.isMetaRegion() && regionInfo.isFirst()) {
|
||||
// mirror the meta location to zookeeper
|
||||
mirrorMetaLocation(regionInfo, regionLocation, state);
|
||||
}
|
||||
}
|
||||
|
||||
private void mirrorMetaLocation(RegionInfo regionInfo, ServerName serverName, State state)
|
||||
throws IOException {
|
||||
try {
|
||||
MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName, regionInfo.getReplicaId(),
|
||||
state);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private void removeMirrorMetaLocation(int oldReplicaCount, int newReplicaCount)
|
||||
throws IOException {
|
||||
try {
|
||||
for (int i = newReplicaCount; i < oldReplicaCount; i++) {
|
||||
MetaTableLocator.deleteMetaLocation(master.getZooKeeper(), i);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private void updateRegionLocation(RegionInfo regionInfo, State state, Put put)
|
||||
throws IOException {
|
||||
try (Table table = getMetaTable()) {
|
||||
debugLogMutation(put);
|
||||
try {
|
||||
if (regionInfo.isMetaRegion()) {
|
||||
masterRegion.update(r -> r.put(put));
|
||||
} else {
|
||||
try (Table table = master.getConnection().getTable(TableName.META_TABLE_NAME)) {
|
||||
table.put(put);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// TODO: Revist!!!! Means that if a server is loaded, then we will abort our host!
|
||||
// In tests we abort the Master!
|
||||
|
@ -283,50 +297,68 @@ public class RegionStateStore {
|
|||
}
|
||||
|
||||
private Scan getScanForUpdateRegionReplicas(TableName tableName) {
|
||||
return MetaTableAccessor.getScanForTableName(master.getConfiguration(), tableName)
|
||||
.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
|
||||
Scan scan;
|
||||
if (TableName.isMetaTableName(tableName)) {
|
||||
// Notice that, we do not use MetaCellComparator for master local region, so we can not use
|
||||
// the same logic to set start key and end key for scanning meta table when locating entries
|
||||
// in master local region. And since there is only one table in master local region(the record
|
||||
// for meta table), so we do not need set start key and end key.
|
||||
scan = new Scan();
|
||||
} else {
|
||||
scan = MetaTableAccessor.getScanForTableName(master.getConfiguration(), tableName);
|
||||
}
|
||||
return scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
|
||||
}
|
||||
|
||||
public void removeRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount)
|
||||
throws IOException {
|
||||
if (TableName.isMetaTableName(tableName)) {
|
||||
ZKWatcher zk = master.getZooKeeper();
|
||||
try {
|
||||
for (int i = newReplicaCount; i < oldReplicaCount; i++) {
|
||||
ZKUtil.deleteNode(zk, zk.getZNodePaths().getZNodeForReplica(i));
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
} else {
|
||||
Scan scan = getScanForUpdateRegionReplicas(tableName);
|
||||
private List<Delete> deleteRegionReplicas(ResultScanner scanner, int oldReplicaCount,
|
||||
int newReplicaCount, long now) throws IOException {
|
||||
List<Delete> deletes = new ArrayList<>();
|
||||
long now = EnvironmentEdgeManager.currentTime();
|
||||
try (Table metaTable = getMetaTable(); ResultScanner scanner = metaTable.getScanner(scan)) {
|
||||
for (;;) {
|
||||
Result result = scanner.next();
|
||||
if (result == null) {
|
||||
break;
|
||||
}
|
||||
RegionInfo primaryRegionInfo = MetaTableAccessor.getRegionInfo(result);
|
||||
if (primaryRegionInfo == null || primaryRegionInfo.isSplitParent()) {
|
||||
if (primaryRegionInfo == null || primaryRegionInfo.isSplit()) {
|
||||
continue;
|
||||
}
|
||||
Delete delete = new Delete(result.getRow());
|
||||
for (int i = newReplicaCount; i < oldReplicaCount; i++) {
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i),
|
||||
now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(i),
|
||||
now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i), now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(i), now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(i),
|
||||
now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerNameColumn(i),
|
||||
now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY,
|
||||
MetaTableAccessor.getRegionStateColumn(i), now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getRegionStateColumn(i),
|
||||
now);
|
||||
}
|
||||
deletes.add(delete);
|
||||
}
|
||||
return deletes;
|
||||
}
|
||||
|
||||
|
||||
public void removeRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount)
|
||||
throws IOException {
|
||||
Scan scan = getScanForUpdateRegionReplicas(tableName);
|
||||
long now = EnvironmentEdgeManager.currentTime();
|
||||
if (TableName.isMetaTableName(tableName)) {
|
||||
List<Delete> deletes;
|
||||
try (ResultScanner scanner = masterRegion.getScanner(scan)) {
|
||||
deletes = deleteRegionReplicas(scanner, oldReplicaCount, newReplicaCount, now);
|
||||
}
|
||||
debugLogMutations(deletes);
|
||||
masterRegion.update(r -> {
|
||||
for (Delete d : deletes) {
|
||||
r.delete(d);
|
||||
}
|
||||
});
|
||||
// also delete the mirrored location on zk
|
||||
removeMirrorMetaLocation(oldReplicaCount, newReplicaCount);
|
||||
} else {
|
||||
try (Table metaTable = getMetaTable(); ResultScanner scanner = metaTable.getScanner(scan)) {
|
||||
List<Delete> deletes = deleteRegionReplicas(scanner, oldReplicaCount, newReplicaCount, now);
|
||||
debugLogMutations(deletes);
|
||||
metaTable.delete(deletes);
|
||||
}
|
||||
|
@ -380,7 +412,7 @@ public class RegionStateStore {
|
|||
}
|
||||
}
|
||||
|
||||
private static byte[] getStateColumn(int replicaId) {
|
||||
public static byte[] getStateColumn(int replicaId) {
|
||||
return replicaId == 0 ? HConstants.STATE_QUALIFIER :
|
||||
Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER +
|
||||
String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Get;
|
|||
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.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -134,7 +135,11 @@ public final class MasterRegion {
|
|||
return region.get(get);
|
||||
}
|
||||
|
||||
public RegionScanner getScanner(Scan scan) throws IOException {
|
||||
public ResultScanner getScanner(Scan scan) throws IOException {
|
||||
return new RegionScannerAsResultScanner(region.getScanner(scan));
|
||||
}
|
||||
|
||||
public RegionScanner getRegionScanner(Scan scan) throws IOException {
|
||||
return region.getScanner(scan);
|
||||
}
|
||||
|
||||
|
|
|
@ -20,11 +20,14 @@ package org.apache.hadoop.hbase.master.region;
|
|||
import java.io.IOException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
|
@ -80,6 +83,10 @@ public final class MasterRegionFactory {
|
|||
public static final byte[] PROC_FAMILY = Bytes.toBytes("proc");
|
||||
|
||||
private static final TableDescriptor TABLE_DESC = TableDescriptorBuilder.newBuilder(TABLE_NAME)
|
||||
.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY)
|
||||
.setMaxVersions(HConstants.DEFAULT_HBASE_META_VERSIONS).setInMemory(true)
|
||||
.setBlocksize(HConstants.DEFAULT_HBASE_META_BLOCK_SIZE).setBloomFilterType(BloomType.ROWCOL)
|
||||
.setDataBlockEncoding(DataBlockEncoding.ROW_INDEX_V1).build())
|
||||
.setColumnFamily(ColumnFamilyDescriptorBuilder.of(PROC_FAMILY)).build();
|
||||
|
||||
public static MasterRegion create(Server server) throws IOException {
|
||||
|
|
|
@ -0,0 +1,88 @@
|
|||
/**
|
||||
* 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.region;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Wrap a {@link RegionScanner} as a {@link ResultScanner}.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class RegionScannerAsResultScanner implements ResultScanner {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(RegionScannerAsResultScanner.class);
|
||||
|
||||
private final RegionScanner scanner;
|
||||
|
||||
private boolean moreRows = true;
|
||||
|
||||
private final List<Cell> cells = new ArrayList<>();
|
||||
|
||||
RegionScannerAsResultScanner(RegionScanner scanner) {
|
||||
this.scanner = scanner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean renewLease() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result next() throws IOException {
|
||||
if (!moreRows) {
|
||||
return null;
|
||||
}
|
||||
for (;;) {
|
||||
moreRows = scanner.next(cells);
|
||||
if (cells.isEmpty()) {
|
||||
if (!moreRows) {
|
||||
return null;
|
||||
} else {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
Result result = Result.create(cells);
|
||||
cells.clear();
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScanMetrics getScanMetrics() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
try {
|
||||
scanner.close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to close scanner", e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -261,7 +261,7 @@ public class RegionProcedureStore extends ProcedureStoreBase {
|
|||
long maxProcId = 0;
|
||||
|
||||
try (RegionScanner scanner =
|
||||
region.getScanner(new Scan().addColumn(PROC_FAMILY, PROC_QUALIFIER))) {
|
||||
region.getRegionScanner(new Scan().addColumn(PROC_FAMILY, PROC_QUALIFIER))) {
|
||||
List<Cell> cells = new ArrayList<>();
|
||||
boolean moreRows;
|
||||
do {
|
||||
|
@ -426,8 +426,8 @@ public class RegionProcedureStore extends ProcedureStoreBase {
|
|||
public void cleanup() {
|
||||
// actually delete the procedures if it is not the one with the max procedure id.
|
||||
List<Cell> cells = new ArrayList<Cell>();
|
||||
try (RegionScanner scanner =
|
||||
region.getScanner(new Scan().addColumn(PROC_FAMILY, PROC_QUALIFIER).setReversed(true))) {
|
||||
try (RegionScanner scanner = region
|
||||
.getRegionScanner(new Scan().addColumn(PROC_FAMILY, PROC_QUALIFIER).setReversed(true))) {
|
||||
// skip the row with max procedure id
|
||||
boolean moreRows = scanner.next(cells);
|
||||
if (cells.isEmpty()) {
|
||||
|
@ -442,8 +442,9 @@ public class RegionProcedureStore extends ProcedureStoreBase {
|
|||
Cell cell = cells.get(0);
|
||||
cells.clear();
|
||||
if (cell.getValueLength() == 0) {
|
||||
region.update(r -> r
|
||||
.delete(new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())));
|
||||
region.update(
|
||||
r -> r.delete(new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())
|
||||
.addFamily(PROC_FAMILY)));
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
|||
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
|
||||
import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
|
||||
|
@ -128,16 +129,18 @@ public class TestFailedMetaReplicaAssigment {
|
|||
}
|
||||
|
||||
@Override
|
||||
public AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new BrokenMasterMetaAssignmentManager(master);
|
||||
public AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new BrokenMasterMetaAssignmentManager(master, masterRegion);
|
||||
}
|
||||
}
|
||||
|
||||
public static class BrokenMasterMetaAssignmentManager extends AssignmentManager {
|
||||
MasterServices master;
|
||||
|
||||
public BrokenMasterMetaAssignmentManager(final MasterServices master) {
|
||||
super(master);
|
||||
public BrokenMasterMetaAssignmentManager(final MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
this.master = master;
|
||||
}
|
||||
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -69,8 +70,9 @@ public class TestCloseAnOpeningRegion {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManager(master) {
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManager(master, masterRegion) {
|
||||
|
||||
@Override
|
||||
public ReportRegionStateTransitionResponse reportRegionStateTransition(
|
||||
|
|
|
@ -27,7 +27,6 @@ import java.util.List;
|
|||
import java.util.Optional;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CompatibilityFactory;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
|
@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
|||
import org.apache.hadoop.hbase.master.assignment.ServerState;
|
||||
import org.apache.hadoop.hbase.master.assignment.ServerStateNode;
|
||||
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
|
||||
|
@ -177,15 +177,16 @@ public class TestClusterRestartFailover extends AbstractTestRestartCluster {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
}
|
||||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,175 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic;
|
||||
import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.StartMiniClusterOption;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
|
||||
|
||||
/**
|
||||
* Testcase for HBASE-26193.
|
||||
*/
|
||||
@Category({ MasterTests.class, LargeTests.class })
|
||||
public class TestMigrateAndMirrorMetaLocations {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestMigrateAndMirrorMetaLocations.class);
|
||||
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
UTIL.startMiniCluster(3);
|
||||
HBaseTestingUtility.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 2);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() throws IOException {
|
||||
UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
private void assertLocationEquals(Result result, int replicaCount) throws Exception {
|
||||
RegionLocations locs = MetaTableAccessor.getRegionLocations(result);
|
||||
assertEquals(replicaCount, locs.size());
|
||||
for (int i = 0; i < replicaCount; i++) {
|
||||
String znode = UTIL.getZooKeeperWatcher().getZNodePaths().getZNodeForReplica(i);
|
||||
byte[] data = ZKUtil.getData(UTIL.getZooKeeperWatcher(), znode);
|
||||
data = removeMetaData(data);
|
||||
int prefixLen = lengthOfPBMagic();
|
||||
ZooKeeperProtos.MetaRegionServer zkProto = ZooKeeperProtos.MetaRegionServer.parser()
|
||||
.parseFrom(data, prefixLen, data.length - prefixLen);
|
||||
ServerName sn = ProtobufUtil.toServerName(zkProto.getServer());
|
||||
assertEquals(locs.getRegionLocation(i).getServerName(), sn);
|
||||
}
|
||||
assertEquals(replicaCount, UTIL.getZooKeeperWatcher().getMetaReplicaNodes().size());
|
||||
}
|
||||
|
||||
private void checkMirrorLocation(int replicaCount) throws Exception {
|
||||
MasterRegion masterRegion = UTIL.getMiniHBaseCluster().getMaster().getMasterRegion();
|
||||
try (RegionScanner scanner =
|
||||
masterRegion.getRegionScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {
|
||||
List<Cell> cells = new ArrayList<>();
|
||||
boolean moreRows = scanner.next(cells);
|
||||
// should only have one row as we have only one meta region, different replicas will be in the
|
||||
// same row
|
||||
assertFalse(moreRows);
|
||||
assertFalse(cells.isEmpty());
|
||||
Result result = Result.create(cells);
|
||||
// make sure we publish the correct location to zookeeper too
|
||||
assertLocationEquals(result, replicaCount);
|
||||
}
|
||||
}
|
||||
|
||||
private void waitUntilNoSCP() throws IOException {
|
||||
UTIL.waitFor(30000, () -> UTIL.getMiniHBaseCluster().getMaster().getProcedures().stream()
|
||||
.filter(p -> p instanceof ServerCrashProcedure).allMatch(Procedure::isSuccess));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test() throws Exception {
|
||||
checkMirrorLocation(2);
|
||||
MasterRegion masterRegion = UTIL.getMiniHBaseCluster().getMaster().getMasterRegion();
|
||||
try (RegionScanner scanner =
|
||||
masterRegion.getRegionScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {
|
||||
List<Cell> cells = new ArrayList<>();
|
||||
scanner.next(cells);
|
||||
Cell cell = cells.get(0);
|
||||
// delete the only row
|
||||
masterRegion.update(
|
||||
r -> r.delete(new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())
|
||||
.addFamily(HConstants.CATALOG_FAMILY)));
|
||||
masterRegion.flush(true);
|
||||
}
|
||||
// restart the whole cluster, to see if we can migrate the data on zookeeper to master local
|
||||
// region
|
||||
UTIL.shutdownMiniHBaseCluster();
|
||||
UTIL.startMiniHBaseCluster(StartMiniClusterOption.builder().numRegionServers(3).build());
|
||||
masterRegion = UTIL.getMiniHBaseCluster().getMaster().getMasterRegion();
|
||||
try (RegionScanner scanner =
|
||||
masterRegion.getRegionScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {
|
||||
List<Cell> cells = new ArrayList<>();
|
||||
boolean moreRows = scanner.next(cells);
|
||||
assertFalse(moreRows);
|
||||
// should have the migrated data
|
||||
assertFalse(cells.isEmpty());
|
||||
}
|
||||
// wait until all meta regions have been assigned
|
||||
UTIL.waitFor(30000,
|
||||
() -> UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).size() == 2);
|
||||
// make sure all the SCPs are finished
|
||||
waitUntilNoSCP();
|
||||
checkMirrorLocation(2);
|
||||
|
||||
// increase replica count to 3
|
||||
HBaseTestingUtility.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 3);
|
||||
checkMirrorLocation(3);
|
||||
|
||||
byte[] replica2Data = ZKUtil.getData(UTIL.getZooKeeperWatcher(),
|
||||
UTIL.getZooKeeperWatcher().getZNodePaths().getZNodeForReplica(2));
|
||||
|
||||
// decrease replica count to 1
|
||||
HBaseTestingUtility.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 1);
|
||||
checkMirrorLocation(1);
|
||||
|
||||
// restart the whole cluster, put an extra replica znode on zookeeper, to see if we will remove
|
||||
// it
|
||||
UTIL.shutdownMiniHBaseCluster();
|
||||
ZKUtil.createAndFailSilent(UTIL.getZooKeeperWatcher(),
|
||||
UTIL.getZooKeeperWatcher().getZNodePaths().getZNodeForReplica(2), replica2Data);
|
||||
UTIL.startMiniHBaseCluster(StartMiniClusterOption.builder().numRegionServers(3).build());
|
||||
// should have removed the extra replica znode as it is part of the start up process, when
|
||||
// initializing AM
|
||||
assertEquals(1, UTIL.getZooKeeperWatcher().getMetaReplicaNodes().size());
|
||||
// make sure all the SCPs are finished
|
||||
waitUntilNoSCP();
|
||||
checkMirrorLocation(1);
|
||||
}
|
||||
}
|
|
@ -51,6 +51,8 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
|
|||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegionFactory;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||
|
@ -87,6 +89,7 @@ public class MockMasterServices extends MockNoopMasterServices {
|
|||
private final SplitWALManager splitWALManager;
|
||||
private final AssignmentManager assignmentManager;
|
||||
private final TableStateManager tableStateManager;
|
||||
private final MasterRegion masterRegion;
|
||||
|
||||
private MasterProcedureEnv procedureEnv;
|
||||
private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
|
||||
|
@ -109,9 +112,10 @@ public class MockMasterServices extends MockNoopMasterServices {
|
|||
this.splitWALManager =
|
||||
conf.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK, DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)?
|
||||
null: new SplitWALManager(this);
|
||||
|
||||
this.masterRegion = MasterRegionFactory.create(this);
|
||||
// Mock an AM.
|
||||
this.assignmentManager = new AssignmentManager(this, new MockRegionStateStore(this));
|
||||
this.assignmentManager =
|
||||
new AssignmentManager(this, masterRegion, new MockRegionStateStore(this, masterRegion));
|
||||
this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
|
||||
this.serverManager = new ServerManager(this);
|
||||
this.tableStateManager = Mockito.mock(TableStateManager.class);
|
||||
|
@ -298,8 +302,8 @@ public class MockMasterServices extends MockNoopMasterServices {
|
|||
}
|
||||
|
||||
private static class MockRegionStateStore extends RegionStateStore {
|
||||
public MockRegionStateStore(final MasterServices master) {
|
||||
super(master);
|
||||
public MockRegionStateStore(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.RegionLocator;
|
|||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -58,8 +59,8 @@ public class TestOpenRegionProcedureBackoff {
|
|||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -78,8 +79,9 @@ public class TestOpenRegionProcedureBackoff {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.master.HMaster;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
|
@ -74,8 +75,8 @@ public class TestOpenRegionProcedureHang {
|
|||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master,MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -110,8 +111,9 @@ public class TestOpenRegionProcedureHang {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.master.HMaster;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
|
||||
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
|
@ -69,8 +70,8 @@ public class TestRaceBetweenSCPAndDTP {
|
|||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master,masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -95,8 +96,9 @@ public class TestRaceBetweenSCPAndDTP {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.master.HMaster;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
|
@ -69,8 +70,8 @@ public class TestRaceBetweenSCPAndTRSP {
|
|||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -108,8 +109,9 @@ public class TestRaceBetweenSCPAndTRSP {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.master.HMaster;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -80,8 +81,8 @@ public class TestRegionAssignedToMultipleRegionServers {
|
|||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -114,8 +115,9 @@ public class TestRegionAssignedToMultipleRegionServers {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
|
|||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -68,8 +69,8 @@ public class TestReportOnlineRegionsRace {
|
|||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -110,8 +111,9 @@ public class TestReportOnlineRegionsRace {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.master.HMaster;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -82,8 +83,8 @@ public class TestReportRegionStateTransitionFromDeadServer {
|
|||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -121,8 +122,9 @@ public class TestReportRegionStateTransitionFromDeadServer {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.master.HMaster;
|
|||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -60,8 +61,8 @@ public class TestReportRegionStateTransitionRetry {
|
|||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -88,8 +89,9 @@ public class TestReportRegionStateTransitionRetry {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
|
|||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -89,8 +90,8 @@ public class TestSCPGetRegionsRace {
|
|||
|
||||
private static final class AssignmentManagerForTest extends AssignmentManager {
|
||||
|
||||
public AssignmentManagerForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -134,8 +135,9 @@ public class TestSCPGetRegionsRace {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AssignmentManagerForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AssignmentManagerForTest(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
|
|||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.region.MasterRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
|
@ -135,8 +136,8 @@ public class TestWakeUpUnexpectedProcedure {
|
|||
|
||||
private static final class AMForTest extends AssignmentManager {
|
||||
|
||||
public AMForTest(MasterServices master) {
|
||||
super(master);
|
||||
public AMForTest(MasterServices master, MasterRegion masterRegion) {
|
||||
super(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -202,8 +203,9 @@ public class TestWakeUpUnexpectedProcedure {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master) {
|
||||
return new AMForTest(master);
|
||||
protected AssignmentManager createAssignmentManager(MasterServices master,
|
||||
MasterRegion masterRegion) {
|
||||
return new AMForTest(master, masterRegion);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -222,7 +222,7 @@ public class TestMasterRegionOnTwoFileSystems {
|
|||
region.close(true);
|
||||
region = createMasterRegion(
|
||||
ServerName.valueOf("localhost", 12345, EnvironmentEdgeManager.currentTime() + round + 1));
|
||||
try (RegionScanner scanner = region.getScanner(new Scan())) {
|
||||
try (RegionScanner scanner = region.getRegionScanner(new Scan())) {
|
||||
List<Cell> cells = new ArrayList<>();
|
||||
boolean moreValues = true;
|
||||
for (int i = 0; i < (round + 1) * countPerRound; i++) {
|
||||
|
|
|
@ -80,7 +80,7 @@ public class TestRegionProcedureStoreMigration {
|
|||
Configuration conf = htu.getConfiguration();
|
||||
conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, false);
|
||||
// Runs on local filesystem. Test does not need sync. Turn off checks.
|
||||
htu.getConfiguration().setBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, false);
|
||||
conf.setBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, false);
|
||||
Path testDir = htu.getDataTestDir();
|
||||
CommonFSUtils.setRootDir(conf, testDir);
|
||||
walStore = new WALProcedureStore(conf, new LeaseRecovery() {
|
||||
|
|
Loading…
Reference in New Issue