Skip to content

Commit

Permalink
ZOOKEEPER-4475: Fix NodeChildrenChanged delivered to recursive watcher
Browse files Browse the repository at this point in the history
The semantics of persistent recursive watch promise no child events on
descendant nodes. When there are standard child watches on descendants
of node being watches in persistent recursive mode, server will deliver
child events to client inevitably. So we have to filter out child events
for persistent recursive watches on client side.
  • Loading branch information
kezhuw committed Oct 10, 2022
1 parent 3daefac commit e33fddf
Show file tree
Hide file tree
Showing 2 changed files with 32 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -404,13 +404,13 @@ public Set<Watcher> materialize(
synchronized (existWatches) {
addTo(existWatches.remove(clientPath), result);
}
addPersistentWatches(clientPath, result);
addPersistentWatches(clientPath, type, result);
break;
case NodeChildrenChanged:
synchronized (childWatches) {
addTo(childWatches.remove(clientPath), result);
}
addPersistentWatches(clientPath, result);
addPersistentWatches(clientPath, type, result);
break;
case NodeDeleted:
synchronized (dataWatches) {
Expand All @@ -427,7 +427,7 @@ public Set<Watcher> materialize(
synchronized (childWatches) {
addTo(childWatches.remove(clientPath), result);
}
addPersistentWatches(clientPath, result);
addPersistentWatches(clientPath, type, result);
break;
default:
String errorMsg = String.format(
Expand All @@ -442,10 +442,17 @@ public Set<Watcher> materialize(
return result;
}

private void addPersistentWatches(String clientPath, Set<Watcher> result) {
private void addPersistentWatches(String clientPath, Watcher.Event.EventType type, Set<Watcher> result) {
synchronized (persistentWatches) {
addTo(persistentWatches.get(clientPath), result);
}
// The semantics of persistent recursive watch promise no child events on descendant nodes. When there
// are standard child watches on descendants of node being watched in persistent recursive mode, server
// will deliver child events to client inevitably. So we have to filter out child events for persistent
// recursive watches on client side.
if (type == Watcher.Event.EventType.NodeChildrenChanged) {
return;
}
synchronized (persistentRecursiveWatches) {
for (String path : PathParentIterator.forAll(clientPath).asIterable()) {
addTo(persistentRecursiveWatches.get(path), result);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,27 @@ public void testRemoval()
}
}

@Test
public void testNoChildEvents() throws Exception {
try (ZooKeeper zk = createClient()) {
zk.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);

zk.addWatch("/", persistentWatcher, PERSISTENT_RECURSIVE);

BlockingQueue<WatchedEvent> childEvents = new LinkedBlockingQueue<>();
zk.getChildren("/a", childEvents::add);

zk.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
zk.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);

assertEvent(childEvents, Watcher.Event.EventType.NodeChildrenChanged, "/a");

assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b");
assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c");
assertTrue(events.isEmpty());
}
}

@Test
public void testDisconnect() throws Exception {
try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) {
Expand Down

0 comments on commit e33fddf

Please sign in to comment.