ACCUMULO-2422 Refine renewal of master lock watcher The first commit for ACCUMULO-2422 succeeds in renewing the watch on another master's lock node when needed. This commit refines the solution:
- The renewal was happening even after the master is able to acquire the lock. This led to a spurious log error message. This commit skips renewing the watch in that case. - If the renewal returns a null status, meaning the other master's lock node disappeared, the master now immediately tries again to acquire the lock. This matches watch establishment in other areas. A lot of logging at the trace level was added to ZooLock to assist future troubleshooting. Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/7eeff02c Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/7eeff02c Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/7eeff02c Branch: refs/heads/ACCUMULO-2061 Commit: 7eeff02c7cf883765a33575a19d208be30e1e17c Parents: 853ed5b Author: Bill Havanki <bhava...@cloudera.com> Authored: Fri Feb 28 14:23:19 2014 -0500 Committer: Bill Havanki <bhava...@cloudera.com> Committed: Fri Feb 28 14:23:19 2014 -0500 ---------------------------------------------------------------------- .../apache/accumulo/fate/zookeeper/ZooLock.java | 43 +++++++++++++++----- .../apache/accumulo/server/master/Master.java | 1 + 2 files changed, 34 insertions(+), 10 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eeff02c/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java ---------------------------------------------------------------------- diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java index 25f1020..03e159f 100644 --- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java +++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java @@ -144,8 +144,15 @@ public class ZooLock implements Watcher { } Collections.sort(children); + if (log.isTraceEnabled()) { + log.trace("Candidate lock nodes"); + for (String child : children) { + log.trace("- " + child); + } + } if (children.get(0).equals(myLock)) { + log.trace("First candidate is my lock, acquiring"); if (!watchingParent) { throw new IllegalStateException("Can not acquire lock, no longer watching parent : " + path); } @@ -166,13 +173,20 @@ public class ZooLock implements Watcher { } final String lockToWatch = path + "/" + prev; - - Stat stat = zooKeeper.getStatus(path + "/" + prev, new Watcher() { + log.trace("Establishing watch on " + lockToWatch); + Stat stat = zooKeeper.getStatus(lockToWatch, new Watcher() { @Override public void process(WatchedEvent event) { - + if (log.isTraceEnabled()) { + log.trace("Processing event:"); + log.trace("- type " + event.getType()); + log.trace("- path " + event.getPath()); + log.trace("- state " + event.getState()); + } + boolean renew = true; if (event.getType() == EventType.NodeDeleted && event.getPath().equals(lockToWatch)) { + log.trace("Detected deletion of " + lockToWatch + ", attempting to acquire lock"); synchronized (ZooLock.this) { try { if (asyncLock != null) { @@ -187,19 +201,28 @@ public class ZooLock implements Watcher { } } } + renew = false; } - if (event.getState() == KeeperState.Expired) { + if (event.getState() == KeeperState.Expired || event.getState() == KeeperState.Disconnected) { synchronized (ZooLock.this) { if (lock == null) { - lw.failedToAcquireLock(new Exception("Zookeeper Session expired")); + lw.failedToAcquireLock(new Exception("Zookeeper Session expired / disconnected")); } } - } else { + renew = false; + } + if (renew) { + log.trace("Renewing watch on " + lockToWatch); try { - zooKeeper.getStatus(event.getPath(), this); - } catch (Exception ex) { - lw.failedToAcquireLock(ex); + Stat restat = zooKeeper.getStatus(lockToWatch, this); + if (restat == null) { + lockAsync(myLock, lw); + } + } catch (KeeperException e) { + lw.failedToAcquireLock(new Exception("Failed to renew watch on other master node")); + } catch (InterruptedException e) { + lw.failedToAcquireLock(new Exception("Failed to renew watch on other master node")); } } } @@ -228,7 +251,7 @@ public class ZooLock implements Watcher { try { final String asyncLockPath = zooKeeper.putEphemeralSequential(path + "/" + LOCK_PREFIX, data); - + log.trace("Ephemeral node " + asyncLockPath + " created"); Stat stat = zooKeeper.getStatus(asyncLockPath, new Watcher() { private void failedToAcquireLock(){ http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eeff02c/server/src/main/java/org/apache/accumulo/server/master/Master.java ---------------------------------------------------------------------- diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java index 039d7fa..70f87ae 100644 --- a/server/src/main/java/org/apache/accumulo/server/master/Master.java +++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java @@ -2232,6 +2232,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt @Override public synchronized void acquiredLock() { + log.debug("Acquired master lock"); if (acquiredLock || failedToAcquireLock) { Halt.halt("Zoolock in unexpected state AL " + acquiredLock + " " + failedToAcquireLock, -1);