|
@@ -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);
|