Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ZOOKEEPER-4475: Fix NodeChildrenChanged delivered to recursive watcher #1820

Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A comment about reason to filter NodeChildrenChanged event helps - it's not quite straightforward.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add comments to explain why this is necessary and why this must be done in client side.

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");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You may assert no outstanding event here.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add assertTrue(events.isEmpty()) to assert that no more events in queue.

assertTrue(events.isEmpty());
}
}

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