瀏覽代碼

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

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.
Kezhu Wang 2 年之前
父節點
當前提交
255b0c9137

+ 11 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/ZKWatchManager.java

@@ -404,13 +404,13 @@ class ZKWatchManager implements ClientWatchManager {
             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) {
@@ -427,7 +427,7 @@ class ZKWatchManager implements ClientWatchManager {
             synchronized (childWatches) {
                 addTo(childWatches.remove(clientPath), result);
             }
-            addPersistentWatches(clientPath, result);
+            addPersistentWatches(clientPath, type, result);
             break;
         default:
             String errorMsg = String.format(
@@ -442,10 +442,17 @@ class ZKWatchManager implements ClientWatchManager {
         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);

+ 21 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java

@@ -114,6 +114,27 @@ public class PersistentRecursiveWatcherTest extends ClientBase {
         }
     }
 
+    @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)) {