You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/10/20 03:05:53 UTC

[12/16] git commit: ACCUMULO-3242 Fix up the retry logic in ZooCache.

ACCUMULO-3242 Fix up the retry logic in ZooCache.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/423cf10a
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/423cf10a
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/423cf10a

Branch: refs/heads/master
Commit: 423cf10a32d062e70ed97d1f85bb3c5c8642acbe
Parents: 1636055
Author: Josh Elser <el...@apache.org>
Authored: Sun Oct 19 12:55:34 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Oct 19 20:16:04 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/fate/zookeeper/ZooCache.java    | 16 ++++++++++++----
 1 file changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/423cf10a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
index d9eb243..b3dd19a 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.fate.zookeeper;
 
-import com.google.common.annotations.VisibleForTesting;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -37,6 +36,8 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * A cache for values stored in ZooKeeper. Values are kept up to date as they
  * change.
@@ -138,6 +139,9 @@ public class ZooCache {
   }
 
   private interface ZooRunnable {
+    /**
+     * Runs an operation against ZooKeeper, automatically retrying in the face of KeeperExceptions
+     */
     void run(ZooKeeper zooKeeper) throws KeeperException, InterruptedException;
   }
 
@@ -154,8 +158,12 @@ public class ZooCache {
         return;
 
       } catch (KeeperException e) {
-        if (e.code() == Code.NONODE) {
+        final Code code = e.code();
+        if (code == Code.NONODE) {
           log.error("Looked up non-existent node in cache " + e.getPath(), e);
+        } else if (code == Code.CONNECTIONLOSS || code == Code.OPERATIONTIMEOUT || code == Code.SESSIONEXPIRED) {
+          log.warn("Saw (possibly) transient exception communicating with ZooKeeper, wil retry", e);
+          continue;
         }
         log.warn("Zookeeper error, will retry", e);
       } catch (InterruptedException e) {
@@ -168,7 +176,7 @@ public class ZooCache {
         // do not hold lock while sleeping
         wait(sleepTime);
       } catch (InterruptedException e) {
-        e.printStackTrace();
+        log.info("Interrupted waiting before retrying ZooKeeper operation", e);
       }
       if (sleepTime < 10000)
         sleepTime = (int) (sleepTime + sleepTime * Math.random());
@@ -244,7 +252,7 @@ public class ZooCache {
         /*
          * The following call to exists() is important, since we are caching that a node does not exist. Once the node comes into existence, it will be added to
          * the cache. But this notification of a node coming into existence will only be given if exists() was previously called.
-         * 
+         *
          * If the call to exists() is bypassed and only getData() is called with a special case that looks for Code.NONODE in the KeeperException, then
          * non-existence can not be cached.
          */