Merge -r 1431250:1431251 from trunk to branch-2. Fixes: HADOOP-9183. Potential deadlock in ActiveStandbyElector.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1431252 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Thomas White 2013-01-10 10:09:06 +00:00
parent b8b1a264ac
commit a22278f23f
2 changed files with 12 additions and 25 deletions

View File

@ -225,6 +225,8 @@ Release 2.0.3-alpha - Unreleased
HADOOP-9155. FsPermission should have different default value, 777 for HADOOP-9155. FsPermission should have different default value, 777 for
directory and 666 for file. (Binglin Chang via atm) directory and 666 for file. (Binglin Chang via atm)
HADOOP-9183. Potential deadlock in ActiveStandbyElector. (tomwhite)
Release 2.0.2-alpha - 2012-09-07 Release 2.0.2-alpha - 2012-09-07
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -613,7 +613,7 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
// Unfortunately, the ZooKeeper constructor connects to ZooKeeper and // Unfortunately, the ZooKeeper constructor connects to ZooKeeper and
// may trigger the Connected event immediately. So, if we register the // may trigger the Connected event immediately. So, if we register the
// watcher after constructing ZooKeeper, we may miss that event. Instead, // 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. // before we can set its ZooKeeper reference.
WatcherWithClientRef watcher = new WatcherWithClientRef(); WatcherWithClientRef watcher = new WatcherWithClientRef();
ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, watcher); ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, watcher);
@ -1002,17 +1002,15 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
private CountDownLatch hasReceivedEvent = new CountDownLatch(1); private CountDownLatch hasReceivedEvent = new CountDownLatch(1);
/** /**
* If any events arrive before the reference to ZooKeeper is set, * Latch used to wait until the reference to ZooKeeper is set.
* they get queued up and later forwarded when the reference is
* available.
*/ */
private final List<WatchedEvent> queuedEvents = Lists.newLinkedList(); private CountDownLatch hasSetZooKeeper = new CountDownLatch(1);
private WatcherWithClientRef() { private WatcherWithClientRef() {
} }
private WatcherWithClientRef(ZooKeeper zk) { private WatcherWithClientRef(ZooKeeper zk) {
this.zk = zk; setZooKeeperRef(zk);
} }
/** /**
@ -1029,9 +1027,7 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
if (!hasReceivedEvent.await(connectionTimeoutMs, TimeUnit.MILLISECONDS)) { if (!hasReceivedEvent.await(connectionTimeoutMs, TimeUnit.MILLISECONDS)) {
LOG.error("Connection timed out: couldn't connect to ZooKeeper in " LOG.error("Connection timed out: couldn't connect to ZooKeeper in "
+ connectionTimeoutMs + " milliseconds"); + connectionTimeoutMs + " milliseconds");
synchronized (this) {
zk.close(); zk.close();
}
throw KeeperException.create(Code.CONNECTIONLOSS); throw KeeperException.create(Code.CONNECTIONLOSS);
} }
} catch (InterruptedException e) { } catch (InterruptedException e) {
@ -1041,29 +1037,18 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
} }
} }
private synchronized void setZooKeeperRef(ZooKeeper zk) { private void setZooKeeperRef(ZooKeeper zk) {
Preconditions.checkState(this.zk == null, Preconditions.checkState(this.zk == null,
"zk already set -- must be set exactly once"); "zk already set -- must be set exactly once");
this.zk = zk; this.zk = zk;
hasSetZooKeeper.countDown();
for (WatchedEvent e : queuedEvents) {
forwardEvent(e);
}
queuedEvents.clear();
} }
@Override @Override
public synchronized void process(WatchedEvent event) { public void process(WatchedEvent event) {
if (zk != null) {
forwardEvent(event);
} else {
queuedEvents.add(event);
}
}
private void forwardEvent(WatchedEvent event) {
hasReceivedEvent.countDown(); hasReceivedEvent.countDown();
try { try {
hasSetZooKeeper.await(zkSessionTimeout, TimeUnit.MILLISECONDS);
ActiveStandbyElector.this.processWatchEvent( ActiveStandbyElector.this.processWatchEvent(
zk, event); zk, event);
} catch (Throwable t) { } catch (Throwable t) {