[
https://issues.apache.org/jira/browse/ZOOKEEPER-4466?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17495823#comment-17495823
]
Kezhu Wang commented on ZOOKEEPER-4466:
---------------------------------------
[~randgalt] No, It does not adds child events to persistent recursive watches.
Actually, it fixed one situation that child event were delivered to persistent
recursive watcher. I give an example here:
# Add {{PERSISTENT_RECURSIVE}} to node "/a".
# Add {{PERSISTENT}} to node "/a/b" which is a child of above node.
Then child events will be delivered to "/a/b"({{NodeChildrenChanged}}), which
will be delivered to {{PERSISTENT_RECURSIVE}} watchers in client
unconditionally. This is the cause#1 I listed in description.
[~eolivelli] [~randgalt] I try to make different modes coexist with each other
on same path and session. I list the combinations here for evaluation:
# Add {{PERSISTENT}} after {{STANDARD}} watcher. No changes as {{PERSISTENT}}
includes all {{STANDARD}} watch events.
# Add {{PERSISTENT_RECURSIVE}} after {{STANDARD}} or {{PERSISTENT}} watcher.
There will be not child event in old behavior. In new behavior, child event
will be delivered to {{STANDARD}} or {{PERSISTENT}} watcher but not
{{PERSISTENT_RECURSIVE}}.
# Add {{STANDARD}} after {{PERSISTENT}} or {{PERSISTENT_RECURSIVE}}. Things go
weird in old behavior as server watchers are override by {{STANDARD}} watcher
but not client side. The results are complicated to describe depends on
different combinations in this case. In new behavior, them work independent
with no interference.
The combination#2 deliver missing child notification to standard and persistent
watcher. The combination#3, deliver missing persistent notifications.
For short, there are more notifications in new behavior and no child events for
recursive watcher.
> So basically you say that if you set a persistent watch we are missing to
> send notifications to regular watches?
No. {{PERSISTENT}} are super set of regular watches. I hope the combinations
listed above answer your question.
> we will be sending more notifications, not fewer
Yes.
> Watchers of different modes interfere on overlapping pathes
> -----------------------------------------------------------
>
> Key: ZOOKEEPER-4466
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4466
> Project: ZooKeeper
> Issue Type: Bug
> Components: java client, server
> Affects Versions: 3.6.3, 3.7, 3.6.4
> Reporter: Kezhu Wang
> Assignee: Kezhu Wang
> Priority: Major
>
> I used to think watchers of different modes are orthogonal. I found there are
> not, when I wrote tests for unfinished rust client. And I wrote [test
> cases|https://github.com/kezhuw/zookeeper/commit/79b05a95d2669a4acd16a4d544f24e2083a264f2#diff-8d31d27ea951fbc1f4fbda48d45748318f7124502839d825b77ad3fb8551bf43L152]
> in java and confirmed.
> I copied test case here for evaluation. You also clone from [my
> fork|https://github.com/kezhuw/zookeeper/tree/watch-overlapping-path-with-different-modes-test-case].
> {code:java}
> //
> zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java
> @Test
> public void testPathOverlapWithStandardWatcher() throws Exception {
> try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) {
> CountDownLatch nodeCreated = new CountDownLatch(1);
> zk.addWatch("/a", persistentWatcher, PERSISTENT_RECURSIVE);
> zk.exists("/a", event -> nodeCreated.countDown());
> zk.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
> CreateMode.PERSISTENT);
> zk.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
> CreateMode.PERSISTENT);
> zk.delete("/a/b", -1);
> zk.delete("/a", -1);
> assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a");
> assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b");
> assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a/b");
> assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a");
> assertTrue(nodeCreated.await(5, TimeUnit.SECONDS));
> }
> }
> @Test
> public void testPathOverlapWithPersistentWatcher() throws Exception {
> try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) {
> zk.addWatch("/a", persistentWatcher, PERSISTENT_RECURSIVE);
> zk.addWatch("/a/b", event -> {}, PERSISTENT);
> zk.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
> CreateMode.PERSISTENT);
> 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);
> zk.delete("/a/b/c", -1);
> zk.delete("/a/b", -1);
> zk.delete("/a", -1);
> assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a");
> assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b");
> assertEvent(events, Watcher.Event.EventType.NodeCreated,
> "/a/b/c");
> assertEvent(events, Watcher.Event.EventType.NodeDeleted,
> "/a/b/c");
> assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a/b");
> assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a");
> }
> }
> {code}
> I skimmed the code and found two possible causes:
> # {{ZKWatchManager.materialize}} materializes all persistent watchers(include
> recursive ones) for {{NodeChildrenChanged}} event.
> # {{WatcherModeManager}} trackes only one watcher mode.
--
This message was sent by Atlassian Jira
(v8.20.1#820001)