Details
Description
While using the ZKUtil methods during testing, I found that watch was not set when it should be set based on the methods and method comments:
createNodeIfNotExistsAndWatch
createEphemeralNodeAndWatch
For example, in createNodeIfNotExistsAndWatch():
public static boolean createNodeIfNotExistsAndWatch( ZooKeeperWatcher zkw, String znode, byte [] data) throws KeeperException { 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; } return false; } catch (InterruptedException e) { zkw.interruptedException(e); return false; } return true; }
The watch is only set via exists() call when the node already exists.
Similarly in createEphemeralNodeAndWatch():
public static boolean createEphemeralNodeAndWatch(ZooKeeperWatcher zkw, String znode, byte [] data) throws KeeperException { 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; } catch (InterruptedException e) { LOG.info("Interrupted", e); Thread.currentThread().interrupt(); } return true; }
Attachments
Attachments
Issue Links
- duplicates
-
HBASE-8937 createEphemeralNodeAndWatch don't set watcher if the node is created successfully
- Closed