HBASE-25164 Make ModifyTableProcedure support changing meta replica count (#2513)
Signed-off-by: Michael Stack <stack@apache.org>
This commit is contained in:
parent
c28220522d
commit
92c3bcd9fb
|
@ -1131,7 +1131,20 @@ public final class HConstants {
|
|||
/** Conf key for enabling meta replication */
|
||||
public static final String USE_META_REPLICAS = "hbase.meta.replicas.use";
|
||||
public static final boolean DEFAULT_USE_META_REPLICAS = false;
|
||||
|
||||
/**
|
||||
* @deprecated Since 2.4.0, will be removed in 4.0.0. Please change the meta replicas number by
|
||||
* altering meta table, i.e, set a new 'region replication' number and call
|
||||
* modifyTable.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String META_REPLICAS_NUM = "hbase.meta.replica.count";
|
||||
/**
|
||||
* @deprecated Since 2.4.0, will be removed in 4.0.0. Please change the meta replicas number by
|
||||
* altering meta table, i.e, set a new 'region replication' number and call
|
||||
* modifyTable.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final int DEFAULT_META_REPLICA_NUM = 1;
|
||||
|
||||
/**
|
||||
|
|
|
@ -53,11 +53,11 @@ public class IntegrationTestMetaReplicas {
|
|||
if (util == null) {
|
||||
util = new IntegrationTestingUtility();
|
||||
}
|
||||
util.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
|
||||
util.getConfiguration().setInt(
|
||||
StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 1000);
|
||||
// Make sure there are three servers.
|
||||
util.initializeCluster(3);
|
||||
HBaseTestingUtility.setReplicas(util.getAdmin(), TableName.META_TABLE_NAME, 3);
|
||||
ZKWatcher zkw = util.getZooKeeperWatcher();
|
||||
Configuration conf = util.getConfiguration();
|
||||
String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
|
||||
|
|
|
@ -1015,10 +1015,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
status.setStatus("Initializing meta table if this is a new deploy");
|
||||
InitMetaProcedure initMetaProc = null;
|
||||
// Print out state of hbase:meta on startup; helps debugging.
|
||||
RegionState rs = this.assignmentManager.getRegionStates().
|
||||
getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO);
|
||||
LOG.info("hbase:meta {}", rs);
|
||||
if (rs != null && rs.isOffline()) {
|
||||
if (!this.assignmentManager.getRegionStates().hasTableRegionStates(TableName.META_TABLE_NAME)) {
|
||||
Optional<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream()
|
||||
.filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
|
||||
initMetaProc = optProc.orElseGet(() -> {
|
||||
|
@ -1070,6 +1067,39 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
this.assignmentManager.joinCluster();
|
||||
// The below depends on hbase:meta being online.
|
||||
this.tableStateManager.start();
|
||||
|
||||
// for migrating from a version without HBASE-25099, and also for honoring the configuration
|
||||
// first.
|
||||
if (conf.get(HConstants.META_REPLICAS_NUM) != null) {
|
||||
int replicasNumInConf =
|
||||
conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM);
|
||||
TableDescriptor metaDesc = tableDescriptors.get(TableName.META_TABLE_NAME);
|
||||
if (metaDesc.getRegionReplication() != replicasNumInConf) {
|
||||
// it is possible that we already have some replicas before upgrading, so we must set the
|
||||
// region replication number in meta TableDescriptor directly first, without creating a
|
||||
// ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas.
|
||||
int existingReplicasCount =
|
||||
assignmentManager.getRegionStates().getRegionsOfTable(TableName.META_TABLE_NAME).size();
|
||||
if (existingReplicasCount > metaDesc.getRegionReplication()) {
|
||||
LOG.info("Update replica count of hbase:meta from {}(in TableDescriptor)" +
|
||||
" to {}(existing ZNodes)", metaDesc.getRegionReplication(), existingReplicasCount);
|
||||
metaDesc = TableDescriptorBuilder.newBuilder(metaDesc)
|
||||
.setRegionReplication(existingReplicasCount).build();
|
||||
tableDescriptors.update(metaDesc);
|
||||
}
|
||||
// check again, and issue a ModifyTableProcedure if needed
|
||||
if (metaDesc.getRegionReplication() != replicasNumInConf) {
|
||||
LOG.info(
|
||||
"The {} config is {} while the replica count in TableDescriptor is {}" +
|
||||
" for hbase:meta, altering...",
|
||||
HConstants.META_REPLICAS_NUM, replicasNumInConf, metaDesc.getRegionReplication());
|
||||
procedureExecutor.submitProcedure(new ModifyTableProcedure(
|
||||
procedureExecutor.getEnvironment(), TableDescriptorBuilder.newBuilder(metaDesc)
|
||||
.setRegionReplication(replicasNumInConf).build(),
|
||||
null, metaDesc, false));
|
||||
}
|
||||
}
|
||||
}
|
||||
// Below has to happen after tablestatemanager has started in the case where this hbase-2.x
|
||||
// is being started over an hbase-1.x dataset. tablestatemanager runs a migration as part
|
||||
// of its 'start' moving table state from zookeeper to hbase:meta. This migration needs to
|
||||
|
@ -1133,13 +1163,6 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
}
|
||||
|
||||
assignmentManager.checkIfShouldMoveSystemRegionAsync();
|
||||
status.setStatus("Assign meta replicas");
|
||||
MasterMetaBootstrap metaBootstrap = createMetaBootstrap();
|
||||
try {
|
||||
metaBootstrap.assignMetaReplicas();
|
||||
} catch (IOException | KeeperException e){
|
||||
LOG.error("Assigning meta replica failed: ", e);
|
||||
}
|
||||
status.setStatus("Starting quota manager");
|
||||
initQuotaManager();
|
||||
if (QuotaUtil.isQuotaEnabled(conf)) {
|
||||
|
@ -1294,21 +1317,6 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
getChoreService().scheduleChore(mobFileCompactionChore);
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Create a {@link MasterMetaBootstrap} instance.
|
||||
* </p>
|
||||
* <p>
|
||||
* Will be overridden in tests.
|
||||
* </p>
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected MasterMetaBootstrap createMetaBootstrap() {
|
||||
// We put this out here in a method so can do a Mockito.spy and stub it out
|
||||
// w/ a mocked up MasterMetaBootstrap.
|
||||
return new MasterMetaBootstrap(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Create a {@link ServerManager} instance.
|
||||
|
|
|
@ -1,111 +0,0 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
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;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Used by the HMaster on startup to split meta logs and assign the meta table.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class MasterMetaBootstrap {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(MasterMetaBootstrap.class);
|
||||
|
||||
private final HMaster master;
|
||||
|
||||
public MasterMetaBootstrap(HMaster master) {
|
||||
this.master = master;
|
||||
}
|
||||
|
||||
/**
|
||||
* For assigning hbase:meta replicas only.
|
||||
* TODO: The way this assign runs, nothing but chance to stop all replicas showing up on same
|
||||
* server as the hbase:meta region.
|
||||
*/
|
||||
void assignMetaReplicas()
|
||||
throws IOException, InterruptedException, KeeperException {
|
||||
int numReplicas = master.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
|
||||
HConstants.DEFAULT_META_REPLICA_NUM);
|
||||
final AssignmentManager assignmentManager = master.getAssignmentManager();
|
||||
if (!assignmentManager.isMetaLoaded()) {
|
||||
throw new IllegalStateException("hbase:meta must be initialized first before we can " +
|
||||
"assign out its replicas");
|
||||
}
|
||||
ServerName metaServername = MetaTableLocator.getMetaRegionLocation(this.master.getZooKeeper());
|
||||
for (int i = 1; i < numReplicas; i++) {
|
||||
// Get current meta state for replica from zk.
|
||||
RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), i);
|
||||
RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
|
||||
RegionInfoBuilder.FIRST_META_REGIONINFO, i);
|
||||
LOG.debug(hri.getRegionNameAsString() + " replica region state from zookeeper=" + metaState);
|
||||
if (metaServername.equals(metaState.getServerName())) {
|
||||
metaState = null;
|
||||
LOG.info(hri.getRegionNameAsString() +
|
||||
" old location is same as current hbase:meta location; setting location as null...");
|
||||
}
|
||||
// These assigns run inline. All is blocked till they complete. Only interrupt is shutting
|
||||
// down hosting server which calls AM#stop.
|
||||
if (metaState != null && metaState.getServerName() != null) {
|
||||
// Try to retain old assignment.
|
||||
assignmentManager.assignAsync(hri, metaState.getServerName());
|
||||
} else {
|
||||
assignmentManager.assignAsync(hri);
|
||||
}
|
||||
}
|
||||
unassignExcessMetaReplica(numReplicas);
|
||||
}
|
||||
|
||||
private void unassignExcessMetaReplica(int numMetaReplicasConfigured) {
|
||||
final ZKWatcher zooKeeper = master.getZooKeeper();
|
||||
// unassign the unneeded replicas (for e.g., if the previous master was configured
|
||||
// with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
|
||||
try {
|
||||
List<String> metaReplicaZnodes = zooKeeper.getMetaReplicaNodes();
|
||||
for (String metaReplicaZnode : metaReplicaZnodes) {
|
||||
int replicaId = zooKeeper.getZNodePaths().getMetaReplicaIdFromZNode(metaReplicaZnode);
|
||||
if (replicaId >= numMetaReplicasConfigured) {
|
||||
RegionState r = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);
|
||||
LOG.info("Closing excess replica of meta region " + r.getRegion());
|
||||
// send a close and wait for a max of 30 seconds
|
||||
ServerManager.closeRegionSilentlyAndWait(master.getAsyncClusterConnection(),
|
||||
r.getServerName(), r.getRegion(), 30000);
|
||||
ZKUtil.deleteNode(zooKeeper, zooKeeper.getZNodePaths().getZNodeForReplica(replicaId));
|
||||
}
|
||||
}
|
||||
} catch (Exception ex) {
|
||||
// ignore the exception since we don't want the master to be wedged due to potential
|
||||
// issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
|
||||
LOG.warn("Ignoring exception " + ex);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
|
|||
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.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.RegionStatesCount;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
@ -228,13 +229,18 @@ public class AssignmentManager {
|
|||
// load meta region state
|
||||
ZKWatcher zkw = master.getZooKeeper();
|
||||
// it could be null in some tests
|
||||
if (zkw != null) {
|
||||
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);
|
||||
// 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);
|
||||
RegionStateNode regionNode =
|
||||
regionStates.getOrCreateRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
|
||||
RegionState regionState = MetaTableLocator.getMetaRegionState(zkw, replicaId);
|
||||
RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionState.getRegion());
|
||||
regionNode.setRegionLocation(regionState.getServerName());
|
||||
regionNode.setState(regionState.getState());
|
||||
if (regionNode.getProcedure() != null) {
|
||||
|
@ -243,8 +249,11 @@ public class AssignmentManager {
|
|||
if (regionState.getServerName() != null) {
|
||||
regionStates.addRegionToServer(regionNode);
|
||||
}
|
||||
if (RegionReplicaUtil.isDefaultReplica(replicaId)) {
|
||||
setMetaAssigned(regionState.getRegion(), regionState.getState() == State.OPEN);
|
||||
}
|
||||
LOG.debug("Loaded hbase:meta {}", regionNode);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -64,6 +64,8 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|||
import org.apache.hadoop.hbase.util.FutureUtils;
|
||||
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;
|
||||
|
@ -126,17 +128,16 @@ public class RegionStateStore {
|
|||
}
|
||||
|
||||
/**
|
||||
* Queries META table for the passed region encoded name,
|
||||
* delegating action upon results to the <code>RegionStateVisitor</code>
|
||||
* passed as second parameter.
|
||||
* Queries META table for the passed region encoded name, delegating action upon results to the
|
||||
* <code>RegionStateVisitor</code> passed as second parameter.
|
||||
* @param regionEncodedName encoded name for the Region we want to query META for.
|
||||
* @param visitor The <code>RegionStateVisitor</code> instance to react over the query results.
|
||||
* @throws IOException If some error occurs while querying META or parsing results.
|
||||
*/
|
||||
public void visitMetaForRegion(final String regionEncodedName, final RegionStateVisitor visitor)
|
||||
throws IOException {
|
||||
Result result = MetaTableAccessor.
|
||||
scanByRegionEncodedName(master.getConnection(), regionEncodedName);
|
||||
Result result =
|
||||
MetaTableAccessor.scanByRegionEncodedName(master.getConnection(), regionEncodedName);
|
||||
if (result != null) {
|
||||
visitMetaEntry(visitor, result);
|
||||
}
|
||||
|
@ -178,13 +179,13 @@ public class RegionStateStore {
|
|||
updateMetaLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(),
|
||||
regionStateNode.getState());
|
||||
} else {
|
||||
long openSeqNum = regionStateNode.getState() == State.OPEN ? regionStateNode.getOpenSeqNum()
|
||||
: HConstants.NO_SEQNUM;
|
||||
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);
|
||||
regionStateNode.getProcedure() != null ? regionStateNode.getProcedure().getProcId() :
|
||||
Procedure.NO_PROC_ID);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -199,8 +200,7 @@ public class RegionStateStore {
|
|||
}
|
||||
|
||||
private void updateUserRegionLocation(RegionInfo regionInfo, State state,
|
||||
ServerName regionLocation, long openSeqNum,
|
||||
long pid) throws IOException {
|
||||
ServerName regionLocation, long openSeqNum, long pid) throws IOException {
|
||||
long time = EnvironmentEdgeManager.currentTime();
|
||||
final int replicaId = regionInfo.getReplicaId();
|
||||
final Put put = new Put(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo), time);
|
||||
|
@ -223,23 +223,16 @@ public class RegionStateStore {
|
|||
} else if (regionLocation != null) {
|
||||
// Ideally, if no regionLocation, write null to the hbase:meta but this will confuse clients
|
||||
// currently; they want a server to hit. TODO: Make clients wait if no location.
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
|
||||
.setRow(put.getRow())
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||
.setFamily(HConstants.CATALOG_FAMILY)
|
||||
.setQualifier(CatalogFamilyFormat.getServerNameColumn(replicaId))
|
||||
.setTimestamp(put.getTimestamp())
|
||||
.setType(Cell.Type.Put)
|
||||
.setValue(Bytes.toBytes(regionLocation.getServerName()))
|
||||
.build());
|
||||
.setTimestamp(put.getTimestamp()).setType(Cell.Type.Put)
|
||||
.setValue(Bytes.toBytes(regionLocation.getServerName())).build());
|
||||
info.append(", regionLocation=").append(regionLocation);
|
||||
}
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
|
||||
.setRow(put.getRow())
|
||||
.setFamily(HConstants.CATALOG_FAMILY)
|
||||
.setQualifier(getStateColumn(replicaId))
|
||||
.setTimestamp(put.getTimestamp())
|
||||
.setType(Cell.Type.Put)
|
||||
.setValue(Bytes.toBytes(state.name()))
|
||||
put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
|
||||
.setFamily(HConstants.CATALOG_FAMILY).setQualifier(getStateColumn(replicaId))
|
||||
.setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(state.name()))
|
||||
.build());
|
||||
LOG.info(info.toString());
|
||||
updateRegionLocation(regionInfo, state, put);
|
||||
|
@ -438,8 +431,7 @@ public class RegionStateStore {
|
|||
* @param connection connection we're using
|
||||
* @param mergeRegion the merged region
|
||||
*/
|
||||
public void deleteMergeQualifiers(RegionInfo mergeRegion)
|
||||
throws IOException {
|
||||
public void deleteMergeQualifiers(RegionInfo mergeRegion) throws IOException {
|
||||
// NOTE: We are doing a new hbase:meta read here.
|
||||
Cell[] cells = getRegionCatalogResult(mergeRegion).rawCells();
|
||||
if (cells == null || cells.length == 0) {
|
||||
|
@ -554,6 +546,16 @@ public class RegionStateStore {
|
|||
|
||||
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);
|
||||
List<Delete> deletes = new ArrayList<>();
|
||||
long now = EnvironmentEdgeManager.currentTime();
|
||||
|
@ -569,14 +571,16 @@ public class RegionStateStore {
|
|||
}
|
||||
Delete delete = new Delete(result.getRow());
|
||||
for (int i = newReplicaCount; i < oldReplicaCount; i++) {
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getServerColumn(i), now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getSeqNumColumn(i), now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getServerColumn(i),
|
||||
now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getSeqNumColumn(i),
|
||||
now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getStartCodeColumn(i),
|
||||
now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getServerNameColumn(i),
|
||||
now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY, CatalogFamilyFormat.getRegionStateColumn(i),
|
||||
now);
|
||||
delete.addColumns(HConstants.CATALOG_FAMILY,
|
||||
CatalogFamilyFormat.getRegionStateColumn(i), now);
|
||||
}
|
||||
deletes.add(delete);
|
||||
}
|
||||
|
@ -584,6 +588,7 @@ public class RegionStateStore {
|
|||
metaTable.delete(deletes);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// ==========================================================================
|
||||
// Table Descriptors helpers
|
||||
|
@ -613,18 +618,19 @@ public class RegionStateStore {
|
|||
* @return the region state, or null if unknown.
|
||||
*/
|
||||
public static State getRegionState(final Result r, RegionInfo regionInfo) {
|
||||
Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY,
|
||||
getStateColumn(regionInfo.getReplicaId()));
|
||||
Cell cell =
|
||||
r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getStateColumn(regionInfo.getReplicaId()));
|
||||
if (cell == null || cell.getValueLength() == 0) {
|
||||
return null;
|
||||
}
|
||||
|
||||
String state = Bytes.toString(cell.getValueArray(), cell.getValueOffset(),
|
||||
cell.getValueLength());
|
||||
String state =
|
||||
Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
|
||||
try {
|
||||
return State.valueOf(state);
|
||||
} catch (IllegalArgumentException e) {
|
||||
LOG.warn("BAD value {} in hbase:meta info:state column for region {} , " +
|
||||
LOG.warn(
|
||||
"BAD value {} in hbase:meta info:state column for region {} , " +
|
||||
"Consider using HBCK2 setRegionState ENCODED_REGION_NAME STATE",
|
||||
state, regionInfo.getEncodedName());
|
||||
return null;
|
||||
|
@ -632,10 +638,9 @@ public class RegionStateStore {
|
|||
}
|
||||
|
||||
private 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));
|
||||
return replicaId == 0 ? HConstants.STATE_QUALIFIER :
|
||||
Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER +
|
||||
String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
|
||||
}
|
||||
|
||||
private static void debugLogMutations(List<? extends Mutation> mutations) throws IOException {
|
||||
|
|
|
@ -28,7 +28,6 @@ import java.util.concurrent.CountDownLatch;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
@ -83,9 +82,8 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
|
|||
// created here in bootstrap and it'll need to be cleaned up. Better to
|
||||
// not make it in first place. Turn off block caching for bootstrap.
|
||||
// Enable after.
|
||||
TableDescriptor metaDescriptor = FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(conf, fs,
|
||||
rootDir, builder -> builder.setRegionReplication(
|
||||
conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM)));
|
||||
TableDescriptor metaDescriptor =
|
||||
FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(conf, fs, rootDir);
|
||||
HRegion
|
||||
.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, rootDir, conf, metaDescriptor, null)
|
||||
.close();
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.function.Function;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import org.apache.commons.lang3.NotImplementedException;
|
||||
|
@ -121,20 +120,16 @@ public class FSTableDescriptors implements TableDescriptors {
|
|||
@VisibleForTesting
|
||||
public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOException {
|
||||
tryUpdateAndGetMetaTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf),
|
||||
CommonFSUtils.getRootDir(conf), null);
|
||||
CommonFSUtils.getRootDir(conf));
|
||||
}
|
||||
|
||||
public static TableDescriptor tryUpdateAndGetMetaTableDescriptor(Configuration conf,
|
||||
FileSystem fs, Path rootdir,
|
||||
Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
|
||||
FileSystem fs, Path rootdir) throws IOException {
|
||||
// see if we already have meta descriptor on fs. Write one if not.
|
||||
try {
|
||||
return getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
|
||||
} catch (TableInfoMissingException e) {
|
||||
TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf);
|
||||
if (metaObserver != null) {
|
||||
builder = metaObserver.apply(builder);
|
||||
}
|
||||
TableDescriptor td = builder.build();
|
||||
LOG.info("Creating new hbase:meta table descriptor {}", td);
|
||||
TableName tableName = td.getTableName();
|
||||
|
|
|
@ -155,8 +155,8 @@
|
|||
Table table = master.getConnection().getTable(TableName.valueOf(fqtn));
|
||||
boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false);
|
||||
boolean readOnly = !InfoServer.canUserModifyUI(request, getServletContext(), conf);
|
||||
int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
|
||||
HConstants.DEFAULT_META_REPLICA_NUM);
|
||||
int numMetaReplicas =
|
||||
master.getTableDescriptors().get(TableName.META_TABLE_NAME).getRegionReplication();
|
||||
Map<String, Integer> frags = null;
|
||||
if (showFragmentation) {
|
||||
frags = FSUtils.getTableFragmentation(master);
|
||||
|
|
|
@ -47,8 +47,8 @@ public abstract class AbstractTestRegionLocator {
|
|||
protected static byte[][] SPLIT_KEYS;
|
||||
|
||||
protected static void startClusterAndCreateTable() throws Exception {
|
||||
UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, REGION_REPLICATION);
|
||||
UTIL.startMiniCluster(3);
|
||||
HBaseTestingUtility.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, REGION_REPLICATION);
|
||||
TableDescriptor td =
|
||||
TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(REGION_REPLICATION)
|
||||
.setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
|
||||
|
|
|
@ -26,9 +26,9 @@ import java.util.Set;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.StartMiniClusterOption;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNameTestRule;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil;
|
||||
|
@ -56,12 +56,13 @@ public class MetaWithReplicasTestBase {
|
|||
|
||||
protected static void startCluster() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt("zookeeper.session.timeout", 30000);
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
|
||||
TEST_UTIL.getConfiguration()
|
||||
.setInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 1000);
|
||||
StartMiniClusterOption option = StartMiniClusterOption.builder().numAlwaysStandByMasters(1)
|
||||
.numMasters(1).numRegionServers(REGIONSERVERS_COUNT).build();
|
||||
TEST_UTIL.startMiniCluster(option);
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
HBaseTestingUtility.setReplicas(admin, TableName.META_TABLE_NAME, 3);
|
||||
AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
|
||||
Set<ServerName> sns = new HashSet<ServerName>();
|
||||
ServerName hbaseMetaServerName =
|
||||
|
|
|
@ -20,13 +20,13 @@ package org.apache.hadoop.hbase.client;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
|
@ -44,10 +44,10 @@ public final class RegionReplicaTestHelper {
|
|||
|
||||
// waits for all replicas to have region location
|
||||
static void waitUntilAllMetaReplicasAreReady(HBaseTestingUtility util,
|
||||
ConnectionRegistry registry) {
|
||||
ConnectionRegistry registry) throws IOException {
|
||||
Configuration conf = util.getConfiguration();
|
||||
int regionReplicaCount = util.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
|
||||
HConstants.DEFAULT_META_REPLICA_NUM);
|
||||
int regionReplicaCount =
|
||||
util.getAdmin().getDescriptor(TableName.META_TABLE_NAME).getRegionReplication();
|
||||
Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true,
|
||||
new ExplainingPredicate<IOException>() {
|
||||
@Override
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.Map;
|
|||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -51,12 +52,11 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase {
|
|||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
|
||||
TestAsyncAdminBase.setUpBeforeClass();
|
||||
HBaseTestingUtility.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3);
|
||||
try (ConnectionRegistry registry =
|
||||
ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration())) {
|
||||
RegionReplicaTestHelper
|
||||
.waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry);
|
||||
RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,7 +22,6 @@ import static org.apache.hadoop.hbase.client.RegionReplicaTestHelper.testLocator
|
|||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -50,8 +49,8 @@ public class TestAsyncMetaRegionLocator {
|
|||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
|
||||
TEST_UTIL.startMiniCluster(3);
|
||||
HBaseTestingUtility.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3);
|
||||
TEST_UTIL.waitUntilNoRegionsInTransition();
|
||||
REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
|
||||
RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY);
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
@ -86,11 +87,11 @@ public class TestAsyncTableUseMetaReplicas {
|
|||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
Configuration conf = UTIL.getConfiguration();
|
||||
conf.setInt(HConstants.META_REPLICAS_NUM, 3);
|
||||
conf.setInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 1000);
|
||||
conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
|
||||
FailPrimaryMetaScanCp.class.getName());
|
||||
UTIL.startMiniCluster(3);
|
||||
HBaseTestingUtility.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 3);
|
||||
try (ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf)) {
|
||||
RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(UTIL, registry);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,55 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({ MiscTests.class, MediumTests.class })
|
||||
public class TestCleanupMetaReplica extends MetaWithReplicasTestBase {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestCleanupMetaReplica.class);
|
||||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
startCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReplicaCleanup() throws Exception {
|
||||
ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||
List<String> metaReplicaZnodes = zkw.getMetaReplicaNodes();
|
||||
assertEquals(3, metaReplicaZnodes.size());
|
||||
HBaseTestingUtility.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 1);
|
||||
metaReplicaZnodes = zkw.getMetaReplicaNodes();
|
||||
assertEquals(1, metaReplicaZnodes.size());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,69 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Make sure we will honor the {@link HConstants#META_REPLICAS_NUM}.
|
||||
*/
|
||||
@Category({ MiscTests.class, MediumTests.class })
|
||||
public class TestCleanupMetaReplicaThroughConfig extends MetaWithReplicasTestBase {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestCleanupMetaReplicaThroughConfig.class);
|
||||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
startCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReplicaCleanup() throws Exception {
|
||||
ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||
List<String> metaReplicaZnodes = zkw.getMetaReplicaNodes();
|
||||
assertEquals(3, metaReplicaZnodes.size());
|
||||
|
||||
final HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
|
||||
master.stop("Restarting");
|
||||
TEST_UTIL.waitFor(30000, () -> master.isStopped());
|
||||
TEST_UTIL.getMiniHBaseCluster().getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 1);
|
||||
|
||||
JVMClusterUtil.MasterThread newMasterThread = TEST_UTIL.getMiniHBaseCluster().startMaster();
|
||||
final HMaster newMaster = newMasterThread.getMaster();
|
||||
|
||||
// wait until new master finished meta replica assignment logic
|
||||
TEST_UTIL.waitFor(30000, () -> newMaster.getMasterQuotaManager() != null);
|
||||
TEST_UTIL.waitFor(30000,
|
||||
() -> TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes().size() == 1);
|
||||
}
|
||||
}
|
|
@ -22,18 +22,23 @@ import static org.junit.Assert.assertNull;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.StartMiniClusterOption;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
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.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -53,7 +58,8 @@ public class TestFailedMetaReplicaAssigment {
|
|||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
// using our rigged master, to force a failed meta replica assignment
|
||||
// using our rigged master, to force a failed meta replica assignment when start up master
|
||||
// this test can be removed once we remove the HConstants.META_REPLICAS_NUM config.
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setInt(HConstants.META_REPLICAS_NUM, 3);
|
||||
StartMiniClusterOption option = StartMiniClusterOption.builder().numAlwaysStandByMasters(1)
|
||||
|
@ -100,9 +106,20 @@ public class TestFailedMetaReplicaAssigment {
|
|||
}
|
||||
|
||||
public static class BrokenTransitRegionStateProcedure extends TransitRegionStateProcedure {
|
||||
protected BrokenTransitRegionStateProcedure() {
|
||||
|
||||
public BrokenTransitRegionStateProcedure() {
|
||||
super(null, null, null, false, TransitionType.ASSIGN);
|
||||
}
|
||||
|
||||
public BrokenTransitRegionStateProcedure(MasterProcedureEnv env, RegionInfo hri) {
|
||||
super(env, hri, null, false, TransitionType.ASSIGN);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Procedure[] execute(MasterProcedureEnv env)
|
||||
throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
|
||||
throw new ProcedureSuspendedException("Never end procedure!");
|
||||
}
|
||||
}
|
||||
|
||||
public static class BrokenMetaReplicaMaster extends HMaster {
|
||||
|
@ -124,12 +141,24 @@ public class TestFailedMetaReplicaAssigment {
|
|||
this.master = master;
|
||||
}
|
||||
|
||||
public Future<byte[]> assignAsync(RegionInfo regionInfo, ServerName sn) throws IOException {
|
||||
RegionStateNode regionNode = getRegionStates().getOrCreateRegionStateNode(regionInfo);
|
||||
if (regionNode.getRegionInfo().getReplicaId() == 2) {
|
||||
regionNode.setProcedure(new BrokenTransitRegionStateProcedure());
|
||||
@Override
|
||||
public TransitRegionStateProcedure[] createAssignProcedures(List<RegionInfo> hris) {
|
||||
List<TransitRegionStateProcedure> procs = new ArrayList<>();
|
||||
for (RegionInfo hri : hris) {
|
||||
if (hri.isMetaRegion() && hri.getReplicaId() == 2) {
|
||||
RegionStateNode regionNode = getRegionStates().getOrCreateRegionStateNode(hri);
|
||||
regionNode.lock();
|
||||
try {
|
||||
procs.add(regionNode.setProcedure(new BrokenTransitRegionStateProcedure(
|
||||
master.getMasterProcedureExecutor().getEnvironment(), hri)));
|
||||
} finally {
|
||||
regionNode.unlock();
|
||||
}
|
||||
return super.assignAsync(regionInfo, sn);
|
||||
} else {
|
||||
procs.add(super.createAssignProcedures(Collections.singletonList(hri))[0]);
|
||||
}
|
||||
}
|
||||
return procs.toArray(new TransitRegionStateProcedure[0]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,68 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Make sure we will honor the {@link HConstants#META_REPLICAS_NUM}.And also test upgrading.
|
||||
*/
|
||||
@Category({ MiscTests.class, MediumTests.class })
|
||||
public class TestIncreaseMetaReplicaThroughConfig extends MetaWithReplicasTestBase {
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestIncreaseMetaReplicaThroughConfig.class);
|
||||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
startCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpgradeAndIncreaseReplicaCount() throws Exception {
|
||||
HMaster oldMaster = TEST_UTIL.getMiniHBaseCluster().getMaster();
|
||||
TableDescriptors oldTds = oldMaster.getTableDescriptors();
|
||||
TableDescriptor oldMetaTd = oldTds.get(TableName.META_TABLE_NAME);
|
||||
assertEquals(3, oldMetaTd.getRegionReplication());
|
||||
// force update the replica count to 1 and then kill the master, to simulate that hen upgrading,
|
||||
// we have no region replication in meta table descriptor but we actually have meta region
|
||||
// replicas
|
||||
oldTds.update(TableDescriptorBuilder.newBuilder(oldMetaTd).setRegionReplication(1).build());
|
||||
oldMaster.stop("Restarting");
|
||||
TEST_UTIL.waitFor(30000, () -> oldMaster.isStopped());
|
||||
|
||||
// increase replica count to 5 through Configuration
|
||||
TEST_UTIL.getMiniHBaseCluster().getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 5);
|
||||
TEST_UTIL.getMiniHBaseCluster().startMaster();
|
||||
TEST_UTIL.waitFor(30000,
|
||||
() -> TEST_UTIL.getZooKeeperWatcher().getMetaReplicaNodes().size() == 5);
|
||||
}
|
||||
}
|
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.StartMiniClusterOption;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Waiter;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
|
@ -45,6 +45,7 @@ import org.junit.BeforeClass;
|
|||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
|
||||
@Category({ MediumTests.class, ClientTests.class })
|
||||
|
@ -57,10 +58,10 @@ public class TestMasterRegistry {
|
|||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
|
||||
StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
|
||||
builder.numMasters(3).numRegionServers(3);
|
||||
TEST_UTIL.startMiniCluster(builder.build());
|
||||
HBaseTestingUtility.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.MultithreadedTestUtil;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MetaRegionLocationCache;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
|
@ -57,8 +59,8 @@ public class TestMetaRegionLocationCache {
|
|||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
|
||||
TEST_UTIL.startMiniCluster(3);
|
||||
HBaseTestingUtility.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3);
|
||||
REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
|
||||
RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY);
|
||||
TEST_UTIL.getAdmin().balancerSwitch(false, true);
|
||||
|
|
|
@ -20,17 +20,15 @@ package org.apache.hadoop.hbase.client;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
|
||||
|
@ -82,27 +80,6 @@ public class TestMetaWithReplicasBasic extends MetaWithReplicasTestBase {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReplicaCleanup() throws Exception {
|
||||
ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||
List<String> metaReplicaZnodes = zkw.getMetaReplicaNodes();
|
||||
assertEquals(3, metaReplicaZnodes.size());
|
||||
|
||||
final HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
|
||||
master.stop("Restarting");
|
||||
TEST_UTIL.waitFor(30000, () -> master.isStopped());
|
||||
TEST_UTIL.getMiniHBaseCluster().getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 1);
|
||||
|
||||
JVMClusterUtil.MasterThread newMasterThread = TEST_UTIL.getMiniHBaseCluster().startMaster();
|
||||
final HMaster newMaster = newMasterThread.getMaster();
|
||||
|
||||
//wait until new master finished meta replica assignment logic
|
||||
TEST_UTIL.waitFor(30000, () -> newMaster.getMasterQuotaManager() != null);
|
||||
zkw = TEST_UTIL.getZooKeeperWatcher();
|
||||
metaReplicaZnodes = zkw.getMetaReplicaNodes();
|
||||
assertEquals(1, metaReplicaZnodes.size());
|
||||
|
||||
}
|
||||
@Test
|
||||
public void testAccessingUnknownTables() throws Exception {
|
||||
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
|
||||
|
|
|
@ -249,9 +249,6 @@ public class TestReplicaWithCluster {
|
|||
HTU.getConfiguration().setInt("hbase.client.primaryCallTimeout.get", 1000000);
|
||||
HTU.getConfiguration().setInt("hbase.client.primaryCallTimeout.scan", 1000000);
|
||||
|
||||
// Enable meta replica at server side
|
||||
HTU.getConfiguration().setInt("hbase.meta.replica.count", 2);
|
||||
|
||||
// Make sure master does not host system tables.
|
||||
HTU.getConfiguration().set("hbase.balancer.tablesOnMaster", "none");
|
||||
|
||||
|
@ -263,6 +260,9 @@ public class TestReplicaWithCluster {
|
|||
META_SCAN_TIMEOUT_IN_MILLISEC * 1000);
|
||||
|
||||
HTU.startMiniCluster(NB_SERVERS);
|
||||
// Enable meta replica at server side
|
||||
HBaseTestingUtility.setReplicas(HTU.getAdmin(), TableName.META_TABLE_NAME, 2);
|
||||
|
||||
HTU.getHBaseCluster().startMaster();
|
||||
}
|
||||
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM;
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotEquals;
|
||||
|
@ -25,6 +24,7 @@ import static org.junit.Assert.assertNotNull;
|
|||
import static org.junit.Assert.assertNotSame;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.stream.IntStream;
|
||||
|
@ -62,13 +62,9 @@ public class TestZKConnectionRegistry {
|
|||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
|
||||
TEST_UTIL.startMiniCluster(3);
|
||||
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
|
||||
// make sure that we do not depend on this config when getting locations for meta replicas, see
|
||||
// HBASE-21658.
|
||||
conf.setInt(META_REPLICAS_NUM, 1);
|
||||
REGISTRY = new ZKConnectionRegistry(conf);
|
||||
HBaseTestingUtility.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3);
|
||||
REGISTRY = new ZKConnectionRegistry(TEST_UTIL.getConfiguration());
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
|
|
@ -64,10 +64,10 @@ public class AlwaysStandByHMaster extends HMaster {
|
|||
if (MasterAddressTracker.getMasterAddress(watcher) != null) {
|
||||
clusterHasActiveMaster.set(true);
|
||||
}
|
||||
Threads.sleepWithoutInterrupt(100);
|
||||
} catch (IOException e) {
|
||||
// pass, we will get notified when some other active master creates the znode.
|
||||
}
|
||||
Threads.sleepWithoutInterrupt(1000);
|
||||
} catch (KeeperException e) {
|
||||
master.abort("Received an unexpected KeeperException, aborting", e);
|
||||
return false;
|
||||
|
|
|
@ -71,7 +71,7 @@ public class TestReadAndWriteRegionInfoFile {
|
|||
RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
|
||||
// Create a region. That'll write the .regioninfo file.
|
||||
FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(FS, ROOT_DIR);
|
||||
FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(CONF, FS, ROOT_DIR, null);
|
||||
FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(CONF, FS, ROOT_DIR);
|
||||
HRegion r = HBaseTestingUtility.createRegionAndWAL(ri, ROOT_DIR, CONF,
|
||||
fsTableDescriptors.get(TableName.META_TABLE_NAME));
|
||||
// Get modtime on the file.
|
||||
|
|
Loading…
Reference in New Issue