You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by Randgalt <gi...@git.apache.org> on 2016/12/26 19:37:01 UTC

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

GitHub user Randgalt opened a pull request:

    https://github.com/apache/zookeeper/pull/136

    [ZOOKEEPER-1416] Persistent Recursive Watch

    Here is a completed implementation for a persistent, recursive watch addition for ZK. These watches are set via a new method, `addPersistentWatch()` and are removed via the existing watcher removal methods. Persistent, recursive watches have these characteristics:
    
    - Once set, they do not auto-remove when triggered
    - They trigger for all event types (child, data, etc.) on the node they are registered for and any child znode recursively.
    - They are efficiently implemented by using the existing watch internals. A new class `PathIterator` walks up the path parent-by-parent when checking if a watcher applies. 
    
    Persistent watcher specific tests are in `PersistentWatcherTest.java`. I'd appreciated feedback on other additional tests that should be added.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/Randgalt/zookeeper ZOOKEEPER-1416

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/136.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #136
    
----
commit 3c05c671d09e5b6df936af8f0a700995d5749e11
Author: randgalt <jo...@jordanzimmerman.com>
Date:   2016-12-25T21:36:13Z

    basic work done. Needs more testing, tuning, etc.

commit ca4a000dcf294aaebd09d3118ebc62cb0783f9cc
Author: randgalt <jo...@jordanzimmerman.com>
Date:   2016-12-26T15:06:55Z

    working on persistent watcher removal

commit bf13deda0b00ca67cd1fa963961d95a22634ed88
Author: randgalt <jo...@jordanzimmerman.com>
Date:   2016-12-26T17:59:04Z

    Support resetting persistent watches

commit 27d8d6cd45cb6adfabf50143f6de62a371447519
Author: randgalt <jo...@jordanzimmerman.com>
Date:   2016-12-26T18:21:17Z

    docs

commit 2766fb1020c600af579a0f701fa3c00ea92b7e22
Author: randgalt <jo...@jordanzimmerman.com>
Date:   2016-12-26T18:44:42Z

    containsWatcher() was broken for STANDARD watchers

commit 86fa1fbcb75021179f80588a2ea46aad2127fb4e
Author: randgalt <jo...@jordanzimmerman.com>
Date:   2016-12-26T19:20:00Z

    removed unused import

