From 3c05c671d09e5b6df936af8f0a700995d5749e11 Mon Sep 17 00:00:00 2001 From: randgalt Date: Sun, 25 Dec 2016 16:36:13 -0500 Subject: [PATCH 01/31] basic work done. Needs more testing, tuning, etc. --- .../main/org/apache/zookeeper/ZooDefs.java | 2 + .../main/org/apache/zookeeper/ZooKeeper.java | 104 ++++++++++ .../org/apache/zookeeper/server/DataTree.java | 21 ++- .../server/FinalRequestProcessor.java | 12 ++ .../apache/zookeeper/server/PathIterator.java | 53 ++++++ .../server/PrepRequestProcessor.java | 1 + .../org/apache/zookeeper/server/Request.java | 3 + .../zookeeper/server/TraceFormatter.java | 2 + .../apache/zookeeper/server/WatchManager.java | 129 ++++++++----- .../apache/zookeeper/server/ZKDatabase.java | 4 + .../zookeeper/test/PathIteratorTest.java | 57 ++++++ .../zookeeper/test/PersistentWatcherTest.java | 177 ++++++++++++++++++ src/zookeeper.jute | 4 + 13 files changed, 516 insertions(+), 53 deletions(-) create mode 100644 src/java/main/org/apache/zookeeper/server/PathIterator.java create mode 100644 src/java/test/org/apache/zookeeper/test/PathIteratorTest.java create mode 100644 src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java diff --git a/src/java/main/org/apache/zookeeper/ZooDefs.java b/src/java/main/org/apache/zookeeper/ZooDefs.java index cd26c73d10d..2d67cdf4741 100644 --- a/src/java/main/org/apache/zookeeper/ZooDefs.java +++ b/src/java/main/org/apache/zookeeper/ZooDefs.java @@ -71,6 +71,8 @@ public interface OpCode { public final int createTTL = 21; + public final int addPersistentWatch = 22; + public final int auth = 100; public final int setWatches = 101; diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index cf274a4460f..11eb1f5ce66 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -41,12 +41,14 @@ import org.apache.zookeeper.common.PathUtils; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.proto.AddPersistentWatcherRequest; import org.apache.zookeeper.proto.CheckWatchesRequest; import org.apache.zookeeper.proto.Create2Response; import org.apache.zookeeper.proto.CreateRequest; import org.apache.zookeeper.proto.CreateResponse; import org.apache.zookeeper.proto.CreateTTLRequest; import org.apache.zookeeper.proto.DeleteRequest; +import org.apache.zookeeper.proto.ErrorResponse; import org.apache.zookeeper.proto.ExistsRequest; import org.apache.zookeeper.proto.GetACLRequest; import org.apache.zookeeper.proto.GetACLResponse; @@ -67,6 +69,7 @@ import org.apache.zookeeper.proto.SyncResponse; import org.apache.zookeeper.server.DataTree; import org.apache.zookeeper.server.EphemeralType; +import org.apache.zookeeper.server.PathIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -254,6 +257,8 @@ static class ZKWatchManager implements ClientWatchManager { new HashMap>(); private final Map> childWatches = new HashMap>(); + private final Map> persistentWatches = + new HashMap>(); private boolean disableAutoWatchReset; ZKWatchManager(boolean disableAutoWatchReset) { @@ -366,6 +371,12 @@ void containsWatcher(String path, Watcher watcher, synchronized (childWatches) { containsWatcher = contains(path, watcher, childWatches); } + + synchronized (persistentWatches) { + boolean contains_temp = contains(path, watcher, + persistentWatches); + containsWatcher |= contains_temp; + } break; } case Data: { @@ -378,6 +389,12 @@ void containsWatcher(String path, Watcher watcher, existWatches); containsWatcher |= contains_temp; } + + synchronized (persistentWatches) { + boolean contains_temp = contains(path, watcher, + persistentWatches); + containsWatcher |= contains_temp; + } break; } case Any: { @@ -394,6 +411,11 @@ void containsWatcher(String path, Watcher watcher, existWatches); containsWatcher |= contains_temp; } + synchronized (persistentWatches) { + boolean contains_temp = contains(path, watcher, + persistentWatches); + containsWatcher |= contains_temp; + } } } // Watcher function doesn't exists for the specified params @@ -482,6 +504,12 @@ public Set materialize(Watcher.Event.KeeperState state, } } + synchronized(persistentWatches) { + for(Set ws: persistentWatches.values()) { + result.addAll(ws); + } + } + return result; case NodeDataChanged: case NodeCreated: @@ -491,11 +519,13 @@ public Set materialize(Watcher.Event.KeeperState state, synchronized (existWatches) { addTo(existWatches.remove(clientPath), result); } + addPersistentWatches(clientPath, result); break; case NodeChildrenChanged: synchronized (childWatches) { addTo(childWatches.remove(clientPath), result); } + addPersistentWatches(clientPath, result); break; case NodeDeleted: synchronized (dataWatches) { @@ -512,6 +542,7 @@ public Set materialize(Watcher.Event.KeeperState state, synchronized (childWatches) { addTo(childWatches.remove(clientPath), result); } + addPersistentWatches(clientPath, result); break; default: String msg = "Unhandled watch event type " + type @@ -522,6 +553,15 @@ public Set materialize(Watcher.Event.KeeperState state, return result; } + + private void addPersistentWatches(String clientPath, Set result) { + synchronized (persistentWatches) { + PathIterator pathIterator = new PathIterator(clientPath); + while (pathIterator.hasNext()) { + addTo(persistentWatches.get(pathIterator.next()), result); + } + } + } } /** @@ -608,6 +648,22 @@ protected Map> getWatches(int rc) { } } + class PersistentWatchRegistration extends WatchRegistration { + public PersistentWatchRegistration(Watcher watcher, String clientPath) { + super(watcher, clientPath); + } + + @Override + protected Map> getWatches(int rc) { + return watchManager.persistentWatches; + } + + @Override + protected boolean shouldAddWatch(int rc) { + return rc == 0 || rc == KeeperException.Code.NONODE.intValue(); + } + } + public enum States { CONNECTING, ASSOCIATING, CONNECTED, CONNECTEDREADONLY, CLOSED, AUTH_FAILED, NOT_CONNECTED; @@ -2649,6 +2705,54 @@ public void sync(final String path, VoidCallback cb, Object ctx){ clientPath, serverPath, ctx, null); } + /** + * Set a watcher on the given path that: a) does not get removed when triggered (i.e. it stays active + * until it is removed); b) applies not only to the registered path but all child paths recursively + * @param basePath the top path that the watcher applies to + * @param watcher the watcher + * @param watcherType registration type: data, children or both + * @throws InterruptedException If the server transaction is interrupted. + * @throws KeeperException If the server signals an error with a non-zero + * error code. + */ + public void addPersistentWatch(String basePath, Watcher watcher, WatcherType watcherType) + throws KeeperException, InterruptedException { + PathUtils.validatePath(basePath); + String serverPath = prependChroot(basePath); + + RequestHeader h = new RequestHeader(); + h.setType(ZooDefs.OpCode.addPersistentWatch); + AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath, watcherType.getIntValue()); + ReplyHeader r = cnxn.submitRequest(h, request, new ErrorResponse(), + new PersistentWatchRegistration(watcher, basePath)); + if (r.getErr() != 0) { + throw KeeperException.create(KeeperException.Code.get(r.getErr()), + basePath); + } + } + + /** + * Async version of {@link #addPersistentWatch(String, Watcher, WatcherType)} (see it for details) + * + * @param basePath the top path that the watcher applies to + * @param watcher the watcher + * @param watcherType registration type: data, children or both + * @param cb a handler for the callback + * @param ctx context to be provided to the callback + * @throws IllegalArgumentException if an invalid path is specified + */ + public void addPersistentWatch(String basePath, Watcher watcher, WatcherType watcherType, + VoidCallback cb, Object ctx) { + PathUtils.validatePath(basePath); + String serverPath = prependChroot(basePath); + + RequestHeader h = new RequestHeader(); + h.setType(ZooDefs.OpCode.addPersistentWatch); + AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath, watcherType.getIntValue()); + cnxn.queuePacket(h, new ReplyHeader(), request, new ErrorResponse(), cb, + basePath, serverPath, ctx, new PersistentWatchRegistration(watcher, basePath)); + } + /** * For the given znode path, removes the specified watcher of given * watcherType. diff --git a/src/java/main/org/apache/zookeeper/server/DataTree.java b/src/java/main/org/apache/zookeeper/server/DataTree.java index 65c30ef4b3f..9f9c42401a2 100644 --- a/src/java/main/org/apache/zookeeper/server/DataTree.java +++ b/src/java/main/org/apache/zookeeper/server/DataTree.java @@ -636,6 +636,15 @@ public String getMaxPrefixWithQuota(String path) { } } + public void addPersistentWatch(String basePath, Watcher watcher, WatcherType type) { + if ( (type == WatcherType.Any) || (type == WatcherType.Data) ) { + dataWatches.addWatch(basePath, watcher, true); + } + if ( (type == WatcherType.Any) || (type == WatcherType.Children) ) { + childWatches.addWatch(basePath, watcher, true); + } + } + public byte[] getData(String path, Stat stat, Watcher watcher) throws KeeperException.NoNodeException { DataNode n = nodes.get(path); @@ -645,7 +654,7 @@ public byte[] getData(String path, Stat stat, Watcher watcher) synchronized (n) { n.copyStat(stat); if (watcher != null) { - dataWatches.addWatch(path, watcher); + dataWatches.addWatch(path, watcher, false); } return n.data; } @@ -656,7 +665,7 @@ public Stat statNode(String path, Watcher watcher) Stat stat = new Stat(); DataNode n = nodes.get(path); if (watcher != null) { - dataWatches.addWatch(path, watcher); + dataWatches.addWatch(path, watcher, false); } if (n == null) { throw new KeeperException.NoNodeException(); @@ -686,7 +695,7 @@ public List getChildren(String path, Stat stat, Watcher watcher) } if (watcher != null) { - childWatches.addWatch(path, watcher); + childWatches.addWatch(path, watcher, false); } return children; } @@ -1350,7 +1359,7 @@ public void setWatches(long relativeZxid, List dataWatches, watcher.process(new WatchedEvent(EventType.NodeDataChanged, KeeperState.SyncConnected, path)); } else { - this.dataWatches.addWatch(path, watcher); + this.dataWatches.addWatch(path, watcher, false); } } for (String path : existWatches) { @@ -1359,7 +1368,7 @@ public void setWatches(long relativeZxid, List dataWatches, watcher.process(new WatchedEvent(EventType.NodeCreated, KeeperState.SyncConnected, path)); } else { - this.dataWatches.addWatch(path, watcher); + this.dataWatches.addWatch(path, watcher, false); } } for (String path : childWatches) { @@ -1371,7 +1380,7 @@ public void setWatches(long relativeZxid, List dataWatches, watcher.process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, path)); } else { - this.childWatches.addWatch(path, watcher); + this.childWatches.addWatch(path, watcher, false); } } } diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java index 49a8273ccb8..7899de8666d 100644 --- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -35,9 +35,11 @@ import org.apache.zookeeper.common.Time; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.proto.AddPersistentWatcherRequest; import org.apache.zookeeper.proto.CheckWatchesRequest; import org.apache.zookeeper.proto.Create2Response; import org.apache.zookeeper.proto.CreateResponse; +import org.apache.zookeeper.proto.ErrorResponse; import org.apache.zookeeper.proto.ExistsRequest; import org.apache.zookeeper.proto.ExistsResponse; import org.apache.zookeeper.proto.GetACLRequest; @@ -341,6 +343,16 @@ public void processRequest(Request request) { setWatches.getChildWatches(), cnxn); break; } + case OpCode.addPersistentWatch: { + lastOp = "APEW"; + AddPersistentWatcherRequest addPersistentWatcherRequest = new AddPersistentWatcherRequest(); + ByteBufferInputStream.byteBuffer2Record(request.request, + addPersistentWatcherRequest); + WatcherType type = WatcherType.fromInt(addPersistentWatcherRequest.getType()); + zks.getZKDatabase().addPersistentWatch(addPersistentWatcherRequest.getPath(), cnxn, type); + rsp = new ErrorResponse(0); + break; + } case OpCode.getACL: { lastOp = "GETA"; GetACLRequest getACLRequest = new GetACLRequest(); diff --git a/src/java/main/org/apache/zookeeper/server/PathIterator.java b/src/java/main/org/apache/zookeeper/server/PathIterator.java new file mode 100644 index 00000000000..a62685c4c5e --- /dev/null +++ b/src/java/main/org/apache/zookeeper/server/PathIterator.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.server; + +public class PathIterator { + private String path; + private int level = -1; + + public PathIterator(String path) { + this.path = path; + } + + public boolean hasNext() { + return path.length() > 0; + } + + public boolean atParentPath() + { + return level > 0; + } + + public String next() { + String localPath = path; + if ( hasNext() ) { + ++level; + if ( path.equals("/") ) { + path = ""; + } else { + path = path.substring(0, path.lastIndexOf('/')); + if ( path.length() == 0 ) { + path = "/"; + } + } + } + return localPath; + } +} diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index 007b0668db2..5f9b19ca675 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -870,6 +870,7 @@ protected void pRequest(Request request) throws RequestProcessorException { case OpCode.setWatches: case OpCode.checkWatches: case OpCode.removeWatches: + case OpCode.addPersistentWatch: zks.sessionTracker.checkSession(request.sessionId, request.getOwner()); break; diff --git a/src/java/main/org/apache/zookeeper/server/Request.java b/src/java/main/org/apache/zookeeper/server/Request.java index ede9280441a..bf5ef935070 100644 --- a/src/java/main/org/apache/zookeeper/server/Request.java +++ b/src/java/main/org/apache/zookeeper/server/Request.java @@ -156,6 +156,7 @@ static boolean isValid(int type) { case OpCode.sync: case OpCode.checkWatches: case OpCode.removeWatches: + case OpCode.addPersistentWatch: return true; default: return false; @@ -243,6 +244,8 @@ static String op2String(int op) { return "checkWatches"; case OpCode.removeWatches: return "removeWatches"; + case OpCode.addPersistentWatch: + return "addPersistentWatch"; default: return "unknown " + op; } diff --git a/src/java/main/org/apache/zookeeper/server/TraceFormatter.java b/src/java/main/org/apache/zookeeper/server/TraceFormatter.java index 63a3edd6ec2..8735c127970 100644 --- a/src/java/main/org/apache/zookeeper/server/TraceFormatter.java +++ b/src/java/main/org/apache/zookeeper/server/TraceFormatter.java @@ -71,6 +71,8 @@ public static String op2String(int op) { return "error"; case OpCode.reconfig: return "reconfig"; + case OpCode.addPersistentWatch: + return "addPersistentWatch"; default: return "unknown " + op; } diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index d00bdb63e6d..9c9e9394dc6 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -19,9 +19,10 @@ package org.apache.zookeeper.server; import java.io.PrintWriter; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; +import java.util.Iterator; import java.util.Map; import java.util.Map.Entry; import java.util.Set; @@ -40,47 +41,52 @@ class WatchManager { private static final Logger LOG = LoggerFactory.getLogger(WatchManager.class); - private final HashMap> watchTable = - new HashMap>(); + private enum Type { + STANDARD, + PERSISTENT + } + + private final Map> watchTable = + new HashMap<>(); - private final HashMap> watch2Paths = - new HashMap>(); + private final Map> watch2Paths = + new HashMap<>(); synchronized int size(){ int result = 0; - for(Set watches : watchTable.values()) { + for(Map watches : watchTable.values()) { result += watches.size(); } return result; } - synchronized void addWatch(String path, Watcher watcher) { - HashSet list = watchTable.get(path); + synchronized void addWatch(String path, Watcher watcher, boolean persistent) { + Map list = watchTable.get(path); if (list == null) { // don't waste memory if there are few watches on a node // rehash when the 4th entry is added, doubling size thereafter // seems like a good compromise - list = new HashSet(4); + list = new HashMap<>(4); watchTable.put(path, list); } - list.add(watcher); + list.put(watcher, persistent ? Type.PERSISTENT : Type.STANDARD); - HashSet paths = watch2Paths.get(watcher); + Set paths = watch2Paths.get(watcher); if (paths == null) { // cnxns typically have many watches, so use default cap here - paths = new HashSet(); + paths = new HashSet<>(); watch2Paths.put(watcher, paths); } paths.add(path); } synchronized void removeWatcher(Watcher watcher) { - HashSet paths = watch2Paths.remove(watcher); + Set paths = watch2Paths.remove(watcher); if (paths == null) { return; } for (String p : paths) { - HashSet list = watchTable.get(p); + Map list = watchTable.get(p); if (list != null) { list.remove(watcher); if (list.size() == 0) { @@ -97,23 +103,44 @@ Set triggerWatch(String path, EventType type) { Set triggerWatch(String path, EventType type, Set supress) { WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path); - HashSet watchers; + Set watchers = new HashSet<>(); + PathIterator pathIterator = new PathIterator(path); synchronized (this) { - watchers = watchTable.remove(path); - if (watchers == null || watchers.isEmpty()) { - if (LOG.isTraceEnabled()) { - ZooTrace.logTraceMessage(LOG, - ZooTrace.EVENT_DELIVERY_TRACE_MASK, - "No watchers for " + path); + while (pathIterator.hasNext()) { + String localPath = pathIterator.next(); + Map thisWatchers = watchTable.get(localPath); + if (thisWatchers == null || thisWatchers.isEmpty()) { + continue; } - return null; - } - for (Watcher w : watchers) { - HashSet paths = watch2Paths.get(w); - if (paths != null) { - paths.remove(path); + Iterator> iterator = thisWatchers.entrySet().iterator(); + while (iterator.hasNext()) { + Entry entry = iterator.next(); + Type entryType = entry.getValue(); + Watcher watcher = entry.getKey(); + if (entryType == Type.PERSISTENT) { + watchers.add(watcher); + } + else if (!pathIterator.atParentPath()) { + watchers.add(watcher); + iterator.remove(); + Set paths = watch2Paths.get(watcher); + if (paths != null) { + paths.remove(localPath); + } + } } + if (thisWatchers.size() == 0) { + watchTable.remove(localPath); + } + } + } + if (watchers.size() == 0) { + if (LOG.isTraceEnabled()) { + ZooTrace.logTraceMessage(LOG, + ZooTrace.EVENT_DELIVERY_TRACE_MASK, + "No watchers for " + path ); } + return null; } for (Watcher w : watchers) { if (supress != null && supress.contains(w)) { @@ -135,7 +162,7 @@ public synchronized String toString() { .append(watchTable.size()).append(" paths\n"); int total = 0; - for (HashSet paths : watch2Paths.values()) { + for (Set paths : watch2Paths.values()) { total += paths.size(); } sb.append("Total watches:").append(total); @@ -147,20 +174,19 @@ public synchronized String toString() { * String representation of watches. Warning, may be large! * @param byPath iff true output watches by paths, otw output * watches by connection - * @return string representation of watches */ synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) { if (byPath) { - for (Entry> e : watchTable.entrySet()) { + for (Entry> e : watchTable.entrySet()) { pwriter.println(e.getKey()); - for (Watcher w : e.getValue()) { + for (Watcher w : e.getValue().keySet()) { pwriter.print("\t0x"); pwriter.print(Long.toHexString(((ServerCnxn)w).getSessionId())); pwriter.print("\n"); } } } else { - for (Entry> e : watch2Paths.entrySet()) { + for (Entry> e : watch2Paths.entrySet()) { pwriter.print("0x"); pwriter.println(Long.toHexString(((ServerCnxn)e.getKey()).getSessionId())); for (String path : e.getValue()) { @@ -181,11 +207,16 @@ synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) { * @return true if the watcher exists, false otherwise */ synchronized boolean containsWatcher(String path, Watcher watcher) { - HashSet paths = watch2Paths.get(watcher); - if (paths == null || !paths.contains(path)) { - return false; + PathIterator pathIterator = new PathIterator(path); + while (pathIterator.hasNext()) { + String localPath = pathIterator.next(); + Map watchers = watchTable.get(localPath); + Type watcherType = (watchers != null) ? watchers.get(watcher) : null; + if ((watcherType == Type.PERSISTENT) || (pathIterator.atParentPath() && (watcherType == Type.STANDARD))) { + return true; + } } - return true; + return false; } /** @@ -198,13 +229,13 @@ synchronized boolean containsWatcher(String path, Watcher watcher) { * @return true if the watcher successfully removed, false otherwise */ synchronized boolean removeWatcher(String path, Watcher watcher) { - HashSet paths = watch2Paths.get(watcher); + Set paths = watch2Paths.get(watcher); if (paths == null || !paths.remove(path)) { return false; } - HashSet list = watchTable.get(path); - if (list == null || !list.remove(watcher)) { + Map list = watchTable.get(path); + if (list == null || (list.remove(watcher) == null)) { return false; } @@ -222,10 +253,10 @@ synchronized boolean removeWatcher(String path, Watcher watcher) { * @see WatchesReport */ synchronized WatchesReport getWatches() { - Map> id2paths = new HashMap>(); - for (Entry> e: watch2Paths.entrySet()) { + Map> id2paths = new HashMap<>(); + for (Entry> e: watch2Paths.entrySet()) { Long id = ((ServerCnxn) e.getKey()).getSessionId(); - HashSet paths = new HashSet(e.getValue()); + HashSet paths = new HashSet<>(e.getValue()); id2paths.put(id, paths); } return new WatchesReport(id2paths); @@ -238,11 +269,11 @@ synchronized WatchesReport getWatches() { * @see WatchesPathReport */ synchronized WatchesPathReport getWatchesByPath() { - Map> path2ids = new HashMap>(); - for (Entry> e : watchTable.entrySet()) { - Set ids = new HashSet(e.getValue().size()); + Map> path2ids = new HashMap<>(); + for (Entry> e : watchTable.entrySet()) { + Set ids = new HashSet<>(e.getValue().size()); path2ids.put(e.getKey(), ids); - for (Watcher watcher : e.getValue()) { + for (Watcher watcher : e.getValue().keySet()) { ids.add(((ServerCnxn) watcher).getSessionId()); } } @@ -257,10 +288,14 @@ synchronized WatchesPathReport getWatchesByPath() { */ synchronized WatchesSummary getWatchesSummary() { int totalWatches = 0; - for (HashSet paths : watch2Paths.values()) { + for (Set paths : watch2Paths.values()) { totalWatches += paths.size(); } return new WatchesSummary (watch2Paths.size(), watchTable.size(), totalWatches); } + + private String getParent(String path) { + return path.substring(0, path.lastIndexOf('/')); + } } diff --git a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java index 05bbb9177a0..a6cf5eab6bb 100644 --- a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java +++ b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java @@ -614,4 +614,8 @@ public boolean containsWatcher(String path, WatcherType type, Watcher watcher) { public boolean removeWatch(String path, WatcherType type, Watcher watcher) { return dataTree.removeWatch(path, type, watcher); } + + public void addPersistentWatch(String basePath, Watcher watcher, WatcherType type) { + dataTree.addPersistentWatch(basePath, watcher, type); + } } diff --git a/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java b/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java new file mode 100644 index 00000000000..d852553a62d --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java @@ -0,0 +1,57 @@ +package org.apache.zookeeper.test; + +import org.apache.zookeeper.server.PathIterator; +import org.junit.Assert; +import org.junit.Test; + +public class PathIteratorTest { + @Test + public void testRoot() { + PathIterator pathIterator = new PathIterator("/"); + Assert.assertTrue(pathIterator.hasNext()); + Assert.assertFalse(pathIterator.atParentPath()); + Assert.assertEquals(pathIterator.next(), "/"); + Assert.assertFalse(pathIterator.hasNext()); + } + + @Test + public void test1Level() { + PathIterator pathIterator = new PathIterator("/a"); + Assert.assertTrue(pathIterator.hasNext()); + Assert.assertFalse(pathIterator.atParentPath()); + Assert.assertEquals(pathIterator.next(), "/a"); + + Assert.assertTrue(pathIterator.hasNext()); + Assert.assertEquals(pathIterator.next(), "/"); + Assert.assertTrue(pathIterator.atParentPath()); + + Assert.assertFalse(pathIterator.hasNext()); + } + + @Test + public void testLong() { + PathIterator pathIterator = new PathIterator("/a/b/c/d"); + + Assert.assertTrue(pathIterator.hasNext()); + Assert.assertEquals(pathIterator.next(), "/a/b/c/d"); + Assert.assertFalse(pathIterator.atParentPath()); + + Assert.assertTrue(pathIterator.hasNext()); + Assert.assertEquals(pathIterator.next(), "/a/b/c"); + Assert.assertTrue(pathIterator.atParentPath()); + + Assert.assertTrue(pathIterator.hasNext()); + Assert.assertEquals(pathIterator.next(), "/a/b"); + Assert.assertTrue(pathIterator.atParentPath()); + + Assert.assertTrue(pathIterator.hasNext()); + Assert.assertEquals(pathIterator.next(), "/a"); + Assert.assertTrue(pathIterator.atParentPath()); + + Assert.assertTrue(pathIterator.hasNext()); + Assert.assertEquals(pathIterator.next(), "/"); + Assert.assertTrue(pathIterator.atParentPath()); + + Assert.assertFalse(pathIterator.hasNext()); + } +} diff --git a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java new file mode 100644 index 00000000000..31788cc99a7 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -0,0 +1,177 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.test; + +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +public class PersistentWatcherTest extends ClientBase { + private static final Logger LOG = LoggerFactory.getLogger(PersistentWatcherTest.class); + private BlockingQueue events; + private Watcher persistentWatcher; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + + events = new LinkedBlockingQueue<>(); + persistentWatcher = new Watcher() { + @Override + public void process(WatchedEvent event) { + events.add(event); + } + }; + } + + @Test + public void testBasic() + throws IOException, InterruptedException, KeeperException { + ZooKeeper zk = null; + try { + zk = createClient(new CountdownWatcher(), hostPort); + + zk.addPersistentWatch("/a/b", persistentWatcher, Watcher.WatcherType.Any); + internalTestBasic(zk); + } finally { + if (zk != null) { + zk.close(); + } + } + } + + @Test + public void testBasicAsync() + throws IOException, InterruptedException, KeeperException { + ZooKeeper zk = null; + try { + zk = createClient(new CountdownWatcher(), hostPort); + + final CountDownLatch latch = new CountDownLatch(1); + AsyncCallback.VoidCallback cb = new AsyncCallback.VoidCallback() { + @Override + public void processResult(int rc, String path, Object ctx) { + if ( rc == 0 ) { + latch.countDown(); + } + } + }; + zk.addPersistentWatch("/a/b", persistentWatcher, Watcher.WatcherType.Any, cb, null); + Assert.assertTrue(latch.await(5, TimeUnit.SECONDS)); + internalTestBasic(zk); + } finally { + if (zk != null) { + zk.close(); + } + } + } + + private void internalTestBasic(ZooKeeper zk) throws KeeperException, InterruptedException { + 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.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.setData("/a/b/c/d/e", new byte[0], -1); + + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c/d"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c/d/e"); + } + + @Test + public void testWatcherTypeChildren() + throws IOException, InterruptedException, KeeperException { + ZooKeeper zk = null; + try { + zk = createClient(new CountdownWatcher(), hostPort); + + zk.addPersistentWatch("/a/b", persistentWatcher, Watcher.WatcherType.Children); + 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.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c/d"); + } finally { + if (zk != null) { + zk.close(); + } + } + } + + @Test + public void testWatcherTypeData() + throws IOException, InterruptedException, KeeperException { + ZooKeeper zk = null; + try { + zk = createClient(new CountdownWatcher(), hostPort); + + zk.addPersistentWatch("/a/b", persistentWatcher, Watcher.WatcherType.Data); + 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.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.setData("/a/b/c/d/e", new byte[0], -1); + + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c/d/e"); + } finally { + if (zk != null) { + zk.close(); + } + } + } + + private void assertEvent(BlockingQueue events, Watcher.Event.EventType eventType, String path) + throws InterruptedException { + WatchedEvent event = events.poll(5, TimeUnit.SECONDS); + Assert.assertNotNull(event); + Assert.assertEquals(eventType, event.getType()); + Assert.assertEquals(path, event.getPath()); + } +} diff --git a/src/zookeeper.jute b/src/zookeeper.jute index a404e787325..6900fc9a188 100644 --- a/src/zookeeper.jute +++ b/src/zookeeper.jute @@ -177,6 +177,10 @@ module org.apache.zookeeper.proto { class SetACLResponse { org.apache.zookeeper.data.Stat stat; } + class AddPersistentWatcherRequest { + ustring path; + int type; // watcher type + } class WatcherEvent { int type; // event type int state; // state of the Keeper client runtime From ca4a000dcf294aaebd09d3118ebc62cb0783f9cc Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 10:06:55 -0500 Subject: [PATCH 02/31] working on persistent watcher removal --- .../main/org/apache/zookeeper/Watcher.java | 4 +- .../main/org/apache/zookeeper/ZooKeeper.java | 19 +++++---- .../org/apache/zookeeper/server/DataTree.java | 10 ++--- .../server/FinalRequestProcessor.java | 3 +- .../apache/zookeeper/server/ZKDatabase.java | 4 +- .../zookeeper/test/PersistentWatcherTest.java | 42 ++++--------------- src/zookeeper.jute | 1 - 7 files changed, 29 insertions(+), 54 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/Watcher.java b/src/java/main/org/apache/zookeeper/Watcher.java index f436f231ad9..126b053d921 100644 --- a/src/java/main/org/apache/zookeeper/Watcher.java +++ b/src/java/main/org/apache/zookeeper/Watcher.java @@ -119,7 +119,8 @@ public enum EventType { NodeDataChanged (3), NodeChildrenChanged (4), DataWatchRemoved (5), - ChildWatchRemoved (6); + ChildWatchRemoved (6), + PersistentWatchRemoved (7); private final int intValue; // Integer representation of value // for sending over wire @@ -141,6 +142,7 @@ public static EventType fromInt(int intValue) { case 4: return EventType.NodeChildrenChanged; case 5: return EventType.DataWatchRemoved; case 6: return EventType.ChildWatchRemoved; + case 7: return EventType.PersistentWatchRemoved; default: throw new RuntimeException("Invalid integer value for conversion to EventType"); diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index 11eb1f5ce66..ac0bf83e5a9 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -286,6 +286,8 @@ public Map> removeWatcher(String clientPath, .put(EventType.ChildWatchRemoved, childWatchersToRem); HashSet dataWatchersToRem = new HashSet(); removedWatchers.put(EventType.DataWatchRemoved, dataWatchersToRem); + HashSet persistentWatchersToRem = new HashSet(); + removedWatchers.put(EventType.PersistentWatchRemoved, persistentWatchersToRem); boolean removedWatcher = false; switch (watcherType) { case Children: { @@ -324,6 +326,11 @@ public Map> removeWatcher(String clientPath, watcher, clientPath, local, rc, dataWatchersToRem); removedWatcher |= removedDataWatcher; } + synchronized (persistentWatches) { + boolean removedPersistenWatcher = removeWatches(persistentWatches, + watcher, clientPath, local, rc, persistentWatchersToRem); + removedWatcher |= removedPersistenWatcher; + } } } // Watcher function doesn't exists for the specified params @@ -2710,19 +2717,18 @@ public void sync(final String path, VoidCallback cb, Object ctx){ * until it is removed); b) applies not only to the registered path but all child paths recursively * @param basePath the top path that the watcher applies to * @param watcher the watcher - * @param watcherType registration type: data, children or both * @throws InterruptedException If the server transaction is interrupted. * @throws KeeperException If the server signals an error with a non-zero * error code. */ - public void addPersistentWatch(String basePath, Watcher watcher, WatcherType watcherType) + public void addPersistentWatch(String basePath, Watcher watcher) throws KeeperException, InterruptedException { PathUtils.validatePath(basePath); String serverPath = prependChroot(basePath); RequestHeader h = new RequestHeader(); h.setType(ZooDefs.OpCode.addPersistentWatch); - AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath, watcherType.getIntValue()); + AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath); ReplyHeader r = cnxn.submitRequest(h, request, new ErrorResponse(), new PersistentWatchRegistration(watcher, basePath)); if (r.getErr() != 0) { @@ -2732,23 +2738,22 @@ public void addPersistentWatch(String basePath, Watcher watcher, WatcherType wat } /** - * Async version of {@link #addPersistentWatch(String, Watcher, WatcherType)} (see it for details) + * Async version of {@link #addPersistentWatch(String, Watcher)} (see it for details) * * @param basePath the top path that the watcher applies to * @param watcher the watcher - * @param watcherType registration type: data, children or both * @param cb a handler for the callback * @param ctx context to be provided to the callback * @throws IllegalArgumentException if an invalid path is specified */ - public void addPersistentWatch(String basePath, Watcher watcher, WatcherType watcherType, + public void addPersistentWatch(String basePath, Watcher watcher, VoidCallback cb, Object ctx) { PathUtils.validatePath(basePath); String serverPath = prependChroot(basePath); RequestHeader h = new RequestHeader(); h.setType(ZooDefs.OpCode.addPersistentWatch); - AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath, watcherType.getIntValue()); + AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath); cnxn.queuePacket(h, new ReplyHeader(), request, new ErrorResponse(), cb, basePath, serverPath, ctx, new PersistentWatchRegistration(watcher, basePath)); } diff --git a/src/java/main/org/apache/zookeeper/server/DataTree.java b/src/java/main/org/apache/zookeeper/server/DataTree.java index 9f9c42401a2..3265dddaf75 100644 --- a/src/java/main/org/apache/zookeeper/server/DataTree.java +++ b/src/java/main/org/apache/zookeeper/server/DataTree.java @@ -636,13 +636,9 @@ public String getMaxPrefixWithQuota(String path) { } } - public void addPersistentWatch(String basePath, Watcher watcher, WatcherType type) { - if ( (type == WatcherType.Any) || (type == WatcherType.Data) ) { - dataWatches.addWatch(basePath, watcher, true); - } - if ( (type == WatcherType.Any) || (type == WatcherType.Children) ) { - childWatches.addWatch(basePath, watcher, true); - } + public void addPersistentWatch(String basePath, Watcher watcher) { + dataWatches.addWatch(basePath, watcher, true); + childWatches.addWatch(basePath, watcher, true); } public byte[] getData(String path, Stat stat, Watcher watcher) diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java index 7899de8666d..d28491a0c96 100644 --- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -348,8 +348,7 @@ public void processRequest(Request request) { AddPersistentWatcherRequest addPersistentWatcherRequest = new AddPersistentWatcherRequest(); ByteBufferInputStream.byteBuffer2Record(request.request, addPersistentWatcherRequest); - WatcherType type = WatcherType.fromInt(addPersistentWatcherRequest.getType()); - zks.getZKDatabase().addPersistentWatch(addPersistentWatcherRequest.getPath(), cnxn, type); + zks.getZKDatabase().addPersistentWatch(addPersistentWatcherRequest.getPath(), cnxn); rsp = new ErrorResponse(0); break; } diff --git a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java index a6cf5eab6bb..e47a6da8fa3 100644 --- a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java +++ b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java @@ -615,7 +615,7 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) { return dataTree.removeWatch(path, type, watcher); } - public void addPersistentWatch(String basePath, Watcher watcher, WatcherType type) { - dataTree.addPersistentWatch(basePath, watcher, type); + public void addPersistentWatch(String basePath, Watcher watcher) { + dataTree.addPersistentWatch(basePath, watcher); } } diff --git a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java index 31788cc99a7..e1564107ea9 100644 --- a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java +++ b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -63,7 +63,7 @@ public void testBasic() try { zk = createClient(new CountdownWatcher(), hostPort); - zk.addPersistentWatch("/a/b", persistentWatcher, Watcher.WatcherType.Any); + zk.addPersistentWatch("/a/b", persistentWatcher); internalTestBasic(zk); } finally { if (zk != null) { @@ -88,7 +88,7 @@ public void processResult(int rc, String path, Object ctx) { } } }; - zk.addPersistentWatch("/a/b", persistentWatcher, Watcher.WatcherType.Any, cb, null); + zk.addPersistentWatch("/a/b", persistentWatcher, cb, null); Assert.assertTrue(latch.await(5, TimeUnit.SECONDS)); internalTestBasic(zk); } finally { @@ -117,49 +117,23 @@ private void internalTestBasic(ZooKeeper zk) throws KeeperException, Interrupted } @Test - public void testWatcherTypeChildren() + public void testRemoval() throws IOException, InterruptedException, KeeperException { ZooKeeper zk = null; try { zk = createClient(new CountdownWatcher(), hostPort); - zk.addPersistentWatch("/a/b", persistentWatcher, Watcher.WatcherType.Children); + zk.addPersistentWatch("/a/b", persistentWatcher); 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.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); - assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c"); - assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c/d"); - } finally { - if (zk != null) { - zk.close(); - } - } - } - - @Test - public void testWatcherTypeData() - throws IOException, InterruptedException, KeeperException { - ZooKeeper zk = null; - try { - zk = createClient(new CountdownWatcher(), hostPort); - zk.addPersistentWatch("/a/b", persistentWatcher, Watcher.WatcherType.Data); - 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.removeWatches("/a/b", persistentWatcher, Watcher.WatcherType.Any, false); zk.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk.setData("/a/b/c/d/e", new byte[0], -1); - - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); - assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.PersistentWatchRemoved, "/a/b"); } finally { if (zk != null) { zk.close(); diff --git a/src/zookeeper.jute b/src/zookeeper.jute index 6900fc9a188..50bb442d189 100644 --- a/src/zookeeper.jute +++ b/src/zookeeper.jute @@ -179,7 +179,6 @@ module org.apache.zookeeper.proto { } class AddPersistentWatcherRequest { ustring path; - int type; // watcher type } class WatcherEvent { int type; // event type From bf13deda0b00ca67cd1fa963961d95a22634ed88 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 12:59:04 -0500 Subject: [PATCH 03/31] Support resetting persistent watches --- .../main/org/apache/zookeeper/ClientCnxn.java | 19 ++++++++++++++----- .../main/org/apache/zookeeper/ZooDefs.java | 2 ++ .../main/org/apache/zookeeper/ZooKeeper.java | 6 ++++++ .../org/apache/zookeeper/server/DataTree.java | 15 +++++++++++++-- .../server/FinalRequestProcessor.java | 15 +++++++++++++++ .../server/PrepRequestProcessor.java | 1 + .../org/apache/zookeeper/server/Request.java | 4 ++++ .../apache/zookeeper/server/ZKDatabase.java | 14 ++++++++++++++ .../zookeeper/test/PersistentWatcherTest.java | 19 +++++++++++++++++++ src/zookeeper.jute | 7 +++++++ 10 files changed, 95 insertions(+), 7 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index 9e6c154eca5..8ac1e1320e7 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -82,6 +82,7 @@ import org.apache.zookeeper.proto.SetACLResponse; import org.apache.zookeeper.proto.SetDataResponse; import org.apache.zookeeper.proto.SetWatches; +import org.apache.zookeeper.proto.SetWatches2; import org.apache.zookeeper.proto.WatcherEvent; import org.apache.zookeeper.server.ByteBufferInputStream; import org.apache.zookeeper.server.ZooKeeperThread; @@ -960,18 +961,22 @@ void primeConnection() throws IOException { List dataWatches = zooKeeper.getDataWatches(); List existWatches = zooKeeper.getExistWatches(); List childWatches = zooKeeper.getChildWatches(); + List persistentWatches = zooKeeper.getPersistentWatches(); if (!dataWatches.isEmpty() - || !existWatches.isEmpty() || !childWatches.isEmpty()) { + || !existWatches.isEmpty() || !childWatches.isEmpty() || !persistentWatches.isEmpty()) { Iterator dataWatchesIter = prependChroot(dataWatches).iterator(); Iterator existWatchesIter = prependChroot(existWatches).iterator(); Iterator childWatchesIter = prependChroot(childWatches).iterator(); + Iterator persistentWatchesIter = prependChroot(persistentWatches).iterator(); long setWatchesLastZxid = lastZxid; while (dataWatchesIter.hasNext() - || existWatchesIter.hasNext() || childWatchesIter.hasNext()) { + || existWatchesIter.hasNext() || childWatchesIter.hasNext() + || persistentWatchesIter.hasNext()) { List dataWatchesBatch = new ArrayList(); List existWatchesBatch = new ArrayList(); List childWatchesBatch = new ArrayList(); + List persistentWatchesBatch = new ArrayList(); int batchLength = 0; // Note, we may exceed our max length by a bit when we add the last @@ -987,17 +992,21 @@ void primeConnection() throws IOException { } else if (childWatchesIter.hasNext()) { watch = childWatchesIter.next(); childWatchesBatch.add(watch); + } else if (persistentWatchesIter.hasNext()) { + watch = persistentWatchesIter.next(); + persistentWatchesBatch.add(watch); } else { break; } batchLength += watch.length(); } - SetWatches sw = new SetWatches(setWatchesLastZxid, + SetWatches2 sw = new SetWatches2(setWatchesLastZxid, dataWatchesBatch, existWatchesBatch, - childWatchesBatch); - RequestHeader header = new RequestHeader(-8, OpCode.setWatches); + childWatchesBatch, + persistentWatchesBatch); + RequestHeader header = new RequestHeader(-8, OpCode.setWatches2); Packet packet = new Packet(header, new ReplyHeader(), sw, null, null); outgoingQueue.addFirst(packet); } diff --git a/src/java/main/org/apache/zookeeper/ZooDefs.java b/src/java/main/org/apache/zookeeper/ZooDefs.java index 2d67cdf4741..85ab2e5dd92 100644 --- a/src/java/main/org/apache/zookeeper/ZooDefs.java +++ b/src/java/main/org/apache/zookeeper/ZooDefs.java @@ -79,6 +79,8 @@ public interface OpCode { public final int sasl = 102; + public final int setWatches2 = 103; + public final int createSession = -10; public final int closeSession = -11; diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index ac0bf83e5a9..1d81320f973 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -242,6 +242,12 @@ protected List getChildWatches() { return rc; } } + protected List getPersistentWatches() { + synchronized(watchManager.persistentWatches) { + List rc = new ArrayList(watchManager.persistentWatches.keySet()); + return rc; + } + } /** * Manage watchers & handle events generated by the ClientCnxn object. diff --git a/src/java/main/org/apache/zookeeper/server/DataTree.java b/src/java/main/org/apache/zookeeper/server/DataTree.java index 3265dddaf75..ca9b3f565e6 100644 --- a/src/java/main/org/apache/zookeeper/server/DataTree.java +++ b/src/java/main/org/apache/zookeeper/server/DataTree.java @@ -1345,6 +1345,13 @@ public void removeCnxn(Watcher watcher) { public void setWatches(long relativeZxid, List dataWatches, List existWatches, List childWatches, Watcher watcher) { + setWatches(relativeZxid, dataWatches, existWatches, childWatches, + Collections.emptyList(), watcher); + } + + public void setWatches(long relativeZxid, List dataWatches, + List existWatches, List childWatches, List persistentWatches, + Watcher watcher) { for (String path : dataWatches) { DataNode node = getNode(path); WatchedEvent e = null; @@ -1378,8 +1385,12 @@ public void setWatches(long relativeZxid, List dataWatches, } else { this.childWatches.addWatch(path, watcher, false); } - } - } + } + for (String path : persistentWatches) { + this.childWatches.addWatch(path, watcher, true); + this.dataWatches.addWatch(path, watcher, true); + } + } /** * This method sets the Cversion and Pzxid for the specified node to the diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java index d28491a0c96..ede177637c3 100644 --- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -55,6 +55,7 @@ import org.apache.zookeeper.proto.SetACLResponse; import org.apache.zookeeper.proto.SetDataResponse; import org.apache.zookeeper.proto.SetWatches; +import org.apache.zookeeper.proto.SetWatches2; import org.apache.zookeeper.proto.SyncRequest; import org.apache.zookeeper.proto.SyncResponse; import org.apache.zookeeper.server.DataTree.ProcessTxnResult; @@ -343,6 +344,20 @@ public void processRequest(Request request) { setWatches.getChildWatches(), cnxn); break; } + case OpCode.setWatches2: { + lastOp = "STW2"; + SetWatches2 setWatches = new SetWatches2(); + // XXX We really should NOT need this!!!! + request.request.rewind(); + ByteBufferInputStream.byteBuffer2Record(request.request, setWatches); + long relativeZxid = setWatches.getRelativeZxid(); + zks.getZKDatabase().setWatches(relativeZxid, + setWatches.getDataWatches(), + setWatches.getExistWatches(), + setWatches.getChildWatches(), + setWatches.getPersistentWatches(), cnxn); + break; + } case OpCode.addPersistentWatch: { lastOp = "APEW"; AddPersistentWatcherRequest addPersistentWatcherRequest = new AddPersistentWatcherRequest(); diff --git a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java index 5f9b19ca675..12bf2ba3104 100644 --- a/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -868,6 +868,7 @@ protected void pRequest(Request request) throws RequestProcessorException { case OpCode.getChildren2: case OpCode.ping: case OpCode.setWatches: + case OpCode.setWatches2: case OpCode.checkWatches: case OpCode.removeWatches: case OpCode.addPersistentWatch: diff --git a/src/java/main/org/apache/zookeeper/server/Request.java b/src/java/main/org/apache/zookeeper/server/Request.java index bf5ef935070..03b775bcf4b 100644 --- a/src/java/main/org/apache/zookeeper/server/Request.java +++ b/src/java/main/org/apache/zookeeper/server/Request.java @@ -153,6 +153,7 @@ static boolean isValid(int type) { case OpCode.setACL: case OpCode.setData: case OpCode.setWatches: + case OpCode.setWatches2: case OpCode.sync: case OpCode.checkWatches: case OpCode.removeWatches: @@ -206,6 +207,8 @@ static String op2String(int op) { return "createContainer"; case OpCode.setWatches: return "setWatches"; + case OpCode.setWatches2: + return "setWatches2"; case OpCode.delete: return "delete"; case OpCode.deleteContainer: @@ -266,6 +269,7 @@ public String toString() { String path = "n/a"; if (type != OpCode.createSession && type != OpCode.setWatches + && type != OpCode.setWatches2 && type != OpCode.closeSession && request != null && request.remaining() >= 4) diff --git a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java index e47a6da8fa3..3f67ee15a3f 100644 --- a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java +++ b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java @@ -450,6 +450,20 @@ public void setWatches(long relativeZxid, List dataWatches, dataTree.setWatches(relativeZxid, dataWatches, existWatches, childWatches, watcher); } + /** + * set watches on the datatree + * @param relativeZxid the relative zxid that client has seen + * @param dataWatches the data watches the client wants to reset + * @param existWatches the exists watches the client wants to reset + * @param childWatches the child watches the client wants to reset + * @param persistentWatches the persistent watches the client wants to reset + * @param watcher the watcher function + */ + public void setWatches(long relativeZxid, List dataWatches, + List existWatches, List childWatches, List persistentWatches, Watcher watcher) { + dataTree.setWatches(relativeZxid, dataWatches, existWatches, childWatches, persistentWatches, watcher); + } + /** * get acl for a path * @param path the path to query for acl diff --git a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java index e1564107ea9..eb741f5acc2 100644 --- a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java +++ b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -141,6 +141,25 @@ public void testRemoval() } } + @Test + public void testDisconnect() throws Exception { + ZooKeeper zk = null; + try { + zk = createClient(new CountdownWatcher(), hostPort); + + zk.addPersistentWatch("/a/b", persistentWatcher); + stopServer(); + assertEvent(events, Watcher.Event.EventType.None, null); + startServer(); + assertEvent(events, Watcher.Event.EventType.None, null); + internalTestBasic(zk); + } finally { + if (zk != null) { + zk.close(); + } + } + } + private void assertEvent(BlockingQueue events, Watcher.Event.EventType eventType, String path) throws InterruptedException { WatchedEvent event = events.poll(5, TimeUnit.SECONDS); diff --git a/src/zookeeper.jute b/src/zookeeper.jute index 50bb442d189..45710033cbb 100644 --- a/src/zookeeper.jute +++ b/src/zookeeper.jute @@ -73,6 +73,13 @@ module org.apache.zookeeper.proto { vectorexistWatches; vectorchildWatches; } + class SetWatches2 { + long relativeZxid; + vectordataWatches; + vectorexistWatches; + vectorchildWatches; + vectorpersistentWatches; + } class RequestHeader { int xid; int type; From 27d8d6cd45cb6adfabf50143f6de62a371447519 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 13:21:17 -0500 Subject: [PATCH 04/31] docs --- src/java/main/org/apache/zookeeper/ZooKeeper.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index 1d81320f973..fd1ef6b20d6 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -2720,7 +2720,9 @@ public void sync(final String path, VoidCallback cb, Object ctx){ /** * Set a watcher on the given path that: a) does not get removed when triggered (i.e. it stays active - * until it is removed); b) applies not only to the registered path but all child paths recursively + * until it is removed); b) applies not only to the registered path but all child paths recursively. This watcher + * is triggered for both data and child events. To remove the watcher, use + * {@link #removeWatches(String, Watcher, WatcherType, boolean)} with {@link WatcherType#Any} * @param basePath the top path that the watcher applies to * @param watcher the watcher * @throws InterruptedException If the server transaction is interrupted. From 2766fb1020c600af579a0f701fa3c00ea92b7e22 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 13:44:42 -0500 Subject: [PATCH 05/31] containsWatcher() was broken for STANDARD watchers --- src/java/main/org/apache/zookeeper/server/WatchManager.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index 9c9e9394dc6..92686ef7e19 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -212,7 +212,10 @@ synchronized boolean containsWatcher(String path, Watcher watcher) { String localPath = pathIterator.next(); Map watchers = watchTable.get(localPath); Type watcherType = (watchers != null) ? watchers.get(watcher) : null; - if ((watcherType == Type.PERSISTENT) || (pathIterator.atParentPath() && (watcherType == Type.STANDARD))) { + if ((watcherType == Type.STANDARD) && !pathIterator.atParentPath()) { + return true; + } + if (watcherType == Type.PERSISTENT) { return true; } } From 86fa1fbcb75021179f80588a2ea46aad2127fb4e Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 14:20:00 -0500 Subject: [PATCH 06/31] removed unused import --- src/java/main/org/apache/zookeeper/server/WatchManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index 92686ef7e19..bf2080321c9 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -19,7 +19,6 @@ package org.apache.zookeeper.server; import java.io.PrintWriter; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; From b490c84d1e56335ba66f9c56d64134886b144451 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 14:20:08 -0500 Subject: [PATCH 07/31] Updated doc for persistent watches --- .../content/xdocs/zookeeperOver.xml | 5 ++++ .../content/xdocs/zookeeperProgrammers.xml | 23 +++++++++++++++++-- 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperOver.xml b/src/docs/src/documentation/content/xdocs/zookeeperOver.xml index f972657489f..a28ae763559 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperOver.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperOver.xml @@ -172,6 +172,11 @@ connection between the client and one of the Zoo Keeper servers is broken, the client will receive a local notification. These can be used to [tbd]. + + New in 3.6.0: Clients can also set + permanent, recursive watches on a znode that are not removed when triggered + and that trigger for changes on the registered znode as well as any children + znodes recursively.
diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index 8dec729dd46..9f23e9bffa5 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -192,6 +192,11 @@ sends the client a notification. More information about watches can be found in the section ZooKeeper Watches. + + New in 3.6.0: Clients can also set + permanent, recursive watches on a znode that are not removed when triggered + and that trigger for changes on the registered znode as well as any children + znodes recursively.
@@ -712,6 +717,20 @@
+
+ Persistent Recursive Watches + New in 3.6.0: There is now a variation on the standard + watch described above whereby you can set a watch that does not get removed when triggered. + Additionally, these watches trigger all event types (created, deleted, changed, child) + recursively for all znodes starting at the znode that the watch is registered for. + + Persistent recursive watches are set using the method addPersistentWatch(). The triggering semantics and guarantees + (other then one-time triggering) are the same as standard watches. Persistent recursive + watches are removed using removeWatches() with watcher type + WatcherType.Any. +
+
Remove Watches We can remove the watches registered on a znode with a call to @@ -767,7 +786,7 @@ - Watches are one time triggers; if you get a watch event and + Standard watches are one time triggers; if you get a watch event and you want to get notified of future changes, you must set another watch. @@ -775,7 +794,7 @@ - Because watches are one time triggers and there is latency + Because standard watches are one time triggers and there is latency between getting the event and sending a new request to get a watch you cannot reliably see every change that happens to a node in ZooKeeper. Be prepared to handle the case where the znode changes From e00b7a9ef1d0e909025b501b19eddcc0c37a2fc0 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 15:02:58 -0500 Subject: [PATCH 08/31] Fixed some Jenkins issues --- .../main/org/apache/zookeeper/ZooKeeper.java | 2 +- .../apache/zookeeper/server/ZKDatabase.java | 8 ++++++++ .../zookeeper/test/PathIteratorTest.java | 18 ++++++++++++++++++ 3 files changed, 27 insertions(+), 1 deletion(-) diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index fd1ef6b20d6..f8714da19cc 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -2722,7 +2722,7 @@ public void sync(final String path, VoidCallback cb, Object ctx){ * Set a watcher on the given path that: a) does not get removed when triggered (i.e. it stays active * until it is removed); b) applies not only to the registered path but all child paths recursively. This watcher * is triggered for both data and child events. To remove the watcher, use - * {@link #removeWatches(String, Watcher, WatcherType, boolean)} with {@link WatcherType#Any} + * removeWatches() with WatcherType.Any * @param basePath the top path that the watcher applies to * @param watcher the watcher * @throws InterruptedException If the server transaction is interrupted. diff --git a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java index 3f67ee15a3f..a9369773e45 100644 --- a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java +++ b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java @@ -629,6 +629,14 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) { return dataTree.removeWatch(path, type, watcher); } + /** + * Add a persistent watch + * + * @param basePath + * watch base + * @param watcher + * the watcher + */ public void addPersistentWatch(String basePath, Watcher watcher) { dataTree.addPersistentWatch(basePath, watcher); } diff --git a/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java b/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java index d852553a62d..c9c986536d5 100644 --- a/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java +++ b/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.zookeeper.test; import org.apache.zookeeper.server.PathIterator; From d60fde71b022693a5b75d08af4d690ccdc2097c2 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 18:17:25 -0500 Subject: [PATCH 09/31] Fix a few nits: --- .../main/org/apache/zookeeper/server/PathIterator.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/server/PathIterator.java b/src/java/main/org/apache/zookeeper/server/PathIterator.java index a62685c4c5e..9149ade8121 100644 --- a/src/java/main/org/apache/zookeeper/server/PathIterator.java +++ b/src/java/main/org/apache/zookeeper/server/PathIterator.java @@ -23,11 +23,12 @@ public class PathIterator { private int level = -1; public PathIterator(String path) { + // NOTE: asserts that the path has already been validated this.path = path; } public boolean hasNext() { - return path.length() > 0; + return !path.isEmpty(); } public boolean atParentPath() @@ -37,9 +38,9 @@ public boolean atParentPath() public String next() { String localPath = path; - if ( hasNext() ) { + if (hasNext()) { ++level; - if ( path.equals("/") ) { + if (path.equals("/")) { path = ""; } else { path = path.substring(0, path.lastIndexOf('/')); From 0b50826b3fc373c8cd3f39e905c351a57c5ef688 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 18:30:05 -0500 Subject: [PATCH 10/31] Based on suggestion from eribeiro make PathIterator usable in for each statements --- .../main/org/apache/zookeeper/ZooKeeper.java | 5 +-- .../apache/zookeeper/server/PathIterator.java | 38 ++++++++++++++++++- .../apache/zookeeper/server/WatchManager.java | 6 +-- 3 files changed, 41 insertions(+), 8 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index f8714da19cc..3e621f1c1fb 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -569,9 +569,8 @@ public Set materialize(Watcher.Event.KeeperState state, private void addPersistentWatches(String clientPath, Set result) { synchronized (persistentWatches) { - PathIterator pathIterator = new PathIterator(clientPath); - while (pathIterator.hasNext()) { - addTo(persistentWatches.get(pathIterator.next()), result); + for (String path : new PathIterator(clientPath).asIterable()) { + addTo(persistentWatches.get(path), result); } } } diff --git a/src/java/main/org/apache/zookeeper/server/PathIterator.java b/src/java/main/org/apache/zookeeper/server/PathIterator.java index 9149ade8121..c92000e7a7d 100644 --- a/src/java/main/org/apache/zookeeper/server/PathIterator.java +++ b/src/java/main/org/apache/zookeeper/server/PathIterator.java @@ -18,24 +18,55 @@ package org.apache.zookeeper.server; -public class PathIterator { +import java.util.Iterator; + +/** + * Iterates over a ZooKeeper path. Each iteration goes up one parent path. Thus, the + * effect of the iterator is to iterate over the initial path and then all of its parents. + */ +public class PathIterator implements Iterator { private String path; private int level = -1; + /** + * @param path initial path + */ public PathIterator(String path) { // NOTE: asserts that the path has already been validated this.path = path; } + /** + * Return an Iterable view so that this Iterator can be used in for each + * statements. IMPORTANT: the returned Iterable is single use only + * @return Iterable + */ + public Iterable asIterable() { + return new Iterable() { + @Override + public Iterator iterator() { + return PathIterator.this; + } + }; + } + + @Override public boolean hasNext() { return !path.isEmpty(); } + /** + * Returns true if this iterator is currently at a parent path as opposed + * to the initial path given to the constructor + * + * @return true/false + */ public boolean atParentPath() { return level > 0; } + @Override public String next() { String localPath = path; if (hasNext()) { @@ -51,4 +82,9 @@ public String next() { } return localPath; } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } } diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index bf2080321c9..605f15602a7 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -105,8 +105,7 @@ Set triggerWatch(String path, EventType type, Set supress) { Set watchers = new HashSet<>(); PathIterator pathIterator = new PathIterator(path); synchronized (this) { - while (pathIterator.hasNext()) { - String localPath = pathIterator.next(); + for (String localPath : pathIterator.asIterable()) { Map thisWatchers = watchTable.get(localPath); if (thisWatchers == null || thisWatchers.isEmpty()) { continue; @@ -207,8 +206,7 @@ synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) { */ synchronized boolean containsWatcher(String path, Watcher watcher) { PathIterator pathIterator = new PathIterator(path); - while (pathIterator.hasNext()) { - String localPath = pathIterator.next(); + for (String localPath : pathIterator.asIterable()) { Map watchers = watchTable.get(localPath); Type watcherType = (watchers != null) ? watchers.get(watcher) : null; if ((watcherType == Type.STANDARD) && !pathIterator.atParentPath()) { From 073d693fbcad21e0adde0c6387f5ff5bfd490081 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 18:32:25 -0500 Subject: [PATCH 11/31] fixed bad newline --- src/java/main/org/apache/zookeeper/server/PathIterator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/server/PathIterator.java b/src/java/main/org/apache/zookeeper/server/PathIterator.java index c92000e7a7d..659e5f1d346 100644 --- a/src/java/main/org/apache/zookeeper/server/PathIterator.java +++ b/src/java/main/org/apache/zookeeper/server/PathIterator.java @@ -61,8 +61,7 @@ public boolean hasNext() { * * @return true/false */ - public boolean atParentPath() - { + public boolean atParentPath() { return level > 0; } From 1272e4fa6b7f67eb9eba97d0702d9bd2c1847e62 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 18:35:15 -0500 Subject: [PATCH 12/31] spacing --- src/java/main/org/apache/zookeeper/server/PathIterator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/main/org/apache/zookeeper/server/PathIterator.java b/src/java/main/org/apache/zookeeper/server/PathIterator.java index 659e5f1d346..b38980ec8b4 100644 --- a/src/java/main/org/apache/zookeeper/server/PathIterator.java +++ b/src/java/main/org/apache/zookeeper/server/PathIterator.java @@ -74,7 +74,7 @@ public String next() { path = ""; } else { path = path.substring(0, path.lastIndexOf('/')); - if ( path.length() == 0 ) { + if (path.length() == 0) { path = "/"; } } From 99a00882640ca04b945a1d76cecbbb417165b941 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 22:23:57 -0500 Subject: [PATCH 13/31] fixed a few nits --- .../main/org/apache/zookeeper/server/WatchManager.java | 8 ++++---- .../org/apache/zookeeper/test/PersistentWatcherTest.java | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index 605f15602a7..84fa6064fd1 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -88,7 +88,7 @@ synchronized void removeWatcher(Watcher watcher) { Map list = watchTable.get(p); if (list != null) { list.remove(watcher); - if (list.size() == 0) { + if (list.isEmpty()) { watchTable.remove(p); } } @@ -127,12 +127,12 @@ else if (!pathIterator.atParentPath()) { } } } - if (thisWatchers.size() == 0) { + if (thisWatchers.isEmpty()) { watchTable.remove(localPath); } } } - if (watchers.size() == 0) { + if (watchers.isEmpty()) { if (LOG.isTraceEnabled()) { ZooTrace.logTraceMessage(LOG, ZooTrace.EVENT_DELIVERY_TRACE_MASK, @@ -239,7 +239,7 @@ synchronized boolean removeWatcher(String path, Watcher watcher) { return false; } - if (list.size() == 0) { + if (list.isEmpty()) { watchTable.remove(path); } diff --git a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java index eb741f5acc2..e9f8a8bb8ee 100644 --- a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java +++ b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -83,7 +83,7 @@ public void testBasicAsync() AsyncCallback.VoidCallback cb = new AsyncCallback.VoidCallback() { @Override public void processResult(int rc, String path, Object ctx) { - if ( rc == 0 ) { + if (rc == 0) { latch.countDown(); } } From 84410e1f456a2230ea2410a5ad8526d4a0ceff70 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 26 Dec 2016 22:37:00 -0500 Subject: [PATCH 14/31] Inlcude deletions in persistent watcher tests --- src/java/main/org/apache/zookeeper/server/WatchManager.java | 3 +-- .../org/apache/zookeeper/test/PersistentWatcherTest.java | 6 ++++++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index 84fa6064fd1..67d03cdfa0e 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -117,8 +117,7 @@ Set triggerWatch(String path, EventType type, Set supress) { Watcher watcher = entry.getKey(); if (entryType == Type.PERSISTENT) { watchers.add(watcher); - } - else if (!pathIterator.atParentPath()) { + } else if (!pathIterator.atParentPath()) { watchers.add(watcher); iterator.remove(); Set paths = watch2Paths.get(watcher); diff --git a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java index e9f8a8bb8ee..25113fa1c65 100644 --- a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java +++ b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -105,6 +105,8 @@ private void internalTestBasic(ZooKeeper zk) throws KeeperException, Interrupted zk.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk.setData("/a/b/c/d/e", new byte[0], -1); + zk.delete("/a/b/c/d/e", -1); + zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); @@ -114,6 +116,10 @@ private void internalTestBasic(ZooKeeper zk) throws KeeperException, Interrupted assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c/d"); assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c/d"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c/d"); } @Test From 48d7f646e3fdcf2eeaaaf1bc24005644b0319109 Mon Sep 17 00:00:00 2001 From: randgalt Date: Tue, 27 Dec 2016 17:56:27 -0500 Subject: [PATCH 15/31] Don't trigger NodeChildrenChanged for persistent watches. It's redundant --- .../content/xdocs/zookeeperProgrammers.xml | 8 +++- .../apache/zookeeper/server/WatchManager.java | 4 +- .../zookeeper/test/PersistentWatcherTest.java | 43 ++++++++++++++++--- 3 files changed, 46 insertions(+), 9 deletions(-) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index 9f23e9bffa5..ecdcbd20adf 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -721,8 +721,12 @@ Persistent Recursive Watches New in 3.6.0: There is now a variation on the standard watch described above whereby you can set a watch that does not get removed when triggered. - Additionally, these watches trigger all event types (created, deleted, changed, child) - recursively for all znodes starting at the znode that the watch is registered for. + Additionally, these watches trigger the event types NodeCreated, + NodeDeleted, NodeDataChanged, + and PersistentWatchRemoved recursively for all znodes starting at the + znode that the watch is registered for. Note that NodeChildrenChanged + are not triggered for persistent recursive watches as it is redundant. + Persistent recursive watches are set using the method addPersistentWatch(). The triggering semantics and guarantees diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index 67d03cdfa0e..d9da07f9d06 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -116,7 +116,9 @@ Set triggerWatch(String path, EventType type, Set supress) { Type entryType = entry.getValue(); Watcher watcher = entry.getKey(); if (entryType == Type.PERSISTENT) { - watchers.add(watcher); + if ( type != EventType.NodeChildrenChanged ) { + watchers.add(watcher); + } } else if (!pathIterator.atParentPath()) { watchers.add(watcher); iterator.remove(); diff --git a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java index 25113fa1c65..5ead8f91542 100644 --- a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java +++ b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -32,9 +32,11 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.SynchronousQueue; import java.util.concurrent.TimeUnit; public class PersistentWatcherTest extends ClientBase { @@ -110,16 +112,11 @@ private void internalTestBasic(ZooKeeper zk) throws KeeperException, Interrupted assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); - assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d"); - assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c"); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); - assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c/d"); assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c/d/e"); assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a/b/c/d/e"); - assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c/d"); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); - assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b/c/d"); } @Test @@ -135,7 +132,6 @@ public void testRemoval() zk.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); - assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); zk.removeWatches("/a/b", persistentWatcher, Watcher.WatcherType.Any, false); zk.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -166,6 +162,41 @@ public void testDisconnect() throws Exception { } } + @Test + public void testMultiClient() + throws IOException, InterruptedException, KeeperException { + ZooKeeper zk1 = null; + ZooKeeper zk2 = null; + try { + zk1 = createClient(new CountdownWatcher(), hostPort); + zk2 = createClient(new CountdownWatcher(), hostPort); + + zk1.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk1.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk1.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + zk1.addPersistentWatch("/a/b", persistentWatcher); + zk1.setData("/a/b/c", "one".getBytes(), -1); + Thread.sleep(1000); // give some time for the event to arrive + + zk2.setData("/a/b/c", "two".getBytes(), -1); + zk2.setData("/a/b/c", "three".getBytes(), -1); + zk2.setData("/a/b/c", "four".getBytes(), -1); + + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + } finally { + if (zk1 != null) { + zk1.close(); + } + if (zk2 != null) { + zk2.close(); + } + } + } + private void assertEvent(BlockingQueue events, Watcher.Event.EventType eventType, String path) throws InterruptedException { WatchedEvent event = events.poll(5, TimeUnit.SECONDS); From 8332eb405a5d1fbf08497d3fd0c5e20abfdf35d1 Mon Sep 17 00:00:00 2001 From: randgalt Date: Tue, 27 Dec 2016 18:20:25 -0500 Subject: [PATCH 16/31] PathIterator.next() should throw NoSuchElementException if no next element --- .../apache/zookeeper/server/PathIterator.java | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/server/PathIterator.java b/src/java/main/org/apache/zookeeper/server/PathIterator.java index b38980ec8b4..efb89a3b0fd 100644 --- a/src/java/main/org/apache/zookeeper/server/PathIterator.java +++ b/src/java/main/org/apache/zookeeper/server/PathIterator.java @@ -19,6 +19,7 @@ package org.apache.zookeeper.server; import java.util.Iterator; +import java.util.NoSuchElementException; /** * Iterates over a ZooKeeper path. Each iteration goes up one parent path. Thus, the @@ -67,16 +68,18 @@ public boolean atParentPath() { @Override public String next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + String localPath = path; - if (hasNext()) { - ++level; - if (path.equals("/")) { - path = ""; - } else { - path = path.substring(0, path.lastIndexOf('/')); - if (path.length() == 0) { - path = "/"; - } + ++level; + if (path.equals("/")) { + path = ""; + } else { + path = path.substring(0, path.lastIndexOf('/')); + if (path.length() == 0) { + path = "/"; } } return localPath; From 23756a14c6ab6be73ba2f033d68ce1d434d77386 Mon Sep 17 00:00:00 2001 From: randgalt Date: Tue, 27 Dec 2016 23:33:36 -0500 Subject: [PATCH 17/31] add root watcher test --- .../zookeeper/test/PersistentWatcherTest.java | 25 +++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java index 5ead8f91542..23196f74843 100644 --- a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java +++ b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -32,11 +32,9 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.SynchronousQueue; import java.util.concurrent.TimeUnit; public class PersistentWatcherTest extends ClientBase { @@ -197,6 +195,29 @@ public void testMultiClient() } } + @Test + public void testRootWatcher() + throws IOException, InterruptedException, KeeperException { + ZooKeeper zk = null; + try { + zk = createClient(new CountdownWatcher(), hostPort); + + zk.addPersistentWatch("/", persistentWatcher); + 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("/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b/c"); + } finally { + if (zk != null) { + zk.close(); + } + } + } + private void assertEvent(BlockingQueue events, Watcher.Event.EventType eventType, String path) throws InterruptedException { WatchedEvent event = events.poll(5, TimeUnit.SECONDS); From 692db2d7628f504e68707e5bdd4a6cce649f4491 Mon Sep 17 00:00:00 2001 From: randgalt Date: Wed, 28 Dec 2016 00:31:43 -0500 Subject: [PATCH 18/31] doc updates --- .../documentation/content/xdocs/zookeeperProgrammers.xml | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index ecdcbd20adf..8c746ac2655 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -722,11 +722,10 @@ New in 3.6.0: There is now a variation on the standard watch described above whereby you can set a watch that does not get removed when triggered. Additionally, these watches trigger the event types NodeCreated, - NodeDeleted, NodeDataChanged, - and PersistentWatchRemoved recursively for all znodes starting at the - znode that the watch is registered for. Note that NodeChildrenChanged - are not triggered for persistent recursive watches as it is redundant. - + NodeDeleted, and NodeDataChanged + recursively for all znodes starting at the znode that the watch is registered for. Note that + NodeChildrenChanged are not triggered for persistent recursive watches as + it is redundant. Persistent recursive watches are set using the method addPersistentWatch(). The triggering semantics and guarantees From 29d3c6cd9b6d5c29e1d20147c18331510019df08 Mon Sep 17 00:00:00 2001 From: randgalt Date: Wed, 28 Dec 2016 02:24:03 -0500 Subject: [PATCH 19/31] more doc --- .../src/documentation/content/xdocs/zookeeperProgrammers.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index 8c746ac2655..e999d02651c 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -752,6 +752,11 @@ Data Remove event: Watcher which was added with a call to exists or getData. + + + Persistent Remove event: + Watcher which was added with a call to add a persistent watch. +
From 61c2211d0781d8177bb5a451f7aa630ee1923c1c Mon Sep 17 00:00:00 2001 From: randgalt Date: Wed, 28 Dec 2016 02:25:23 -0500 Subject: [PATCH 20/31] fixed misspelling --- .../src/documentation/content/xdocs/zookeeperProgrammers.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index e999d02651c..3e8ca7554c2 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -729,7 +729,7 @@ Persistent recursive watches are set using the method addPersistentWatch(). The triggering semantics and guarantees - (other then one-time triggering) are the same as standard watches. Persistent recursive + (other than one-time triggering) are the same as standard watches. Persistent recursive watches are removed using removeWatches() with watcher type WatcherType.Any. From 335436910d1a549fdd3402d9e881491e481edb4c Mon Sep 17 00:00:00 2001 From: randgalt Date: Wed, 28 Dec 2016 12:29:38 -0500 Subject: [PATCH 21/31] doc clarification --- .../src/documentation/content/xdocs/zookeeperProgrammers.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index 3e8ca7554c2..13ae8ba682e 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -729,8 +729,9 @@ Persistent recursive watches are set using the method addPersistentWatch(). The triggering semantics and guarantees - (other than one-time triggering) are the same as standard watches. Persistent recursive - watches are removed using removeWatches() with watcher type + (other than one-time triggering) are the same as standard watches. The only exception regarding events is that + persistent watchers never throw child changed events as they are redundant. + Persistent recursive watches are removed using removeWatches() with watcher type WatcherType.Any. From 088dfb912fb393e4baf6dab23c7008b5a1139d4b Mon Sep 17 00:00:00 2001 From: randgalt Date: Wed, 9 Aug 2017 13:12:07 -0500 Subject: [PATCH 22/31] Renamed PathIterator to PathParentIterator for better clarity --- .../main/org/apache/zookeeper/ZooKeeper.java | 4 +- ...hIterator.java => PathParentIterator.java} | 6 +- .../apache/zookeeper/server/WatchManager.java | 12 +-- .../zookeeper/test/PathIteratorTest.java | 75 ------------------- .../test/PathParentIteratorTest.java | 75 +++++++++++++++++++ 5 files changed, 86 insertions(+), 86 deletions(-) rename src/java/main/org/apache/zookeeper/server/{PathIterator.java => PathParentIterator.java} (94%) delete mode 100644 src/java/test/org/apache/zookeeper/test/PathIteratorTest.java create mode 100644 src/java/test/org/apache/zookeeper/test/PathParentIteratorTest.java diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index 9b54cd53ab0..33373eefc62 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -70,7 +70,7 @@ import org.apache.zookeeper.proto.SyncResponse; import org.apache.zookeeper.server.DataTree; import org.apache.zookeeper.server.EphemeralType; -import org.apache.zookeeper.server.PathIterator; +import org.apache.zookeeper.server.PathParentIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -579,7 +579,7 @@ public Set materialize(Watcher.Event.KeeperState state, private void addPersistentWatches(String clientPath, Set result) { synchronized (persistentWatches) { - for (String path : new PathIterator(clientPath).asIterable()) { + for (String path : new PathParentIterator(clientPath).asIterable()) { addTo(persistentWatches.get(path), result); } } diff --git a/src/java/main/org/apache/zookeeper/server/PathIterator.java b/src/java/main/org/apache/zookeeper/server/PathParentIterator.java similarity index 94% rename from src/java/main/org/apache/zookeeper/server/PathIterator.java rename to src/java/main/org/apache/zookeeper/server/PathParentIterator.java index efb89a3b0fd..2f117886eb4 100644 --- a/src/java/main/org/apache/zookeeper/server/PathIterator.java +++ b/src/java/main/org/apache/zookeeper/server/PathParentIterator.java @@ -25,14 +25,14 @@ * Iterates over a ZooKeeper path. Each iteration goes up one parent path. Thus, the * effect of the iterator is to iterate over the initial path and then all of its parents. */ -public class PathIterator implements Iterator { +public class PathParentIterator implements Iterator { private String path; private int level = -1; /** * @param path initial path */ - public PathIterator(String path) { + public PathParentIterator(String path) { // NOTE: asserts that the path has already been validated this.path = path; } @@ -46,7 +46,7 @@ public Iterable asIterable() { return new Iterable() { @Override public Iterator iterator() { - return PathIterator.this; + return PathParentIterator.this; } }; } diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index d9da07f9d06..5b880294a41 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -103,9 +103,9 @@ Set triggerWatch(String path, EventType type, Set supress) { WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path); Set watchers = new HashSet<>(); - PathIterator pathIterator = new PathIterator(path); + PathParentIterator pathParentIterator = new PathParentIterator(path); synchronized (this) { - for (String localPath : pathIterator.asIterable()) { + for (String localPath : pathParentIterator.asIterable()) { Map thisWatchers = watchTable.get(localPath); if (thisWatchers == null || thisWatchers.isEmpty()) { continue; @@ -119,7 +119,7 @@ Set triggerWatch(String path, EventType type, Set supress) { if ( type != EventType.NodeChildrenChanged ) { watchers.add(watcher); } - } else if (!pathIterator.atParentPath()) { + } else if (!pathParentIterator.atParentPath()) { watchers.add(watcher); iterator.remove(); Set paths = watch2Paths.get(watcher); @@ -206,11 +206,11 @@ synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) { * @return true if the watcher exists, false otherwise */ synchronized boolean containsWatcher(String path, Watcher watcher) { - PathIterator pathIterator = new PathIterator(path); - for (String localPath : pathIterator.asIterable()) { + PathParentIterator pathParentIterator = new PathParentIterator(path); + for (String localPath : pathParentIterator.asIterable()) { Map watchers = watchTable.get(localPath); Type watcherType = (watchers != null) ? watchers.get(watcher) : null; - if ((watcherType == Type.STANDARD) && !pathIterator.atParentPath()) { + if ((watcherType == Type.STANDARD) && !pathParentIterator.atParentPath()) { return true; } if (watcherType == Type.PERSISTENT) { diff --git a/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java b/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java deleted file mode 100644 index c9c986536d5..00000000000 --- a/src/java/test/org/apache/zookeeper/test/PathIteratorTest.java +++ /dev/null @@ -1,75 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.zookeeper.test; - -import org.apache.zookeeper.server.PathIterator; -import org.junit.Assert; -import org.junit.Test; - -public class PathIteratorTest { - @Test - public void testRoot() { - PathIterator pathIterator = new PathIterator("/"); - Assert.assertTrue(pathIterator.hasNext()); - Assert.assertFalse(pathIterator.atParentPath()); - Assert.assertEquals(pathIterator.next(), "/"); - Assert.assertFalse(pathIterator.hasNext()); - } - - @Test - public void test1Level() { - PathIterator pathIterator = new PathIterator("/a"); - Assert.assertTrue(pathIterator.hasNext()); - Assert.assertFalse(pathIterator.atParentPath()); - Assert.assertEquals(pathIterator.next(), "/a"); - - Assert.assertTrue(pathIterator.hasNext()); - Assert.assertEquals(pathIterator.next(), "/"); - Assert.assertTrue(pathIterator.atParentPath()); - - Assert.assertFalse(pathIterator.hasNext()); - } - - @Test - public void testLong() { - PathIterator pathIterator = new PathIterator("/a/b/c/d"); - - Assert.assertTrue(pathIterator.hasNext()); - Assert.assertEquals(pathIterator.next(), "/a/b/c/d"); - Assert.assertFalse(pathIterator.atParentPath()); - - Assert.assertTrue(pathIterator.hasNext()); - Assert.assertEquals(pathIterator.next(), "/a/b/c"); - Assert.assertTrue(pathIterator.atParentPath()); - - Assert.assertTrue(pathIterator.hasNext()); - Assert.assertEquals(pathIterator.next(), "/a/b"); - Assert.assertTrue(pathIterator.atParentPath()); - - Assert.assertTrue(pathIterator.hasNext()); - Assert.assertEquals(pathIterator.next(), "/a"); - Assert.assertTrue(pathIterator.atParentPath()); - - Assert.assertTrue(pathIterator.hasNext()); - Assert.assertEquals(pathIterator.next(), "/"); - Assert.assertTrue(pathIterator.atParentPath()); - - Assert.assertFalse(pathIterator.hasNext()); - } -} diff --git a/src/java/test/org/apache/zookeeper/test/PathParentIteratorTest.java b/src/java/test/org/apache/zookeeper/test/PathParentIteratorTest.java new file mode 100644 index 00000000000..438fb1e1f0b --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/PathParentIteratorTest.java @@ -0,0 +1,75 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.test; + +import org.apache.zookeeper.server.PathParentIterator; +import org.junit.Assert; +import org.junit.Test; + +public class PathParentIteratorTest { + @Test + public void testRoot() { + PathParentIterator pathParentIterator = new PathParentIterator("/"); + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertFalse(pathParentIterator.atParentPath()); + Assert.assertEquals(pathParentIterator.next(), "/"); + Assert.assertFalse(pathParentIterator.hasNext()); + } + + @Test + public void test1Level() { + PathParentIterator pathParentIterator = new PathParentIterator("/a"); + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertFalse(pathParentIterator.atParentPath()); + Assert.assertEquals(pathParentIterator.next(), "/a"); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertFalse(pathParentIterator.hasNext()); + } + + @Test + public void testLong() { + PathParentIterator pathParentIterator = new PathParentIterator("/a/b/c/d"); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a/b/c/d"); + Assert.assertFalse(pathParentIterator.atParentPath()); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a/b/c"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a/b"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertFalse(pathParentIterator.hasNext()); + } +} From 9a11ee5b979ac8e38f0b4122672410f744386768 Mon Sep 17 00:00:00 2001 From: randgalt Date: Wed, 9 Aug 2017 13:41:55 -0500 Subject: [PATCH 23/31] Optimization: don't iterate through to parent paths unless the system has seen a persistent watcher. i.e. the performance is exactly the same for those not using the new feature --- .../main/org/apache/zookeeper/ZooKeeper.java | 2 +- .../zookeeper/server/PathParentIterator.java | 17 +++++++++++++++-- .../apache/zookeeper/server/WatchManager.java | 18 ++++++++++++++++-- .../zookeeper/test/PathParentIteratorTest.java | 16 +++++++++++++--- 4 files changed, 45 insertions(+), 8 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index 33373eefc62..6e0014c77a1 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -579,7 +579,7 @@ public Set materialize(Watcher.Event.KeeperState state, private void addPersistentWatches(String clientPath, Set result) { synchronized (persistentWatches) { - for (String path : new PathParentIterator(clientPath).asIterable()) { + for (String path : PathParentIterator.forAll(clientPath).asIterable()) { addTo(persistentWatches.get(path), result); } } diff --git a/src/java/main/org/apache/zookeeper/server/PathParentIterator.java b/src/java/main/org/apache/zookeeper/server/PathParentIterator.java index 2f117886eb4..4dd489c0983 100644 --- a/src/java/main/org/apache/zookeeper/server/PathParentIterator.java +++ b/src/java/main/org/apache/zookeeper/server/PathParentIterator.java @@ -27,14 +27,27 @@ */ public class PathParentIterator implements Iterator { private String path; + private final int maxLevel; private int level = -1; /** * @param path initial path */ - public PathParentIterator(String path) { + public static PathParentIterator forAll(String path) { + return new PathParentIterator(path, Integer.MAX_VALUE); + } + + /** + * @param path initial path + */ + public static PathParentIterator forPathOnly(String path) { + return new PathParentIterator(path, 0); + } + + private PathParentIterator(String path, int maxLevel) { // NOTE: asserts that the path has already been validated this.path = path; + this.maxLevel = maxLevel; } /** @@ -53,7 +66,7 @@ public Iterator iterator() { @Override public boolean hasNext() { - return !path.isEmpty(); + return !path.isEmpty() && (level < maxLevel); } /** diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index 5b880294a41..159bf59a726 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -51,6 +51,8 @@ private enum Type { private final Map> watch2Paths = new HashMap<>(); + private boolean hasHadPersistentWatches = false; // guarded by sync + synchronized int size(){ int result = 0; for(Map watches : watchTable.values()) { @@ -77,6 +79,10 @@ synchronized void addWatch(String path, Watcher watcher, boolean persistent) { watch2Paths.put(watcher, paths); } paths.add(path); + + if (persistent) { + hasHadPersistentWatches = true; + } } synchronized void removeWatcher(Watcher watcher) { @@ -103,8 +109,8 @@ Set triggerWatch(String path, EventType type, Set supress) { WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path); Set watchers = new HashSet<>(); - PathParentIterator pathParentIterator = new PathParentIterator(path); synchronized (this) { + PathParentIterator pathParentIterator = getPathParentIterator(path); for (String localPath : pathParentIterator.asIterable()) { Map thisWatchers = watchTable.get(localPath); if (thisWatchers == null || thisWatchers.isEmpty()) { @@ -206,7 +212,7 @@ synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) { * @return true if the watcher exists, false otherwise */ synchronized boolean containsWatcher(String path, Watcher watcher) { - PathParentIterator pathParentIterator = new PathParentIterator(path); + PathParentIterator pathParentIterator = getPathParentIterator(path); for (String localPath : pathParentIterator.asIterable()) { Map watchers = watchTable.get(localPath); Type watcherType = (watchers != null) ? watchers.get(watcher) : null; @@ -296,6 +302,14 @@ synchronized WatchesSummary getWatchesSummary() { totalWatches); } + private PathParentIterator getPathParentIterator(String path) { + if ( !hasHadPersistentWatches ) { + // optimization - if we've never seen a Persistent Watch, there's no need to iterate through parent nodes + return PathParentIterator.forPathOnly(path); + } + return PathParentIterator.forAll(path); + } + private String getParent(String path) { return path.substring(0, path.lastIndexOf('/')); } diff --git a/src/java/test/org/apache/zookeeper/test/PathParentIteratorTest.java b/src/java/test/org/apache/zookeeper/test/PathParentIteratorTest.java index 438fb1e1f0b..6f551d171a6 100644 --- a/src/java/test/org/apache/zookeeper/test/PathParentIteratorTest.java +++ b/src/java/test/org/apache/zookeeper/test/PathParentIteratorTest.java @@ -25,7 +25,7 @@ public class PathParentIteratorTest { @Test public void testRoot() { - PathParentIterator pathParentIterator = new PathParentIterator("/"); + PathParentIterator pathParentIterator = PathParentIterator.forAll("/"); Assert.assertTrue(pathParentIterator.hasNext()); Assert.assertFalse(pathParentIterator.atParentPath()); Assert.assertEquals(pathParentIterator.next(), "/"); @@ -34,7 +34,7 @@ public void testRoot() { @Test public void test1Level() { - PathParentIterator pathParentIterator = new PathParentIterator("/a"); + PathParentIterator pathParentIterator = PathParentIterator.forAll("/a"); Assert.assertTrue(pathParentIterator.hasNext()); Assert.assertFalse(pathParentIterator.atParentPath()); Assert.assertEquals(pathParentIterator.next(), "/a"); @@ -48,7 +48,7 @@ public void test1Level() { @Test public void testLong() { - PathParentIterator pathParentIterator = new PathParentIterator("/a/b/c/d"); + PathParentIterator pathParentIterator = PathParentIterator.forAll("/a/b/c/d"); Assert.assertTrue(pathParentIterator.hasNext()); Assert.assertEquals(pathParentIterator.next(), "/a/b/c/d"); @@ -72,4 +72,14 @@ public void testLong() { Assert.assertFalse(pathParentIterator.hasNext()); } + + @Test + public void testForPathOnly() { + PathParentIterator pathParentIterator = PathParentIterator.forPathOnly("/a/b/c/d"); + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a/b/c/d"); + Assert.assertFalse(pathParentIterator.atParentPath()); + + Assert.assertFalse(pathParentIterator.hasNext()); + } } From 51ef9be62e765e96f0b4c51cbf9b261bd92b4e5a Mon Sep 17 00:00:00 2001 From: randgalt Date: Wed, 9 Aug 2017 13:44:37 -0500 Subject: [PATCH 24/31] Use try-with-resources for cleaner code --- .../zookeeper/test/PersistentWatcherTest.java | 45 +++---------------- 1 file changed, 5 insertions(+), 40 deletions(-) diff --git a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java index 23196f74843..aceb42f1c64 100644 --- a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java +++ b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -59,26 +59,16 @@ public void process(WatchedEvent event) { @Test public void testBasic() throws IOException, InterruptedException, KeeperException { - ZooKeeper zk = null; - try { - zk = createClient(new CountdownWatcher(), hostPort); - + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { zk.addPersistentWatch("/a/b", persistentWatcher); internalTestBasic(zk); - } finally { - if (zk != null) { - zk.close(); - } } } @Test public void testBasicAsync() throws IOException, InterruptedException, KeeperException { - ZooKeeper zk = null; - try { - zk = createClient(new CountdownWatcher(), hostPort); - + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { final CountDownLatch latch = new CountDownLatch(1); AsyncCallback.VoidCallback cb = new AsyncCallback.VoidCallback() { @Override @@ -91,10 +81,6 @@ public void processResult(int rc, String path, Object ctx) { zk.addPersistentWatch("/a/b", persistentWatcher, cb, null); Assert.assertTrue(latch.await(5, TimeUnit.SECONDS)); internalTestBasic(zk); - } finally { - if (zk != null) { - zk.close(); - } } } @@ -120,10 +106,7 @@ private void internalTestBasic(ZooKeeper zk) throws KeeperException, Interrupted @Test public void testRemoval() throws IOException, InterruptedException, KeeperException { - ZooKeeper zk = null; - try { - zk = createClient(new CountdownWatcher(), hostPort); - + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { zk.addPersistentWatch("/a/b", persistentWatcher); 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); @@ -134,29 +117,18 @@ public void testRemoval() zk.removeWatches("/a/b", persistentWatcher, Watcher.WatcherType.Any, false); zk.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); assertEvent(events, Watcher.Event.EventType.PersistentWatchRemoved, "/a/b"); - } finally { - if (zk != null) { - zk.close(); - } } } @Test public void testDisconnect() throws Exception { - ZooKeeper zk = null; - try { - zk = createClient(new CountdownWatcher(), hostPort); - + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { zk.addPersistentWatch("/a/b", persistentWatcher); stopServer(); assertEvent(events, Watcher.Event.EventType.None, null); startServer(); assertEvent(events, Watcher.Event.EventType.None, null); internalTestBasic(zk); - } finally { - if (zk != null) { - zk.close(); - } } } @@ -198,10 +170,7 @@ public void testMultiClient() @Test public void testRootWatcher() throws IOException, InterruptedException, KeeperException { - ZooKeeper zk = null; - try { - zk = createClient(new CountdownWatcher(), hostPort); - + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { zk.addPersistentWatch("/", persistentWatcher); 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); @@ -211,10 +180,6 @@ public void testRootWatcher() assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b"); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b/c"); - } finally { - if (zk != null) { - zk.close(); - } } } From 4c9f24b64b2bec6de21987bb7f1eab9c449f2c00 Mon Sep 17 00:00:00 2001 From: randgalt Date: Wed, 9 Aug 2017 13:48:25 -0500 Subject: [PATCH 25/31] added javadoc --- .../org/apache/zookeeper/server/PathParentIterator.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/java/main/org/apache/zookeeper/server/PathParentIterator.java b/src/java/main/org/apache/zookeeper/server/PathParentIterator.java index 4dd489c0983..d1aaf8cfb6f 100644 --- a/src/java/main/org/apache/zookeeper/server/PathParentIterator.java +++ b/src/java/main/org/apache/zookeeper/server/PathParentIterator.java @@ -31,6 +31,9 @@ public class PathParentIterator implements Iterator { private int level = -1; /** + * Return a new PathParentIterator that iterates from the + * given path to all parents. + * * @param path initial path */ public static PathParentIterator forAll(String path) { @@ -38,6 +41,9 @@ public static PathParentIterator forAll(String path) { } /** + * Return a new PathParentIterator that only returns the given path - i.e. + * does not iterate to parent paths. + * * @param path initial path */ public static PathParentIterator forPathOnly(String path) { From 072df539976f02133abd40906aa694af7e756102 Mon Sep 17 00:00:00 2001 From: randgalt Date: Sun, 20 Aug 2017 12:20:46 +0200 Subject: [PATCH 26/31] 1. made the recursion feature of persistent watches optional. i.e. you can now have a persistent watch or a persistent recursive watch. 2. Much better optimization for recursive behavior. The small PathParentIterator performance penalty only occurs if there are recursive watches registered. i.e. a count of recursive watches is kept and the PathParentIterator behavior only occurs if that count is not zero --- .../main/org/apache/zookeeper/ClientCnxn.java | 14 +- .../main/org/apache/zookeeper/ZooKeeper.java | 87 ++++++-- .../org/apache/zookeeper/server/DataTree.java | 31 +-- .../server/FinalRequestProcessor.java | 7 +- .../apache/zookeeper/server/WatchManager.java | 96 +++++++-- .../apache/zookeeper/server/ZKDatabase.java | 13 +- .../server/RecursiveWatchQtyTest.java | 88 ++++++++ .../test/PersistentRecursiveWatcherTest.java | 193 ++++++++++++++++++ .../zookeeper/test/PersistentWatcherTest.java | 63 +++--- src/zookeeper.jute | 2 + 10 files changed, 501 insertions(+), 93 deletions(-) create mode 100644 src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java create mode 100644 src/java/test/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java diff --git a/src/java/main/org/apache/zookeeper/ClientCnxn.java b/src/java/main/org/apache/zookeeper/ClientCnxn.java index cae072c52a5..c1eab07b951 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxn.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxn.java @@ -960,21 +960,25 @@ void primeConnection() throws IOException { List existWatches = zooKeeper.getExistWatches(); List childWatches = zooKeeper.getChildWatches(); List persistentWatches = zooKeeper.getPersistentWatches(); + List persistentRecursiveWatches = zooKeeper.getPersistentRecursiveWatches(); if (!dataWatches.isEmpty() - || !existWatches.isEmpty() || !childWatches.isEmpty() || !persistentWatches.isEmpty()) { + || !existWatches.isEmpty() || !childWatches.isEmpty() + || !persistentWatches.isEmpty() || !persistentRecursiveWatches.isEmpty()) { Iterator dataWatchesIter = prependChroot(dataWatches).iterator(); Iterator existWatchesIter = prependChroot(existWatches).iterator(); Iterator childWatchesIter = prependChroot(childWatches).iterator(); Iterator persistentWatchesIter = prependChroot(persistentWatches).iterator(); + Iterator persistentRecursiveWatchesIter = prependChroot(persistentRecursiveWatches).iterator(); long setWatchesLastZxid = lastZxid; while (dataWatchesIter.hasNext() || existWatchesIter.hasNext() || childWatchesIter.hasNext() - || persistentWatchesIter.hasNext()) { + || persistentWatchesIter.hasNext() || persistentRecursiveWatchesIter.hasNext()) { List dataWatchesBatch = new ArrayList(); List existWatchesBatch = new ArrayList(); List childWatchesBatch = new ArrayList(); List persistentWatchesBatch = new ArrayList(); + List persistentRecursiveWatchesBatch = new ArrayList(); int batchLength = 0; // Note, we may exceed our max length by a bit when we add the last @@ -993,6 +997,9 @@ void primeConnection() throws IOException { } else if (persistentWatchesIter.hasNext()) { watch = persistentWatchesIter.next(); persistentWatchesBatch.add(watch); + } else if (persistentRecursiveWatchesIter.hasNext()) { + watch = persistentRecursiveWatchesIter.next(); + persistentRecursiveWatchesBatch.add(watch); } else { break; } @@ -1003,7 +1010,8 @@ void primeConnection() throws IOException { dataWatchesBatch, existWatchesBatch, childWatchesBatch, - persistentWatchesBatch); + persistentWatchesBatch, + persistentRecursiveWatchesBatch); RequestHeader header = new RequestHeader(-8, OpCode.setWatches2); Packet packet = new Packet(header, new ReplyHeader(), sw, null, null); outgoingQueue.addFirst(packet); diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index 6e0014c77a1..2455662ee69 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -258,6 +258,12 @@ protected List getPersistentWatches() { return rc; } } + protected List getPersistentRecursiveWatches() { + synchronized(watchManager.persistentRecursiveWatches) { + List rc = new ArrayList(watchManager.persistentRecursiveWatches.keySet()); + return rc; + } + } /** * Manage watchers & handle events generated by the ClientCnxn object. @@ -275,6 +281,8 @@ static class ZKWatchManager implements ClientWatchManager { new HashMap>(); private final Map> persistentWatches = new HashMap>(); + private final Map> persistentRecursiveWatches = + new HashMap>(); private boolean disableAutoWatchReset; ZKWatchManager(boolean disableAutoWatchReset) { @@ -298,8 +306,7 @@ public Map> removeWatcher(String clientPath, Map> removedWatchers = new HashMap>(); HashSet childWatchersToRem = new HashSet(); - removedWatchers - .put(EventType.ChildWatchRemoved, childWatchersToRem); + removedWatchers.put(EventType.ChildWatchRemoved, childWatchersToRem); HashSet dataWatchersToRem = new HashSet(); removedWatchers.put(EventType.DataWatchRemoved, dataWatchersToRem); HashSet persistentWatchersToRem = new HashSet(); @@ -343,9 +350,14 @@ public Map> removeWatcher(String clientPath, removedWatcher |= removedDataWatcher; } synchronized (persistentWatches) { - boolean removedPersistenWatcher = removeWatches(persistentWatches, + boolean removedPersistentWatcher = removeWatches(persistentWatches, watcher, clientPath, local, rc, persistentWatchersToRem); - removedWatcher |= removedPersistenWatcher; + removedWatcher |= removedPersistentWatcher; + } + synchronized (persistentRecursiveWatches) { + boolean removedPersistentRecursiveWatcher = removeWatches(persistentRecursiveWatches, + watcher, clientPath, local, rc, persistentWatchersToRem); + removedWatcher |= removedPersistentRecursiveWatcher; } } } @@ -400,6 +412,12 @@ void containsWatcher(String path, Watcher watcher, persistentWatches); containsWatcher |= contains_temp; } + + synchronized (persistentRecursiveWatches) { + boolean contains_temp = contains(path, watcher, + persistentRecursiveWatches); + containsWatcher |= contains_temp; + } break; } case Data: { @@ -418,6 +436,12 @@ void containsWatcher(String path, Watcher watcher, persistentWatches); containsWatcher |= contains_temp; } + + synchronized (persistentRecursiveWatches) { + boolean contains_temp = contains(path, watcher, + persistentRecursiveWatches); + containsWatcher |= contains_temp; + } break; } case Any: { @@ -439,6 +463,11 @@ void containsWatcher(String path, Watcher watcher, persistentWatches); containsWatcher |= contains_temp; } + synchronized (persistentRecursiveWatches) { + boolean contains_temp = contains(path, watcher, + persistentRecursiveWatches); + containsWatcher |= contains_temp; + } } } // Watcher function doesn't exists for the specified params @@ -533,6 +562,12 @@ public Set materialize(Watcher.Event.KeeperState state, } } + synchronized(persistentRecursiveWatches) { + for(Set ws: persistentRecursiveWatches.values()) { + result.addAll(ws); + } + } + return result; case NodeDataChanged: case NodeCreated: @@ -579,8 +614,11 @@ public Set materialize(Watcher.Event.KeeperState state, private void addPersistentWatches(String clientPath, Set result) { synchronized (persistentWatches) { + addTo(persistentWatches.get(clientPath), result); + } + synchronized (persistentRecursiveWatches) { for (String path : PathParentIterator.forAll(clientPath).asIterable()) { - addTo(persistentWatches.get(path), result); + addTo(persistentRecursiveWatches.get(path), result); } } } @@ -671,13 +709,16 @@ protected Map> getWatches(int rc) { } class PersistentWatchRegistration extends WatchRegistration { - public PersistentWatchRegistration(Watcher watcher, String clientPath) { + private final boolean recursive; + + public PersistentWatchRegistration(Watcher watcher, String clientPath, boolean recursive) { super(watcher, clientPath); + this.recursive = recursive; } @Override protected Map> getWatches(int rc) { - return watchManager.persistentWatches; + return recursive ? watchManager.persistentRecursiveWatches : watchManager.persistentWatches; } @Override @@ -2753,26 +2794,41 @@ public void sync(final String path, VoidCallback cb, Object ctx){ } /** + *

* Set a watcher on the given path that: a) does not get removed when triggered (i.e. it stays active - * until it is removed); b) applies not only to the registered path but all child paths recursively. This watcher + * until it is removed); b) optionally applies not only to the registered path but all child paths recursively. This watcher * is triggered for both data and child events. To remove the watcher, use * removeWatches() with WatcherType.Any + *

+ * + *

+ * If recursive is false, the watcher behaves as if you placed an exists() watch and + * a getData() watch on the ZNode at the given path. + *

+ * + *

+ * If recursive is true, the watcher behaves as if you placed an exists() watch and + * a getData() watch on the ZNode at the given path and any ZNodes that are children + * of the given path including children added later. + *

+ * * @param basePath the top path that the watcher applies to * @param watcher the watcher + * @param recursive if true applies not only to the registered path but all child paths recursively * @throws InterruptedException If the server transaction is interrupted. * @throws KeeperException If the server signals an error with a non-zero * error code. */ - public void addPersistentWatch(String basePath, Watcher watcher) + public void addPersistentWatch(String basePath, Watcher watcher, boolean recursive) throws KeeperException, InterruptedException { PathUtils.validatePath(basePath); String serverPath = prependChroot(basePath); RequestHeader h = new RequestHeader(); h.setType(ZooDefs.OpCode.addPersistentWatch); - AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath); + AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath, recursive); ReplyHeader r = cnxn.submitRequest(h, request, new ErrorResponse(), - new PersistentWatchRegistration(watcher, basePath)); + new PersistentWatchRegistration(watcher, basePath, recursive)); if (r.getErr() != 0) { throw KeeperException.create(KeeperException.Code.get(r.getErr()), basePath); @@ -2780,24 +2836,25 @@ public void addPersistentWatch(String basePath, Watcher watcher) } /** - * Async version of {@link #addPersistentWatch(String, Watcher)} (see it for details) + * Async version of {@link #addPersistentWatch(String, Watcher, boolean)} (see it for details) * * @param basePath the top path that the watcher applies to * @param watcher the watcher + * @param recursive if true applies not only to the registered path but all child paths recursively * @param cb a handler for the callback * @param ctx context to be provided to the callback * @throws IllegalArgumentException if an invalid path is specified */ - public void addPersistentWatch(String basePath, Watcher watcher, + public void addPersistentWatch(String basePath, Watcher watcher, boolean recursive, VoidCallback cb, Object ctx) { PathUtils.validatePath(basePath); String serverPath = prependChroot(basePath); RequestHeader h = new RequestHeader(); h.setType(ZooDefs.OpCode.addPersistentWatch); - AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath); + AddPersistentWatcherRequest request = new AddPersistentWatcherRequest(serverPath, recursive); cnxn.queuePacket(h, new ReplyHeader(), request, new ErrorResponse(), cb, - basePath, serverPath, ctx, new PersistentWatchRegistration(watcher, basePath)); + basePath, serverPath, ctx, new PersistentWatchRegistration(watcher, basePath, recursive)); } /** diff --git a/src/java/main/org/apache/zookeeper/server/DataTree.java b/src/java/main/org/apache/zookeeper/server/DataTree.java index fa4745e512c..67448cb2f65 100644 --- a/src/java/main/org/apache/zookeeper/server/DataTree.java +++ b/src/java/main/org/apache/zookeeper/server/DataTree.java @@ -633,9 +633,10 @@ public String getMaxPrefixWithQuota(String path) { } } - public void addPersistentWatch(String basePath, Watcher watcher) { - dataWatches.addWatch(basePath, watcher, true); - childWatches.addWatch(basePath, watcher, true); + public void addPersistentWatch(String basePath, Watcher watcher, boolean recursive) { + WatchManager.Type type = recursive ? WatchManager.Type.PERSISTENT_RECURSIVE : WatchManager.Type.PERSISTENT; + dataWatches.addWatch(basePath, watcher, type); + childWatches.addWatch(basePath, watcher, type); } public byte[] getData(String path, Stat stat, Watcher watcher) @@ -647,7 +648,7 @@ public byte[] getData(String path, Stat stat, Watcher watcher) synchronized (n) { n.copyStat(stat); if (watcher != null) { - dataWatches.addWatch(path, watcher, false); + dataWatches.addWatch(path, watcher, WatchManager.Type.STANDARD); } return n.data; } @@ -658,7 +659,7 @@ public Stat statNode(String path, Watcher watcher) Stat stat = new Stat(); DataNode n = nodes.get(path); if (watcher != null) { - dataWatches.addWatch(path, watcher, false); + dataWatches.addWatch(path, watcher, WatchManager.Type.STANDARD); } if (n == null) { throw new KeeperException.NoNodeException(); @@ -682,7 +683,7 @@ public List getChildren(String path, Stat stat, Watcher watcher) List children=new ArrayList(n.getChildren()); if (watcher != null) { - childWatches.addWatch(path, watcher, false); + childWatches.addWatch(path, watcher, WatchManager.Type.STANDARD); } return children; } @@ -1326,12 +1327,12 @@ public void setWatches(long relativeZxid, List dataWatches, List existWatches, List childWatches, Watcher watcher) { setWatches(relativeZxid, dataWatches, existWatches, childWatches, - Collections.emptyList(), watcher); + Collections.emptyList(), Collections.emptyList(), watcher); } public void setWatches(long relativeZxid, List dataWatches, List existWatches, List childWatches, List persistentWatches, - Watcher watcher) { + List persistentRecursiveWatches, Watcher watcher) { for (String path : dataWatches) { DataNode node = getNode(path); WatchedEvent e = null; @@ -1342,7 +1343,7 @@ public void setWatches(long relativeZxid, List dataWatches, watcher.process(new WatchedEvent(EventType.NodeDataChanged, KeeperState.SyncConnected, path)); } else { - this.dataWatches.addWatch(path, watcher, false); + this.dataWatches.addWatch(path, watcher, WatchManager.Type.STANDARD); } } for (String path : existWatches) { @@ -1351,7 +1352,7 @@ public void setWatches(long relativeZxid, List dataWatches, watcher.process(new WatchedEvent(EventType.NodeCreated, KeeperState.SyncConnected, path)); } else { - this.dataWatches.addWatch(path, watcher, false); + this.dataWatches.addWatch(path, watcher, WatchManager.Type.STANDARD); } } for (String path : childWatches) { @@ -1363,12 +1364,16 @@ public void setWatches(long relativeZxid, List dataWatches, watcher.process(new WatchedEvent(EventType.NodeChildrenChanged, KeeperState.SyncConnected, path)); } else { - this.childWatches.addWatch(path, watcher, false); + this.childWatches.addWatch(path, watcher, WatchManager.Type.STANDARD); } } for (String path : persistentWatches) { - this.childWatches.addWatch(path, watcher, true); - this.dataWatches.addWatch(path, watcher, true); + this.childWatches.addWatch(path, watcher, WatchManager.Type.PERSISTENT); + this.dataWatches.addWatch(path, watcher, WatchManager.Type.PERSISTENT); + } + for (String path : persistentRecursiveWatches) { + this.childWatches.addWatch(path, watcher, WatchManager.Type.PERSISTENT_RECURSIVE); + this.dataWatches.addWatch(path, watcher, WatchManager.Type.PERSISTENT_RECURSIVE); } } diff --git a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java index 0a8fd908f8c..70c9251fb54 100644 --- a/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -355,7 +355,9 @@ public void processRequest(Request request) { setWatches.getDataWatches(), setWatches.getExistWatches(), setWatches.getChildWatches(), - setWatches.getPersistentWatches(), cnxn); + setWatches.getPersistentWatches(), + setWatches.getPersistentRecursiveWatches(), + cnxn); break; } case OpCode.addPersistentWatch: { @@ -363,7 +365,8 @@ public void processRequest(Request request) { AddPersistentWatcherRequest addPersistentWatcherRequest = new AddPersistentWatcherRequest(); ByteBufferInputStream.byteBuffer2Record(request.request, addPersistentWatcherRequest); - zks.getZKDatabase().addPersistentWatch(addPersistentWatcherRequest.getPath(), cnxn); + zks.getZKDatabase().addPersistentWatch(addPersistentWatcherRequest.getPath(), cnxn, + addPersistentWatcherRequest.getRecursive()); rsp = new ErrorResponse(0); break; } diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index 159bf59a726..51d88bfd803 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -40,9 +40,44 @@ class WatchManager { private static final Logger LOG = LoggerFactory.getLogger(WatchManager.class); - private enum Type { - STANDARD, - PERSISTENT + enum Type { + STANDARD() { + @Override + boolean isPersistent() { + return false; + } + + @Override + boolean isRecursive() { + return false; + } + }, + PERSISTENT() { + @Override + boolean isPersistent() { + return true; + } + + @Override + boolean isRecursive() { + return false; + } + }, + PERSISTENT_RECURSIVE() { + @Override + boolean isPersistent() { + return true; + } + + @Override + boolean isRecursive() { + return true; + } + } + ; + + abstract boolean isPersistent(); + abstract boolean isRecursive(); } private final Map> watchTable = @@ -51,7 +86,12 @@ private enum Type { private final Map> watch2Paths = new HashMap<>(); - private boolean hasHadPersistentWatches = false; // guarded by sync + private int recursiveWatchQty = 0; // guarded by sync + + // visible for testing + synchronized int getRecursiveWatchQty() { + return recursiveWatchQty; + } synchronized int size(){ int result = 0; @@ -61,7 +101,7 @@ synchronized int size(){ return result; } - synchronized void addWatch(String path, Watcher watcher, boolean persistent) { + synchronized void addWatch(String path, Watcher watcher, WatchManager.Type type) { Map list = watchTable.get(path); if (list == null) { // don't waste memory if there are few watches on a node @@ -70,7 +110,13 @@ synchronized void addWatch(String path, Watcher watcher, boolean persistent) { list = new HashMap<>(4); watchTable.put(path, list); } - list.put(watcher, persistent ? Type.PERSISTENT : Type.STANDARD); + Type previousType = list.put(watcher, type); + if ((previousType != null) && previousType.isRecursive()) { + --recursiveWatchQty; + } + if (type.isRecursive()) { + ++recursiveWatchQty; + } Set paths = watch2Paths.get(watcher); if (paths == null) { @@ -79,10 +125,6 @@ synchronized void addWatch(String path, Watcher watcher, boolean persistent) { watch2Paths.put(watcher, paths); } paths.add(path); - - if (persistent) { - hasHadPersistentWatches = true; - } } synchronized void removeWatcher(Watcher watcher) { @@ -93,7 +135,10 @@ synchronized void removeWatcher(Watcher watcher) { for (String p : paths) { Map list = watchTable.get(p); if (list != null) { - list.remove(watcher); + Type removedType = list.remove(watcher); + if (removedType.isRecursive()) { + --recursiveWatchQty; + } if (list.isEmpty()) { watchTable.remove(p); } @@ -121,16 +166,18 @@ Set triggerWatch(String path, EventType type, Set supress) { Entry entry = iterator.next(); Type entryType = entry.getValue(); Watcher watcher = entry.getKey(); - if (entryType == Type.PERSISTENT) { + if (entryType.isRecursive()) { if ( type != EventType.NodeChildrenChanged ) { watchers.add(watcher); } } else if (!pathParentIterator.atParentPath()) { watchers.add(watcher); - iterator.remove(); - Set paths = watch2Paths.get(watcher); - if (paths != null) { - paths.remove(localPath); + if (!entryType.isPersistent()) { + iterator.remove(); + Set paths = watch2Paths.get(watcher); + if (paths != null) { + paths.remove(localPath); + } } } } @@ -216,10 +263,12 @@ synchronized boolean containsWatcher(String path, Watcher watcher) { for (String localPath : pathParentIterator.asIterable()) { Map watchers = watchTable.get(localPath); Type watcherType = (watchers != null) ? watchers.get(watcher) : null; - if ((watcherType == Type.STANDARD) && !pathParentIterator.atParentPath()) { - return true; + if ( !pathParentIterator.atParentPath() ) { + if ( watcherType != null ) { + return true; // at the leaf node, all watcher types match + } } - if (watcherType == Type.PERSISTENT) { + if (watcherType == Type.PERSISTENT_RECURSIVE) { return true; } } @@ -242,9 +291,13 @@ synchronized boolean removeWatcher(String path, Watcher watcher) { } Map list = watchTable.get(path); - if (list == null || (list.remove(watcher) == null)) { + Type removedType = (list != null) ? list.remove(watcher) : null; + if (removedType == null) { return false; } + if (removedType.isRecursive()) { + --recursiveWatchQty; + } if (list.isEmpty()) { watchTable.remove(path); @@ -303,8 +356,7 @@ synchronized WatchesSummary getWatchesSummary() { } private PathParentIterator getPathParentIterator(String path) { - if ( !hasHadPersistentWatches ) { - // optimization - if we've never seen a Persistent Watch, there's no need to iterate through parent nodes + if (recursiveWatchQty == 0) { return PathParentIterator.forPathOnly(path); } return PathParentIterator.forAll(path); diff --git a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java index 2eae1357f4b..f1fc02e5b0c 100644 --- a/src/java/main/org/apache/zookeeper/server/ZKDatabase.java +++ b/src/java/main/org/apache/zookeeper/server/ZKDatabase.java @@ -482,11 +482,14 @@ public void setWatches(long relativeZxid, List dataWatches, * @param existWatches the exists watches the client wants to reset * @param childWatches the child watches the client wants to reset * @param persistentWatches the persistent watches the client wants to reset + * @param persistentRecursiveWatches the persistent recursive watches the client wants to reset * @param watcher the watcher function */ public void setWatches(long relativeZxid, List dataWatches, - List existWatches, List childWatches, List persistentWatches, Watcher watcher) { - dataTree.setWatches(relativeZxid, dataWatches, existWatches, childWatches, persistentWatches, watcher); + List existWatches, List childWatches, + List persistentWatches, List persistentRecursiveWatches, Watcher watcher) { + dataTree.setWatches(relativeZxid, dataWatches, existWatches, childWatches, persistentWatches, + persistentRecursiveWatches, watcher); } /** @@ -661,8 +664,10 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) { * watch base * @param watcher * the watcher + * @param recursive + * true if recursive */ - public void addPersistentWatch(String basePath, Watcher watcher) { - dataTree.addPersistentWatch(basePath, watcher); + public void addPersistentWatch(String basePath, Watcher watcher, boolean recursive) { + dataTree.addPersistentWatch(basePath, watcher, recursive); } } diff --git a/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java b/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java new file mode 100644 index 00000000000..08148ffade1 --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java @@ -0,0 +1,88 @@ +package org.apache.zookeeper.server; + +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class RecursiveWatchQtyTest { + private WatchManager watchManager; + + private static class DummyWatcher implements Watcher { + @Override + public void process(WatchedEvent event) { + // NOP + } + } + + @Before + public void setup() { + watchManager = new WatchManager(); + } + + @Test + public void testAddRemove() { + Watcher watcher1 = new DummyWatcher(); + Watcher watcher2 = new DummyWatcher(); + + watchManager.addWatch("/a", watcher1, WatchManager.Type.PERSISTENT_RECURSIVE); + watchManager.addWatch("/b", watcher2, WatchManager.Type.PERSISTENT_RECURSIVE); + Assert.assertEquals(2, watchManager.getRecursiveWatchQty()); + watchManager.removeWatcher("/a", watcher1); + watchManager.removeWatcher("/b", watcher2); + Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); + } + + @Test + public void testAddRemoveAlt() { + Watcher watcher1 = new DummyWatcher(); + Watcher watcher2 = new DummyWatcher(); + + watchManager.addWatch("/a", watcher1, WatchManager.Type.PERSISTENT_RECURSIVE); + watchManager.addWatch("/b", watcher2, WatchManager.Type.PERSISTENT_RECURSIVE); + Assert.assertEquals(2, watchManager.getRecursiveWatchQty()); + watchManager.removeWatcher(watcher1); + watchManager.removeWatcher(watcher2); + Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); + } + + @Test + public void testDoubleAdd() { + Watcher watcher = new DummyWatcher(); + + watchManager.addWatch("/a", watcher, WatchManager.Type.PERSISTENT_RECURSIVE); + watchManager.addWatch("/a", watcher, WatchManager.Type.PERSISTENT_RECURSIVE); + Assert.assertEquals(1, watchManager.getRecursiveWatchQty()); + watchManager.removeWatcher(watcher); + Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); + } + + @Test + public void testSameWatcherMultiPath() { + Watcher watcher = new DummyWatcher(); + + watchManager.addWatch("/a", watcher, WatchManager.Type.PERSISTENT_RECURSIVE); + watchManager.addWatch("/a/b", watcher, WatchManager.Type.PERSISTENT_RECURSIVE); + watchManager.addWatch("/a/b/c", watcher, WatchManager.Type.PERSISTENT_RECURSIVE); + Assert.assertEquals(3, watchManager.getRecursiveWatchQty()); + watchManager.removeWatcher("/a/b", watcher); + Assert.assertEquals(2, watchManager.getRecursiveWatchQty()); + watchManager.removeWatcher(watcher); + Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); + } + + @Test + public void testChangeType() { + Watcher watcher = new DummyWatcher(); + + watchManager.addWatch("/a", watcher, WatchManager.Type.PERSISTENT); + Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); + watchManager.addWatch("/a", watcher, WatchManager.Type.PERSISTENT_RECURSIVE); + Assert.assertEquals(1, watchManager.getRecursiveWatchQty()); + watchManager.addWatch("/a", watcher, WatchManager.Type.STANDARD); + Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); + watchManager.removeWatcher("/a", watcher); + Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); + } +} diff --git a/src/java/test/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java new file mode 100644 index 00000000000..55b67337cdf --- /dev/null +++ b/src/java/test/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java @@ -0,0 +1,193 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.test; + +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +public class PersistentRecursiveWatcherTest extends ClientBase { + private static final Logger LOG = LoggerFactory.getLogger(PersistentRecursiveWatcherTest.class); + private BlockingQueue events; + private Watcher persistentWatcher; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + + events = new LinkedBlockingQueue<>(); + persistentWatcher = new Watcher() { + @Override + public void process(WatchedEvent event) { + events.add(event); + } + }; + } + + @Test + public void testBasic() + throws IOException, InterruptedException, KeeperException { + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { + zk.addPersistentWatch("/a/b", persistentWatcher, true); + internalTestBasic(zk); + } + } + + @Test + public void testBasicAsync() + throws IOException, InterruptedException, KeeperException { + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { + final CountDownLatch latch = new CountDownLatch(1); + AsyncCallback.VoidCallback cb = new AsyncCallback.VoidCallback() { + @Override + public void processResult(int rc, String path, Object ctx) { + if (rc == 0) { + latch.countDown(); + } + } + }; + zk.addPersistentWatch("/a/b", persistentWatcher, true, cb, null); + Assert.assertTrue(latch.await(5, TimeUnit.SECONDS)); + internalTestBasic(zk); + } + } + + private void internalTestBasic(ZooKeeper zk) throws KeeperException, InterruptedException { + 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.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.setData("/a/b/c/d/e", new byte[0], -1); + zk.delete("/a/b/c/d/e", -1); + zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); + } + + @Test + public void testRemoval() + throws IOException, InterruptedException, KeeperException { + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { + zk.addPersistentWatch("/a/b", persistentWatcher, true); + 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); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); + + zk.removeWatches("/a/b", persistentWatcher, Watcher.WatcherType.Any, false); + zk.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + assertEvent(events, Watcher.Event.EventType.PersistentWatchRemoved, "/a/b"); + } + } + + @Test + public void testDisconnect() throws Exception { + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { + zk.addPersistentWatch("/a/b", persistentWatcher, true); + stopServer(); + assertEvent(events, Watcher.Event.EventType.None, null); + startServer(); + assertEvent(events, Watcher.Event.EventType.None, null); + internalTestBasic(zk); + } + } + + @Test + public void testMultiClient() + throws IOException, InterruptedException, KeeperException { + ZooKeeper zk1 = null; + ZooKeeper zk2 = null; + try { + zk1 = createClient(new CountdownWatcher(), hostPort); + zk2 = createClient(new CountdownWatcher(), hostPort); + + zk1.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk1.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk1.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + zk1.addPersistentWatch("/a/b", persistentWatcher, true); + zk1.setData("/a/b/c", "one".getBytes(), -1); + Thread.sleep(1000); // give some time for the event to arrive + + zk2.setData("/a/b/c", "two".getBytes(), -1); + zk2.setData("/a/b/c", "three".getBytes(), -1); + zk2.setData("/a/b/c", "four".getBytes(), -1); + + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + } finally { + if (zk1 != null) { + zk1.close(); + } + if (zk2 != null) { + zk2.close(); + } + } + } + + @Test + public void testRootWatcher() + throws IOException, InterruptedException, KeeperException { + try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { + zk.addPersistentWatch("/", persistentWatcher, true); + 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("/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b/c"); + } + } + + private void assertEvent(BlockingQueue events, Watcher.Event.EventType eventType, String path) + throws InterruptedException { + WatchedEvent event = events.poll(5, TimeUnit.SECONDS); + Assert.assertNotNull(event); + Assert.assertEquals(eventType, event.getType()); + Assert.assertEquals(path, event.getPath()); + } +} diff --git a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java index aceb42f1c64..6e4885cbd8f 100644 --- a/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java +++ b/src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -60,7 +60,7 @@ public void process(WatchedEvent event) { public void testBasic() throws IOException, InterruptedException, KeeperException { try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { - zk.addPersistentWatch("/a/b", persistentWatcher); + zk.addPersistentWatch("/a/b", persistentWatcher, false); internalTestBasic(zk); } } @@ -78,7 +78,7 @@ public void processResult(int rc, String path, Object ctx) { } } }; - zk.addPersistentWatch("/a/b", persistentWatcher, cb, null); + zk.addPersistentWatch("/a/b", persistentWatcher, false, cb, null); Assert.assertTrue(latch.await(5, TimeUnit.SECONDS)); internalTestBasic(zk); } @@ -88,34 +88,33 @@ private void internalTestBasic(ZooKeeper zk) throws KeeperException, Interrupted 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.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk.setData("/a/b/c/d/e", new byte[0], -1); - zk.delete("/a/b/c/d/e", -1); - zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.setData("/a/b", new byte[0], -1); + zk.delete("/a/b/c", -1); + zk.delete("/a/b", -1); + zk.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); - assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c/d/e"); - assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a/b/c/d/e"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); } @Test public void testRemoval() throws IOException, InterruptedException, KeeperException { try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { - zk.addPersistentWatch("/a/b", persistentWatcher); + zk.addPersistentWatch("/a/b", persistentWatcher, false); 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); assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); zk.removeWatches("/a/b", persistentWatcher, Watcher.WatcherType.Any, false); - zk.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.delete("/a/b/c", -1); + zk.delete("/a/b", -1); assertEvent(events, Watcher.Event.EventType.PersistentWatchRemoved, "/a/b"); } } @@ -123,7 +122,7 @@ public void testRemoval() @Test public void testDisconnect() throws Exception { try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { - zk.addPersistentWatch("/a/b", persistentWatcher); + zk.addPersistentWatch("/a/b", persistentWatcher, false); stopServer(); assertEvent(events, Watcher.Event.EventType.None, null); startServer(); @@ -143,20 +142,19 @@ public void testMultiClient() zk1.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk1.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk1.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk1.addPersistentWatch("/a/b", persistentWatcher); - zk1.setData("/a/b/c", "one".getBytes(), -1); + zk1.addPersistentWatch("/a/b", persistentWatcher, false); + zk1.setData("/a/b", "one".getBytes(), -1); Thread.sleep(1000); // give some time for the event to arrive - zk2.setData("/a/b/c", "two".getBytes(), -1); - zk2.setData("/a/b/c", "three".getBytes(), -1); - zk2.setData("/a/b/c", "four".getBytes(), -1); + zk2.setData("/a/b", "two".getBytes(), -1); + zk2.setData("/a/b", "three".getBytes(), -1); + zk2.setData("/a/b", "four".getBytes(), -1); - assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); - assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); - assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); - assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); } finally { if (zk1 != null) { zk1.close(); @@ -171,15 +169,12 @@ public void testMultiClient() public void testRootWatcher() throws IOException, InterruptedException, KeeperException { try ( ZooKeeper zk = createClient(new CountdownWatcher(), hostPort) ) { - zk.addPersistentWatch("/", persistentWatcher); + zk.addPersistentWatch("/", persistentWatcher, false); 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.setData("/a", new byte[0], -1); zk.create("/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk.create("/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b"); - assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/"); } } diff --git a/src/zookeeper.jute b/src/zookeeper.jute index 45710033cbb..de745db31ac 100644 --- a/src/zookeeper.jute +++ b/src/zookeeper.jute @@ -79,6 +79,7 @@ module org.apache.zookeeper.proto { vectorexistWatches; vectorchildWatches; vectorpersistentWatches; + vectorpersistentRecursiveWatches; } class RequestHeader { int xid; @@ -186,6 +187,7 @@ module org.apache.zookeeper.proto { } class AddPersistentWatcherRequest { ustring path; + boolean recursive; } class WatcherEvent { int type; // event type From 97a9f3e2a0fc5dc38d17a566817d58fa06427259 Mon Sep 17 00:00:00 2001 From: randgalt Date: Sun, 20 Aug 2017 14:08:45 +0200 Subject: [PATCH 27/31] doc updates for recursive watches --- .../content/xdocs/zookeeperProgrammers.xml | 10 +++++----- src/java/main/org/apache/zookeeper/ZooKeeper.java | 5 +++++ 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml index 6844a64efdf..af0431a95ca 100644 --- a/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml +++ b/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml @@ -718,20 +718,20 @@
- Persistent Recursive Watches + Persistent Watches New in 3.6.0: There is now a variation on the standard watch described above whereby you can set a watch that does not get removed when triggered. Additionally, these watches trigger the event types NodeCreated, NodeDeleted, and NodeDataChanged - recursively for all znodes starting at the znode that the watch is registered for. Note that + and, optionally, recursively for all znodes starting at the znode that the watch is registered for. Note that NodeChildrenChanged are not triggered for persistent recursive watches as it is redundant. - Persistent recursive watches are set using the method Persistent watches are set using the method addPersistentWatch(). The triggering semantics and guarantees (other than one-time triggering) are the same as standard watches. The only exception regarding events is that - persistent watchers never throw child changed events as they are redundant. - Persistent recursive watches are removed using removeWatches() with watcher type + recursive persistent watchers never trigger child changed events as they are redundant. + Persistent watches are removed using removeWatches() with watcher type WatcherType.Any.
diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index 2455662ee69..cf029135783 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -2812,6 +2812,11 @@ public void sync(final String path, VoidCallback cb, Object ctx){ * of the given path including children added later. *

* + *

+ * NOTE: when there are active recursive watches there is a small performance loss as all segments + * of ZNode paths must be checked for watch triggering. + *

+ * * @param basePath the top path that the watcher applies to * @param watcher the watcher * @param recursive if true applies not only to the registered path but all child paths recursively From 19fc7663509d25f0af433f7d97acc35d6ca28215 Mon Sep 17 00:00:00 2001 From: randgalt Date: Sun, 20 Aug 2017 14:16:01 +0200 Subject: [PATCH 28/31] doc updates for recursive watches --- src/java/main/org/apache/zookeeper/ZooKeeper.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index cf029135783..7f5a20ecded 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -2813,13 +2813,14 @@ public void sync(final String path, VoidCallback cb, Object ctx){ *

* *

- * NOTE: when there are active recursive watches there is a small performance loss as all segments + * NOTE: when there are active recursive watches there is a small performance decrease as all segments * of ZNode paths must be checked for watch triggering. *

* * @param basePath the top path that the watcher applies to * @param watcher the watcher - * @param recursive if true applies not only to the registered path but all child paths recursively + * @param recursive if true applies not only to the registered path but all child paths recursively including + * any child nodes added in the future * @throws InterruptedException If the server transaction is interrupted. * @throws KeeperException If the server signals an error with a non-zero * error code. From 30ef7af364fcd06d18c0d5b2ac308ea0d53b4978 Mon Sep 17 00:00:00 2001 From: randgalt Date: Sun, 20 Aug 2017 14:16:31 +0200 Subject: [PATCH 29/31] doc updates for recursive watches --- src/java/main/org/apache/zookeeper/ZooKeeper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/java/main/org/apache/zookeeper/ZooKeeper.java b/src/java/main/org/apache/zookeeper/ZooKeeper.java index 7f5a20ecded..21ee7c2ffa8 100644 --- a/src/java/main/org/apache/zookeeper/ZooKeeper.java +++ b/src/java/main/org/apache/zookeeper/ZooKeeper.java @@ -2846,7 +2846,8 @@ public void addPersistentWatch(String basePath, Watcher watcher, boolean recursi * * @param basePath the top path that the watcher applies to * @param watcher the watcher - * @param recursive if true applies not only to the registered path but all child paths recursively + * @param recursive if true applies not only to the registered path but all child paths recursively including + * any child nodes added in the future * @param cb a handler for the callback * @param ctx context to be provided to the callback * @throws IllegalArgumentException if an invalid path is specified From 63458818a5f32bac23141612470944389745ad32 Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 21 Aug 2017 11:00:18 +0200 Subject: [PATCH 30/31] Cleaner method of checking for recursive watches --- .../main/org/apache/zookeeper/server/WatchManager.java | 10 +++++++--- .../apache/zookeeper/server/RecursiveWatchQtyTest.java | 8 ++++---- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/java/main/org/apache/zookeeper/server/WatchManager.java b/src/java/main/org/apache/zookeeper/server/WatchManager.java index 51d88bfd803..de1cb398bdb 100644 --- a/src/java/main/org/apache/zookeeper/server/WatchManager.java +++ b/src/java/main/org/apache/zookeeper/server/WatchManager.java @@ -111,7 +111,7 @@ synchronized void addWatch(String path, Watcher watcher, WatchManager.Type type) watchTable.put(path, list); } Type previousType = list.put(watcher, type); - if ((previousType != null) && previousType.isRecursive()) { + if (safeIsRecursive(previousType)) { --recursiveWatchQty; } if (type.isRecursive()) { @@ -127,6 +127,10 @@ synchronized void addWatch(String path, Watcher watcher, WatchManager.Type type) paths.add(path); } + private boolean safeIsRecursive(Type type) { + return (type != null) && type.isRecursive(); + } + synchronized void removeWatcher(Watcher watcher) { Set paths = watch2Paths.remove(watcher); if (paths == null) { @@ -136,7 +140,7 @@ synchronized void removeWatcher(Watcher watcher) { Map list = watchTable.get(p); if (list != null) { Type removedType = list.remove(watcher); - if (removedType.isRecursive()) { + if (safeIsRecursive(removedType)) { --recursiveWatchQty; } if (list.isEmpty()) { @@ -295,7 +299,7 @@ synchronized boolean removeWatcher(String path, Watcher watcher) { if (removedType == null) { return false; } - if (removedType.isRecursive()) { + if (safeIsRecursive(removedType)) { --recursiveWatchQty; } diff --git a/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java b/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java index 08148ffade1..417c80a4728 100644 --- a/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java +++ b/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java @@ -29,8 +29,8 @@ public void testAddRemove() { watchManager.addWatch("/a", watcher1, WatchManager.Type.PERSISTENT_RECURSIVE); watchManager.addWatch("/b", watcher2, WatchManager.Type.PERSISTENT_RECURSIVE); Assert.assertEquals(2, watchManager.getRecursiveWatchQty()); - watchManager.removeWatcher("/a", watcher1); - watchManager.removeWatcher("/b", watcher2); + Assert.assertTrue(watchManager.removeWatcher("/a", watcher1)); + Assert.assertTrue(watchManager.removeWatcher("/b", watcher2)); Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); } @@ -66,7 +66,7 @@ public void testSameWatcherMultiPath() { watchManager.addWatch("/a/b", watcher, WatchManager.Type.PERSISTENT_RECURSIVE); watchManager.addWatch("/a/b/c", watcher, WatchManager.Type.PERSISTENT_RECURSIVE); Assert.assertEquals(3, watchManager.getRecursiveWatchQty()); - watchManager.removeWatcher("/a/b", watcher); + Assert.assertTrue(watchManager.removeWatcher("/a/b", watcher)); Assert.assertEquals(2, watchManager.getRecursiveWatchQty()); watchManager.removeWatcher(watcher); Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); @@ -82,7 +82,7 @@ public void testChangeType() { Assert.assertEquals(1, watchManager.getRecursiveWatchQty()); watchManager.addWatch("/a", watcher, WatchManager.Type.STANDARD); Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); - watchManager.removeWatcher("/a", watcher); + Assert.assertTrue(watchManager.removeWatcher("/a", watcher)); Assert.assertEquals(0, watchManager.getRecursiveWatchQty()); } } From 15e0a1d332c46499cc77f85eaecf8cb4412a986c Mon Sep 17 00:00:00 2001 From: randgalt Date: Mon, 25 Sep 2017 11:23:42 -0500 Subject: [PATCH 31/31] Needed license header --- .../server/RecursiveWatchQtyTest.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java b/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java index 417c80a4728..3b0f88efdc0 100644 --- a/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java +++ b/src/java/test/org/apache/zookeeper/server/RecursiveWatchQtyTest.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.zookeeper.server; import org.apache.zookeeper.WatchedEvent;