HBASE-21191 Add a holding-pattern if no assign for meta or namespace (Can happen if masterprocwals have been cleared).

Add a check for hbase:meta being online before we go to read it.
If not online, move into a holding-pattern until rectified, probably
by external operator.

Incorporates bulk of patch made by Allan Yang over on HBASE-21035.

M hbase-common/src/main/java/org/apache/hadoop/hbase/util/RetryCounterFactory.java

 Add a Constructor for case where retries are for ever.

M hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
 Move stuff around so that the first hbase:meta read is the AM#loadMeta.
 Previously, checking table state and/or favored nodes could end up
 trying to read a meta that was not onlined holding up master startup.
 Do similar for the namespace table. Adds new methods isMeta and
 isNamespace which check that the regions/tables are online.. if not,
 we wait logging with a back-off that assigns need to be run.

Signed-off-by: Allan Yang <allan163@apache.org>
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
Michael Stack 2018-09-12 10:47:33 -07:00
parent a925a4ce16
commit 39e0b8515f
12 changed files with 298 additions and 35 deletions

View File

@ -28,6 +28,10 @@ import org.apache.yetus.audience.InterfaceAudience;
public class RetryCounterFactory {
private final RetryConfig retryConfig;
public RetryCounterFactory(int sleepIntervalMillis) {
this(Integer.MAX_VALUE, sleepIntervalMillis);
}
public RetryCounterFactory(int maxAttempts, int sleepIntervalMillis) {
this(maxAttempts, sleepIntervalMillis, -1);
}

View File

@ -55,6 +55,7 @@ import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@ -191,6 +192,8 @@ import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.IdLock;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
@ -882,11 +885,13 @@ public class HMaster extends HRegionServer implements MasterServices {
// Start RegionServerTracker with listing of servers found with exiting SCPs -- these should
// be registered in the deadServers set -- and with the list of servernames out on the
// filesystem that COULD BE 'alive' (we'll schedule SCPs for each and let SCP figure it out).
// We also pass dirs that are already 'splitting'... so we can do some checks down in tracker.
// TODO: Generate the splitting and live Set in one pass instead of two as we currently do.
this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
this.regionServerTracker.start(
procedureExecutor.getProcedures().stream().filter(p -> p instanceof ServerCrashProcedure)
.map(p -> ((ServerCrashProcedure) p).getServerName()).collect(Collectors.toSet()),
walManager.getLiveServersFromWALDir());
walManager.getLiveServersFromWALDir(), walManager.getSplittingServersFromWALDir());
// This manager will be started AFTER hbase:meta is confirmed on line.
// hbase.mirror.table.state.to.zookeeper is so hbase1 clients can connect. They read table
// state from zookeeper while hbase2 reads it from hbase:meta. Disable if no hbase1 clients.
@ -916,10 +921,14 @@ public class HMaster extends HRegionServer implements MasterServices {
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");
InitMetaProcedure initMetaProc = null;
if (assignmentManager.getRegionStates().getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO)
.isOffline()) {
// 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.isOffline()) {
Optional<Procedure<MasterProcedureEnv>> optProc = procedureExecutor.getProcedures().stream()
.filter(p -> p instanceof InitMetaProcedure).findAny();
if (optProc.isPresent()) {
@ -946,7 +955,6 @@ public class HMaster extends HRegionServer implements MasterServices {
if (initMetaProc != null) {
initMetaProc.await();
}
tableStateManager.start();
// Wake up this server to check in
sleeper.skipSleepCycle();
@ -963,7 +971,20 @@ public class HMaster extends HRegionServer implements MasterServices {
return;
}
//Initialize after meta as it scans meta
status.setStatus("Starting assignment manager");
// FIRST HBASE:META READ!!!!
// The below cannot make progress w/o hbase:meta being online.
// 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.
if (!waitUntilMetaOnline()) {
return;
}
this.assignmentManager.joinCluster();
// The below depends on hbase:meta being online.
this.tableStateManager.start();
// Initialize after meta is up as below scans meta
if (favoredNodesManager != null) {
SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
new SnapshotOfRegionAssignmentFromMeta(getConnection());
@ -971,10 +992,6 @@ public class HMaster extends HRegionServer implements MasterServices {
favoredNodesManager.initialize(snapshotOfRegionAssignment);
}
// Fix up assignment manager status
status.setStatus("Starting assignment manager");
this.assignmentManager.joinCluster();
// set cluster status again after user regions are assigned
this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
@ -989,6 +1006,13 @@ public class HMaster extends HRegionServer implements MasterServices {
this.catalogJanitorChore = new CatalogJanitor(this);
getChoreService().scheduleChore(catalogJanitorChore);
// NAMESPACE READ!!!!
// Here we expect hbase:namespace to be online. See inside initClusterSchemaService.
// TODO: Fix this. Namespace is a pain being a sort-of system table. Fold it in to hbase:meta.
// isNamespace does like isMeta and waits until namespace is onlined before allowing progress.
if (!waitUntilNamespaceOnline()) {
return;
}
status.setStatus("Starting cluster schema service");
initClusterSchemaService();
@ -1064,6 +1088,68 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
/**
* Check hbase:meta is up and ready for reading. For use during Master startup only.
* @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
* and we will hold here until operator intervention.
*/
@VisibleForTesting
public boolean waitUntilMetaOnline() throws InterruptedException {
return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
}
/**
* @return True if region is online and scannable else false if an error or shutdown (Otherwise
* we just block in here holding up all forward-progess).
*/
private boolean isRegionOnline(RegionInfo ri) throws InterruptedException {
RetryCounter rc = null;
while (!isStopped()) {
RegionState rs = this.assignmentManager.getRegionStates().getRegionState(ri);
if (rs.isOpened()) {
if (this.getServerManager().isServerOnline(rs.getServerName())) {
return true;
}
}
// Region is not OPEN.
Optional<Procedure<MasterProcedureEnv>> optProc = this.procedureExecutor.getProcedures().
stream().filter(p -> p instanceof ServerCrashProcedure).findAny();
// TODO: Add a page to refguide on how to do repair. Have this log message point to it.
// Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
// then how to assign including how to break region lock if one held.
LOG.warn("{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot " +
"progress, in holding-pattern until region onlined; operator intervention required. " +
"Schedule an assign.", ri.getRegionNameAsString(), rs, optProc.isPresent());
// Check once-a-minute.
if (rc == null) {
rc = new RetryCounterFactory(1000).create();
}
Threads.sleep(rc.getBackoffTimeAndIncrementAttempts());
}
return false;
}
/**
* Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table
* (TODO: Fix this! NS should not hold-up startup).
* @return True if namespace table is up/online.
*/
@VisibleForTesting
public boolean waitUntilNamespaceOnline() throws InterruptedException {
List<RegionInfo> ris = this.assignmentManager.getRegionStates().
getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);
if (ris.isEmpty()) {
// If empty, means we've not assigned the namespace table yet... Just return true so startup
// continues and the namespace table gets created.
return true;
}
// Else there are namespace regions up in meta. Ensure they are assigned before we go on.
for (RegionInfo ri: ris) {
isRegionOnline(ri);
}
return true;
}
/**
* Adds the {@code MasterQuotasObserver} to the list of configured Master observers to
* automatically remove quotas for a table when that table is deleted.

View File

@ -530,9 +530,8 @@ public class MasterRpcServices extends RSRpcServices
RpcController controller, ReportRSFatalErrorRequest request) throws ServiceException {
String errorText = request.getErrorMessage();
ServerName sn = ProtobufUtil.toServerName(request.getServer());
String msg = "Region server " + sn
+ " reported a fatal error:\n" + errorText;
LOG.error(msg);
String msg = sn + " reported a fatal error:\n" + errorText;
LOG.warn(msg);
master.rsFatals.add(msg);
return ReportRSFatalErrorResponse.newBuilder().build();
}

View File

@ -144,18 +144,33 @@ public class MasterWalManager {
}
/**
* @return listing of ServerNames found in the filesystem under the WAL directory
* that COULD BE 'alive'; excludes those that have a '-splitting' suffix as these are already
* being split -- they cannot be 'alive'.
* Get Servernames which are currently splitting; paths have a '-splitting' suffix.
* @return ServerName
* @throws IOException IOException
*/
public Set<ServerName> getSplittingServersFromWALDir() throws IOException {
return getServerNamesFromWALDirPath(
p -> p.getName().endsWith(AbstractFSWALProvider.SPLITTING_EXT));
}
/**
* Get Servernames that COULD BE 'alive'; excludes those that have a '-splitting' suffix as these
* are already being split -- they cannot be 'alive'.
* @return ServerName
* @throws IOException IOException
*/
public Set<ServerName> getLiveServersFromWALDir() throws IOException {
Path walDirPath = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
FileStatus[] walDirForLiveServers = FSUtils.listStatus(fs, walDirPath,
return getServerNamesFromWALDirPath(
p -> !p.getName().endsWith(AbstractFSWALProvider.SPLITTING_EXT));
if (walDirForLiveServers == null) {
return Collections.emptySet();
}
return Stream.of(walDirForLiveServers).map(s -> {
/**
* @return listing of ServerNames found by parsing WAL directory paths in FS.
*
*/
public Set<ServerName> getServerNamesFromWALDirPath(final PathFilter filter) throws IOException {
FileStatus[] walDirForServerNames = getWALDirPaths(filter);
return Stream.of(walDirForServerNames).map(s -> {
ServerName serverName = AbstractFSWALProvider.getServerNameFromWALDirectoryName(s.getPath());
if (serverName == null) {
LOG.warn("Log folder {} doesn't look like its name includes a " +
@ -167,6 +182,15 @@ public class MasterWalManager {
}).filter(s -> s != null).collect(Collectors.toSet());
}
/**
* @return List of all RegionServer WAL dirs; i.e. this.rootDir/HConstants.HREGION_LOGDIR_NAME.
*/
public FileStatus[] getWALDirPaths(final PathFilter filter) throws IOException {
Path walDirPath = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
FileStatus[] walDirForServerNames = FSUtils.listStatus(fs, walDirPath, filter);
return walDirForServerNames == null? new FileStatus[0]: walDirForServerNames;
}
/**
* Inspect the log directory to find dead servers which need recovery work
* @return A set of ServerNames which aren't running but still have WAL files left in file system

View File

@ -115,11 +115,19 @@ public class RegionServerTracker extends ZKListener {
* protection to prevent concurrency issues with server expiration operation.
* @param deadServersFromPE the region servers which already have SCP associated.
* @param liveServersFromWALDir the live region servers from wal directory.
* @param splittingServersFromWALDir Servers whose WALs are being actively 'split'.
*/
public void start(Set<ServerName> deadServersFromPE, Set<ServerName> liveServersFromWALDir)
public void start(Set<ServerName> deadServersFromPE, Set<ServerName> liveServersFromWALDir,
Set<ServerName> splittingServersFromWALDir)
throws KeeperException, IOException {
LOG.info("Starting RegionServerTracker; {} have existing ServerCrashProcedures, {} " +
"possibly 'live' servers.", deadServersFromPE.size(), liveServersFromWALDir.size());
"possibly 'live' servers, and {} 'splitting'.", deadServersFromPE.size(),
liveServersFromWALDir.size(), splittingServersFromWALDir.size());
// deadServersFromPE is made from a list of outstanding ServerCrashProcedures.
// splittingServersFromWALDir are being actively split -- the directory in the FS ends in
// '-SPLITTING'. Each splitting server should have a corresponding SCP. Log if not.
splittingServersFromWALDir.stream().map(s -> !deadServersFromPE.contains(s)).
forEach(s -> LOG.error("{} has no matching ServerCrashProcedure", s));
watcher.registerListener(this);
synchronized (this) {
List<String> servers =

View File

@ -383,12 +383,16 @@ public class TableNamespaceManager implements Stoppable {
return;
}
try {
if (this.zkNamespaceManager != null) {
this.zkNamespaceManager.stop();
}
} catch (IOException ioe) {
LOG.warn("Failed NamespaceManager close", ioe);
}
try {
if (this.nsTable != null) {
this.nsTable.close();
}
} catch (IOException ioe) {
LOG.warn("Failed Namespace Table close", ioe);
}

View File

@ -1186,8 +1186,9 @@ public class AssignmentManager implements ServerListener {
long startTime = System.nanoTime();
LOG.debug("Joining cluster...");
// Scan hbase:meta to build list of existing regions, servers, and assignment
// hbase:meta is online when we get to here and TableStateManager has been started.
// Scan hbase:meta to build list of existing regions, servers, and assignment.
// hbase:meta is online now or will be. Inside loadMeta, we keep trying. Can't make progress
// w/o meta.
loadMeta();
while (master.getServerManager().countOfRegionServers() < 1) {

View File

@ -256,7 +256,7 @@ public class DisableTableProcedure
TableStateManager tsm = env.getMasterServices().getTableStateManager();
TableState ts = tsm.getTableState(tableName);
if (!ts.isEnabled()) {
LOG.info("Not ENABLED skipping {}", this);
LOG.info("Not ENABLED, state={}, skipping disable; {}", ts.getState(), this);
setFailure("master-disable-table", new TableNotEnabledException(ts.toString()));
canTableBeDisabled = false;
}

View File

@ -335,7 +335,7 @@ public class EnableTableProcedure
TableStateManager tsm = env.getMasterServices().getTableStateManager();
TableState ts = tsm.getTableState(tableName);
if(!ts.isDisabled()){
LOG.info("Not DISABLED tableState=" + ts + "; skipping enable");
LOG.info("Not DISABLED tableState={}; skipping enable; {}", ts.getState(), this);
setFailure("master-enable-table", new TableNotDisabledException(ts.toString()));
canTableBeEnabled = false;
}

View File

@ -711,8 +711,12 @@ public class HRegionServer extends HasThread implements
"hbase.regionserver.kerberos.principal", host);
}
protected void waitForMasterActive() {
}
/**
* Wait for an active Master.
* See override in Master superclass for how it is used.
*/
protected void waitForMasterActive() {}
protected String getProcessName() {
return REGIONSERVER;
@ -873,10 +877,6 @@ public class HRegionServer extends HasThread implements
}
}
// In case colocated master, wait here till it's active.
// So backup masters won't start as regionservers.
// This is to avoid showing backup masters as regionservers
// in master web UI, or assigning any region to them.
waitForMasterActive();
if (isStopped() || isAborted()) {
return; // No need for further initialization

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.ipc.CallRunner;
import org.apache.hadoop.hbase.ipc.DelegatingRpcScheduler;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.ipc.RpcScheduler;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory;
@ -104,6 +105,21 @@ public class TestMetaTableAccessor {
UTIL.shutdownMiniCluster();
}
@Test
public void testIsMetaWhenAllHealthy() throws InterruptedException {
HMaster m = UTIL.getMiniHBaseCluster().getMaster();
assertTrue(m.waitUntilMetaOnline());
}
@Test
public void testIsMetaWhenMetaGoesOffline() throws InterruptedException {
HMaster m = UTIL.getMiniHBaseCluster().getMaster();
int index = UTIL.getMiniHBaseCluster().getServerWithMeta();
HRegionServer rsWithMeta = UTIL.getMiniHBaseCluster().getRegionServer(index);
rsWithMeta.abort("TESTING");
assertTrue(m.waitUntilMetaOnline());
}
/**
* Does {@link MetaTableAccessor#getRegion(Connection, byte[])} and a write
* against hbase:meta while its hosted server is restarted to prove our retrying

View File

@ -0,0 +1,121 @@
/**
* 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.assignment;
import static org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore.MASTER_PROCEDURE_LOGDIR;
import java.util.List;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Category({MasterTests.class, MediumTests.class})
public class TestMetaInitIfAllProceduresLost {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetaInitIfAllProceduresLost.class);
private static final Logger LOG = LoggerFactory
.getLogger(TestMetaInitIfAllProceduresLost.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@BeforeClass
public static void setupCluster() throws Exception {
UTIL.startMiniCluster(3);
}
@AfterClass
public static void cleanupTest() throws Exception {
try {
UTIL.shutdownMiniCluster();
} catch (Exception e) {
LOG.warn("failure shutting down cluster", e);
}
}
@Test
public void test() throws Exception {
for (JVMClusterUtil.RegionServerThread rst : UTIL.getMiniHBaseCluster()
.getRegionServerThreads()) {
rst.getRegionServer().abort("killAll");
}
//wait for a while, until all dirs are changed to '-splitting'
UTIL.waitFor(30000, () ->
UTIL.getMiniHBaseCluster().getMaster().getMasterWalManager()
.getLiveServersFromWALDir().size() == 0);
Thread.sleep(1000);
Path procedureWals = new Path(
UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem()
.getRootDir(), MASTER_PROCEDURE_LOGDIR);
//Kill the master
UTIL.getMiniHBaseCluster().killAll();
//Delte all procedure log to create an anomaly
for (FileStatus file : UTIL.getTestFileSystem().listStatus(procedureWals)) {
LOG.info("removing " + file);
UTIL.getTestFileSystem().delete(file.getPath());
}
UTIL.getMiniHBaseCluster().startMaster();
UTIL.getMiniHBaseCluster().startRegionServer();
UTIL.getMiniHBaseCluster().startRegionServer();
UTIL.getMiniHBaseCluster().startRegionServer();
ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1);
// Schedule an assign of meta after ten seconds. Then a few seconds later, do namespace assign.
// The meta table needs to be online before the namespace can succeed.
final HMaster master = UTIL.getHBaseCluster().getMaster();
final AssignmentManager am = master.getAssignmentManager();
final AssignProcedure ap = am.createAssignProcedure(RegionInfoBuilder.FIRST_META_REGIONINFO);
scheduler.schedule(() -> master.getMasterProcedureExecutor().submitProcedure(ap), 10,
TimeUnit.SECONDS);
scheduler.schedule(() -> {
// hbase:meta should be online by the time this runs. That means we should have read the
// regions that make up the namespace table so below query should return results.
List<RegionInfo> ris = am.getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);
if (ris.isEmpty()) {
throw new RuntimeException("No namespace regions found!");
}
for (RegionInfo ri: ris) {
AssignProcedure riap = am.createAssignProcedure(ri);
master.getMasterProcedureExecutor().submitProcedure(riap);
}
}, 20 /*Must run AFTER meta is online*/, TimeUnit.SECONDS);
// Master should able to finish init even if all procedures are lost
UTIL.waitFor(180000, () -> UTIL.getMiniHBaseCluster().getMaster() != null && UTIL
.getMiniHBaseCluster().getMaster().isInitialized());
}
}