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

svn commit: r1563567 - /hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java

Author: stack
Date: Sun Feb  2 06:04:01 2014
New Revision: 1563567

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

Modified:
    hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java

Modified: hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1563567&r1=1563566&r2=1563567&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/branches/0.96/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Sun Feb  2 06:04:01 2014
@@ -993,20 +993,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;
   }
 
   /**
@@ -1032,22 +1033,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;
   }
 
   /**