You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by "xyuanlu (via GitHub)" <gi...@apache.org> on 2023/04/05 06:51:58 UTC

[GitHub] [helix] xyuanlu opened a new pull request, #2432: persist watcher draft

xyuanlu opened a new pull request, #2432:
URL: https://github.com/apache/helix/pull/2432

   ### Issues
   
   - [ ] My PR addresses the following Helix issues and references them in the PR description:
   
   (#200 - Link your issue number here: You can write "Fixes #XXX". Please use the proper keyword so that the issue gets closed automatically. See https://docs.github.com/en/github/managing-your-work-on-github/linking-a-pull-request-to-an-issue
   Any of the following keywords can be used: close, closes, closed, fix, fixes, fixed, resolve, resolves, resolved)
   
   ### Description
   
   - [ ] Here are some details about my PR, including screenshots of any UI changes:
   
   (Write a concise description including what, why, how)
   
   ### Tests
   
   - [ ] The following tests are written for this issue:
   
   (List the names of added unit/integration tests)
   
   - The following is the result of the "mvn test" command on the appropriate module:
   
   (If CI test fails due to known issue, please specify the issue and test PR locally. Then copy & paste the result of "mvn test" to here.)
   
   ### Changes that Break Backward Compatibility (Optional)
   
   - My PR contains changes that break backward compatibility or previous assumptions for certain methods or API. They include:
   
   (Consider including all behavior changes for public methods or API. Also include these changes in merge description so that other developers are aware of these changes. This allows them to make relevant code changes in feature branches accounting for the new method/API behavior.)
   
   ### Documentation (Optional)
   
   - In case of new functionality, my PR adds documentation in the following wiki page:
   
   (Link the GitHub wiki you added)
   
   ### Commits
   
   - My commits all reference appropriate Apache Helix GitHub issues in their subject lines. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Code Quality
   
   - My diff has been formatted using helix-style.xml 
   (helix-style-intellij.xml if IntelliJ IDE is used)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1166074910


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2921,10 +2988,47 @@ private void addChildListener(String path, IZkChildListener listener) {
     listeners.add(listener);
   }
 
