HADOOP-9220. Unnecessary transition to standby in ActiveStandbyElector. Contributed by Tom White and Todd Lipcon.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1482401 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
e29170e771
commit
d5a6e764dc
@ -720,6 +720,9 @@ Release 2.0.5-beta - UNRELEASED
|
||||
HADOOP-9307. BufferedFSInputStream.read returns wrong results
|
||||
after certain seeks. (todd)
|
||||
|
||||
HADOOP-9220. Unnecessary transition to standby in ActiveStandbyElector.
|
||||
(tom and todd via todd)
|
||||
|
||||
Release 2.0.4-alpha - 2013-04-25
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -159,6 +159,7 @@ static enum State {
|
||||
private int createRetryCount = 0;
|
||||
private int statRetryCount = 0;
|
||||
private ZooKeeper zkClient;
|
||||
private WatcherWithClientRef watcher;
|
||||
private ConnectionState zkConnectionState = ConnectionState.TERMINATED;
|
||||
|
||||
private final ActiveStandbyElectorCallback appClient;
|
||||
@ -247,6 +248,11 @@ public synchronized void joinElection(byte[] data)
|
||||
throw new HadoopIllegalArgumentException("data cannot be null");
|
||||
}
|
||||
|
||||
if (wantToBeInElection) {
|
||||
LOG.info("Already in election. Not re-connecting.");
|
||||
return;
|
||||
}
|
||||
|
||||
appData = new byte[data.length];
|
||||
System.arraycopy(data, 0, appData, 0, data.length);
|
||||
|
||||
@ -615,7 +621,7 @@ protected synchronized ZooKeeper getNewZooKeeper() throws IOException,
|
||||
// watcher after constructing ZooKeeper, we may miss that event. Instead,
|
||||
// we construct the watcher first, and have it block any events it receives
|
||||
// before we can set its ZooKeeper reference.
|
||||
WatcherWithClientRef watcher = new WatcherWithClientRef();
|
||||
watcher = new WatcherWithClientRef();
|
||||
ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, watcher);
|
||||
watcher.setZooKeeperRef(zk);
|
||||
|
||||
@ -753,6 +759,7 @@ private void createConnection() throws IOException, KeeperException {
|
||||
e);
|
||||
}
|
||||
zkClient = null;
|
||||
watcher = null;
|
||||
}
|
||||
zkClient = getNewZooKeeper();
|
||||
LOG.debug("Created new connection for " + this);
|
||||
@ -765,12 +772,14 @@ void terminateConnection() {
|
||||
LOG.debug("Terminating ZK connection for " + this);
|
||||
ZooKeeper tempZk = zkClient;
|
||||
zkClient = null;
|
||||
watcher = null;
|
||||
try {
|
||||
tempZk.close();
|
||||
} catch(InterruptedException e) {
|
||||
LOG.warn(e);
|
||||
}
|
||||
zkConnectionState = ConnectionState.TERMINATED;
|
||||
wantToBeInElection = false;
|
||||
}
|
||||
|
||||
private void reset() {
|
||||
@ -914,7 +923,7 @@ private void createLockNodeAsync() {
|
||||
|
||||
private void monitorLockNodeAsync() {
|
||||
zkClient.exists(zkLockFilePath,
|
||||
new WatcherWithClientRef(zkClient), this,
|
||||
watcher, this,
|
||||
zkClient);
|
||||
}
|
||||
|
||||
@ -1016,13 +1025,6 @@ private final class WatcherWithClientRef implements Watcher {
|
||||
*/
|
||||
private CountDownLatch hasSetZooKeeper = new CountDownLatch(1);
|
||||
|
||||
private WatcherWithClientRef() {
|
||||
}
|
||||
|
||||
private WatcherWithClientRef(ZooKeeper zk) {
|
||||
setZooKeeperRef(zk);
|
||||
}
|
||||
|
||||
/**
|
||||
* Waits for the next event from ZooKeeper to arrive.
|
||||
*
|
||||
|
@ -49,6 +49,7 @@ class DummyHAService extends HAServiceTarget {
|
||||
|
||||
DummySharedResource sharedResource;
|
||||
public int fenceCount = 0;
|
||||
public int activeTransitionCount = 0;
|
||||
|
||||
static ArrayList<DummyHAService> instances = Lists.newArrayList();
|
||||
int index;
|
||||
@ -139,6 +140,7 @@ public void monitorHealth() throws HealthCheckFailedException,
|
||||
@Override
|
||||
public void transitionToActive(StateChangeRequestInfo req) throws ServiceFailedException,
|
||||
AccessControlException, IOException {
|
||||
activeTransitionCount++;
|
||||
checkUnreachable();
|
||||
if (failToBecomeActive) {
|
||||
throw new ServiceFailedException("injected failure");
|
||||
|
@ -422,7 +422,7 @@ public void testCedeActive() throws Exception {
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=15000)
|
||||
@Test(timeout=25000)
|
||||
public void testGracefulFailover() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
@ -430,11 +430,16 @@ public void testGracefulFailover() throws Exception {
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
cluster.waitForActiveLockHolder(1);
|
||||
|
||||
cluster.getService(0).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
Thread.sleep(10000); // allow to quiesce
|
||||
|
||||
assertEquals(0, cluster.getService(0).fenceCount);
|
||||
assertEquals(0, cluster.getService(1).fenceCount);
|
||||
assertEquals(2, cluster.getService(0).activeTransitionCount);
|
||||
assertEquals(1, cluster.getService(1).activeTransitionCount);
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user