[
https://issues.apache.org/jira/browse/ZOOKEEPER-4466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kezhu Wang updated ZOOKEEPER-4466:
----------------------------------
Release Note: Previously, there is only one watch mode per path and
session. Later watch will ruin existing watch. For example, getData after
AddWatchMode::Persistent will result in one-time data watch and persistent
children watch. This is not our intend and it fears clients from arbitrary
watch. So we decide to support multiple different watch modes on one path in
one session. This way, users of client can watch whatever paths in whatever
modes. They will not ruin each other.
> Support different watch modes on same path
> ------------------------------------------
>
> 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
> Labels: pull-request-available
> Fix For: 3.9.0
>
> Time Spent: 3h 10m
> Remaining Estimate: 0h
>
> 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.10#820010)