commit b490c84d1e56335ba66f9c56d64134886b144451
Author: randgalt <jo...@jordanzimmerman.com>
Date:   2016-12-26T19:20:08Z

    Updated doc for persistent watches

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    @eribeiro Have a look at the change to `PathIterator` I think this solves your desire while not breaking implicit contracts.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132034244
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -181,11 +206,18 @@ synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) {
          * @return true if the watcher exists, false otherwise
          */
         synchronized boolean containsWatcher(String path, Watcher watcher) {
    -        HashSet<String> paths = watch2Paths.get(watcher);
    -        if (paths == null || !paths.contains(path)) {
    -            return false;
    +        PathIterator pathIterator = new PathIterator(path);
    +        for (String localPath : pathIterator.asIterable()) {
    +            Map<Watcher, Type> watchers = watchTable.get(localPath);
    +            Type watcherType = (watchers != null) ? watchers.get(watcher) : null;
    +            if ((watcherType == Type.STANDARD) && !pathIterator.atParentPath()) {
    --- End diff --
    
    I'll check this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93887652
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -40,47 +40,52 @@
     class WatchManager {
         private static final Logger LOG = LoggerFactory.getLogger(WatchManager.class);
     
    -    private final HashMap<String, HashSet<Watcher>> watchTable =
    -        new HashMap<String, HashSet<Watcher>>();
    +    private enum Type {
    --- End diff --
    
    Sincere question: wdyt about changing the name to WatchType? It is doable? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886777
  
    --- Diff: 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 {
    --- End diff --
    
    The problem with this version of the class is that it breaks the implied contract of `Iterable`. I've always assumed that `Iterable#iterator` returns a new iterator. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132579217
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -97,23 +102,44 @@ synchronized void removeWatcher(Watcher watcher) {
         Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
             WatchedEvent e = new WatchedEvent(type,
                     KeeperState.SyncConnected, path);
    -        HashSet<Watcher> watchers;
    +        Set<Watcher> 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);
    +            for (String localPath : pathIterator.asIterable()) {
    +                Map<Watcher, Type> thisWatchers = watchTable.get(localPath);
    +                if (thisWatchers == null || thisWatchers.isEmpty()) {
    +                    continue;
                     }
    -                return null;
    -            }
    -            for (Watcher w : watchers) {
    -                HashSet<String> paths = watch2Paths.get(w);
    -                if (paths != null) {
    -                    paths.remove(path);
    +                Iterator<Entry<Watcher, Type>> iterator = thisWatchers.entrySet().iterator();
    +                while (iterator.hasNext()) {
    +                    Entry<Watcher, Type> entry = iterator.next();
    +                    Type entryType = entry.getValue();
    +                    Watcher watcher = entry.getKey();
    +                    if (entryType == Type.PERSISTENT) {
    +                        if ( type != EventType.NodeChildrenChanged ) {
    +                            watchers.add(watcher);
    --- End diff --
    
    The problem is that it makes 3.5.4 clients incompatible with 3.5.3 servers. That might make sense for 3.6.0 but it's unreasonable for .x.N release.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    @Randgalt it is required to add a C client change in this patch too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93889759
  
    --- Diff: 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() ) {
    --- End diff --
    
    @Rangdalt please, ignore the comment just above, you did it already. But the preceding comment about the ``else`` clause would be a plus, I guess.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93883200
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -97,23 +102,44 @@ synchronized void removeWatcher(Watcher watcher) {
         Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
             WatchedEvent e = new WatchedEvent(type,
                     KeeperState.SyncConnected, path);
    -        HashSet<Watcher> watchers;
    +        Set<Watcher> 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()) {
    --- End diff --
    
    Same here, if ``pathIterator`` implements ``Iterable`` and ``Iterator`` then we can use:
    ```
    for (String localPath : pathIterator) {
       (...)
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    Thanks @eribeiro - at minimum it would be good to get feedback. This feature will really help the community.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r94030738
  
    --- Diff: src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml ---
    @@ -729,7 +729,7 @@
     
           <para>Persistent recursive watches are set using the method <emphasis
           role="bold">addPersistentWatch()</emphasis>. 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
    --- End diff --
    
          (only exception regarding events is that persistent watchers **never** throw child changed event).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886426
  
    --- Diff: 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;
    --- End diff --
    
    I'd say the path, by definition, is validated. But, it would be good to add comment saying that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132576447
  
    --- Diff: src/java/main/org/apache/zookeeper/ZooDefs.java ---
    @@ -74,12 +74,16 @@
     
             public final int createTTL = 21;
     
    +        public final int addPersistentWatch = 22;
    +
             public final int auth = 100;
     
             public final int setWatches = 101;
     
             public final int sasl = 102;
     
    +        public final int setWatches2 = 103;
    --- End diff --
    
    I don't think just because we are backporting this we need to keep this the same between the branches. I feel we should take advantage of the new version and keep the code as clean as possible. @hanm do you have an opinion on the matter?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    re: ACL and recursive watches...
    
    ZooKeeper.exists() does not check ACLs so I assert that the current behavior of this PR doesn't violate the current behavior. i.e. in ZooKeeper you can have knowledge of the existence of every ZNode in the database irrespective of ACL. I can update the docs to make this clear however if people think that that's needed.


---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    Another goal is feature parity with other consensus tools such as etcd/consul. I added TTL nodes with this (and other) goals earlier in the year (or was it last year?). Watches in consul are persistent and optionally recursive.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by skamille <gi...@git.apache.org>.
Github user skamille commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    > Per 3. This PR does not guarantee that you will see all events. I'll double check the doc to make sure that that's clear. These watches behave exactly as other watches in ZK other than they don't remove themselves when triggered.
    
    Well, I would personally expect that a watch that doesn't remove itself when triggered is going to never drop an event so long as the session is alive. My understanding of the raft-based stores is that they can give you the historical change log that guarantees it won't lose events. I haven't done the mental exercise of determining if we could do that in ZK (probably? maybe?) but it is certainly a difference that partially arises from the underlying design differences between the systems, and we should be more specific about what will really happen with this feature.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by skamille <gi...@git.apache.org>.
Github user skamille commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r136637514
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -40,50 +40,110 @@
     class WatchManager {
         private static final Logger LOG = LoggerFactory.getLogger(WatchManager.class);
     
    -    private final HashMap<String, HashSet<Watcher>> watchTable =
    -        new HashMap<String, HashSet<Watcher>>();
    +    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<String, Map<Watcher, Type>> watchTable =
    +        new HashMap<>();
    +
    +    private final Map<Watcher, Set<String>> watch2Paths =
    +        new HashMap<>();
     
    -    private final HashMap<Watcher, HashSet<String>> watch2Paths =
    -        new HashMap<Watcher, HashSet<String>>();
    +    private int recursiveWatchQty = 0;    // guarded by sync
    +
    +    // visible for testing
    +    synchronized int getRecursiveWatchQty() {
    +        return recursiveWatchQty;
    +    }
     
         synchronized int size(){
             int result = 0;
    -        for(Set<Watcher> watches : watchTable.values()) {
    +        for(Map<Watcher, Type> watches : watchTable.values()) {
                 result += watches.size();
             }
             return result;
         }
     
    -    synchronized void addWatch(String path, Watcher watcher) {
    -        HashSet<Watcher> list = watchTable.get(path);
    +    synchronized void addWatch(String path, Watcher watcher, WatchManager.Type type) {
    +        Map<Watcher, Type> 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<Watcher>(4);
    +            list = new HashMap<>(4);
    --- End diff --
    
    Does the assumption still hold re: memory management now that we have a hashmap instead of a hashset?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93960053
  
    --- Diff: src/java/main/org/apache/zookeeper/server/PathIterator.java ---
    @@ -0,0 +1,89 @@
    +/**
    + * 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 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<String> {
    +    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<String> asIterable() {
    +        return new Iterable<String>() {
    +            @Override
    +            public Iterator<String> 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()) {
    +            ++level;
    +            if (path.equals("/")) {
    +                path = "";
    +            } else {
    +                path = path.substring(0, path.lastIndexOf('/'));
    +                if (path.length() == 0) {
    +                    path = "/";
    +                }
    +            }
    +        }
    --- End diff --
    
    @Randgalt I had written about this previously, but I think it got lost in my comment flood yesterday, excuse me:
    
    By the ``Iterator#next()`` contract, if we call next() and there's no next element it should throw a ``NoSuchElementException``, so an ``else`` clause throwing this exception would be nice. ;)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93887048
  
    --- Diff: 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 {
    --- End diff --
    
    Good point!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93883352
  
    --- Diff: 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 ) {
    --- End diff --
    
    nit: ``if (path.isEmpty())``


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by stuhood <gi...@git.apache.org>.
Github user stuhood commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    Driveby comment: would be great to see a "recipe" or some real world example of correctly using this API to watch a tree. Is the goal that consumption of only the watch events would be sufficient for a client to stay synchronized for some tree?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886806
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -97,23 +102,44 @@ synchronized void removeWatcher(Watcher watcher) {
         Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
             WatchedEvent e = new WatchedEvent(type,
                     KeeperState.SyncConnected, path);
    -        HashSet<Watcher> watchers;
    +        Set<Watcher> 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()) {
    --- End diff --
    
    See my comment above why this isn't practical


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93887170
  
    --- Diff: src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java ---
    @@ -0,0 +1,170 @@
    +/**
    + * 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<WatchedEvent> 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);
    +            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 ) {
    --- End diff --
    
    nit: if expression with extra spaces


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r140815418
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -40,50 +40,110 @@
     class WatchManager {
         private static final Logger LOG = LoggerFactory.getLogger(WatchManager.class);
     
    -    private final HashMap<String, HashSet<Watcher>> watchTable =
    -        new HashMap<String, HashSet<Watcher>>();
    +    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<String, Map<Watcher, Type>> watchTable =
    +        new HashMap<>();
    +
    +    private final Map<Watcher, Set<String>> watch2Paths =
    +        new HashMap<>();
     
    -    private final HashMap<Watcher, HashSet<String>> watch2Paths =
    -        new HashMap<Watcher, HashSet<String>>();
    +    private int recursiveWatchQty = 0;    // guarded by sync
    +
    +    // visible for testing
    +    synchronized int getRecursiveWatchQty() {
    +        return recursiveWatchQty;
    +    }
     
         synchronized int size(){
             int result = 0;
    -        for(Set<Watcher> watches : watchTable.values()) {
    +        for(Map<Watcher, Type> watches : watchTable.values()) {
                 result += watches.size();
             }
             return result;
         }
     
    -    synchronized void addWatch(String path, Watcher watcher) {
    -        HashSet<Watcher> list = watchTable.get(path);
    +    synchronized void addWatch(String path, Watcher watcher, WatchManager.Type type) {
    +        Map<Watcher, Type> 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<Watcher>(4);
    +            list = new HashMap<>(4);
    --- End diff --
    
    Good point - this was just cargo-cult coding on my part. However, internally `HashSet` uses a `HashMap` so the implementation is the same.


---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886805
  
    --- Diff: src/java/main/org/apache/zookeeper/ZooKeeper.java ---
    @@ -522,6 +566,15 @@ protected boolean removeWatches(Map<String, Set<Watcher>> pathVsWatcher,
     
                 return result;
             }
    +
    +        private void addPersistentWatches(String clientPath, Set<Watcher> result) {
    +            synchronized (persistentWatches) {
    +                PathIterator pathIterator = new PathIterator(clientPath);
    +                while (pathIterator.hasNext()) {
    --- End diff --
    
    See my comment above why this isn't practical


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    This PR and #332 are ready. How can we get this merged? @skamille ?


---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    @Randgalt yup, very cool the ``asIterable()`` solution. +1. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886810
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -181,11 +206,19 @@ synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) {
          * @return true if the watcher exists, false otherwise
          */
         synchronized boolean containsWatcher(String path, Watcher watcher) {
    -        HashSet<String> paths = watch2Paths.get(watcher);
    -        if (paths == null || !paths.contains(path)) {
    -            return false;
    +        PathIterator pathIterator = new PathIterator(path);
    +        while (pathIterator.hasNext()) {
    --- End diff --
    
    See my comment above why this isn't practical


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r131271126
  
    --- Diff: src/java/main/org/apache/zookeeper/ZooDefs.java ---
    @@ -74,12 +74,16 @@
     
             public final int createTTL = 21;
     
    +        public final int addPersistentWatch = 22;
    +
             public final int auth = 100;
     
             public final int setWatches = 101;
     
             public final int sasl = 102;
     
    +        public final int setWatches2 = 103;
    --- End diff --
    
    Since we are targeting a new major release with this change, can we just make changes to the preexisting `setWatches`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886519
  
    --- Diff: src/java/test/org/apache/zookeeper/test/PathIteratorTest.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.PathIterator;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +public class PathIteratorTest {
    +    @Test
    +    public void testRoot() {
    +        PathIterator pathIterator = new PathIterator("/");
    +        Assert.assertTrue(pathIterator.hasNext());
    --- End diff --
    
    Sorry, you right. I am used to static imports on other projects.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    News on this great feature?


---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    @afine issues addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886802
  
    --- Diff: 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 ) {
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    @stuhood - FYI - here's a new version of Curator's cache recipes consolidated into one-recipe-to-rule-them-all that relies on this new Persistent Recursive Watch implementation. https://github.com/apache/curator/pull/181


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132010827
  
    --- Diff: src/java/main/org/apache/zookeeper/server/PathIterator.java ---
    @@ -0,0 +1,92 @@
    +/**
    + * 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 java.util.Iterator;
    +import java.util.NoSuchElementException;
    +
    +/**
    + * 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<String> {
    --- End diff --
    
    Can we call this `PathParentIterator` or something else a little clearer?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    Per 1. I posted some performance numbers in the issue. There's a definite hit but it's worth it in my view. We should discuss this.
    
    Per 2. What this PR is aimed at is users of Curator's TreeCache - one of the most widely used "recipes" in the library. Many users want to know everything that happens to a tree of ZNodes. With the current APIs this is extraordinarily difficult (thus the complexity of the TreeCache code) and inefficient. You must set 2 watches for every single node in the tree (data and children) and then work very hard to keep those watches set as they trigger, through network issues, etc.
    
    Per 3. This PR does not guarantee that you will see all events. I'll double check the doc to make sure that that's clear. These watches behave exactly as other watches in ZK other than they don't remove themselves when triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132588619
  
    --- Diff: src/java/main/org/apache/zookeeper/ZooDefs.java ---
    @@ -74,12 +74,16 @@
     
             public final int createTTL = 21;
     
    +        public final int addPersistentWatch = 22;
    +
             public final int auth = 100;
     
             public final int setWatches = 101;
     
             public final int sasl = 102;
     
    +        public final int setWatches2 = 103;
    --- End diff --
    
    @Randgalt im guessing you meant to put "The problem is that it makes 3.5.4 clients incompatible with 3.5.3 servers. That might make sense for 3.6.0 but it's unreasonable for .x.N release." here. I agree totally. This code should stay the same in the 3.5 line but there is no reason it needs to be identical to the 3.6 line which is why i think it should be changed "here". 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93885429
  
    --- Diff: 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;
    --- End diff --
    
    This is being used only by this patch now, but it makes sense to include an invalid parameter checking here? I mean:
    
    ```
    this.path = path;
    PathUtils.validatePath(this.path);
    ```
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886789
  
    --- Diff: 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;
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132592200
  
    --- Diff: src/java/main/org/apache/zookeeper/ZooDefs.java ---
    @@ -74,12 +74,16 @@
     
             public final int createTTL = 21;
     
    +        public final int addPersistentWatch = 22;
    +
             public final int auth = 100;
     
             public final int setWatches = 101;
     
             public final int sasl = 102;
     
    +        public final int setWatches2 = 103;
    --- End diff --
    
    I see - sure that's OK with me. Frankly, anything to get this PR moving. It's been almost impossible to get a ZK committer interested in this and I see this as a vital addition to ZK.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    > does the new map in the watch structure break rolling upgrades
    
    It would depend how you did it. If you upgraded all the ZK servers before adding any persistent watches you'd be OK.


---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132027605
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -97,23 +102,44 @@ synchronized void removeWatcher(Watcher watcher) {
         Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
             WatchedEvent e = new WatchedEvent(type,
                     KeeperState.SyncConnected, path);
    -        HashSet<Watcher> watchers;
    +        Set<Watcher> 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);
    +            for (String localPath : pathIterator.asIterable()) {
    --- End diff --
    
    It occurs to me that a check could be made to see if there are any Persistent watchers registered and skip if there aren't. I'll make that change.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    >  it is required to add a C client change in this patch too?
    
    @eribeiro C client work is separate. I usually add a new Jira for them. If/when this patch is accepted I'll do that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93887190
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -97,23 +102,43 @@ synchronized void removeWatcher(Watcher watcher) {
         Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
             WatchedEvent e = new WatchedEvent(type,
                     KeeperState.SyncConnected, path);
    -        HashSet<Watcher> watchers;
    +        Set<Watcher> 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);
    +            for (String localPath : pathIterator.asIterable()) {
    +                Map<Watcher, Type> thisWatchers = watchTable.get(localPath);
    +                if (thisWatchers == null || thisWatchers.isEmpty()) {
    +                    continue;
                     }
    -                return null;
    -            }
    -            for (Watcher w : watchers) {
    -                HashSet<String> paths = watch2Paths.get(w);
    -                if (paths != null) {
    -                    paths.remove(path);
    +                Iterator<Entry<Watcher, Type>> iterator = thisWatchers.entrySet().iterator();
    +                while (iterator.hasNext()) {
    +                    Entry<Watcher, Type> 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<String> paths = watch2Paths.get(watcher);
    +                        if (paths != null) {
    +                            paths.remove(localPath);
    +                        }
    +                    }
                     }
    +                if (thisWatchers.size() == 0) {
    +                    watchTable.remove(localPath);
    +                }
    +            }
    +        }
    +        if (watchers.size() == 0) {
    --- End diff --
    
    Nit: ``watchers.isEmpty()``


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93887080
  
    --- Diff: 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() ) {
    --- End diff --
    
    As you explained above, implementing Iterable would break the implicit contract, but can we still implement Iterator<String>? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    Hey @fpj, @phunt, @breed, @skamille (or any PMC/commiter), it would be really cool to have this patch by @Randgalt merged, wouldn't it? If you have any cycles would you mind to review this one? 
    
    Thanks!!!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93883345
  
    --- Diff: 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;
    --- End diff --
    
    nit: ``return !path.isEmpty();``


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93883225
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -181,11 +206,19 @@ synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) {
          * @return true if the watcher exists, false otherwise
          */
         synchronized boolean containsWatcher(String path, Watcher watcher) {
    -        HashSet<String> paths = watch2Paths.get(watcher);
    -        if (paths == null || !paths.contains(path)) {
    -            return false;
    +        PathIterator pathIterator = new PathIterator(path);
    +        while (pathIterator.hasNext()) {
    --- End diff --
    
    Same here: 
    ```
    for (String localPath : pathIterator) {
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    One of our devs, @alexbrasetvik, asked about ACLs and recursive watches. It turns out this PR is not handling that and needs to. I'll submit support for this soon.


---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93883126
  
    --- Diff: src/java/main/org/apache/zookeeper/ZooKeeper.java ---
    @@ -522,6 +566,15 @@ protected boolean removeWatches(Map<String, Set<Watcher>> pathVsWatcher,
     
                 return result;
             }
    +
    +        private void addPersistentWatches(String clientPath, Set<Watcher> result) {
    +            synchronized (persistentWatches) {
    +                PathIterator pathIterator = new PathIterator(clientPath);
    +                while (pathIterator.hasNext()) {
    --- End diff --
    
    If you change ``PathIterator`` to implement both ``Iterable`` and ``Iterator`` (see below) then this line can be:
    
    ```
    for (String path: pathIterator) {
        addTo(persistentWatches.get(path), result);
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    @stuhood I'll end up writing one for Apache Curator. The short answer is that large ZK users end up with 100s of thousands or millions of watchers when a few dozen would suffice. The use case is watching a tree of znodes. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by skamille <gi...@git.apache.org>.
Github user skamille commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    Questions I have about this from a high level design perspective:
    1. As I asked on the mailing list, have we done load/performance testing or addressed what that might look like in the design? (Jordan to get back to us on that)
    2. I'm not sure I understand why persistent watches are both persistent and always set for all children of a node. Is it not useful to imagine that I would want a persistent watch on some node but not care about its children? Some clarification on that choice would be helpful.
    3. What does it really mean to guarantee sending of all watch events? What are the implications for a disconnected client upon reconnect? How much do we expect ZK to potentially be storing in order to be able to fulfill this guarantee? Will this potentially cause unbounded memory overhead or lead to full GC? Can we realistically bound this guarantee in order to provide the other operational guarantees people expect from ZK such as generally predictable memory usage based on size of data tree?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93885330
  
    --- Diff: 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() ) {
    --- End diff --
    
    If we are going to preserve the ``java.util.Iterator`` contract then this ``if`` condition should have an ``else`` as below:
    
    ```
    else { 
        throw new NoSuchElementException();
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by skamille <gi...@git.apache.org>.
Github user skamille commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    I want to make it clear that I'm not arguing against the feature, I just want us to make sure that we're implementing it in a way that makes sense absent the TreeSet use case, and that we at least document the performance and operational considerations of the feature. This seems important to helping people use it appropriately.
    
    When you had to do the overhead of implementing this in Curator, what did the cost in terms of memory usage/performance to track this across large numbers of nodes?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886440
  
    --- Diff: src/java/test/org/apache/zookeeper/test/PathIteratorTest.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.PathIterator;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +public class PathIteratorTest {
    +    @Test
    +    public void testRoot() {
    +        PathIterator pathIterator = new PathIterator("/");
    +        Assert.assertTrue(pathIterator.hasNext());
    --- End diff --
    
    It's not done elsewhere in the code. So, I was trying to be consistent.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by skamille <gi...@git.apache.org>.
Github user skamille commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    We have to remember that people who don't use TreeCache will still use this feature. Not to say that we shouldn't keep it in mind as an important user, but presumably people who don't actually do anything with curator will decide to use this feature. Does the design make sense absent that consideration? Specifically, if you weren't thinking of this as a feature for TreeCache, would we implement it to automatically watch children changes as well, or would it be broken up into two modes: persistent no children, persistent children.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132024228
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -181,11 +206,18 @@ synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) {
          * @return true if the watcher exists, false otherwise
          */
         synchronized boolean containsWatcher(String path, Watcher watcher) {
    -        HashSet<String> paths = watch2Paths.get(watcher);
    -        if (paths == null || !paths.contains(path)) {
    -            return false;
    +        PathIterator pathIterator = new PathIterator(path);
    +        for (String localPath : pathIterator.asIterable()) {
    +            Map<Watcher, Type> watchers = watchTable.get(localPath);
    +            Type watcherType = (watchers != null) ? watchers.get(watcher) : null;
    +            if ((watcherType == Type.STANDARD) && !pathIterator.atParentPath()) {
    --- End diff --
    
    NIT: I feel this is likely to be more performant if we short circuit on atParentPath. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93883250
  
    --- Diff: src/java/test/org/apache/zookeeper/test/PathIteratorTest.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.PathIterator;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +public class PathIteratorTest {
    +    @Test
    +    public void testRoot() {
    +        PathIterator pathIterator = new PathIterator("/");
    +        Assert.assertTrue(pathIterator.hasNext());
    --- End diff --
    
    nit (really!): we can ``import static Assert.*`` so that we can have only:
    ```
     assertTrue(...);
     assertFalse(...);
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93889832
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -97,23 +102,43 @@ synchronized void removeWatcher(Watcher watcher) {
         Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
             WatchedEvent e = new WatchedEvent(type,
                     KeeperState.SyncConnected, path);
    -        HashSet<Watcher> watchers;
    +        Set<Watcher> 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);
    +            for (String localPath : pathIterator.asIterable()) {
    +                Map<Watcher, Type> thisWatchers = watchTable.get(localPath);
    +                if (thisWatchers == null || thisWatchers.isEmpty()) {
    +                    continue;
                     }
    -                return null;
    -            }
    -            for (Watcher w : watchers) {
    -                HashSet<String> paths = watch2Paths.get(w);
    -                if (paths != null) {
    -                    paths.remove(path);
    +                Iterator<Entry<Watcher, Type>> iterator = thisWatchers.entrySet().iterator();
    +                while (iterator.hasNext()) {
    +                    Entry<Watcher, Type> 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<String> paths = watch2Paths.get(watcher);
    +                        if (paths != null) {
    +                            paths.remove(localPath);
    +                        }
    +                    }
                     }
    +                if (thisWatchers.size() == 0) {
    +                    watchTable.remove(localPath);
    +                }
    +            }
    +        }
    +        if (watchers.size() == 0) {
    --- End diff --
    
    And L#130


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93960121
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -40,47 +40,52 @@
     class WatchManager {
         private static final Logger LOG = LoggerFactory.getLogger(WatchManager.class);
     
    -    private final HashMap<String, HashSet<Watcher>> watchTable =
    -        new HashMap<String, HashSet<Watcher>>();
    +    private enum Type {
    --- End diff --
    
    Sure, np. :+1: 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93883280
  
    --- Diff: 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 {
    --- End diff --
    
    Does it make sense to change this class to be **final** ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132023758
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -97,23 +102,44 @@ synchronized void removeWatcher(Watcher watcher) {
         Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
             WatchedEvent e = new WatchedEvent(type,
                     KeeperState.SyncConnected, path);
    -        HashSet<Watcher> watchers;
    +        Set<Watcher> 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);
    +            for (String localPath : pathIterator.asIterable()) {
    --- End diff --
    
    I'm not sure if we should consider a potential performance impact for tall data trees since we now always iterate all the way up every time a watcher is triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    This patch requires a rebase.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93893162
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -40,47 +40,52 @@
     class WatchManager {
         private static final Logger LOG = LoggerFactory.getLogger(WatchManager.class);
     
    -    private final HashMap<String, HashSet<Watcher>> watchTable =
    -        new HashMap<String, HashSet<Watcher>>();
    +    private enum Type {
    --- End diff --
    
    It's private, I didn't think it mattered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132034154
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -97,23 +102,44 @@ synchronized void removeWatcher(Watcher watcher) {
         Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
             WatchedEvent e = new WatchedEvent(type,
                     KeeperState.SyncConnected, path);
    -        HashSet<Watcher> watchers;
    +        Set<Watcher> 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);
    +            for (String localPath : pathIterator.asIterable()) {
    +                Map<Watcher, Type> thisWatchers = watchTable.get(localPath);
    +                if (thisWatchers == null || thisWatchers.isEmpty()) {
    +                    continue;
                     }
    -                return null;
    -            }
    -            for (Watcher w : watchers) {
    -                HashSet<String> paths = watch2Paths.get(w);
    -                if (paths != null) {
    -                    paths.remove(path);
    +                Iterator<Entry<Watcher, Type>> iterator = thisWatchers.entrySet().iterator();
    +                while (iterator.hasNext()) {
    +                    Entry<Watcher, Type> entry = iterator.next();
    +                    Type entryType = entry.getValue();
    +                    Watcher watcher = entry.getKey();
    +                    if (entryType == Type.PERSISTENT) {
    +                        if ( type != EventType.NodeChildrenChanged ) {
    +                            watchers.add(watcher);
    --- End diff --
    
    Well, other watchers are in a set so I don't think it matters. Sets solve other guarantees of watchers.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132006551
  
    --- Diff: src/java/test/org/apache/zookeeper/test/PersistentWatcherTest.java ---
    @@ -0,0 +1,228 @@
    +/**
    + * 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<WatchedEvent> 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);
    +            internalTestBasic(zk);
    +        } finally {
    +            if (zk != null) {
    +                zk.close();
    --- End diff --
    
    ZooKeeper supports try-with-resources so using that may make this code cleaner


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886801
  
    --- Diff: 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() {
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    As promised - here's the project with the benchmarking stuff:
    
    https://github.com/Randgalt/zkbench


---

[GitHub] zookeeper issue #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on the issue:

    https://github.com/apache/zookeeper/pull/136
  
    @hanm done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by Randgalt <gi...@git.apache.org>.
Github user Randgalt commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132034008
  
    --- Diff: src/java/main/org/apache/zookeeper/ZooDefs.java ---
    @@ -74,12 +74,16 @@
     
             public final int createTTL = 21;
     
    +        public final int addPersistentWatch = 22;
    +
             public final int auth = 100;
     
             public final int setWatches = 101;
     
             public final int sasl = 102;
     
    +        public final int setWatches2 = 103;
    --- End diff --
    
    I plan on backporting this to 3.5.x so I'd vote against that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93883095
  
    --- Diff: 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 {
    --- End diff --
    
    nit: 
    ```
    public class PathIterator implements Iterator<String>, Iterable<String> {
    
    (...)
    
    @Override
    public boolean hasNext() {
    
    (...)
    
    @Override
    public String next() {
    
    (...)
    
    @Override
    public Iterator<String> iterator() {
         return this;
    }
    
    (...)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93886566
  
    --- Diff: 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;
    --- End diff --
    
    Cool. I was extrapolating its current use, but what you find most suitable. I am fine as it is. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r132025725
  
    --- Diff: src/java/main/org/apache/zookeeper/server/WatchManager.java ---
    @@ -97,23 +102,44 @@ synchronized void removeWatcher(Watcher watcher) {
         Set<Watcher> triggerWatch(String path, EventType type, Set<Watcher> supress) {
             WatchedEvent e = new WatchedEvent(type,
                     KeeperState.SyncConnected, path);
    -        HashSet<Watcher> watchers;
    +        Set<Watcher> 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);
    +            for (String localPath : pathIterator.asIterable()) {
    +                Map<Watcher, Type> thisWatchers = watchTable.get(localPath);
    +                if (thisWatchers == null || thisWatchers.isEmpty()) {
    +                    continue;
                     }
    -                return null;
    -            }
    -            for (Watcher w : watchers) {
    -                HashSet<String> paths = watch2Paths.get(w);
    -                if (paths != null) {
    -                    paths.remove(path);
    +                Iterator<Entry<Watcher, Type>> iterator = thisWatchers.entrySet().iterator();
    +                while (iterator.hasNext()) {
    +                    Entry<Watcher, Type> entry = iterator.next();
    +                    Type entryType = entry.getValue();
    +                    Watcher watcher = entry.getKey();
    +                    if (entryType == Type.PERSISTENT) {
    +                        if ( type != EventType.NodeChildrenChanged ) {
    +                            watchers.add(watcher);
    --- End diff --
    
    Are there any guarantees around the order the watchers are fired? Perhaps `watchers` should be a list?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #136: [ZOOKEEPER-1416] Persistent Recursive Watch

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/136#discussion_r93883319
  
    --- Diff: 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() {
    --- End diff --
    
    nit: extra spaces around the ``if`` expressions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---