HBASE-12467 Master joins cluster but never completes initialization
Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
1be63539f1
commit
cc8bdcb498
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.net.InetAddress;
|
||||
|
@ -35,6 +36,7 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
@ -119,6 +121,7 @@ import org.apache.hadoop.hbase.util.CompressionTest;
|
|||
import org.apache.hadoop.hbase.util.EncryptionTest;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
|
||||
import org.apache.hadoop.hbase.util.HasThread;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.util.VersionInfo;
|
||||
|
@ -162,6 +165,59 @@ import com.google.protobuf.Service;
|
|||
public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||
private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
|
||||
|
||||
/**
|
||||
* Protection against zombie master. Started once Master accepts active responsibility and
|
||||
* starts taking over responsibilities. Allows a finite time window before giving up ownership.
|
||||
*/
|
||||
private static class InitializationMonitor extends HasThread {
|
||||
/** The amount of time in milliseconds to sleep before checking initialization status. */
|
||||
public static final String TIMEOUT_KEY = "hbase.master.initializationmonitor.timeout";
|
||||
public static final long TIMEOUT_DEFAULT = TimeUnit.MILLISECONDS.convert(15, TimeUnit.MINUTES);
|
||||
|
||||
/**
|
||||
* When timeout expired and initialization has not complete, call {@link System#exit(int)} when
|
||||
* true, do nothing otherwise.
|
||||
*/
|
||||
public static final String HALT_KEY = "hbase.master.initializationmonitor.haltontimeout";
|
||||
public static final boolean HALT_DEFAULT = false;
|
||||
|
||||
private final HMaster master;
|
||||
private final long timeout;
|
||||
private final boolean haltOnTimeout;
|
||||
|
||||
/** Creates a Thread that monitors the {@link #isInitialized()} state. */
|
||||
InitializationMonitor(HMaster master) {
|
||||
super("MasterInitializationMonitor");
|
||||
this.master = master;
|
||||
this.timeout = master.getConfiguration().getLong(TIMEOUT_KEY, TIMEOUT_DEFAULT);
|
||||
this.haltOnTimeout = master.getConfiguration().getBoolean(HALT_KEY, HALT_DEFAULT);
|
||||
this.setDaemon(true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
while (!master.isStopped() && master.isActiveMaster()) {
|
||||
Thread.sleep(timeout);
|
||||
if (master.isInitialized()) {
|
||||
LOG.debug("Initialization completed within allotted tolerance. Monitor exiting.");
|
||||
} else {
|
||||
LOG.error("Master failed to complete initialization after " + timeout + "ms. Please"
|
||||
+ " consider submitting a bug report including a thread dump of this process.");
|
||||
if (haltOnTimeout) {
|
||||
LOG.error("Zombie Master exiting. Thread dump to stdout");
|
||||
org.apache.hadoop.util.ReflectionUtils.printThreadInfo(
|
||||
new PrintWriter(System.out), "Zombie HMaster");
|
||||
System.exit(-1);
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.trace("InitMonitor thread interrupted. Existing.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// MASTER is name of the webapp and the attribute name used stuffing this
|
||||
//instance into web context.
|
||||
public static final String MASTER = "master";
|
||||
|
@ -521,6 +577,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
|
||||
|
||||
isActiveMaster = true;
|
||||
Thread zombieDetector = new Thread(new InitializationMonitor(this));
|
||||
zombieDetector.start();
|
||||
|
||||
/*
|
||||
* We are active master now... go initialize components we need to run.
|
||||
|
@ -686,6 +744,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
LOG.error("Coprocessor postStartMaster() hook failed", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
zombieDetector.interrupt();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue