From d863f7a1e452ecb26c3cb92bc6c90e552731304b Mon Sep 17 00:00:00 2001 From: Thomas White Date: Thu, 10 Jan 2013 10:05:53 +0000 Subject: [PATCH] HADOOP-9183. Potential deadlock in ActiveStandbyElector. git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1431251 13f79535-47bb-0310-9956-ffa450edef68 --- .../hadoop-common/CHANGES.txt | 2 ++ .../hadoop/ha/ActiveStandbyElector.java | 35 ++++++------------- 2 files changed, 12 insertions(+), 25 deletions(-) diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index 618933b6c4..984f7acac7 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -528,6 +528,8 @@ Release 2.0.3-alpha - Unreleased HADOOP-9155. FsPermission should have different default value, 777 for directory and 666 for file. (Binglin Chang via atm) + HADOOP-9183. Potential deadlock in ActiveStandbyElector. (tomwhite) + Release 2.0.2-alpha - 2012-09-07 INCOMPATIBLE CHANGES 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 bb4d4b5809..6badc5e84c 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 @@ -613,7 +613,7 @@ protected synchronized ZooKeeper getNewZooKeeper() throws IOException, // Unfortunately, the ZooKeeper constructor connects to ZooKeeper and // may trigger the Connected event immediately. So, if we register the // watcher after constructing ZooKeeper, we may miss that event. Instead, - // we construct the watcher first, and have it queue any events it receives + // we construct the watcher first, and have it block any events it receives // before we can set its ZooKeeper reference. WatcherWithClientRef watcher = new WatcherWithClientRef(); ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, watcher); @@ -1002,19 +1002,17 @@ private final class WatcherWithClientRef implements Watcher { private CountDownLatch hasReceivedEvent = new CountDownLatch(1); /** - * If any events arrive before the reference to ZooKeeper is set, - * they get queued up and later forwarded when the reference is - * available. + * Latch used to wait until the reference to ZooKeeper is set. */ - private final List queuedEvents = Lists.newLinkedList(); + private CountDownLatch hasSetZooKeeper = new CountDownLatch(1); private WatcherWithClientRef() { } private WatcherWithClientRef(ZooKeeper zk) { - this.zk = zk; + setZooKeeperRef(zk); } - + /** * Waits for the next event from ZooKeeper to arrive. * @@ -1029,9 +1027,7 @@ private void waitForZKConnectionEvent(int connectionTimeoutMs) if (!hasReceivedEvent.await(connectionTimeoutMs, TimeUnit.MILLISECONDS)) { LOG.error("Connection timed out: couldn't connect to ZooKeeper in " + connectionTimeoutMs + " milliseconds"); - synchronized (this) { - zk.close(); - } + zk.close(); throw KeeperException.create(Code.CONNECTIONLOSS); } } catch (InterruptedException e) { @@ -1041,29 +1037,18 @@ private void waitForZKConnectionEvent(int connectionTimeoutMs) } } - private synchronized void setZooKeeperRef(ZooKeeper zk) { + private void setZooKeeperRef(ZooKeeper zk) { Preconditions.checkState(this.zk == null, "zk already set -- must be set exactly once"); this.zk = zk; - - for (WatchedEvent e : queuedEvents) { - forwardEvent(e); - } - queuedEvents.clear(); + hasSetZooKeeper.countDown(); } @Override - public synchronized void process(WatchedEvent event) { - if (zk != null) { - forwardEvent(event); - } else { - queuedEvents.add(event); - } - } - - private void forwardEvent(WatchedEvent event) { + public void process(WatchedEvent event) { hasReceivedEvent.countDown(); try { + hasSetZooKeeper.await(zkSessionTimeout, TimeUnit.MILLISECONDS); ActiveStandbyElector.this.processWatchEvent( zk, event); } catch (Throwable t) {