You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by GitBox <gi...@apache.org> on 2020/03/31 19:07:13 UTC

[GitHub] [helix] kaisun2000 opened a new pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

kaisun2000 opened a new pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924
 
 
   ### Issues
   
   - [x] My PR addresses the following Helix issues and references them in the PR description:
   
   Fixes #921 
   
   ### Description
   
   - [x] Here are some details about my PR, including screenshots of any UI changes:
   
       ZkHelixPropertyStore loses ZK notification after session expires.
       The issue was caused by a bug in Share ZkClient code path. More
       specifically, Share ZkClient would not call fireAllEvent when ZK
       session expires. Thus, ZkHelixPropertyStore would not install
       watches for corresponding ZkPath. Thus, lose Zookeeper
       nofiticaition when changes happens.
   
   ### Tests
   
   - [x] The following tests are written for this issue:
   
   testSessionExpirationWithSharedZkClient
   
   - [ ] The following is the result of the "mvn test" command on the appropriate module:
   
   Test still running, while attach the result here later, please review first.
   
   ### Commits
   
   - [x] 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
   
   - [x] 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r406389934
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1080,22 +1080,21 @@ protected void processStateChanged(WatchedEvent event) {
      */
     if (event.getState() == KeeperState.SyncConnected) {
       if (!_isNewSessionEventFired && !"0".equals(getHexSessionId())) {
-        if (isManagingZkConnection()) {
-          /*
-           * Before the new zookeeper instance is connected to the zookeeper service and its session
-           * is established, its session id is 0.
-           * New session event is not fired until the new zookeeper session receives the first
-           * SyncConnected state(the zookeeper session is established).
-           * Now the session id is available and non-zero, and we can fire new session events.
-           */
-          fireNewSessionEvents();
-        }
         /*
          * Set it true to avoid firing events again for the same session next time
          * when SyncConnected events are received.
          */
         _isNewSessionEventFired = true;
 
+        /*
+         * Before the new zookeeper instance is connected to the zookeeper service and its session
+         * is established, its session id is 0.
+         * New session event is not fired until the new zookeeper session receives the first
+         * SyncConnected state(the zookeeper session is established).
+         * Now the session id is available and non-zero, and we can fire new session events.
+         */
+        fireNewSessionEvents();
 
 Review comment:
   I changed it back to the original way. The previous changing order should also work. But to be on the safe side, as you commented, let us change it back to original way.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401182893
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
       return;
 
 Review comment:
   Strictly speaking, this is not from sub-class. This is one object of a class that can act on different roles. But I get what you are thinking.
   
   That said, traditionally it is like this. 
   
   I can write in an another concise way wrapping all this logic into the if block of 1093. 
   
   Thought about it. The drawback of that way would be losing readability, and thus the intention of the logic. This would pave the way of next hidden bug when people do refactoring later, just as how this bug is introduced.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405970789
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,83 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
 
 Review comment:
   Style?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405741651
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
 
 Review comment:
   Discussed wit Kai offline, this should be fine as long as processStateChanged is synchronized called in one thread.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on issue #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on issue #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#issuecomment-613685365
 
 
   I did run another round of mvn test; same results. 4 flaking test. Running independently they succeed. The same as before this fix and added test,

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405970457
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,99 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    final long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(()-> {
 
 Review comment:
   I've seen multiple places not formatted. Can you format your changes here?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405884477
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
+      if (isManagingZkConnection()) {
 
 Review comment:
   done. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401167168
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
+    // kill the session one more time to cover Shared ZkClient resetting flag
+    // indicating first time synconnect happened.
+    ZkTestHelper.expireSession(testClient);
+
+    listener.reset();
+    int expectDeleteNodes = 1 + firstLevelNr + firstLevelNr * secondLevelNr;
+    store.remove("/", 0);
+    // wait until all callbacks have been received
+    for (int i = 0; i < 10; i++) {
+      if (listener._deleteKeys.size() == expectDeleteNodes)
 
 Review comment:
   Style:
   ```
   if () {
   }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405742497
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
 
 Review comment:
   BTW, please update the comment of _isNewSessionEventFired, it is out of date.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401928444
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
 
 Review comment:
   You are right, here this latch guarantees expiring happens. But later, the code still ensures new session established.
   
   `     waitExpireSession.await();
       zkClient.unsubscribeStateChanges(listener);
   
       connection = (ZkConnection) zkClient.getConnection();
       curZookeeper = connection.getZookeeper();
   
       // wait util connected
       while (curZookeeper.getState() != States.CONNECTED) {
         Thread.sleep(10);
       } `

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401188555
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/store/zk/ZkHelixPropertyStore.java
 ##########
 @@ -47,4 +48,9 @@ public ZkHelixPropertyStore(String zkAddress, ZkSerializer serializer, String ch
       ZkBaseDataAccessor.ZkClientType zkClientType) {
     super(zkAddress, serializer, chrootPath, null, null, MONITOR_TYPE, chrootPath, zkClientType);
   }
+
+  // test only
+  protected HelixZkClient getTestZkClient() {
+    return getZkClient();
+  }
 
 Review comment:
   We have another thread of this discussion in this pull. Close this one

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401183078
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
 ##########
 @@ -72,6 +72,11 @@
 
   private HelixZkClient _zkClient = null;
 
+  // test only
+  public HelixZkClient getZkClient() {
 
 Review comment:
   +1, we should not add a public method to a public class just for testing, specially this is a public API.  If you test class is sitting in the same module with this class, protected will work.  Another way is to make it protected, and wrap ZkCacheBaseDataAccessor to a new test class. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r406001302
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1080,22 +1080,21 @@ protected void processStateChanged(WatchedEvent event) {
      */
     if (event.getState() == KeeperState.SyncConnected) {
       if (!_isNewSessionEventFired && !"0".equals(getHexSessionId())) {
-        if (isManagingZkConnection()) {
-          /*
-           * Before the new zookeeper instance is connected to the zookeeper service and its session
-           * is established, its session id is 0.
-           * New session event is not fired until the new zookeeper session receives the first
-           * SyncConnected state(the zookeeper session is established).
-           * Now the session id is available and non-zero, and we can fire new session events.
-           */
-          fireNewSessionEvents();
-        }
         /*
          * Set it true to avoid firing events again for the same session next time
          * when SyncConnected events are received.
          */
         _isNewSessionEventFired = true;
 
+        /*
+         * Before the new zookeeper instance is connected to the zookeeper service and its session
+         * is established, its session id is 0.
+         * New session event is not fired until the new zookeeper session receives the first
+         * SyncConnected state(the zookeeper session is established).
+         * Now the session id is available and non-zero, and we can fire new session events.
+         */
+        fireNewSessionEvents();
 
 Review comment:
   Thanks for put the check inside the function. But why we need to move it to after _isNewSessionEventFired = true;?
   Is it concerning? Because when _isNewSessionEventFired is set to true, the event is not sent yet.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402028968
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for ZkClient no managing connection
     if (!isManagingZkConnection()) {
+      /*
+       * For SharedZkClient, we will not invoke fireNewSessionEvents but will invoke fireAllEvents
+       * This keeps the same behavior as original SharedZkClient.
+       * Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+       */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
       return;
     }
 
+    // The following is the case for ZkClient managing the connection
 
 Review comment:
   This comment is a bit confusing.  You mean "The following only applies to dedicated ZkClient in which it manages ZkConnection directly", right?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401176227
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
 
 Review comment:
   If we remove ` _isNewSessionEventFired = false;` in `reconnect()` and do below code, it seems more concise and easier to maintain. `_isNewSessionEventFired` is only changed in this code block. What do you think?
   
   ```java
       if (event.getState() == KeeperState.SyncConnected && !_isNewSessionEventFired && !"0".equals(getHexSessionId())) {
         if (isManagingZkConnection()) {
           /*
            * Before the new zookeeper instance is connected to the zookeeper service and its session
            * is established, its session id is 0.
            * New session event is not fired until the new zookeeper session receives the first
            * SyncConnected state(the zookeeper session is established).
            * Now the session id is available and non-zero, and we can fire new session events.
            */
           fireNewSessionEvents();
         }
         /*
          * Set it true to avoid firing events again for the same session next time
          * when SyncConnected events are received.
          */
         _isNewSessionEventFired = true;
   
         /*
          * With this first SyncConnected state, we just get connected to zookeeper service after
          * reconnecting when the session expired. Because previous session expired, we also have to
          * notify all listeners that something might have changed.
          */
         fireAllEvents();
       } else if (event.getState() == KeeperState.Expired) {
         if (isManagingZkConnection()) {
           reconnectOnExpiring();
         }
          _isNewSessionEventFired = false;
       }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 edited a comment on issue #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 edited a comment on issue #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#issuecomment-606938078
 
 
   @jiajunwang 
   
   > I see the main issue is about the additional public method for fetching the client. I don't like that either.
   > An alternative way is that, since you are for sure given a shared zkclient in that property store, you can call ShareZkClientFactory with the same parameters and it will return you with another shared ZkClient instance. But, since they will use the same connection manager, when you expires the newly created client, the one that in the HelixPropertyStore will also be expired for once.
   > Can you have a try? In this way, no new method is required.
   
   How do we get the zkconnection manager from the new sharedZkClient? It is not public. 
   
   If we make zkConnectionManager protected, we need to access it from the module in zookeeper-api, which means move (rewrite) this test case there. But this case the test case uses the code from HelixPropertyStore, which is helix-core. This is circular dependency.
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 opened a new pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 opened a new pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924
 
 
   ### Issues
   
   - [x] My PR addresses the following Helix issues and references them in the PR description:
   
   Fixes #921 
   
   ### Description
   
   - [x] Here are some details about my PR, including screenshots of any UI changes:
   
       ZkHelixPropertyStore loses ZK notification after session expires.
       The issue was caused by a bug in Share ZkClient code path. More
       specifically, Share ZkClient would not call fireAllEvent when ZK
       session expires. Thus, ZkHelixPropertyStore would not install
       watches for corresponding ZkPath. Thus, lose Zookeeper
       nofiticaition when changes happens.
   
   ### Tests
   
   - [x] The following tests are written for this issue:
   
   testSessionExpirationWithSharedZkClient
   
   - [x] The following is the result of the "mvn test" command on the appropriate module:
   
   ksun-mn1:helix-core ksun$ mvn test
   
   Failed tests: 
     TestZkConnectionLost.testLostZkConnection » ThreadTimeout Method org.testng.in...
     TestJobFailureDependence.testWorkflowFailureJobThreshold » ThreadTimeout Metho...
     TestHelixAdminCli.testInstanceOperations:469 » ZkClient Failed to delete /Test...
   
   Tests run: 1093, Failures: 3, Errors: 0, Skipped: 4
   
   [INFO] ------------------------------------------------------------------------
   [INFO] BUILD FAILURE
   [INFO] ------------------------------------------------------------------------
   [INFO] Total time:  01:37 h
   [INFO] Finished at: 2020-04-06T18:55:19-07:00
   [INFO] ------------------------------------------------------------------------
   [ERROR] Failed to execute goal org.apache.maven.plugins:maven-surefire-plugin:2.18.1:test (default-test) on project helix-core: There are test failures.
   [ERROR] 
   [ERROR] Please refer to /Users/ksun/dev_branch_helix/helix/helix-core/target/surefire-reports for the individual test results.
   [ERROR] -> [Help 1]
   [ERROR] 
   [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.
   [ERROR] Re-run Maven using the -X switch to enable full debug logging.
   [ERROR] 
   [ERROR] For more information about the errors and possible solutions, please read the following articles:
   [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException
   
   Individual test case run will all succeed.
   ### Commits
   
   - [x] 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
   
   - [x] 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405971820
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,11 +1103,16 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
+      _isNewSessionEventFired = false;
       reconnectOnExpiring();
     }
   }
 
   private void reconnectOnExpiring() {
+    // only managing zkclient fire handleNewSession event
+    if (isManagingZkConnection()) {
 
 Review comment:
   Reversed.
   And please correct the comment.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on issue #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on issue #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#issuecomment-606942634
 
 
   > @jiajunwang
   > 
   > > I see the main issue is about the additional public method for fetching the client. I don't like that either.
   > > An alternative way is that, since you are for sure given a shared zkclient in that property store, you can call ShareZkClientFactory with the same parameters and it will return you with another shared ZkClient instance. But, since they will use the same connection manager, when you expires the newly created client, the one that in the HelixPropertyStore will also be expired for once.
   > > Can you have a try? In this way, no new method is required.
   > 
   > How do we get the zkconnection manager from the new sharedZkClient? It is not public.
   > 
   > If we make zkConnectionManager protected, we need to access it from the module in zookeeper-api, which means move (rewrite) this test case there. But this case use the code from HelixPropertyStore, which is helix-core. This is circular dependency.
   
   Why you need to get zkconnection manager? The object you operate on is the send shared zkclient that you get from the factory.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405866648
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,100 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    while (true) {
+      long curSessionId = dupZkclient.getSessionId();
+      if (curSessionId == sessionId || curSessionId == 0) {
+        Thread.sleep(500);
+      } else {
+        sessionId = curSessionId;
+        break;
 
 Review comment:
   fixed. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401407845
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
+    // kill the session one more time to cover Shared ZkClient resetting flag
+    // indicating first time synconnect happened.
+    ZkTestHelper.expireSession(testClient);
+
+    listener.reset();
+    int expectDeleteNodes = 1 + firstLevelNr + firstLevelNr * secondLevelNr;
+    store.remove("/", 0);
+    // wait until all callbacks have been received
+    for (int i = 0; i < 10; i++) {
+      if (listener._deleteKeys.size() == expectDeleteNodes)
+        break;
+      Thread.sleep(500);
 
 Review comment:
   New approach use TestHelper.verifyWithTimeout().

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401410875
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
 
 Review comment:
   @jiajunwang, what is your take here? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401175988
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
+    // kill the session one more time to cover Shared ZkClient resetting flag
+    // indicating first time synconnect happened.
+    ZkTestHelper.expireSession(testClient);
+
+    listener.reset();
+    int expectDeleteNodes = 1 + firstLevelNr + firstLevelNr * secondLevelNr;
+    store.remove("/", 0);
+    // wait until all callbacks have been received
+    for (int i = 0; i < 10; i++) {
+      if (listener._deleteKeys.size() == expectDeleteNodes)
+        break;
+      Thread.sleep(500);
 
 Review comment:
   This is copied from the other test in this file. I know you guys will pick it up. Let us draw a line here. I will only change this one, not the other test for now. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401155689
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
+    // kill the session one more time to cover Shared ZkClient resetting flag
+    // indicating first time synconnect happened.
+    ZkTestHelper.expireSession(testClient);
+
+    listener.reset();
+    int expectDeleteNodes = 1 + firstLevelNr + firstLevelNr * secondLevelNr;
+    store.remove("/", 0);
+    // wait until all callbacks have been received
+    for (int i = 0; i < 10; i++) {
+      if (listener._deleteKeys.size() == expectDeleteNodes)
+        break;
+      Thread.sleep(500);
 
 Review comment:
   `sleep()` doesn't look solid and makes the test unstable. I suggest using `TestHelper.varify()` to replace sleep().

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401187909
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java
 ##########
 @@ -112,4 +112,9 @@ public String create(final String path, Object datat, final List<ACL> acl,
   protected boolean isManagingZkConnection() {
     return false;
   }
+
+  // test only
+  public HelixZkClient getConnectionManager() {
+    return _connectionManager;
+  }
 
 Review comment:
   Another way is, if your test class is put in the same module with sharedZkClient, such as org.apache.helix.zookeeper.impl.client,  you should be able to access protected instance variables or methods from the test code.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401167566
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
 
 Review comment:
   Extra spaces.
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401928444
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
 
 Review comment:
   You are right, here this latch guarantees expiring happens. But later, the code still ensures new session established.
   
       waitExpireSession.await();
       zkClient.unsubscribeStateChanges(listener);
   
       connection = (ZkConnection) zkClient.getConnection();
       curZookeeper = connection.getZookeeper();
   
       // wait util connected
       while (curZookeeper.getState() != States.CONNECTED) {
         Thread.sleep(10);
       } 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402537367
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
 
 Review comment:
   By they way, setting _isNewSessionEventFired = false should be above reconnect(). Otherwise, maybe there is a small chance of race condition. In reconnect() or current code, maybe we still have this race condition. Put it ahead definitely don't.
   ```
       } else if (event.getState() == KeeperState.Expired) {
          _isNewSessionEventFired = false;
         if (isManagingZkConnection()) {
           reconnectOnExpiring();
         }
       }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402529790
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,83 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
+
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    ZkTestHelper.expireSharedZkClientSession(dupZkclient);
+
+    // kill the session one more time to cover code path ZkClient resetting flag that
+    // indicates first time synconnect happened.
+    ZkTestHelper.expireSharedZkClientSession(dupZkclient);
+
+    // remove the currentstates
+    paths.clear();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      paths.add(path);
+    }
+    success = extBaseAccessor.remove(paths, 0);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in remove:" + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
 
 Review comment:
   See above.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401173555
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java
 ##########
 @@ -112,4 +112,9 @@ public String create(final String path, Object datat, final List<ACL> acl,
   protected boolean isManagingZkConnection() {
     return false;
   }
+
+  // test only
+  public HelixZkClient getConnectionManager() {
+    return _connectionManager;
+  }
 
 Review comment:
   Can you avoid adding public methods like this that shouldn't otherwise be used in production?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402537367
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
 
 Review comment:
   By they way, setting _isNewSessionEventFired = false should be above reconnect(). Otherwise, there is a small chance of race condition. In reconnect() or current code, we still have this race condition.
   ```
       } else if (event.getState() == KeeperState.Expired) {
          _isNewSessionEventFired = false;
         if (isManagingZkConnection()) {
           reconnectOnExpiring();
         }
       }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401918194
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
 
 Review comment:
   I guess this wait only check if there is an expiring happened. The original wait is to ensure the session has been re-established. So if you want to achieve the same goal, maybe we need to compare the older session and newer session then end the wait.
   Given that saying, I guess you can just modify the existing method to include this check. Note that if this not happen, we can directly fail the test by throwing an exception.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401172861
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/store/zk/ZkHelixPropertyStore.java
 ##########
 @@ -47,4 +48,9 @@ public ZkHelixPropertyStore(String zkAddress, ZkSerializer serializer, String ch
       ZkBaseDataAccessor.ZkClientType zkClientType) {
     super(zkAddress, serializer, chrootPath, null, null, MONITOR_TYPE, chrootPath, zkClientType);
   }
+
+  // test only
+  protected HelixZkClient getTestZkClient() {
+    return getZkClient();
+  }
 
 Review comment:
   Exposing the internal HelixZkClient is not desirable

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402030011
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
 
 Review comment:
   This work. Since it should work for all the client, can we just modify the original expiring method? 2 similar methods for the same purpose is not good.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401173312
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
+    // kill the session one more time to cover Shared ZkClient resetting flag
+    // indicating first time synconnect happened.
+    ZkTestHelper.expireSession(testClient);
+
+    listener.reset();
+    int expectDeleteNodes = 1 + firstLevelNr + firstLevelNr * secondLevelNr;
+    store.remove("/", 0);
+    // wait until all callbacks have been received
+    for (int i = 0; i < 10; i++) {
+      if (listener._deleteKeys.size() == expectDeleteNodes)
+        break;
+      Thread.sleep(500);
+    }
+
+    System.out.println("createKey#:" + listener._createKeys.size() + ", changeKey#:"
+        + listener._changeKeys.size() + ", deleteKey#:" + listener._deleteKeys.size());
 
 Review comment:
   Possible to avoid print statements?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402042848
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for ZkClient no managing connection
     if (!isManagingZkConnection()) {
+      /*
+       * For SharedZkClient, we will not invoke fireNewSessionEvents but will invoke fireAllEvents
+       * This keeps the same behavior as original SharedZkClient.
+       * Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+       */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
       return;
     }
 
+    // The following is the case for ZkClient managing the connection
 
 Review comment:
   @lei-xia, right. Your description is more readable. But @jiajunwang and @hulu like the more concise way. Unless you like this separation of two branch way, I will adopt the more concise way. And add another comment about the goal fo what we will achieve. (Intention)
   
   So later if people refactor this code. They know the intention at lease. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402031952
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for ZkClient no managing connection
     if (!isManagingZkConnection()) {
+      /*
+       * For SharedZkClient, we will not invoke fireNewSessionEvents but will invoke fireAllEvents
+       * This keeps the same behavior as original SharedZkClient.
+       * Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+       */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
       return;
     }
 
+    // The following is the case for ZkClient managing the connection
 
 Review comment:
   I guess it can be simplified to:
   // else, if managing the connection
   
   Anyway, if we do what Huizhi suggested, it should be clearer.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401205633
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
 
 Review comment:
   I applied the style file. And it did not pick this up. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402147083
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
 
 Review comment:
   Naming *Shared*ZkClient doesn’t match the parameter type HelixZkClient.
   
   FYI, it seems the only difference between expireSession and this is, this does not have handleNewSession implementation logic. Would you consider reducing the duplicate code?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401172569
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
 ##########
 @@ -72,6 +72,11 @@
 
   private HelixZkClient _zkClient = null;
 
+  // test only
+  public HelixZkClient getZkClient() {
 
 Review comment:
   This should *not* be introduced.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405883107
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
+      if (isManagingZkConnection()) {
 
 Review comment:
   Yeah, this is what I meant.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401178796
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/store/zk/ZkHelixPropertyStore.java
 ##########
 @@ -47,4 +48,9 @@ public ZkHelixPropertyStore(String zkAddress, ZkSerializer serializer, String ch
       ZkBaseDataAccessor.ZkClientType zkClientType) {
     super(zkAddress, serializer, chrootPath, null, null, MONITOR_TYPE, chrootPath, zkClientType);
   }
+
+  // test only
+  protected HelixZkClient getTestZkClient() {
+    return getZkClient();
+  }
 
 Review comment:
   Java has not way to access otherwise private members in the class. In this case, we have to expire the session inside this class. This is a common problem I did not see addressed in our code base. Unless you guys can identify another pattern, I guess we have to leave it this way.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401269062
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
 
 Review comment:
   To me, it is easier to understand:
   1. only managed connection will do: fireNewSessionEvents() and reconnectOnExpiring.
   2. first sync connected: _isNewSessionEventFired = true; expired: _isNewSessionEventFired = false.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r408327499
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,93 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder
+          .instanceCurrentState(clusterName, "localhost_8901", "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000, zkCacheInitPaths, accessor._zkCache._cache,
+        _gZkClient, true);
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    final long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(() -> {
+      long curSessionId = dupZkclient.getSessionId();
+      return curSessionId != sessionId && curSessionId != 0 ? true : false;
 
 Review comment:
   addressed.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r408327321
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,93 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
 
 Review comment:
   addressed.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402031507
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
 
 Review comment:
   I like less code : )
   
   And actually I can understand the concise version. So I guess it's better.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402030316
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
+        }
+      }
+
+      @Override
+      public void handleNewSession(final String sessionId) throws Exception {
+      }
+
+      @Override
+      public void handleSessionEstablishmentError(Throwable var1) throws Exception {
+      }
+    };
+
+    zkClient.subscribeStateChanges(listener);
+
+    ZkConnection connection = ((ZkConnection) zkClient.getConnection());
+    ZooKeeper curZookeeper = connection.getZookeeper();
+    String oldSessionId = Long.toHexString(curZookeeper.getSessionId());
+    LOG.info("Before session expiry. sessionId: " + oldSessionId + ", zk: " + curZookeeper);
+
+    Watcher watcher = new Watcher() {
+      @Override
+      public void process(WatchedEvent event) {
+        LOG.info("Watcher#process, event: " + event);
+      }
+    };
+
+    final ZooKeeper dupZookeeper =
+        new ZooKeeper(connection.getServers(), curZookeeper.getSessionTimeout(), watcher,
+            curZookeeper.getSessionId(), curZookeeper.getSessionPasswd());
+    // wait until connected, then close
+    while (dupZookeeper.getState() != States.CONNECTED) {
+      Thread.sleep(10);
+    }
+    Assert.assertEquals(dupZookeeper.getState(), States.CONNECTED,
+        "Fail to connect to zk using current session info");
+    dupZookeeper.close();
+
+    // make sure session expiry really happens
+    waitExpireSession.await();
+    zkClient.unsubscribeStateChanges(listener);
+
+    connection = (ZkConnection) zkClient.getConnection();
+    curZookeeper = connection.getZookeeper();
+
+    // wait util connected
+    while (curZookeeper.getState() != States.CONNECTED) {
 
 Review comment:
   Can we subscribe to the state change instead of busy loop?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405880693
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
+      if (isManagingZkConnection()) {
 
 Review comment:
   Can you be more specific as what you want to do with the change? Sketch how you think about the fireNewSessionEvent, fireAllEvent, reconnectOnExpiring look like? 
   
   To me, there are many ways to put the if and else check. Either way works for me.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on issue #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on issue #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#issuecomment-612336474
 
 
   @kaisun2000, please follow the merge steps to finish this PR merge process if you think it is good enough.
   
   @pkuwm, please take another look to ensure this looks good to you.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401188387
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
 
 Review comment:
   I think we have the format style file? You can apply the style file in intellij.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on issue #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on issue #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#issuecomment-612344941
 
 
   This PR is ready to be merged, approved by @jiajunwang 
   
   Final message:
       fix ZkHelixPropertyStore loses Zookeeper notification issue
       
       ZkHelixPropertyStore loses ZK notification after session expires.
       THe issue was caused by a bug in Share ZkClient code path. More
       specifically, Share ZkClient would not call fireAllEvent when ZK
       session expires. Thus, ZkHelixPropertyStore would not install
       watches for corresponding ZkPath. Thus, lose Zookeeper
       nofiticaition when changes happens.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401247502
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java
 ##########
 @@ -112,4 +112,9 @@ public String create(final String path, Object datat, final List<ACL> acl,
   protected boolean isManagingZkConnection() {
     return false;
   }
+
+  // test only
+  public HelixZkClient getConnectionManager() {
+    return _connectionManager;
+  }
 
 Review comment:
   
   Here is the case:
   
   test case is in package org.apache.helix.store.zk, here is the how it is constructed.
   
   `    ZkSerializer serializer = new ZNRecordSerializer();
       ZkHelixPropertyStore<ZNRecord> store =
           new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);`
   
   We need the internal shared ZkClient in the store; and further the internal _ZkconnectionManager.
   
   Note, ZkHelixPropertyStore uses ZkCacheBasedDataAccessor which lives in package org.apache.helix.manager.zk;
   
   Note, ShareZkClient and ZkConnectionManager lives in package org.apache.helix.zookeeper.impl.client;
   
   So the solution (in this case, but not in general) can be:
   
   I can put the test in the org.apache.helix.zookeeper.impl.client, zookeeper-api module
   
   I can also initialize the HelixPropertystore like this:
   
   1/ construct SharedZkClient 
   
   2/ construct ZkBaseDataAccessor using this sharedZkCLient.
   
   3/ construct ZkHelixPropertyStore (using ZkBaseDataAccessor.
   
   by making SharedZkClient's _zkConnectionManager protected, we can use it here.  
   
   The problem is that org.apache.helix.zookeeper.impl.client is in Zookeeper-api, we need a test scope dependency to Helix-core and Helix-common module. Does this test dependency work here?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405728245
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
 
 Review comment:
   What if reconnect is not done successfully in the following reconnectOnExpiring() call?
   In that case, if we change this flag to false, will it cause additional fireNewSessionEvents()? For example if we have a stale new session event in the queue.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401178552
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
+    // kill the session one more time to cover Shared ZkClient resetting flag
+    // indicating first time synconnect happened.
+    ZkTestHelper.expireSession(testClient);
+
+    listener.reset();
+    int expectDeleteNodes = 1 + firstLevelNr + firstLevelNr * secondLevelNr;
+    store.remove("/", 0);
+    // wait until all callbacks have been received
+    for (int i = 0; i < 10; i++) {
+      if (listener._deleteKeys.size() == expectDeleteNodes)
+        break;
+      Thread.sleep(500);
+    }
+
+    System.out.println("createKey#:" + listener._createKeys.size() + ", changeKey#:"
+        + listener._changeKeys.size() + ", deleteKey#:" + listener._deleteKeys.size());
 
 Review comment:
   Java has not way to access otherwise private members in the class. In this case, we have to expire the session inside this class. This is a common problem I did not see addressed in our code base. Unless you guys can identify another pattern, I guess we have to leave it this way.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405969528
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,99 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    final long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(()-> {
+      long curSessionId = dupZkclient.getSessionId();
+      if (curSessionId != sessionId && curSessionId != 0) {
+        return true;
+      }
+      return false;
 
 Review comment:
   It could be simplified to `return curSessionId != sessionId && curSessionId != 0`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 closed pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 closed pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401174432
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
       return;
 
 Review comment:
   In general, it's not good practice to add subclass-specific logic in its parent class. Can you think of another way to do this?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401166894
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
 
 Review comment:
   Remove extra empty line.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402514956
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,83 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
 
 Review comment:
   good point. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405882303
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
+      if (isManagingZkConnection()) {
 
 Review comment:
   maybe something like this:
   
   ```
   private void reconnectOnExpiring() {
     if (!isManagingZkConnection()) {
       return;
     }
     ... // the rest is the same
   }
   ```
   
   ```
   private void fireNewSessionEvents() {
     if (!isManagingZkConnection()) {
       return;
     }
    ...// the rest is the same.
   
     }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402037660
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
 
 Review comment:
   The I will use this concise version. I will add a verbose note about the intention too.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r407808397
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,93 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder
+          .instanceCurrentState(clusterName, "localhost_8901", "session_0", "TestDB" + i);
 
 Review comment:
   @kaisun2000 My understanding is, the current states created by this invalid session "session_0" would also be invalid. Can you help me understand if these invalid current states would work for this testing purpose? Thanks.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402145324
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,83 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
 
 Review comment:
   Should assign the result to ret?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402041626
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
+        }
+      }
+
+      @Override
+      public void handleNewSession(final String sessionId) throws Exception {
+      }
+
+      @Override
+      public void handleSessionEstablishmentError(Throwable var1) throws Exception {
+      }
+    };
+
+    zkClient.subscribeStateChanges(listener);
+
+    ZkConnection connection = ((ZkConnection) zkClient.getConnection());
+    ZooKeeper curZookeeper = connection.getZookeeper();
+    String oldSessionId = Long.toHexString(curZookeeper.getSessionId());
+    LOG.info("Before session expiry. sessionId: " + oldSessionId + ", zk: " + curZookeeper);
+
+    Watcher watcher = new Watcher() {
+      @Override
+      public void process(WatchedEvent event) {
+        LOG.info("Watcher#process, event: " + event);
+      }
+    };
+
+    final ZooKeeper dupZookeeper =
+        new ZooKeeper(connection.getServers(), curZookeeper.getSessionTimeout(), watcher,
+            curZookeeper.getSessionId(), curZookeeper.getSessionPasswd());
+    // wait until connected, then close
+    while (dupZookeeper.getState() != States.CONNECTED) {
+      Thread.sleep(10);
+    }
+    Assert.assertEquals(dupZookeeper.getState(), States.CONNECTED,
+        "Fail to connect to zk using current session info");
+    dupZookeeper.close();
+
+    // make sure session expiry really happens
+    waitExpireSession.await();
+    zkClient.unsubscribeStateChanges(listener);
+
+    connection = (ZkConnection) zkClient.getConnection();
+    curZookeeper = connection.getZookeeper();
+
+    // wait util connected
+    while (curZookeeper.getState() != States.CONNECTED) {
 
 Review comment:
   There are many ways to do it. Let us be specific to avoid delaying this important fix. 
   So the code can be something like this:
   
     ```
   public void handleStateChanged(KeeperState state) throws Exception {
           LOG.info("IZkStateListener#handleStateChanged, state: " + state);
           if (state == KeeperState.Expired) {
             waitExpireSession.countDown();
           } 
           if (state == KeeperState.Synconnected) {
             waitNewSession.countDown();
           }
   
   ```
   
   Later something like this to wait for latch?
   
   ```
   boolean expiredCompleted = waitExpireSession.await(5L, TimeUnit.SECONDS);
   if (!expiredCompleted)  {
      throw Exception(" Expiration of Session failed.");
   }
   boolean newSessionCompleted = waitNewSession.await(5L, TimeUnit.SECONDS);
   if (!newSessionCompleted) {
     throw Exception(" Expiration of Session failed.");
   }
   zkClient.unsubscribeStateChanges(listener);
   ```
   
   Note, the wait for expiration and synconnected are both necessary. The reason is that disconnect can give you sysconnected too. 
   
   Note, the timeout in await() is to prevent hanging here forever to block test from proceeding. 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405769135
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,100 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    while (true) {
+      long curSessionId = dupZkclient.getSessionId();
+      if (curSessionId == sessionId || curSessionId == 0) {
+        Thread.sleep(500);
+      } else {
+        sessionId = curSessionId;
+        break;
 
 Review comment:
   If you have to use sleep to wait for the sync connected, I do see duplicate code and it could replaced by TestHelper.varify(), which saves you the duplicate code and also offers a timeout option in case it is forever blocked here.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r407793211
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,93 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
 
 Review comment:
   I still think `CURSTATECNT` should follow the style: it is not a constant variable.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401175988
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
+    // kill the session one more time to cover Shared ZkClient resetting flag
+    // indicating first time synconnect happened.
+    ZkTestHelper.expireSession(testClient);
+
+    listener.reset();
+    int expectDeleteNodes = 1 + firstLevelNr + firstLevelNr * secondLevelNr;
+    store.remove("/", 0);
+    // wait until all callbacks have been received
+    for (int i = 0; i < 10; i++) {
+      if (listener._deleteKeys.size() == expectDeleteNodes)
+        break;
+      Thread.sleep(500);
 
 Review comment:
   This is copied from the other test. I can change this a little bit. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405971613
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1150,6 +1156,10 @@ private void reconnect() {
   }
 
   private void fireNewSessionEvents() {
+    // only managing zkclient fire handleNewSession event
+    if (isManagingZkConnection()) {
 
 Review comment:
   Reversed way? Managing connection should fire handle new session events.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402042037
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
 
 Review comment:
   See my comment later about how to enhance with handleStateChanged. If we agree with that approach. I will use this logic in TestHelper.expireSession() and remove this one. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405969873
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,99 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
 
 Review comment:
   Unnecessary extra space before `)`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401175129
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
 
 Review comment:
   This is not how you would enter JavaDoc in a public method. Could you please review the javadoc best practice guidelines?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401159442
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
 ##########
 @@ -72,6 +72,11 @@
 
   private HelixZkClient _zkClient = null;
 
+  // test only
+  public HelixZkClient getZkClient() {
 
 Review comment:
   This should not be introduced. If getZkClient() is really necessary, you can make `_zkClient` as protected.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401178173
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
 ##########
 @@ -72,6 +72,11 @@
 
   private HelixZkClient _zkClient = null;
 
+  // test only
+  public HelixZkClient getZkClient() {
 
 Review comment:
   Protect is not going to work, as the test case live in another package. 
   
   Java has not way to access otherwise private members in the class. In this case, we have to expire the session inside this class. This is a common problem I did not see addressed in our code base. Unless you guys can identify another pattern, I guess we have to leave it this way.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r406380246
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,99 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    final long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(()-> {
 
 Review comment:
   done.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r408358596
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,93 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int curstateCnt = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < curstateCnt; i++) {
+      String path = PropertyPathBuilder
+          .instanceCurrentState(clusterName, "localhost_8901", "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < curstateCnt; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000, zkCacheInitPaths, accessor._zkCache._cache,
+        _gZkClient, true);
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    final long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(() -> {
+      long curSessionId = dupZkclient.getSessionId();
+      return curSessionId != sessionId && curSessionId != 0;
+    }, 10000);
+    Assert.assertTrue(ret, "kill session timed out!");
+
+    // kill the session one more time to cover code path ZkClient resetting flag that
+    // indicates first time synconnect happened.
+    final long sessionId1 = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(() -> {
+      long curSessionId = dupZkclient.getSessionId();
+      return curSessionId != sessionId1 && curSessionId != 0;
+    }, 10000);
+    Assert.assertTrue(ret, "kill session second time timed out!");
+
+    // remove the currentstates
+    paths.clear();
+    for (int i = 0; i < curstateCnt; i++) {
+      String path = PropertyPathBuilder
+          .instanceCurrentState(clusterName, "localhost_8901", "session_0", "TestDB" + i);
+      paths.add(path);
+    }
 
 Review comment:
   This block of clearing current state paths is unnecessary, as the paths list is not changed?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
lei-xia commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401186639
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java
 ##########
 @@ -112,4 +112,9 @@ public String create(final String path, Object datat, final List<ACL> acl,
   protected boolean isManagingZkConnection() {
     return false;
   }
+
+  // test only
+  public HelixZkClient getConnectionManager() {
+    return _connectionManager;
+  }
 
 Review comment:
   Agreed here, we should never add a public method to a public class just for testing.  If you really need to expose ZkClient, either change it to protected, and inherit this class in your test, such that:
   
   class SharedZkClient {
      protected _zkClient;
   }
   
   class testSharedZkClient extends SharedZkClient {
   
    Protected getZkClient() {
         return _zkClient;
    }

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401175238
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
 
 Review comment:
   empty line?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on issue #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on issue #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#issuecomment-606994922
 
 
   As discussed offline, ZkTestHelper.expireSession() only works with zkConnectionmanager, but not shared zkclient. 
   
   But we can add another expireSharedZkClientSession() though for ZkTestHelper.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401173166
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
+    // kill the session one more time to cover Shared ZkClient resetting flag
+    // indicating first time synconnect happened.
+    ZkTestHelper.expireSession(testClient);
+
+    listener.reset();
+    int expectDeleteNodes = 1 + firstLevelNr + firstLevelNr * secondLevelNr;
+    store.remove("/", 0);
+    // wait until all callbacks have been received
+    for (int i = 0; i < 10; i++) {
+      if (listener._deleteKeys.size() == expectDeleteNodes)
+        break;
+      Thread.sleep(500);
 
 Review comment:
   +1. This is not a show of good craftsmanship..

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401178870
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java
 ##########
 @@ -112,4 +112,9 @@ public String create(final String path, Object datat, final List<ACL> acl,
   protected boolean isManagingZkConnection() {
     return false;
   }
+
+  // test only
+  public HelixZkClient getConnectionManager() {
+    return _connectionManager;
+  }
 
 Review comment:
   Java has not way to access otherwise private members in the class. In this case, we have to expire the session inside this class. This is a common problem I did not see addressed in our code base. Unless you guys can identify another pattern, I guess we have to leave it this way.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401159442
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
 ##########
 @@ -72,6 +72,11 @@
 
   private HelixZkClient _zkClient = null;
 
+  // test only
+  public HelixZkClient getZkClient() {
 
 Review comment:
   This should be introduced. If getZkClient() is really necessary, you can make `_zkClient` as protected.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r408360025
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,93 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder
+          .instanceCurrentState(clusterName, "localhost_8901", "session_0", "TestDB" + i);
 
 Review comment:
   My point was, I just wanted to understand it is not only reading the current state paths that have a valid session. 
   I double checked. It should work as it would read the paths recursively so "session_0" would be read. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401178552
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/store/zk/TestZkHelixPropertyStore.java
 ##########
 @@ -88,6 +92,57 @@ public void afterClass() {
     deleteCluster(getShortClassName());
   }
 
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    /*
+    This test is to make sure SharedZkClient would work with
+    HelixPropertyStore when session expiration happens. More specifically
+    HelixPropertyStore would register all the callbacks after session
+    expiration.
+     */
+    String subRoot = _root + "/" + "localCallback";
+    List<String> subscribedPaths = new ArrayList<>();
+    subscribedPaths.add(subRoot);
+
+    ZkSerializer serializer = new ZNRecordSerializer();
+    ZkHelixPropertyStore<ZNRecord> store =
+        new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);
+
+    TestListener listener = new TestListener();
+    store.subscribe("/", listener);
+
+    // test dataCreate callbacks
+    listener.reset();
+    setNodes(store, 'a', true);
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    // Note, current shared ZkClient does not issue new session. Thus, we need
+    // kill _zkConnectionManager's session. Otherwise, ZkTestHelper.expireSesson
+    // would not work.
+    SharedZkClient sharedClient = (SharedZkClient) store.getTestZkClient();
+    HelixZkClient  testClient =  sharedClient.getConnectionManager();
+    ZkTestHelper.expireSession(testClient);
+
+
+    // kill the session one more time to cover Shared ZkClient resetting flag
+    // indicating first time synconnect happened.
+    ZkTestHelper.expireSession(testClient);
+
+    listener.reset();
+    int expectDeleteNodes = 1 + firstLevelNr + firstLevelNr * secondLevelNr;
+    store.remove("/", 0);
+    // wait until all callbacks have been received
+    for (int i = 0; i < 10; i++) {
+      if (listener._deleteKeys.size() == expectDeleteNodes)
+        break;
+      Thread.sleep(500);
+    }
+
+    System.out.println("createKey#:" + listener._createKeys.size() + ", changeKey#:"
+        + listener._changeKeys.size() + ", deleteKey#:" + listener._deleteKeys.size());
 
 Review comment:
   Java has not way to access otherwise private members in the class. In this case, we have to expire the session inside this class. This is a common problem I did not see addressed in our code base. Unless you guys can identify another pattern, I guess we have to leave it this way.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r406380385
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,99 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
 
 Review comment:
   formatted.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401178173
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
 ##########
 @@ -72,6 +72,11 @@
 
   private HelixZkClient _zkClient = null;
 
+  // test only
+  public HelixZkClient getZkClient() {
 
 Review comment:
   Protect is not going to work, as the test case live in another package. 
   
   Java has not way to access otherwise private members in the class. In this case, we have to expire the session inside this class. This is a common pattern I did not see addressed in our code base. Unless you guys can identify another pattern, I guess we have to leave it this way.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401174618
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
       return;
     }
 
+    // The following is the case for non sharedZkClient
 
 Review comment:
   parent class shouldn't know about what its sub classes are doing?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405767696
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,83 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
 
 Review comment:
   Do you think this should be changed, since I've seen you agreement but no 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r406389934
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1080,22 +1080,21 @@ protected void processStateChanged(WatchedEvent event) {
      */
     if (event.getState() == KeeperState.SyncConnected) {
       if (!_isNewSessionEventFired && !"0".equals(getHexSessionId())) {
-        if (isManagingZkConnection()) {
-          /*
-           * Before the new zookeeper instance is connected to the zookeeper service and its session
-           * is established, its session id is 0.
-           * New session event is not fired until the new zookeeper session receives the first
-           * SyncConnected state(the zookeeper session is established).
-           * Now the session id is available and non-zero, and we can fire new session events.
-           */
-          fireNewSessionEvents();
-        }
         /*
          * Set it true to avoid firing events again for the same session next time
          * when SyncConnected events are received.
          */
         _isNewSessionEventFired = true;
 
+        /*
+         * Before the new zookeeper instance is connected to the zookeeper service and its session
+         * is established, its session id is 0.
+         * New session event is not fired until the new zookeeper session receives the first
+         * SyncConnected state(the zookeeper session is established).
+         * Now the session id is available and non-zero, and we can fire new session events.
+         */
+        fireNewSessionEvents();
 
 Review comment:
   I changed it back to the original way. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r408439012
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,93 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int curstateCnt = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < curstateCnt; i++) {
+      String path = PropertyPathBuilder
+          .instanceCurrentState(clusterName, "localhost_8901", "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < curstateCnt; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000, zkCacheInitPaths, accessor._zkCache._cache,
+        _gZkClient, true);
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    final long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(() -> {
+      long curSessionId = dupZkclient.getSessionId();
+      return curSessionId != sessionId && curSessionId != 0;
+    }, 10000);
+    Assert.assertTrue(ret, "kill session timed out!");
+
+    // kill the session one more time to cover code path ZkClient resetting flag that
+    // indicates first time synconnect happened.
+    final long sessionId1 = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(() -> {
+      long curSessionId = dupZkclient.getSessionId();
+      return curSessionId != sessionId1 && curSessionId != 0;
+    }, 10000);
+    Assert.assertTrue(ret, "kill session second time timed out!");
+
+    // remove the currentstates
+    paths.clear();
+    for (int i = 0; i < curstateCnt; i++) {
+      String path = PropertyPathBuilder
+          .instanceCurrentState(clusterName, "localhost_8901", "session_0", "TestDB" + i);
+      paths.add(path);
+    }
 
 Review comment:
   This is necessary.  This removal of currentstate should trigger propertystore to reflect this removal in its cache as well. 
   
   Without this fix, this is not going to be true because the watcher to zookeeper is lost. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401184161
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
       return;
     }
 
+    // The following is the case for non sharedZkClient
 
 Review comment:
   See my comment in the other similar comment. We can discuss it over there.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405767321
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
 
 Review comment:
   I was also thinking about the same concern.  It should be fine as it is protected by the eventLock. And also if reconnecting fails because of IOException, the old/expired zk is kept, but the expired zk could not do anything.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405742254
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
+      if (isManagingZkConnection()) {
 
 Review comment:
   As we moved the isManagingZkConnection closer to these methods, it might be cleaner if we just move it inside reconnectOnExpiring() and fireNewSessionEvents(). So we have better protection and the code looks cleaner.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405876161
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on issue #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on issue #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#issuecomment-613651021
 
 
   @kaisun2000 Just a reminder: please run the tests in an appropriate module before merging the PR. Thanks.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405876751
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
+      if (isManagingZkConnection()) {
 
 Review comment:
   As @pkuwm pointed out, processStateChanged is private method always called under lock protection, the same as fireAllEvent, and reconnectionOnExpiring. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402514561
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
 
 Review comment:
   It is different for a reason. Shared ZkClient does not have handleNewSession. Will combine see the above discussion with JJ.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r406385357
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,99 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    final long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(()-> {
+      long curSessionId = dupZkclient.getSessionId();
+      if (curSessionId != sessionId && curSessionId != 0) {
+        return true;
+      }
+      return false;
 
 Review comment:
   changed. 
   
   there are many ways to write same bool logic. there is no preferred way by all the people. (say what if JJ prefer another way, how do we resolve it?) let us refrain from pick this kind stuff up. It incurs cost without adding value.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401928444
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
 
 Review comment:
   You are right, here this latch guarantees expiring happens. But later, the code still ensures new session established.
   
   `  waitExpireSession.await();
       zkClient.unsubscribeStateChanges(listener);
   
       connection = (ZkConnection) zkClient.getConnection();
       curZookeeper = connection.getZookeeper();
   
       // wait util connected
       while (curZookeeper.getState() != States.CONNECTED) {
         Thread.sleep(10);
       } '

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402131845
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,83 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
 
 Review comment:
   If it is just a local var, no need to make it capitalized. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r405878950
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1101,7 +1104,10 @@ protected void processStateChanged(WatchedEvent event) {
         fireAllEvents();
       }
     } else if (event.getState() == KeeperState.Expired) {
-      reconnectOnExpiring();
+      _isNewSessionEventFired = false;
+      if (isManagingZkConnection()) {
 
 Review comment:
   Yeah, but it is a good practice to check closer to where the real business logic exists.
   
   One example could be, if in the future we want to call reconnectOnExpiring() somewhere else, we don't need to add one more check there.
   
   High-level, these 2 methods shall only be called for non-shared clients. But we don't have a way to block them for shared client implementation. So we just put the check inside, then they won't cause any more problems even if we want to do anything to change the zkclient.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402153657
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,83 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
+        zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true  );
+
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    ZkTestHelper.expireSharedZkClientSession(dupZkclient);
+
+    // kill the session one more time to cover code path ZkClient resetting flag that
+    // indicates first time synconnect happened.
+    ZkTestHelper.expireSharedZkClientSession(dupZkclient);
+
+    // remove the currentstates
+    paths.clear();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901",
+          "session_0", "TestDB" + i);
+      paths.add(path);
+    }
+    success = extBaseAccessor.remove(paths, 0);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in remove:" + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000,
 
 Review comment:
   Assert this?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang merged pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang merged pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r407791617
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,93 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder
+          .instanceCurrentState(clusterName, "localhost_8901", "session_0", "TestDB" + i);
+      ZNRecord record = new ZNRecord("TestDB" + i);
+
+      paths.add(path);
+      records.add(record);
+    }
+
+    boolean[] success = extBaseAccessor.createChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < CURSTATECNT; i++) {
+      Assert.assertTrue(success[i], "Should succeed in create: " + paths.get(i));
+    }
+
+    TestHelper.verifyWithTimeout("verifyZkCache", 5000, zkCacheInitPaths, accessor._zkCache._cache,
+        _gZkClient, true);
+
+    // dup shared ZkClient
+    HelixZkClient dupZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+
+    // kill the session to make sure shared zkClient re-installs watcher
+    final long sessionId = dupZkclient.getSessionId();
+    ZkTestHelper.asyncExpireSession(dupZkclient);
+    ret = TestHelper.verify(() -> {
+      long curSessionId = dupZkclient.getSessionId();
+      return curSessionId != sessionId && curSessionId != 0 ? true : false;
 
 Review comment:
   Nit, just make my point clear: this is kind of redundant, not what I meant. I meant 
   ```
   return curSessionId != sessionId && curSessionId != 0;
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401187416
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
+       For SharedZkClient, we will not invoke fireNewSessionEvents and we will invoke fireAllEvents
+       This keeps the same behavior as original SharedZkClient.
+       Invoking fireAllEvents is important to make sure HelixPropertyStore working correctly.
+      */
+      if ((event.getState() == KeeperState.SyncConnected) && (!_isNewSessionEventFired) && (!"0"
+          .equals(getHexSessionId()))) {
+        fireAllEvents();
+        _isNewSessionEventFired = true;
+      } else if (event.getState() == KeeperState.Expired) {
+        _isNewSessionEventFired = false;
+      }
 
 Review comment:
   Good point. 
   Yes, this is another way to do it, thought about it.
   Pro: concise
   Con: hard to understand the intention. This may pave the way to another hidden bug when people do refactoring next time, as how we get to this issue.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401160731
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1073,10 +1073,24 @@ protected void processStateChanged(WatchedEvent event) {
 
     fireStateChangedEvent(event.getState());
 
+    // This following is the case for sharedZkClient
     if (!isManagingZkConnection()) {
+      /*
 
 Review comment:
   Comment block format:
   ```
   /*
    * comments
    */
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402142668
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
+        }
+      }
+
+      @Override
+      public void handleNewSession(final String sessionId) throws Exception {
+      }
+
+      @Override
+      public void handleSessionEstablishmentError(Throwable var1) throws Exception {
+      }
+    };
+
+    zkClient.subscribeStateChanges(listener);
+
+    ZkConnection connection = ((ZkConnection) zkClient.getConnection());
+    ZooKeeper curZookeeper = connection.getZookeeper();
+    String oldSessionId = Long.toHexString(curZookeeper.getSessionId());
+    LOG.info("Before session expiry. sessionId: " + oldSessionId + ", zk: " + curZookeeper);
+
+    Watcher watcher = new Watcher() {
+      @Override
+      public void process(WatchedEvent event) {
+        LOG.info("Watcher#process, event: " + event);
+      }
+    };
+
+    final ZooKeeper dupZookeeper =
+        new ZooKeeper(connection.getServers(), curZookeeper.getSessionTimeout(), watcher,
+            curZookeeper.getSessionId(), curZookeeper.getSessionPasswd());
+    // wait until connected, then close
+    while (dupZookeeper.getState() != States.CONNECTED) {
+      Thread.sleep(10);
+    }
+    Assert.assertEquals(dupZookeeper.getState(), States.CONNECTED,
+        "Fail to connect to zk using current session info");
+    dupZookeeper.close();
+
+    // make sure session expiry really happens
+    waitExpireSession.await();
+    zkClient.unsubscribeStateChanges(listener);
+
+    connection = (ZkConnection) zkClient.getConnection();
+    curZookeeper = connection.getZookeeper();
+
+    // wait util connected
+    while (curZookeeper.getState() != States.CONNECTED) {
 
 Review comment:
   +1 to get rid of busy sleeping.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on issue #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on issue #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#issuecomment-606938078
 
 
   @jiajunwang 
   
   > I see the main issue is about the additional public method for fetching the client. I don't like that either.
   > An alternative way is that, since you are for sure given a shared zkclient in that property store, you can call ShareZkClientFactory with the same parameters and it will return you with another shared ZkClient instance. But, since they will use the same connection manager, when you expires the newly created client, the one that in the HelixPropertyStore will also be expired for once.
   > Can you have a try? In this way, no new method is required.
   
   How do we get the zkconnection manager from the new sharedZkClient? It is not public. 
   
   If we make zkConnectionManager protected, we need to access it from the module in zookeeper-api, which means move (rewrite) this test case there. But this case use the code from HelixPropertyStore, which is helix-core. This is circular dependency.
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r406382875
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/zkclient/ZkClient.java
 ##########
 @@ -1080,22 +1080,21 @@ protected void processStateChanged(WatchedEvent event) {
      */
     if (event.getState() == KeeperState.SyncConnected) {
       if (!_isNewSessionEventFired && !"0".equals(getHexSessionId())) {
-        if (isManagingZkConnection()) {
-          /*
-           * Before the new zookeeper instance is connected to the zookeeper service and its session
-           * is established, its session id is 0.
-           * New session event is not fired until the new zookeeper session receives the first
-           * SyncConnected state(the zookeeper session is established).
-           * Now the session id is available and non-zero, and we can fire new session events.
-           */
-          fireNewSessionEvents();
-        }
         /*
          * Set it true to avoid firing events again for the same session next time
          * when SyncConnected events are received.
          */
         _isNewSessionEventFired = true;
 
+        /*
+         * Before the new zookeeper instance is connected to the zookeeper service and its session
+         * is established, its session id is 0.
+         * New session event is not fired until the new zookeeper session receives the first
+         * SyncConnected state(the zookeeper session is established).
+         * Now the session id is available and non-zero, and we can fire new session events.
+         */
+        fireNewSessionEvents();
 
 Review comment:
   Any comments on "why we need to move it to after _isNewSessionEventFired = true;? Is it concerning?" ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r401247502
 
 

 ##########
 File path: zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java
 ##########
 @@ -112,4 +112,9 @@ public String create(final String path, Object datat, final List<ACL> acl,
   protected boolean isManagingZkConnection() {
     return false;
   }
+
+  // test only
+  public HelixZkClient getConnectionManager() {
+    return _connectionManager;
+  }
 
 Review comment:
   Here is the case:
   
   test case is in package org.apache.helix.store.zk, here is the how it is constructed.
   
   `    ZkSerializer serializer = new ZNRecordSerializer();
       ZkHelixPropertyStore<ZNRecord> store =
           new ZkHelixPropertyStore<>(ZK_ADDR, serializer, subRoot, subscribedPaths);`
   
   We need the internal shared ZkClient in the store; and further the internal _ZkconnectionManager.
   
   Note, ZkHelixPropertyStore uses ZkCacheBasedDataAccessor which lives in package org.apache.helix.manager.zk;
   
   Note, ShareZkClient and ZkConnectionManager lives in package org.apache.helix.zookeeper.impl.client;
   
   So the solution (in this case, but not in general) can be:
   
   I can put the test in the org.apache.helix.zookeeper.impl.client, zookeeper-api module
   
   I can also initialize the HelixPropertystore like this:
   
   1/ construct SharedZkClient 
   
   2/ construct ZkBaseDataAccessor using this sharedZkCLient.
   
   3/ construct ZkHelixPropertyStore (using ZkBaseDataAccessor.
   
   by making SharedZkClient's _zkConnectionManager protected, we can use it here.  
   
   The problem is that org.apache.helix.zookeeper.impl.client is in Zookeeper-api, we need a test scope dependency to Helix-core and Helix-common module. Does this test dependency work here?
   @lei-xia @narendly ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #924: Fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r408325307
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/manager/zk/TestZkCacheAsyncOpSingleThread.java
 ##########
 @@ -37,6 +38,93 @@
 import org.testng.annotations.Test;
 
 public class TestZkCacheAsyncOpSingleThread extends ZkUnitTestBase {
+  @Test
+  public void testSessionExpirationWithSharedZkClient() throws Exception {
+    int CURSTATECNT = 10;
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    // init external base data accessor
+    HelixZkClient sharedZkclient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(ZK_ADDR));
+    sharedZkclient.setZkSerializer(new ZNRecordSerializer());
+    ZkBaseDataAccessor<ZNRecord> sharedBaseAccessor = new ZkBaseDataAccessor<>(sharedZkclient);
+
+    // init zkCacheBaseDataAccessor
+    String curStatePath = PropertyPathBuilder.instanceCurrentState(clusterName, "localhost_8901");
+    String extViewPath = PropertyPathBuilder.externalView(clusterName);
+
+    ZkBaseDataAccessor<ZNRecord> extBaseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+    extBaseAccessor.create(curStatePath, null, AccessOption.PERSISTENT);
+
+    List<String> zkCacheInitPaths = Arrays.asList(curStatePath, extViewPath);
+    ZkCacheBaseDataAccessor<ZNRecord> accessor =
+        new ZkCacheBaseDataAccessor<>(sharedBaseAccessor, null, null, zkCacheInitPaths);
+
+    boolean ret =
+        TestHelper.verifyZkCache(zkCacheInitPaths, accessor._zkCache._cache, _gZkClient, true);
+    Assert.assertTrue(ret, "zkCache doesn't match data on Zk");
+
+    // create 10 current states using external base accessor
+    List<String> paths = new ArrayList<>();
+    List<ZNRecord> records = new ArrayList<>();
+    for (int i = 0; i < CURSTATECNT; i++) {
+      String path = PropertyPathBuilder
+          .instanceCurrentState(clusterName, "localhost_8901", "session_0", "TestDB" + i);
 
 Review comment:
   CurrentStates here only serves the purpose of some directory in zookeeper. This test has nothing to do with controller rebalancing logic.  Note, TestHelper.verifyZkCache only valid the data in zookeeper and propertycache for this currentstates path are the same. That is the gist of this test.
   
   Put it another way, using currentstate property builder is just a convenience. All the other test in this suite is like this.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [helix] jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #924: WIP: fix ZkHelixPropertyStore loses Zookeeper notification issue
URL: https://github.com/apache/helix/pull/924#discussion_r402030011
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
 ##########
 @@ -146,6 +146,70 @@ public void process(WatchedEvent event) {
     LOG.info("After expiry. sessionId: " + Long.toHexString(curZookeeper.getSessionId()));
   }
 
+  public static void expireSharedZkClientSession(HelixZkClient client) throws Exception {
+    final CountDownLatch waitExpireSession = new CountDownLatch(1);
+    final ZkClient zkClient = (ZkClient) client;
+
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(KeeperState state) throws Exception {
+        LOG.info("IZkStateListener#handleStateChanged, state: " + state);
+        if (state == KeeperState.Expired) {
+          waitExpireSession.countDown();
 
 Review comment:
   This works. Since it should work for all the client, can we just modify the original expiring method? 2 similar methods for the same purpose is not good.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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