HBASE-21073 Redo concept of maintenance mode

Instead of being an ephemeral state set by hbck, maintenance mode is now
an explicit toggle set by either configuration property or environment
variable. In maintenance mode, master will host system tables and not
assign any user-space tables to RSs. This gives operators the ability to
affect repairs to meta table with fewer moving parts.
This commit is contained in:
Mike Drob 2018-10-08 14:28:23 -05:00
parent 5fbb227deb
commit bc7628a8c3
No known key found for this signature in database
GPG Key ID: 3E48C0C6EF362B9E
12 changed files with 260 additions and 92 deletions

View File

@ -207,7 +207,6 @@ 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.MasterMaintenanceModeTracker;
import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@ -329,9 +328,6 @@ public class HMaster extends HRegionServer implements MasterServices {
// Tracker for region normalizer state
private RegionNormalizerTracker regionNormalizerTracker;
//Tracker for master maintenance mode setting
private MasterMaintenanceModeTracker maintenanceModeTracker;
private ClusterSchemaService clusterSchemaService;
public static final String HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS =
@ -441,6 +437,11 @@ public class HMaster extends HRegionServer implements MasterServices {
/** jetty server for master to redirect requests to regionserver infoServer */
private Server masterJettyServer;
// Determine if we should do normal startup or minimal "single-user" mode with no region
// servers and no user tables. Useful for repair and recovery of hbase:meta
private final boolean maintenanceMode;
static final String MAINTENANCE_MODE = "hbase.master.maintenance_mode";
public static class RedirectServlet extends HttpServlet {
private static final long serialVersionUID = 2894774810058302473L;
private final int regionServerInfoPort;
@ -500,6 +501,16 @@ public class HMaster extends HRegionServer implements MasterServices {
super(conf);
TraceUtil.initTracer(conf);
try {
if (conf.getBoolean(MAINTENANCE_MODE, false)) {
LOG.info("Detected {}=true via configuration.", MAINTENANCE_MODE);
maintenanceMode = true;
} else if (Boolean.getBoolean(MAINTENANCE_MODE)) {
LOG.info("Detected {}=true via environment variables.", MAINTENANCE_MODE);
maintenanceMode = true;
} else {
maintenanceMode = false;
}
this.rsFatals = new MemoryBoundedLogMessageBuffer(
conf.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024));
LOG.info("hbase.rootdir=" + getRootDir() +
@ -684,6 +695,9 @@ public class HMaster extends HRegionServer implements MasterServices {
*/
@Override
protected void waitForMasterActive(){
if (maintenanceMode) {
return;
}
boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(conf);
while (!(tablesOnMaster && activeMaster) && !isStopped() && !isAborted()) {
sleeper.sleep();
@ -769,9 +783,6 @@ public class HMaster extends HRegionServer implements MasterServices {
this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, this.serverManager);
this.drainingServerTracker.start();
this.maintenanceModeTracker = new MasterMaintenanceModeTracker(zooKeeper);
this.maintenanceModeTracker.start();
String clientQuorumServers = conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM);
boolean clientZkObserverMode = conf.getBoolean(HConstants.CLIENT_ZOOKEEPER_OBSERVER_MODE,
HConstants.DEFAULT_CLIENT_ZOOKEEPER_OBSERVER_MODE);
@ -808,7 +819,7 @@ public class HMaster extends HRegionServer implements MasterServices {
this.mpmHost.initialize(this, this.metricsMaster);
}
private static final ImmutableSet<Class<?>> UNSUPPORTED_PROCEDURES =
private static final ImmutableSet<Class<? extends Procedure>> UNSUPPORTED_PROCEDURES =
ImmutableSet.of(RecoverMetaProcedure.class, AssignProcedure.class, UnassignProcedure.class,
MoveRegionProcedure.class);
@ -825,7 +836,7 @@ public class HMaster extends HRegionServer implements MasterServices {
// support both the old assign/unassign procedures and the new TransitRegionStateProcedure as
// there will be conflict in the code for AM. We should finish all these procedures before
// upgrading.
for (Class<?> clazz : UNSUPPORTED_PROCEDURES) {
for (Class<? extends Procedure> clazz : UNSUPPORTED_PROCEDURES) {
List<Procedure<MasterProcedureEnv>> procs = procsByType.get(clazz);
if (procs != null) {
LOG.error(
@ -994,14 +1005,16 @@ public class HMaster extends HRegionServer implements MasterServices {
new ReplicationPeerConfigUpgrader(zooKeeper, conf);
tableCFsUpdater.copyTableCFs();
// Add the Observer to delete quotas on table deletion before starting all CPs by
// default with quota support, avoiding if user specifically asks to not load this Observer.
if (QuotaUtil.isQuotaEnabled(conf)) {
updateConfigurationForQuotasObserver(conf);
if (!maintenanceMode) {
// Add the Observer to delete quotas on table deletion before starting all CPs by
// default with quota support, avoiding if user specifically asks to not load this Observer.
if (QuotaUtil.isQuotaEnabled(conf)) {
updateConfigurationForQuotasObserver(conf);
}
// initialize master side coprocessors before we start handling requests
status.setStatus("Initializing master coprocessors");
this.cpHost = new MasterCoprocessorHost(this, this.conf);
}
// initialize master side coprocessors before we start handling requests
status.setStatus("Initializing master coprocessors");
this.cpHost = new MasterCoprocessorHost(this, this.conf);
// Checking if meta needs initializing.
status.setStatus("Initializing meta table if this is a new deploy");
@ -1011,15 +1024,14 @@ public class HMaster extends HRegionServer implements MasterServices {
getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO);
LOG.info("hbase:meta {}", rs);
if (rs.isOffline()) {
Optional<Procedure<MasterProcedureEnv>> optProc = procedureExecutor.getProcedures().stream()
.filter(p -> p instanceof InitMetaProcedure).findAny();
if (optProc.isPresent()) {
initMetaProc = (InitMetaProcedure) optProc.get();
} else {
Optional<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream()
.filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
initMetaProc = optProc.orElseGet(() -> {
// schedule an init meta procedure if meta has not been deployed yet
initMetaProc = new InitMetaProcedure();
procedureExecutor.submitProcedure(initMetaProc);
}
InitMetaProcedure temp = new InitMetaProcedure();
procedureExecutor.submitProcedure(temp);
return temp;
});
}
if (this.balancer instanceof FavoredNodesPromoter) {
favoredNodesManager = new FavoredNodesManager(this);
@ -1059,7 +1071,7 @@ public class HMaster extends HRegionServer implements MasterServices {
// This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
// as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
// if it is down. It may take a while to come online. So, wait here until meta if for sure
// available. Thats what waitUntilMetaOnline does.
// available. That's what waitForMetaOnline does.
if (!waitForMetaOnline()) {
return;
}
@ -1067,7 +1079,7 @@ public class HMaster extends HRegionServer implements MasterServices {
// The below depends on hbase:meta being online.
this.tableStateManager.start();
// Initialize after meta is up as below scans meta
if (favoredNodesManager != null) {
if (favoredNodesManager != null && !maintenanceMode) {
SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
new SnapshotOfRegionAssignmentFromMeta(getConnection());
snapshotOfRegionAssignment.initialize();
@ -1116,6 +1128,12 @@ public class HMaster extends HRegionServer implements MasterServices {
configurationManager.registerObserver(this.logCleaner);
// Set master as 'initialized'.
setInitialized(true);
if (maintenanceMode) {
LOG.info("Detected repair mode, skipping final initialization steps.");
return;
}
assignmentManager.checkIfShouldMoveSystemRegionAsync();
status.setStatus("Assign meta replicas");
MasterMetaBootstrap metaBootstrap = createMetaBootstrap();
@ -3083,11 +3101,8 @@ public class HMaster extends HRegionServer implements MasterServices {
* @return true if master is in maintenanceMode
*/
@Override
public boolean isInMaintenanceMode() throws IOException {
if (!isInitialized()) {
throw new PleaseHoldException("Master is initializing");
}
return maintenanceModeTracker.isInMaintenanceMode();
public boolean isInMaintenanceMode() {
return maintenanceMode;
}
@VisibleForTesting
@ -3606,14 +3621,9 @@ public class HMaster extends HRegionServer implements MasterServices {
* @return The state of the load balancer, or false if the load balancer isn't defined.
*/
public boolean isBalancerOn() {
try {
if (null == loadBalancerTracker || isInMaintenanceMode()) {
return false;
}
} catch (IOException e) {
return false;
}
return loadBalancerTracker.isBalancerOn();
return !isInMaintenanceMode()
&& loadBalancerTracker != null
&& loadBalancerTracker.isBalancerOn();
}
/**
@ -3621,12 +3631,9 @@ public class HMaster extends HRegionServer implements MasterServices {
* false is returned.
*/
public boolean isNormalizerOn() {
try {
return (null == regionNormalizerTracker || isInMaintenanceMode()) ?
false: regionNormalizerTracker.isNormalizerOn();
} catch (IOException e) {
return false;
}
return !isInMaintenanceMode()
&& regionNormalizerTracker != null
&& regionNormalizerTracker.isNormalizerOn();
}
/**
@ -3637,14 +3644,9 @@ public class HMaster extends HRegionServer implements MasterServices {
*/
@Override
public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
try {
if (null == splitOrMergeTracker || isInMaintenanceMode()) {
return false;
}
} catch (IOException e) {
return false;
}
return splitOrMergeTracker.isSplitOrMergeEnabled(switchType);
return !isInMaintenanceMode()
&& splitOrMergeTracker != null
&& splitOrMergeTracker.isSplitOrMergeEnabled(switchType);
}
/**

View File

@ -53,13 +53,12 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
* By default, it carries no tables.
* TODO: Add any | system as flags to indicate what it can do.
*/
public static final String TABLES_ON_MASTER = "hbase.balancer.tablesOnMaster";
String TABLES_ON_MASTER = "hbase.balancer.tablesOnMaster";
/**
* Master carries system tables.
*/
public static final String SYSTEM_TABLES_ON_MASTER =
"hbase.balancer.tablesOnMaster.systemTablesOnly";
String SYSTEM_TABLES_ON_MASTER = "hbase.balancer.tablesOnMaster.systemTablesOnly";
// Used to signal to the caller that the region(s) cannot be assigned
// We deliberately use 'localhost' so the operation will fail fast
@ -177,4 +176,8 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
static boolean isSystemTablesOnlyOnMaster(Configuration conf) {
return conf.getBoolean(SYSTEM_TABLES_ON_MASTER, false);
}
static boolean isMasterCanHostUserRegions(Configuration conf) {
return isTablesOnMaster(conf) && !isSystemTablesOnlyOnMaster(conf);
}
}

View File

@ -353,7 +353,7 @@ public class MasterRpcServices extends RSRpcServices
throws IOException {
// RpcServer at HM by default enable ByteBufferPool iff HM having user table region in it
boolean reservoirEnabled = conf.getBoolean(RESERVOIR_ENABLED_KEY,
(LoadBalancer.isTablesOnMaster(conf) && !LoadBalancer.isSystemTablesOnlyOnMaster(conf)));
LoadBalancer.isMasterCanHostUserRegions(conf));
try {
return RpcServerFactory.createRpcServer(server, name, getServices(),
bindAddress, // use final bindAddress for this server.
@ -1524,11 +1524,7 @@ public class MasterRpcServices extends RSRpcServices
final RpcController controller,
final IsInMaintenanceModeRequest request) throws ServiceException {
IsInMaintenanceModeResponse.Builder response = IsInMaintenanceModeResponse.newBuilder();
try {
response.setInMaintenanceMode(master.isInMaintenanceMode());
} catch (IOException e) {
throw new ServiceException(e);
}
response.setInMaintenanceMode(master.isInMaintenanceMode());
return response.build();
}

View File

@ -351,7 +351,7 @@ public interface MasterServices extends Server {
* @return true if master is in maintanceMode
* @throws IOException if the inquiry failed due to an IO problem
*/
boolean isInMaintenanceMode() throws IOException;
boolean isInMaintenanceMode();
/**
* Abort a procedure.

View File

@ -761,19 +761,22 @@ public class ServerManager {
* RegionServers to check-in.
*/
private int getMinToStart() {
// One server should be enough to get us off the ground.
int requiredMinToStart = 1;
if (LoadBalancer.isTablesOnMaster(master.getConfiguration())) {
if (LoadBalancer.isSystemTablesOnlyOnMaster(master.getConfiguration())) {
// If Master is carrying regions but NOT user-space regions, it
// still shows as a 'server'. We need at least one more server to check
// in before we can start up so set defaultMinToStart to 2.
requiredMinToStart = requiredMinToStart + 1;
}
if (master.isInMaintenanceMode()) {
// If in maintenance mode, then master hosting meta will be the only server available
return 1;
}
int minimumRequired = 1;
if (LoadBalancer.isTablesOnMaster(master.getConfiguration()) &&
LoadBalancer.isSystemTablesOnlyOnMaster(master.getConfiguration())) {
// If Master is carrying regions it will show up as a 'server', but is not handling user-
// space regions, so we need a second server.
minimumRequired = 2;
}
int minToStart = this.master.getConfiguration().getInt(WAIT_ON_REGIONSERVERS_MINTOSTART, -1);
// Ensure we are never less than requiredMinToStart else stuff won't work.
return minToStart == -1 || minToStart < requiredMinToStart? requiredMinToStart: minToStart;
// Ensure we are never less than minimumRequired else stuff won't work.
return Math.max(minToStart, minimumRequired);
}
/**

View File

@ -1012,8 +1012,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
protected ClusterMetrics clusterStatus = null;
protected ServerName masterServerName;
protected MasterServices services;
protected boolean tablesOnMaster;
protected boolean onlySystemTablesOnMaster;
protected boolean maintenanceMode;
@Override
public void setConf(Configuration conf) {
@ -1025,20 +1025,15 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
if (overallSlop < 0) overallSlop = 0;
else if (overallSlop > 1) overallSlop = 1;
this.tablesOnMaster = LoadBalancer.isTablesOnMaster(this.config);
this.onlySystemTablesOnMaster = LoadBalancer.isSystemTablesOnlyOnMaster(this.config);
// If system tables on master, implies tablesOnMaster = true.
if (this.onlySystemTablesOnMaster && !this.tablesOnMaster) {
LOG.warn("Set " + TABLES_ON_MASTER + "=true because " + SYSTEM_TABLES_ON_MASTER + "=true");
this.tablesOnMaster = true;
}
this.rackManager = new RackManager(getConf());
if (useRegionFinder) {
regionFinder.setConf(conf);
}
// Print out base configs. Don't print overallSlop since it for simple balancer exclusively.
LOG.info("slop=" + this.slop + ", tablesOnMaster=" + this.tablesOnMaster +
", systemTablesOnMaster=" + this.onlySystemTablesOnMaster);
LOG.info("slop={}, systemTablesOnMaster={}",
this.slop, this.onlySystemTablesOnMaster);
}
protected void setSlop(Configuration conf) {
@ -1051,7 +1046,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
* If so, the primary replica may be expected to be put on the master regionserver.
*/
public boolean shouldBeOnMaster(RegionInfo region) {
return this.onlySystemTablesOnMaster && region.getTable().isSystemTable();
return (this.maintenanceMode || this.onlySystemTablesOnMaster)
&& region.getTable().isSystemTable();
}
/**
@ -1112,7 +1108,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
return null;
}
Map<ServerName, List<RegionInfo>> assignments = new TreeMap<>();
if (this.onlySystemTablesOnMaster) {
if (this.maintenanceMode || this.onlySystemTablesOnMaster) {
if (masterServerName != null && servers.contains(masterServerName)) {
assignments.put(masterServerName, new ArrayList<>());
for (RegionInfo region : regions) {
@ -1150,6 +1146,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
if (useRegionFinder) {
this.regionFinder.setServices(masterServices);
}
if (this.services.isInMaintenanceMode()) {
this.maintenanceMode = true;
}
}
@Override
@ -1242,7 +1241,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
regions.removeAll(masterRegions);
}
}
if (regions == null || regions.isEmpty()) {
if (this.maintenanceMode || regions == null || regions.isEmpty()) {
return assignments;
}
@ -1415,7 +1414,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
regions = regions.entrySet().stream().filter(e -> !masterRegions.contains(e.getKey()))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
}
if (regions.isEmpty()) {
if (this.maintenanceMode || regions.isEmpty()) {
return assignments;
}

View File

@ -948,6 +948,7 @@ public class HRegionServer extends HasThread implements
// Try and register with the Master; tell it we are here. Break if server is stopped or the
// clusterup flag is down or hdfs went wacky. Once registered successfully, go ahead and start
// up all Services. Use RetryCounter to get backoff in case Master is struggling to come up.
LOG.debug("About to register with Master.");
RetryCounterFactory rcf = new RetryCounterFactory(Integer.MAX_VALUE,
this.sleeper.getPeriod(), 1000 * 60 * 5);
RetryCounter rc = rcf.create();
@ -1812,7 +1813,7 @@ public class HRegionServer extends HasThread implements
*/
private void setupWALAndReplication() throws IOException {
boolean isMasterNoTableOrSystemTableOnly = this instanceof HMaster &&
(!LoadBalancer.isTablesOnMaster(conf) || LoadBalancer.isSystemTablesOnlyOnMaster(conf));
!LoadBalancer.isMasterCanHostUserRegions(conf);
WALFactory factory =
new WALFactory(conf, serverName.toString(), !isMasterNoTableOrSystemTableOnly);
if (!isMasterNoTableOrSystemTableOnly) {

View File

@ -2485,8 +2485,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
private boolean shouldRejectRequestsFromClient(HRegion region) {
return regionServer.getReplicationSourceService().getSyncReplicationPeerInfoProvider()
.checkState(region.getRegionInfo().getTable(), RejectRequestsFromClientStateChecker.get());
TableName table = region.getRegionInfo().getTable();
ReplicationSourceService service = regionServer.getReplicationSourceService();
return service != null && service.getSyncReplicationPeerInfoProvider()
.checkState(table, RejectRequestsFromClientStateChecker.get());
}
private void rejectIfInStandByState(HRegion region) throws DoNotRetryIOException {

View File

@ -74,8 +74,7 @@ public class TestMaster {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final Logger LOG = LoggerFactory.getLogger(TestMaster.class);
private static final TableName TABLENAME =
TableName.valueOf("TestMaster");
private static final TableName TABLENAME = TableName.valueOf("TestMaster");
private static final byte[] FAMILYNAME = Bytes.toBytes("fam");
private static Admin admin;

View File

@ -0,0 +1,141 @@
/*
* 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.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.util.Arrays;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
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.StartMiniClusterOption;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
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.Table;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Category({MasterTests.class, MediumTests.class})
public class TestMasterRepairMode {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMasterRepairMode.class);
@Rule
public TestName name = new TestName();
private static final Logger LOG = LoggerFactory.getLogger(TestMasterRepairMode.class);
private static final byte[] FAMILYNAME = Bytes.toBytes("fam");
private static HBaseTestingUtility TEST_UTIL;
@Before
public void setUp() throws Exception {
TEST_UTIL = new HBaseTestingUtility();
}
@After
public void tearDown() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
private void enableMaintenanceMode() {
Configuration c = TEST_UTIL.getConfiguration();
c.setBoolean(HMaster.MAINTENANCE_MODE, true);
c.setInt("hbase.master.init.timeout.localHBaseCluster", 30000);
}
@Test
public void testNewCluster() throws Exception {
enableMaintenanceMode();
TEST_UTIL.startMiniCluster(StartMiniClusterOption.builder()
.numRegionServers(0)
.numDataNodes(3)
.build());
Connection conn = TEST_UTIL.getConnection();
assertTrue(conn.getAdmin().isMasterInMaintenanceMode());
try (Table table = conn.getTable(TableName.META_TABLE_NAME);
ResultScanner scanner = table.getScanner(new Scan())) {
assertNotNull("Could not read meta.", scanner.next());
}
}
@Test
public void testExistingCluster() throws Exception {
TableName testRepairMode = TableName.valueOf(name.getMethodName());
TEST_UTIL.startMiniCluster();
Table t = TEST_UTIL.createTable(testRepairMode, FAMILYNAME);
Put p = new Put(Bytes.toBytes("r"));
p.addColumn(FAMILYNAME, Bytes.toBytes("c"), new byte[0]);
t.put(p);
TEST_UTIL.shutdownMiniHBaseCluster();
LOG.info("Starting master-only");
enableMaintenanceMode();
TEST_UTIL.startMiniHBaseCluster(StartMiniClusterOption.builder()
.numRegionServers(0).createRootDir(false).build());
Connection conn = TEST_UTIL.getConnection();
assertTrue(conn.getAdmin().isMasterInMaintenanceMode());
try (Table table = conn.getTable(TableName.META_TABLE_NAME);
ResultScanner scanner = table.getScanner(HConstants.TABLE_FAMILY);
Stream<Result> results = StreamSupport.stream(scanner.spliterator(), false)) {
assertTrue("Did not find user table records while reading hbase:meta",
results.anyMatch(r -> Arrays.equals(r.getRow(), testRepairMode.getName())));
}
try (Table table = conn.getTable(testRepairMode);
ResultScanner scanner = table.getScanner(new Scan())) {
scanner.next();
fail("Should not be able to access user-space tables in repair mode.");
} catch (Exception e) {
// Expected
}
}
}

View File

@ -25,6 +25,10 @@ import org.apache.zookeeper.KeeperException;
/**
* Tracks the master Maintenance Mode via ZK.
*
* Unused. Used to be set by hbck to prevent concurrent splits/merges, but those use PV2 now and
* HBCK2 uses it's own service, so no longer an issue. Left in, in case we need to use this for
* the incomplete parts of HBCK2...
*/
@InterfaceAudience.Private
public class MasterMaintenanceModeTracker extends ZKListener {

View File

@ -311,6 +311,24 @@ The commands (and arguments) are:
setquota -n|-b val path
----
[[trouble.tools.maintenancemode]]
==== Maintenance Mode
If the cluster has gotten stuck in some state and the standard techniques aren't making progress,
it is possible to restart the cluster in "maintenance mode." This mode features drastically
reduced capabilities and surface area, making it easier to enact very low-level changes such
as repairing/recovering the `hbase:meta` table.
To enter maintenance mode, set `hbase.master.maintenance_mode` to `true` either in your
`hbase-site.xml` or via system propery when starting the master process (`-D...=true`). Entering
and exiting this mode requires a service restart, however the typical use will be when HBase Master
is already facing startup difficulties.
When maintenance mode is enabled, the master will host all system tables - ensure that it has
enough memory to do so. RegionServers will not be assigned any regions from user-space tables;
in fact, they will go completely unused while in maintenance mode. Additionally, the master will
not load any coprocessors, will not run any normalization or merge/split operations, and will not
enforce quotas.
[[trouble.tools.external]]
=== External Tools