Author: todd Date: Tue May 14 15:37:18 2013 New Revision: 1482402 URL: http://svn.apache.org/r1482402 Log: HADOOP-9220. Unnecessary transition to standby in ActiveStandbyElector. Contributed by Tom White and Todd Lipcon.
Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/DummyHAService.java hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1482402&r1=1482401&r2=1482402&view=diff ============================================================================== --- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt (original) +++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt Tue May 14 15:37:18 2013 @@ -165,6 +165,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 Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java?rev=1482402&r1=1482401&r2=1482402&view=diff ============================================================================== --- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java (original) +++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java Tue May 14 15:37:18 2013 @@ -159,6 +159,7 @@ public class ActiveStandbyElector implem private int createRetryCount = 0; private int statRetryCount = 0; private ZooKeeper zkClient; + private WatcherWithClientRef watcher; private ConnectionState zkConnectionState = ConnectionState.TERMINATED; private final ActiveStandbyElectorCallback appClient; @@ -246,6 +247,11 @@ public class ActiveStandbyElector implem if (data == null) { 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 @@ public class ActiveStandbyElector implem // 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 @@ public class ActiveStandbyElector implem e); } zkClient = null; + watcher = null; } zkClient = getNewZooKeeper(); LOG.debug("Created new connection for " + this); @@ -765,12 +772,14 @@ public class ActiveStandbyElector implem 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 @@ public class ActiveStandbyElector implem private void monitorLockNodeAsync() { zkClient.exists(zkLockFilePath, - new WatcherWithClientRef(zkClient), this, + watcher, this, zkClient); } @@ -1015,13 +1024,6 @@ public class ActiveStandbyElector implem * Latch used to wait until the reference to ZooKeeper is set. */ private CountDownLatch hasSetZooKeeper = new CountDownLatch(1); - - private WatcherWithClientRef() { - } - - private WatcherWithClientRef(ZooKeeper zk) { - setZooKeeperRef(zk); - } /** * Waits for the next event from ZooKeeper to arrive. Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/DummyHAService.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/DummyHAService.java?rev=1482402&r1=1482401&r2=1482402&view=diff ============================================================================== --- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/DummyHAService.java (original) +++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/DummyHAService.java Tue May 14 15:37:18 2013 @@ -49,6 +49,7 @@ class DummyHAService extends HAServiceTa DummySharedResource sharedResource; public int fenceCount = 0; + public int activeTransitionCount = 0; static ArrayList<DummyHAService> instances = Lists.newArrayList(); int index; @@ -139,6 +140,7 @@ class DummyHAService extends HAServiceTa @Override public void transitionToActive(StateChangeRequestInfo req) throws ServiceFailedException, AccessControlException, IOException { + activeTransitionCount++; checkUnreachable(); if (failToBecomeActive) { throw new ServiceFailedException("injected failure"); Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java?rev=1482402&r1=1482401&r2=1482402&view=diff ============================================================================== --- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java (original) +++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java Tue May 14 15:37:18 2013 @@ -418,7 +418,7 @@ public class TestZKFailoverController ex } } - @Test(timeout=15000) + @Test(timeout=25000) public void testGracefulFailover() throws Exception { try { cluster.start(); @@ -426,11 +426,16 @@ public class TestZKFailoverController ex 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(); }