You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2014/02/03 06:39:56 UTC

svn commit: r1563783 - /hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java

Author: larsh
Date: Mon Feb  3 05:39:55 2014
New Revision: 1563783

URL: http://svn.apache.org/r1563783
Log:
HBASE-10448 ZKUtil create and watch methods don't set watch in some cases. (Jerry He)

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1563783&r1=1563782&r2=1563783&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Mon Feb  3 05:39:55 2014
@@ -960,20 +960,21 @@ public class ZKUtil {
   public static boolean createEphemeralNodeAndWatch(ZooKeeperWatcher zkw,
       String znode, byte [] data)
   throws KeeperException {
+    boolean ret = true;
     try {
       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
           CreateMode.EPHEMERAL);
     } catch (KeeperException.NodeExistsException nee) {
-      if(!watchAndCheckExists(zkw, znode)) {
-        // It did exist but now it doesn't, try again
-        return createEphemeralNodeAndWatch(zkw, znode, data);
-      }
-      return false;
+      ret = false;
     } catch (InterruptedException e) {
       LOG.info("Interrupted", e);
       Thread.currentThread().interrupt();
     }
-    return true;
+    if(!watchAndCheckExists(zkw, znode)) {
+      // It did exist but now it doesn't, try again
+      return createEphemeralNodeAndWatch(zkw, znode, data);
+    }
+    return ret;
   }
 
   /**
@@ -999,22 +1000,23 @@ public class ZKUtil {
   public static boolean createNodeIfNotExistsAndWatch(
       ZooKeeperWatcher zkw, String znode, byte [] data)
   throws KeeperException {
+    boolean ret = true;
     try {
       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
           CreateMode.PERSISTENT);
     } catch (KeeperException.NodeExistsException nee) {
-      try {
-        zkw.getRecoverableZooKeeper().exists(znode, zkw);
-      } catch (InterruptedException e) {
-        zkw.interruptedException(e);
-        return false;
-      }
+      ret = false;
+    } catch (InterruptedException e) {
+      zkw.interruptedException(e);
       return false;
+    }
+    try {
+      zkw.getRecoverableZooKeeper().exists(znode, zkw);
     } catch (InterruptedException e) {
       zkw.interruptedException(e);
       return false;
     }
-    return true;
+    return ret;
   }
 
   /**