From 04416861eb068f5d0a4ba1361924714d89f2d73c Mon Sep 17 00:00:00 2001 From: Todd Lipcon Date: Thu, 29 Mar 2012 21:49:37 +0000 Subject: [PATCH] Amend HADOOP-8212 (Improve ActiveStandbyElector's behavior when session expires) Amendment patch incorporates following feedback from Bikas Saha: - adds a new functional test for session expiration while in the standby state - adds a safety check in the StatCallback for session expiration - improves some comments git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3042@1307128 13f79535-47bb-0310-9956-ffa450edef68 --- .../hadoop/ha/ActiveStandbyElector.java | 7 ++- .../ha/TestActiveStandbyElectorRealZK.java | 45 +++++++++++++++++++ 2 files changed, 51 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java index ef054562492..e23260ecf97 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java @@ -470,13 +470,18 @@ public synchronized void processResult(int rc, String path, Object ctx, } errorMessage = errorMessage + ". Not retrying further znode monitoring connection errors."; + } else if (isSessionExpired(code)) { + // This isn't fatal - the client Watcher will re-join the election + LOG.warn("Lock monitoring failed because session was lost"); + return; } fatalError(errorMessage); } /** - * interface implementation of Zookeeper watch events (connection and node) + * interface implementation of Zookeeper watch events (connection and node), + * proxied by {@link WatcherWithClientRef}. */ synchronized void processWatchEvent(ZooKeeper zk, WatchedEvent event) { Event.EventType eventType = event.getType(); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java index 3a0fa5f981f..1256e028a0e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java @@ -26,6 +26,7 @@ import org.apache.commons.logging.impl.Log4JLogger; import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback; +import org.apache.hadoop.ha.ActiveStandbyElector.State; import org.apache.log4j.Level; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.server.ZooKeeperServer; @@ -196,4 +197,48 @@ public void testHandleSessionExpiration() throws Exception { checkFatalsAndReset(); } + + @Test(timeout=15000) + public void testHandleSessionExpirationOfStandby() throws Exception { + // Let elector 0 be active + electors[0].ensureParentZNode(); + electors[0].joinElection(appDatas[0]); + ZooKeeperServer zks = getServer(serverFactory); + ActiveStandbyElectorTestUtil.waitForActiveLockData(null, + zks, PARENT_DIR, appDatas[0]); + Mockito.verify(cbs[0], Mockito.timeout(1000)).becomeActive(); + checkFatalsAndReset(); + + // Let elector 1 be standby + electors[1].joinElection(appDatas[1]); + ActiveStandbyElectorTestUtil.waitForElectorState(null, electors[1], + State.STANDBY); + + LOG.info("========================== Expiring standby's session"); + zks.closeSession(electors[1].getZKSessionIdForTests()); + + // Should enter neutral mode when disconnected + Mockito.verify(cbs[1], Mockito.timeout(1000)).enterNeutralMode(); + + // Should re-join the election and go back to STANDBY + ActiveStandbyElectorTestUtil.waitForElectorState(null, electors[1], + State.STANDBY); + checkFatalsAndReset(); + + LOG.info("========================== Quitting election"); + electors[1].quitElection(false); + + // Double check that we don't accidentally re-join the election + // by quitting elector 0 and ensuring elector 1 doesn't become active + electors[0].quitElection(false); + + // due to receiving the "expired" event. + Thread.sleep(1000); + Mockito.verify(cbs[1], Mockito.never()).becomeActive(); + ActiveStandbyElectorTestUtil.waitForActiveLockData(null, + zks, PARENT_DIR, null); + + checkFatalsAndReset(); + } + }