|
@@ -34,6 +34,7 @@ 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.HAServiceState;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
|
|
|
import org.apache.hadoop.util.ZKUtil;
|
|
@@ -105,6 +106,8 @@ public abstract class ZKFailoverController {
|
|
|
|
|
|
private State lastHealthState = State.INITIALIZING;
|
|
|
|
|
|
+ private volatile HAServiceState serviceState = HAServiceState.INITIALIZING;
|
|
|
+
|
|
|
/** Set if a fatal error occurs */
|
|
|
private String fatalError = null;
|
|
|
|
|
@@ -294,6 +297,7 @@ public abstract class ZKFailoverController {
|
|
|
private void initHM() {
|
|
|
healthMonitor = new HealthMonitor(conf, localTarget);
|
|
|
healthMonitor.addCallback(new HealthCallbacks());
|
|
|
+ healthMonitor.addServiceStateCallback(new ServiceStateCallBacks());
|
|
|
healthMonitor.start();
|
|
|
}
|
|
|
|
|
@@ -376,6 +380,7 @@ public abstract class ZKFailoverController {
|
|
|
String msg = "Successfully transitioned " + localTarget +
|
|
|
" to active state";
|
|
|
LOG.info(msg);
|
|
|
+ serviceState = HAServiceState.ACTIVE;
|
|
|
recordActiveAttempt(new ActiveAttemptRecord(true, msg));
|
|
|
|
|
|
} catch (Throwable t) {
|
|
@@ -484,6 +489,7 @@ public abstract class ZKFailoverController {
|
|
|
// TODO handle this. It's a likely case since we probably got fenced
|
|
|
// at the same time.
|
|
|
}
|
|
|
+ serviceState = HAServiceState.STANDBY;
|
|
|
}
|
|
|
|
|
|
|
|
@@ -574,6 +580,7 @@ public abstract class ZKFailoverController {
|
|
|
delayJoiningUntilNanotime = System.nanoTime() +
|
|
|
TimeUnit.MILLISECONDS.toNanos(millisToCede);
|
|
|
elector.quitElection(needFence);
|
|
|
+ serviceState = HAServiceState.INITIALIZING;
|
|
|
}
|
|
|
}
|
|
|
recheckElectability();
|
|
@@ -739,12 +746,16 @@ public abstract class ZKFailoverController {
|
|
|
switch (lastHealthState) {
|
|
|
case SERVICE_HEALTHY:
|
|
|
elector.joinElection(targetToData(localTarget));
|
|
|
+ if (quitElectionOnBadState) {
|
|
|
+ quitElectionOnBadState = false;
|
|
|
+ }
|
|
|
break;
|
|
|
|
|
|
case INITIALIZING:
|
|
|
LOG.info("Ensuring that " + localTarget + " does not " +
|
|
|
"participate in active master election");
|
|
|
elector.quitElection(false);
|
|
|
+ serviceState = HAServiceState.INITIALIZING;
|
|
|
break;
|
|
|
|
|
|
case SERVICE_UNHEALTHY:
|
|
@@ -752,6 +763,7 @@ public abstract class ZKFailoverController {
|
|
|
LOG.info("Quitting master election for " + localTarget +
|
|
|
" and marking that fencing is necessary");
|
|
|
elector.quitElection(true);
|
|
|
+ serviceState = HAServiceState.INITIALIZING;
|
|
|
break;
|
|
|
|
|
|
case HEALTH_MONITOR_FAILED:
|
|
@@ -784,6 +796,44 @@ public abstract class ZKFailoverController {
|
|
|
whenNanos, TimeUnit.NANOSECONDS);
|
|
|
}
|
|
|
|
|
|
+ int serviceStateMismatchCount = 0;
|
|
|
+ boolean quitElectionOnBadState = false;
|
|
|
+
|
|
|
+ void verifyChangedServiceState(HAServiceState changedState) {
|
|
|
+ synchronized (elector) {
|
|
|
+ synchronized (this) {
|
|
|
+ if (serviceState == HAServiceState.INITIALIZING) {
|
|
|
+ if (quitElectionOnBadState) {
|
|
|
+ LOG.debug("rechecking for electability from bad state");
|
|
|
+ recheckElectability();
|
|
|
+ }
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ if (changedState == serviceState) {
|
|
|
+ serviceStateMismatchCount = 0;
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ if (serviceStateMismatchCount == 0) {
|
|
|
+ // recheck one more time. As this might be due to parallel transition.
|
|
|
+ serviceStateMismatchCount++;
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ // quit the election as the expected state and reported state
|
|
|
+ // mismatches.
|
|
|
+ LOG.error("Local service " + localTarget
|
|
|
+ + " has changed the serviceState to " + changedState
|
|
|
+ + ". Expected was " + serviceState
|
|
|
+ + ". Quitting election marking fencing necessary.");
|
|
|
+ delayJoiningUntilNanotime = System.nanoTime()
|
|
|
+ + TimeUnit.MILLISECONDS.toNanos(1000);
|
|
|
+ elector.quitElection(true);
|
|
|
+ quitElectionOnBadState = true;
|
|
|
+ serviceStateMismatchCount = 0;
|
|
|
+ serviceState = HAServiceState.INITIALIZING;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* @return the last health state passed to the FC
|
|
|
* by the HealthMonitor.
|
|
@@ -855,7 +905,17 @@ public abstract class ZKFailoverController {
|
|
|
recheckElectability();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Callbacks for HAServiceStatus
|
|
|
+ */
|
|
|
+ class ServiceStateCallBacks implements HealthMonitor.ServiceStateCallback {
|
|
|
+ @Override
|
|
|
+ public void reportServiceStatus(HAServiceStatus status) {
|
|
|
+ verifyChangedServiceState(status.getState());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private static class ActiveAttemptRecord {
|
|
|
private final boolean succeeded;
|
|
|
private final String status;
|