|
@@ -18,21 +18,32 @@
|
|
|
package org.apache.hadoop.ha;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.net.InetSocketAddress;
|
|
|
import java.security.PrivilegedAction;
|
|
|
+import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
+import java.util.concurrent.ScheduledExecutorService;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.ha.ActiveStandbyElector.ActiveNotFoundException;
|
|
|
import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
|
|
|
import org.apache.hadoop.ha.HAZKUtil.ZKAuthInfo;
|
|
|
import org.apache.hadoop.ha.HealthMonitor.State;
|
|
|
+import org.apache.hadoop.ipc.Server;
|
|
|
+import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.security.authorize.PolicyProvider;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.Tool;
|
|
|
import org.apache.zookeeper.KeeperException;
|
|
|
import org.apache.zookeeper.ZooDefs.Ids;
|
|
@@ -41,6 +52,8 @@ import org.apache.zookeeper.data.ACL;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
+import com.google.common.base.Throwables;
|
|
|
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
|
|
|
@InterfaceAudience.LimitedPrivate("HDFS")
|
|
|
public abstract class ZKFailoverController implements Tool {
|
|
@@ -85,6 +98,7 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
|
|
|
private HealthMonitor healthMonitor;
|
|
|
private ActiveStandbyElector elector;
|
|
|
+ protected ZKFCRpcServer rpcServer;
|
|
|
|
|
|
private HAServiceTarget localTarget;
|
|
|
|
|
@@ -93,6 +107,22 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
/** Set if a fatal error occurs */
|
|
|
private String fatalError = null;
|
|
|
|
|
|
+ /**
|
|
|
+ * A future nanotime before which the ZKFC will not join the election.
|
|
|
+ * This is used during graceful failover.
|
|
|
+ */
|
|
|
+ private long delayJoiningUntilNanotime = 0;
|
|
|
+
|
|
|
+ /** Executor on which {@link #scheduleRecheck(long)} schedules events */
|
|
|
+ private ScheduledExecutorService delayExecutor =
|
|
|
+ Executors.newScheduledThreadPool(1,
|
|
|
+ new ThreadFactoryBuilder().setDaemon(true)
|
|
|
+ .setNameFormat("ZKFC Delay timer #%d")
|
|
|
+ .build());
|
|
|
+
|
|
|
+ private ActiveAttemptRecord lastActiveAttemptRecord;
|
|
|
+ private Object activeAttemptRecordLock = new Object();
|
|
|
+
|
|
|
@Override
|
|
|
public void setConf(Configuration conf) {
|
|
|
this.conf = conf;
|
|
@@ -104,6 +134,10 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
protected abstract HAServiceTarget getLocalTarget();
|
|
|
protected abstract HAServiceTarget dataToTarget(byte[] data);
|
|
|
protected abstract void loginAsFCUser() throws IOException;
|
|
|
+ protected abstract void checkRpcAdminAccess()
|
|
|
+ throws AccessControlException, IOException;
|
|
|
+ protected abstract InetSocketAddress getRpcAddressToBindTo();
|
|
|
+ protected abstract PolicyProvider getPolicyProvider();
|
|
|
|
|
|
/**
|
|
|
* Return the name of a znode inside the configured parent znode in which
|
|
@@ -194,10 +228,14 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
return ERR_CODE_NO_FENCER;
|
|
|
}
|
|
|
|
|
|
+ initRPC();
|
|
|
initHM();
|
|
|
+ startRPC();
|
|
|
try {
|
|
|
mainLoop();
|
|
|
} finally {
|
|
|
+ rpcServer.stopAndJoin();
|
|
|
+
|
|
|
elector.quitElection(true);
|
|
|
healthMonitor.shutdown();
|
|
|
healthMonitor.join();
|
|
@@ -262,6 +300,16 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
healthMonitor.addCallback(new HealthCallbacks());
|
|
|
healthMonitor.start();
|
|
|
}
|
|
|
+
|
|
|
+ protected void initRPC() throws IOException {
|
|
|
+ InetSocketAddress bindAddr = getRpcAddressToBindTo();
|
|
|
+ rpcServer = new ZKFCRpcServer(conf, bindAddr, this, getPolicyProvider());
|
|
|
+ }
|
|
|
+
|
|
|
+ protected void startRPC() throws IOException {
|
|
|
+ rpcServer.start();
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
private void initZK() throws HadoopIllegalArgumentException, IOException {
|
|
|
zkQuorum = conf.get(ZK_QUORUM_KEY);
|
|
@@ -328,10 +376,18 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
HAServiceProtocolHelper.transitionToActive(localTarget.getProxy(
|
|
|
conf, FailoverController.getRpcTimeoutToNewActive(conf)),
|
|
|
createReqInfo());
|
|
|
- LOG.info("Successfully transitioned " + localTarget +
|
|
|
- " to active state");
|
|
|
+ String msg = "Successfully transitioned " + localTarget +
|
|
|
+ " to active state";
|
|
|
+ LOG.info(msg);
|
|
|
+ recordActiveAttempt(new ActiveAttemptRecord(true, msg));
|
|
|
+
|
|
|
} catch (Throwable t) {
|
|
|
- LOG.fatal("Couldn't make " + localTarget + " active", t);
|
|
|
+ String msg = "Couldn't make " + localTarget + " active";
|
|
|
+ LOG.fatal(msg, t);
|
|
|
+
|
|
|
+ recordActiveAttempt(new ActiveAttemptRecord(false, msg + "\n" +
|
|
|
+ StringUtils.stringifyException(t)));
|
|
|
+
|
|
|
if (t instanceof ServiceFailedException) {
|
|
|
throw (ServiceFailedException)t;
|
|
|
} else {
|
|
@@ -350,6 +406,69 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Store the results of the last attempt to become active.
|
|
|
+ * This is used so that, during manually initiated failover,
|
|
|
+ * we can report back the results of the attempt to become active
|
|
|
+ * to the initiator of the failover.
|
|
|
+ */
|
|
|
+ private void recordActiveAttempt(
|
|
|
+ ActiveAttemptRecord record) {
|
|
|
+ synchronized (activeAttemptRecordLock) {
|
|
|
+ lastActiveAttemptRecord = record;
|
|
|
+ activeAttemptRecordLock.notifyAll();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Wait until one of the following events:
|
|
|
+ * <ul>
|
|
|
+ * <li>Another thread publishes the results of an attempt to become active
|
|
|
+ * using {@link #recordActiveAttempt(ActiveAttemptRecord)}</li>
|
|
|
+ * <li>The node enters bad health status</li>
|
|
|
+ * <li>The specified timeout elapses</li>
|
|
|
+ * </ul>
|
|
|
+ *
|
|
|
+ * @param timeoutMillis number of millis to wait
|
|
|
+ * @return the published record, or null if the timeout elapses or the
|
|
|
+ * service becomes unhealthy
|
|
|
+ * @throws InterruptedException if the thread is interrupted.
|
|
|
+ */
|
|
|
+ private ActiveAttemptRecord waitForActiveAttempt(int timeoutMillis)
|
|
|
+ throws InterruptedException {
|
|
|
+ long st = System.nanoTime();
|
|
|
+ long waitUntil = st + TimeUnit.NANOSECONDS.convert(
|
|
|
+ timeoutMillis, TimeUnit.MILLISECONDS);
|
|
|
+
|
|
|
+ do {
|
|
|
+ // periodically check health state, because entering an
|
|
|
+ // unhealthy state could prevent us from ever attempting to
|
|
|
+ // become active. We can detect this and respond to the user
|
|
|
+ // immediately.
|
|
|
+ synchronized (this) {
|
|
|
+ if (lastHealthState != State.SERVICE_HEALTHY) {
|
|
|
+ // early out if service became unhealthy
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ synchronized (activeAttemptRecordLock) {
|
|
|
+ if ((lastActiveAttemptRecord != null &&
|
|
|
+ lastActiveAttemptRecord.nanoTime >= st)) {
|
|
|
+ return lastActiveAttemptRecord;
|
|
|
+ }
|
|
|
+ // Only wait 1sec so that we periodically recheck the health state
|
|
|
+ // above.
|
|
|
+ activeAttemptRecordLock.wait(1000);
|
|
|
+ }
|
|
|
+ } while (System.nanoTime() < waitUntil);
|
|
|
+
|
|
|
+ // Timeout elapsed.
|
|
|
+ LOG.warn(timeoutMillis + "ms timeout elapsed waiting for an attempt " +
|
|
|
+ "to become active");
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
private StateChangeRequestInfo createReqInfo() {
|
|
|
return new StateChangeRequestInfo(RequestSource.REQUEST_BY_ZKFC);
|
|
|
}
|
|
@@ -369,6 +488,304 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
// at the same time.
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+
|
|
|
+ private synchronized void fenceOldActive(byte[] data) {
|
|
|
+ HAServiceTarget target = dataToTarget(data);
|
|
|
+
|
|
|
+ try {
|
|
|
+ doFence(target);
|
|
|
+ } catch (Throwable t) {
|
|
|
+ recordActiveAttempt(new ActiveAttemptRecord(false, "Unable to fence old active: " + StringUtils.stringifyException(t)));
|
|
|
+ Throwables.propagate(t);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void doFence(HAServiceTarget target) {
|
|
|
+ LOG.info("Should fence: " + target);
|
|
|
+ boolean gracefulWorked = new FailoverController(conf,
|
|
|
+ RequestSource.REQUEST_BY_ZKFC).tryGracefulFence(target);
|
|
|
+ if (gracefulWorked) {
|
|
|
+ // It's possible that it's in standby but just about to go into active,
|
|
|
+ // no? Is there some race here?
|
|
|
+ LOG.info("Successfully transitioned " + target + " to standby " +
|
|
|
+ "state without fencing");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ target.checkFencingConfigured();
|
|
|
+ } catch (BadFencingConfigurationException e) {
|
|
|
+ LOG.error("Couldn't fence old active " + target, e);
|
|
|
+ recordActiveAttempt(new ActiveAttemptRecord(false, "Unable to fence old active"));
|
|
|
+ throw new RuntimeException(e);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!target.getFencer().fence(target)) {
|
|
|
+ throw new RuntimeException("Unable to fence " + target);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Request from graceful failover to cede active role. Causes
|
|
|
+ * this ZKFC to transition its local node to standby, then quit
|
|
|
+ * the election for the specified period of time, after which it
|
|
|
+ * will rejoin iff it is healthy.
|
|
|
+ */
|
|
|
+ void cedeActive(final int millisToCede)
|
|
|
+ throws AccessControlException, ServiceFailedException, IOException {
|
|
|
+ try {
|
|
|
+ UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<Void>() {
|
|
|
+ @Override
|
|
|
+ public Void run() throws Exception {
|
|
|
+ doCedeActive(millisToCede);
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw new IOException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void doCedeActive(int millisToCede)
|
|
|
+ throws AccessControlException, ServiceFailedException, IOException {
|
|
|
+ int timeout = FailoverController.getGracefulFenceTimeout(conf);
|
|
|
+
|
|
|
+ // Lock elector to maintain lock ordering of elector -> ZKFC
|
|
|
+ synchronized (elector) {
|
|
|
+ synchronized (this) {
|
|
|
+ if (millisToCede <= 0) {
|
|
|
+ delayJoiningUntilNanotime = 0;
|
|
|
+ recheckElectability();
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ LOG.info("Requested by " + UserGroupInformation.getCurrentUser() +
|
|
|
+ " at " + Server.getRemoteAddress() + " to cede active role.");
|
|
|
+ boolean needFence = false;
|
|
|
+ try {
|
|
|
+ localTarget.getProxy(conf, timeout).transitionToStandby(createReqInfo());
|
|
|
+ LOG.info("Successfully ensured local node is in standby mode");
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.warn("Unable to transition local node to standby: " +
|
|
|
+ ioe.getLocalizedMessage());
|
|
|
+ LOG.warn("Quitting election but indicating that fencing is " +
|
|
|
+ "necessary");
|
|
|
+ needFence = true;
|
|
|
+ }
|
|
|
+ delayJoiningUntilNanotime = System.nanoTime() +
|
|
|
+ TimeUnit.MILLISECONDS.toNanos(millisToCede);
|
|
|
+ elector.quitElection(needFence);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ recheckElectability();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Coordinate a graceful failover to this node.
|
|
|
+ * @throws ServiceFailedException if the node fails to become active
|
|
|
+ * @throws IOException some other error occurs
|
|
|
+ */
|
|
|
+ void gracefulFailoverToYou() throws ServiceFailedException, IOException {
|
|
|
+ try {
|
|
|
+ UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<Void>() {
|
|
|
+ @Override
|
|
|
+ public Void run() throws Exception {
|
|
|
+ doGracefulFailover();
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ });
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw new IOException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Coordinate a graceful failover. This proceeds in several phases:
|
|
|
+ * 1) Pre-flight checks: ensure that the local node is healthy, and
|
|
|
+ * thus a candidate for failover.
|
|
|
+ * 2) Determine the current active node. If it is the local node, no
|
|
|
+ * need to failover - return success.
|
|
|
+ * 3) Ask that node to yield from the election for a number of seconds.
|
|
|
+ * 4) Allow the normal election path to run in other threads. Wait until
|
|
|
+ * we either become unhealthy or we see an election attempt recorded by
|
|
|
+ * the normal code path.
|
|
|
+ * 5) Allow the old active to rejoin the election, so a future
|
|
|
+ * failback is possible.
|
|
|
+ */
|
|
|
+ private void doGracefulFailover()
|
|
|
+ throws ServiceFailedException, IOException, InterruptedException {
|
|
|
+ int timeout = FailoverController.getGracefulFenceTimeout(conf) * 2;
|
|
|
+
|
|
|
+ // Phase 1: pre-flight checks
|
|
|
+ checkEligibleForFailover();
|
|
|
+
|
|
|
+ // Phase 2: determine old/current active node. Check that we're not
|
|
|
+ // ourselves active, etc.
|
|
|
+ HAServiceTarget oldActive = getCurrentActive();
|
|
|
+ if (oldActive == null) {
|
|
|
+ // No node is currently active. So, if we aren't already
|
|
|
+ // active ourselves by means of a normal election, then there's
|
|
|
+ // probably something preventing us from becoming active.
|
|
|
+ throw new ServiceFailedException(
|
|
|
+ "No other node is currently active.");
|
|
|
+ }
|
|
|
+
|
|
|
+ if (oldActive.getAddress().equals(localTarget.getAddress())) {
|
|
|
+ LOG.info("Local node " + localTarget + " is already active. " +
|
|
|
+ "No need to failover. Returning success.");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Phase 3: ask the old active to yield from the election.
|
|
|
+ LOG.info("Asking " + oldActive + " to cede its active state for " +
|
|
|
+ timeout + "ms");
|
|
|
+ ZKFCProtocol oldZkfc = oldActive.getZKFCProxy(conf, timeout);
|
|
|
+ oldZkfc.cedeActive(timeout);
|
|
|
+
|
|
|
+ // Phase 4: wait for the normal election to make the local node
|
|
|
+ // active.
|
|
|
+ ActiveAttemptRecord attempt = waitForActiveAttempt(timeout + 60000);
|
|
|
+
|
|
|
+ if (attempt == null) {
|
|
|
+ // We didn't even make an attempt to become active.
|
|
|
+ synchronized(this) {
|
|
|
+ if (lastHealthState != State.SERVICE_HEALTHY) {
|
|
|
+ throw new ServiceFailedException("Unable to become active. " +
|
|
|
+ "Service became unhealthy while trying to failover.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ throw new ServiceFailedException("Unable to become active. " +
|
|
|
+ "Local node did not get an opportunity to do so from ZooKeeper, " +
|
|
|
+ "or the local node took too long to transition to active.");
|
|
|
+ }
|
|
|
+
|
|
|
+ // Phase 5. At this point, we made some attempt to become active. So we
|
|
|
+ // can tell the old active to rejoin if it wants. This allows a quick
|
|
|
+ // fail-back if we immediately crash.
|
|
|
+ oldZkfc.cedeActive(-1);
|
|
|
+
|
|
|
+ if (attempt.succeeded) {
|
|
|
+ LOG.info("Successfully became active. " + attempt.status);
|
|
|
+ } else {
|
|
|
+ // Propagate failure
|
|
|
+ String msg = "Failed to become active. " + attempt.status;
|
|
|
+ throw new ServiceFailedException(msg);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Ensure that the local node is in a healthy state, and thus
|
|
|
+ * eligible for graceful failover.
|
|
|
+ * @throws ServiceFailedException if the node is unhealthy
|
|
|
+ */
|
|
|
+ private synchronized void checkEligibleForFailover()
|
|
|
+ throws ServiceFailedException {
|
|
|
+ // Check health
|
|
|
+ if (this.getLastHealthState() != State.SERVICE_HEALTHY) {
|
|
|
+ throw new ServiceFailedException(
|
|
|
+ localTarget + " is not currently healthy. " +
|
|
|
+ "Cannot be failover target");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * @return an {@link HAServiceTarget} for the current active node
|
|
|
+ * in the cluster, or null if no node is active.
|
|
|
+ * @throws IOException if a ZK-related issue occurs
|
|
|
+ * @throws InterruptedException if thread is interrupted
|
|
|
+ */
|
|
|
+ private HAServiceTarget getCurrentActive()
|
|
|
+ throws IOException, InterruptedException {
|
|
|
+ synchronized (elector) {
|
|
|
+ synchronized (this) {
|
|
|
+ byte[] activeData;
|
|
|
+ try {
|
|
|
+ activeData = elector.getActiveData();
|
|
|
+ } catch (ActiveNotFoundException e) {
|
|
|
+ return null;
|
|
|
+ } catch (KeeperException ke) {
|
|
|
+ throw new IOException(
|
|
|
+ "Unexpected ZooKeeper issue fetching active node info", ke);
|
|
|
+ }
|
|
|
+
|
|
|
+ HAServiceTarget oldActive = dataToTarget(activeData);
|
|
|
+ return oldActive;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Check the current state of the service, and join the election
|
|
|
+ * if it should be in the election.
|
|
|
+ */
|
|
|
+ private void recheckElectability() {
|
|
|
+ // Maintain lock ordering of elector -> ZKFC
|
|
|
+ synchronized (elector) {
|
|
|
+ synchronized (this) {
|
|
|
+ boolean healthy = lastHealthState == State.SERVICE_HEALTHY;
|
|
|
+
|
|
|
+ long remainingDelay = delayJoiningUntilNanotime - System.nanoTime();
|
|
|
+ if (remainingDelay > 0) {
|
|
|
+ if (healthy) {
|
|
|
+ LOG.info("Would have joined master election, but this node is " +
|
|
|
+ "prohibited from doing so for " +
|
|
|
+ TimeUnit.NANOSECONDS.toMillis(remainingDelay) + " more ms");
|
|
|
+ }
|
|
|
+ scheduleRecheck(remainingDelay);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ switch (lastHealthState) {
|
|
|
+ case SERVICE_HEALTHY:
|
|
|
+ elector.joinElection(targetToData(localTarget));
|
|
|
+ break;
|
|
|
+
|
|
|
+ case INITIALIZING:
|
|
|
+ LOG.info("Ensuring that " + localTarget + " does not " +
|
|
|
+ "participate in active master election");
|
|
|
+ elector.quitElection(false);
|
|
|
+ break;
|
|
|
+
|
|
|
+ case SERVICE_UNHEALTHY:
|
|
|
+ case SERVICE_NOT_RESPONDING:
|
|
|
+ LOG.info("Quitting master election for " + localTarget +
|
|
|
+ " and marking that fencing is necessary");
|
|
|
+ elector.quitElection(true);
|
|
|
+ break;
|
|
|
+
|
|
|
+ case HEALTH_MONITOR_FAILED:
|
|
|
+ fatalError("Health monitor failed!");
|
|
|
+ break;
|
|
|
+
|
|
|
+ default:
|
|
|
+ throw new IllegalArgumentException("Unhandled state:" + lastHealthState);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Schedule a call to {@link #recheckElectability()} in the future.
|
|
|
+ */
|
|
|
+ private void scheduleRecheck(long whenNanos) {
|
|
|
+ delayExecutor.schedule(
|
|
|
+ new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ recheckElectability();
|
|
|
+ } catch (Throwable t) {
|
|
|
+ fatalError("Failed to recheck electability: " +
|
|
|
+ StringUtils.stringifyException(t));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ },
|
|
|
+ whenNanos, TimeUnit.NANOSECONDS);
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* @return the last health state passed to the FC
|
|
@@ -383,6 +800,11 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
ActiveStandbyElector getElectorForTests() {
|
|
|
return elector;
|
|
|
}
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ ZKFCRpcServer getRpcServerForTests() {
|
|
|
+ return rpcServer;
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Callbacks from elector
|
|
@@ -409,32 +831,7 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
|
|
|
@Override
|
|
|
public void fenceOldActive(byte[] data) {
|
|
|
- HAServiceTarget target = dataToTarget(data);
|
|
|
-
|
|
|
- LOG.info("Should fence: " + target);
|
|
|
- boolean gracefulWorked = new FailoverController(conf,
|
|
|
- RequestSource.REQUEST_BY_ZKFC).tryGracefulFence(target);
|
|
|
- if (gracefulWorked) {
|
|
|
- // It's possible that it's in standby but just about to go into active,
|
|
|
- // no? Is there some race here?
|
|
|
- LOG.info("Successfully transitioned " + target + " to standby " +
|
|
|
- "state without fencing");
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- try {
|
|
|
- target.checkFencingConfigured();
|
|
|
- } catch (BadFencingConfigurationException e) {
|
|
|
- LOG.error("Couldn't fence old active " + target, e);
|
|
|
- // TODO: see below todo
|
|
|
- throw new RuntimeException(e);
|
|
|
- }
|
|
|
-
|
|
|
- if (!target.getFencer().fence(target)) {
|
|
|
- // TODO: this will end up in some kind of tight loop,
|
|
|
- // won't it? We need some kind of backoff
|
|
|
- throw new RuntimeException("Unable to fence " + target);
|
|
|
- }
|
|
|
+ ZKFailoverController.this.fenceOldActive(data);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -451,34 +848,21 @@ public abstract class ZKFailoverController implements Tool {
|
|
|
public void enteredState(HealthMonitor.State newState) {
|
|
|
LOG.info("Local service " + localTarget +
|
|
|
" entered state: " + newState);
|
|
|
- switch (newState) {
|
|
|
- case SERVICE_HEALTHY:
|
|
|
- LOG.info("Joining master election for " + localTarget);
|
|
|
- elector.joinElection(targetToData(localTarget));
|
|
|
- break;
|
|
|
-
|
|
|
- case INITIALIZING:
|
|
|
- LOG.info("Ensuring that " + localTarget + " does not " +
|
|
|
- "participate in active master election");
|
|
|
- elector.quitElection(false);
|
|
|
- break;
|
|
|
-
|
|
|
- case SERVICE_UNHEALTHY:
|
|
|
- case SERVICE_NOT_RESPONDING:
|
|
|
- LOG.info("Quitting master election for " + localTarget +
|
|
|
- " and marking that fencing is necessary");
|
|
|
- elector.quitElection(true);
|
|
|
- break;
|
|
|
-
|
|
|
- case HEALTH_MONITOR_FAILED:
|
|
|
- fatalError("Health monitor failed!");
|
|
|
- break;
|
|
|
-
|
|
|
- default:
|
|
|
- throw new IllegalArgumentException("Unhandled state:" + newState);
|
|
|
- }
|
|
|
-
|
|
|
lastHealthState = newState;
|
|
|
+ recheckElectability();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private static class ActiveAttemptRecord {
|
|
|
+ private final boolean succeeded;
|
|
|
+ private final String status;
|
|
|
+ private final long nanoTime;
|
|
|
+
|
|
|
+ public ActiveAttemptRecord(boolean succeeded, String status) {
|
|
|
+ this.succeeded = succeeded;
|
|
|
+ this.status = status;
|
|
|
+ this.nanoTime = System.nanoTime();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
}
|