Author: tomwhite
Date: Thu Jan 10 10:09:06 2013
New Revision: 1431252
URL: http://svn.apache.org/viewvc?rev=1431252&view=rev
Log:
Merge -r 1431250:1431251 from trunk to branch-2. Fixes: HADOOP-9183. Potential
deadlock in ActiveStandbyElector.
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
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=1431252&r1=1431251&r2=1431252&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
Thu Jan 10 10:09:06 2013
@@ -225,6 +225,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
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=1431252&r1=1431251&r2=1431252&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
Thu Jan 10 10:09:06 2013
@@ -613,7 +613,7 @@ public class ActiveStandbyElector implem
// 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 @@ public class ActiveStandbyElector implem
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<WatchedEvent> 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 @@ public class ActiveStandbyElector implem
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 @@ public class ActiveStandbyElector implem
}
}
- 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) {