+  private void addPersistListener(String path, Object listener) {
+    try {
+      _persistListenerMutex.lockInterruptibly();
+      if (listener instanceof IZkChildListener) {
+        addChildListener(path, (IZkChildListener) listener);
+      } else if (listener instanceof IZkDataListener) {
+        addDataListener(path, (IZkDataListener) listener);
+      }
+    } catch (InterruptedException ex) {
+      throw new ZkInterruptedException(ex);
+    } finally {
+      _persistListenerMutex.unlock();

Review Comment:
   TFTR. Good point. Will update. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1169336181


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -384,6 +405,21 @@ public void unsubscribeStateChanges(
   }
 
   public void unsubscribeAll() {
+    if (_usePersistWatcher) {

Review Comment:
    persist watcher only refers to non recursive data/child watcher. 
   Same as data/child watcher, recursive watcher will has its own registry, we will need to clean that registry as well here, but it won't change current impl. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu merged pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu merged PR #2432:
URL: https://github.com/apache/helix/pull/2432


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1171923003


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run(String path, Object listener) throws KeeperException, InterruptedException;
+  }
+
+  private void addPersistListener(String path, Object listener) {
+    ManipulateListener addListeners = (String, Object) -> {
+      if (listener instanceof IZkChildListener) {
+        addChildListener(path, (IZkChildListener) listener);
+      } else if (listener instanceof IZkDataListener) {
+        addDataListener(path, (IZkDataListener) listener);
+      }
+    };
+    executeWithInPersistListenerMutex(addListeners, path, listener);
+  }
+
+  private void removePersistListener(String path, Object listener) {
+
+    ManipulateListener removeListeners = (String, Object) -> {
+      try {
+        if (listener instanceof IZkChildListener) {
+          removeChildListener(path, (IZkChildListener) listener);
+        } else if (listener instanceof IZkDataListener) {
+          removeDataListener(path, (IZkDataListener) listener);
+        }
+        if (!hasListeners(path)) {
+          // TODO: update hasListeners logic when recursive persist listener is added
+          getConnection().removeWatches(path, this, WatcherType.Any);
+        }
+      } catch (KeeperException.NoWatcherException e) {
+        LOG.warn("Persist watcher is already removed");
+      }
+    };
+
+    executeWithInPersistListenerMutex(removeListeners, path, listener);
+  }
+
+  private void executeWithInPersistListenerMutex(ManipulateListener runnable, String path,
+      Object listener) {
+    try {
+      _persistListenerMutex.lockInterruptibly();
+      runnable.run(path, listener);

Review Comment:
   Updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] qqu0127 commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "qqu0127 (via GitHub)" <gi...@apache.org>.
qqu0127 commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1171749866


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -135,6 +138,8 @@ public class ZkClient implements Watcher {
   private ZkClientMonitor _monitor;
   private boolean _usePersistWatcher;
 
+  private ReentrantLock _persistListenerMutex;

Review Comment:
   nit: can this be final?



##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run() throws KeeperException, InterruptedException;
+  }

Review Comment:
   Do we still need this? Can we simply use Runnable?



##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -1313,7 +1354,7 @@ private void fireAllEvents(WatchedEvent event) {
    * are deleted before the last page is fetched. The upstream caller should be able to handle this.
    */
   public List<String> getChildren(String path) {
-    return getChildren(path, hasListeners(path));
+    return getChildren(path, (hasListeners(path) && !_usePersistWatcher));

Review Comment:
   check `_usePersistWatcher` first, this should save some computation!



##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -1785,7 +1826,7 @@ public void run() throws Exception {
                 }
                 try {
                   // TODO: the data is redundantly read multiple times when multiple listeners exist
-                  data = readData(path, null, true);
+                  data = readData(path, null, false); ///true);

Review Comment:
   cleanup the unused change?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] desaikomal commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "desaikomal (via GitHub)" <gi...@apache.org>.
desaikomal commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1173868733


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -315,7 +315,7 @@ public ConnectState getClientConnectionState() {
   @Override
   public boolean subscribeDataChange(String key, DataChangeListener listener, boolean skipWatchingNonExistNode) {
     _zkClient.subscribeDataChanges(key, new DataListenerAdapter(listener));
-    return false;
+    return true;

Review Comment:
   default "true" return type means that this method should be void.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1171922713


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -1313,7 +1354,7 @@ private void fireAllEvents(WatchedEvent event) {
    * are deleted before the last page is fetched. The upstream caller should be able to handle this.
    */
   public List<String> getChildren(String path) {
-    return getChildren(path, hasListeners(path));
+    return getChildren(path, (hasListeners(path) && !_usePersistWatcher));

Review Comment:
   Good suggestion! Updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1169319084


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run(String path, Object listener) throws KeeperException, InterruptedException;
+  }
+
+  private void addPersistListener(String path, Object listener) {
+    ManipulateListener addListeners = (String, Object) -> {
+      if (listener instanceof IZkChildListener) {
+        addChildListener(path, (IZkChildListener) listener);
+      } else if (listener instanceof IZkDataListener) {
+        addDataListener(path, (IZkDataListener) listener);
+      }
+    };

Review Comment:
   TFTR. Missed that when update signature. Updated. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1171923650


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run() throws KeeperException, InterruptedException;
+  }

Review Comment:
   Thanks for the suggestion. I think Runnable.run don't have exception signature? We need to handle these exceptions explicitly. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on PR #2432:
URL: https://github.com/apache/helix/pull/2432#issuecomment-1522087796

   This change is ready to be merged. Approved by @desaikomal and @qqu0127 // Thanks!
   
   Commit message: Use persist watcher for listener registration in ZkClient 
   Before ZK 3.6, all zk watchers are one time watcher. ZkClient used to resubscribe every time when an event happens.
   This change adopts ZK persist watcher.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1176041872


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run(String path, Object listener) throws KeeperException, InterruptedException;
+  }
+
+  private void addPersistListener(String path, Object listener) {
+    ManipulateListener addListeners = (String, Object) -> {
+      if (listener instanceof IZkChildListener) {
+        addChildListener(path, (IZkChildListener) listener);
+      } else if (listener instanceof IZkDataListener) {
+        addDataListener(path, (IZkDataListener) listener);
+      }
+    };
+    executeWithInPersistListenerMutex(addListeners, path, listener);
+  }
+
+  private void removePersistListener(String path, Object listener) {
+
+    ManipulateListener removeListeners = (String, Object) -> {
+      try {
+        if (listener instanceof IZkChildListener) {
+          removeChildListener(path, (IZkChildListener) listener);
+        } else if (listener instanceof IZkDataListener) {
+          removeDataListener(path, (IZkDataListener) listener);
+        }
+        if (!hasListeners(path)) {
+          // TODO: update hasListeners logic when recursive persist listener is added
+          getConnection().removeWatches(path, this, WatcherType.Any);
+        }
+      } catch (KeeperException.NoWatcherException e) {
+        LOG.warn("Persist watcher is already removed");
+      }
+    };
+
+    executeWithInPersistListenerMutex(removeListeners, path, listener);
+  }
+
+  private void executeWithInPersistListenerMutex(ManipulateListener runnable, String path,
+      Object listener) {

Review Comment:
   Thanks for the suggestion. I will add a todo here and may change in following PR as this change is already large. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] junkaixue commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "junkaixue (via GitHub)" <gi...@apache.org>.
junkaixue commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1166067173


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2921,10 +2988,47 @@ private void addChildListener(String path, IZkChildListener listener) {
     listeners.add(listener);
   }
 
+  private void addPersistListener(String path, Object listener) {
+    try {
+      _persistListenerMutex.lockInterruptibly();
+      if (listener instanceof IZkChildListener) {
+        addChildListener(path, (IZkChildListener) listener);
+      } else if (listener instanceof IZkDataListener) {
+        addDataListener(path, (IZkDataListener) listener);
+      }
+    } catch (InterruptedException ex) {
+      throw new ZkInterruptedException(ex);
+    } finally {
+      _persistListenerMutex.unlock();

Review Comment:
   Would it be possible to make these pieces of code generic?
   
   _mutx.lock();
   passedin method.invoke(xxxx)
   
   catch(e){}
   finally{
   _mutex.unlock();
   }
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] qqu0127 commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "qqu0127 (via GitHub)" <gi...@apache.org>.
qqu0127 commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1169180845


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -384,6 +405,21 @@ public void unsubscribeStateChanges(
   }
 
   public void unsubscribeAll() {
+    if (_usePersistWatcher) {
+      ManipulateListener removeAllListeners = (String, Object) -> {
+        Set<String> paths = new HashSet<>();
+        _childListener.forEach((k, v) -> paths.add(k));
+        _dataListener.forEach((k, v) -> paths.add(k));
+        paths.forEach(p -> {
+          try {
+            getConnection().removeWatches(p, this, WatcherType.Any);
+          } catch (InterruptedException | KeeperException e) {
+            LOG.info("Failed to remove persistent watcher for {} ", p, e);
+          }
+        });
+      };

Review Comment:
   This looks like BiFunciton out of box, https://docs.oracle.com/javase/8/docs/api/java/util/function/BiFunction.html, can we use it by any chance?



##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run(String path, Object listener) throws KeeperException, InterruptedException;
+  }
+
+  private void addPersistListener(String path, Object listener) {
+    ManipulateListener addListeners = (String, Object) -> {
+      if (listener instanceof IZkChildListener) {
+        addChildListener(path, (IZkChildListener) listener);
+      } else if (listener instanceof IZkDataListener) {
+        addDataListener(path, (IZkDataListener) listener);
+      }
+    };
+    executeWithInPersistListenerMutex(addListeners, path, listener);
+  }
+
+  private void removePersistListener(String path, Object listener) {
+
+    ManipulateListener removeListeners = (String, Object) -> {
+      try {
+        if (listener instanceof IZkChildListener) {
+          removeChildListener(path, (IZkChildListener) listener);
+        } else if (listener instanceof IZkDataListener) {
+          removeDataListener(path, (IZkDataListener) listener);
+        }
+        if (!hasListeners(path)) {
+          // TODO: update hasListeners logic when recursive persist listener is added
+          getConnection().removeWatches(path, this, WatcherType.Any);
+        }
+      } catch (KeeperException.NoWatcherException e) {
+        LOG.warn("Persist watcher is already removed");
+      }
+    };
+
+    executeWithInPersistListenerMutex(removeListeners, path, listener);
+  }
+
+  private void executeWithInPersistListenerMutex(ManipulateListener runnable, String path,
+      Object listener) {

Review Comment:
   Not an issue, but thought on improving the raw Object class type: 
   what if we create an empty interface and let the two listeners interface extend that as marker interface? It only looks slightly better.



##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run(String path, Object listener) throws KeeperException, InterruptedException;
+  }
+
+  private void addPersistListener(String path, Object listener) {
+    ManipulateListener addListeners = (String, Object) -> {
+      if (listener instanceof IZkChildListener) {
+        addChildListener(path, (IZkChildListener) listener);
+      } else if (listener instanceof IZkDataListener) {
+        addDataListener(path, (IZkDataListener) listener);
+      }
+    };
+    executeWithInPersistListenerMutex(addListeners, path, listener);
+  }
+
+  private void removePersistListener(String path, Object listener) {
+
+    ManipulateListener removeListeners = (String, Object) -> {
+      try {
+        if (listener instanceof IZkChildListener) {
+          removeChildListener(path, (IZkChildListener) listener);
+        } else if (listener instanceof IZkDataListener) {
+          removeDataListener(path, (IZkDataListener) listener);
+        }
+        if (!hasListeners(path)) {
+          // TODO: update hasListeners logic when recursive persist listener is added
+          getConnection().removeWatches(path, this, WatcherType.Any);
+        }
+      } catch (KeeperException.NoWatcherException e) {
+        LOG.warn("Persist watcher is already removed");
+      }
+    };
+
+    executeWithInPersistListenerMutex(removeListeners, path, listener);
+  }
+
+  private void executeWithInPersistListenerMutex(ManipulateListener runnable, String path,
+      Object listener) {
+    try {
+      _persistListenerMutex.lockInterruptibly();
+      runnable.run(path, listener);

Review Comment:
   Similar concern as brought up above. Correct me if I'm wrong, I'm afraid the input (path, listener) here isn't actually being used.



##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -384,6 +405,21 @@ public void unsubscribeStateChanges(
   }
 
   public void unsubscribeAll() {
+    if (_usePersistWatcher) {
+      ManipulateListener removeAllListeners = (String, Object) -> {
+        Set<String> paths = new HashSet<>();
+        _childListener.forEach((k, v) -> paths.add(k));
+        _dataListener.forEach((k, v) -> paths.add(k));
+        paths.forEach(p -> {
+          try {
+            getConnection().removeWatches(p, this, WatcherType.Any);
+          } catch (InterruptedException | KeeperException e) {
+            LOG.info("Failed to remove persistent watcher for {} ", p, e);
+          }
+        });
+      };
+      executeWithInPersistListenerMutex(removeAllListeners, null, null);
+    } else {
     synchronized (_childListener) {
       _childListener.clear();

Review Comment:
   nit: you can just return after the first "if", so no need to do "else". Same for below.



##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run(String path, Object listener) throws KeeperException, InterruptedException;
+  }
+
+  private void addPersistListener(String path, Object listener) {
+    ManipulateListener addListeners = (String, Object) -> {
+      if (listener instanceof IZkChildListener) {
+        addChildListener(path, (IZkChildListener) listener);
+      } else if (listener instanceof IZkDataListener) {
+        addDataListener(path, (IZkDataListener) listener);
+      }
+    };

Review Comment:
   Hmmm, I'm so confused by the syntax.
   On one hand you are defining the lambda expression with two inputs, on the other hand, the ManipulateListener defined here doesn't actually use the input. What's the point of using such bi-function?
   (Plus, why use capital "String" as input name? It also confuses with the class name.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1172814829


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run() throws KeeperException, InterruptedException;
+  }

Review Comment:
   Yea already updated. :D 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1172814829


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run() throws KeeperException, InterruptedException;
+  }

Review Comment:
   Yea already removed generic in most recent commit. :D 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] junkaixue commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "junkaixue (via GitHub)" <gi...@apache.org>.
junkaixue commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1173131258


##########
meta-client/src/test/java/org/apache/helix/metaclient/impl/zk/TestUtil.java:
##########
@@ -0,0 +1,97 @@
+package org.apache.helix.metaclient.impl.zk;
+
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.zkclient.ZkClient;
+import org.apache.helix.zookeeper.zkclient.ZkConnection;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+
+
+public class TestUtil {
+
+  static java.lang.reflect.Field getField(Class clazz, String fieldName)
+      throws NoSuchFieldException {
+    try {
+      return clazz.getDeclaredField(fieldName);
+    } catch (NoSuchFieldException e) {
+      Class superClass = clazz.getSuperclass();
+      if (superClass == null) {
+        throw e;
+      } else {
+        return getField(superClass, fieldName);
+      }

Review Comment:
   No need for else clause.
   
   It could be:
   if (superClass == null) {
           throw e;
    }
   return getField(superClass, fieldName);
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] desaikomal commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "desaikomal (via GitHub)" <gi...@apache.org>.
desaikomal commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1169326565


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -384,6 +405,21 @@ public void unsubscribeStateChanges(
   }
 
   public void unsubscribeAll() {
+    if (_usePersistWatcher) {

Review Comment:
   I am getting a bit confused. if we have recursive watcher and persist watcher, this code will change, correct?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "xyuanlu (via GitHub)" <gi...@apache.org>.
xyuanlu commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1176038509


##########
meta-client/src/main/java/org/apache/helix/metaclient/impl/zk/ZkMetaClient.java:
##########
@@ -315,7 +315,7 @@ public ConnectState getClientConnectionState() {
   @Override
   public boolean subscribeDataChange(String key, DataChangeListener listener, boolean skipWatchingNonExistNode) {
     _zkClient.subscribeDataChanges(key, new DataListenerAdapter(listener));
-    return false;
+    return true;

Review Comment:
   Good point. Will do interface change in following PR. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] qqu0127 commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "qqu0127 (via GitHub)" <gi...@apache.org>.
qqu0127 commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1172014241


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -2927,4 +2990,54 @@ private void removeChildListener(String path, IZkChildListener listener) {
       listeners.remove(listener);
     }
   }
+
+  interface ManipulateListener<T> {
+    void run() throws KeeperException, InterruptedException;
+  }

Review Comment:
   Ok, that's a valid point. 
   We can still use Callable with null return, to remove this wrapper. It's up to you.
   (If you decide to stay with this interface, let's remove the generic, it's not used anyway)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] qqu0127 commented on a diff in pull request #2432: Use persist watcher for listener registration in ZkClient (when configured)

Posted by "qqu0127 (via GitHub)" <gi...@apache.org>.
qqu0127 commented on code in PR #2432:
URL: https://github.com/apache/helix/pull/2432#discussion_r1172859057


##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -384,6 +405,21 @@ public void unsubscribeStateChanges(
   }
 
   public void unsubscribeAll() {
+    if (_usePersistWatcher) {
+      ManipulateListener removeAllListeners = () -> {
+        Set<String> paths = new HashSet<>();
+        _childListener.forEach((k, v) -> paths.add(k));
+        _dataListener.forEach((k, v) -> paths.add(k));

Review Comment:
   This may have some memory overhead.
   Will this be better with Stream.concat(s1.stream(), s2.stream()).forEach()?



##########
zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java:
##########
@@ -384,6 +405,21 @@ public void unsubscribeStateChanges(
   }
 
   public void unsubscribeAll() {
+    if (_usePersistWatcher) {
+      ManipulateListener removeAllListeners = () -> {
+        Set<String> paths = new HashSet<>();
+        _childListener.forEach((k, v) -> paths.add(k));
+        _dataListener.forEach((k, v) -> paths.add(k));
+        paths.forEach(p -> {
+          try {
+            getConnection().removeWatches(p, this, WatcherType.Any);
+          } catch (InterruptedException | KeeperException e) {
+            LOG.info("Failed to remove persistent watcher for {} ", p, e);

Review Comment:
   This should be error



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org