You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@curator.apache.org by "kezhuw (via GitHub)" <gi...@apache.org> on 2023/05/30 04:00:28 UTC

[GitHub] [curator] kezhuw opened a new pull request, #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

kezhuw opened a new pull request, #464:
URL: https://github.com/apache/curator/pull/464

   Previously, if curator framework is closed while there are background operations, these operations are dropped with no dedicated events.
   
   It is hard for blocking clients to stop waiting after framework closed. `CuratorListener` seems like a circumvent. But apparently, it is hard and I don't think this is by design.
   
   This pr introduces a `clockLock` to synchronize framework closing and operation queuing. So we are sure that no operations could be queued after framework marked as closing.


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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1212564545


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -694,17 +714,46 @@ <DATA_TYPE> void processBackgroundOperation(OperationAndData<DATA_TYPE> operatio
         }
     }
 
+    private void closeOperation(OperationAndData<?> operation) {
+        if (operation.getCallback() == null) {
+            return;
+        }
+        CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.CLOSING, KeeperException.Code.SESSIONEXPIRED.intValue(), null, null, operation.getContext(), null, null, null, null, null, null);
+        sendToBackgroundCallback(operation, event);
+    }
+
+    private void requeueSleepOperation(OperationAndData<?> operationAndData) {
+        operationAndData.clearSleep();
+        synchronized (closeLock)
+        {
+            if (getState() == CuratorFrameworkState.STARTED)
+            {
+                if (backgroundOperations.remove(operationAndData))
+                {
+                    // due to the internals of DelayQueue, operation must be removed/re-added so that re-sorting occurs
+                    backgroundOperations.offer(operationAndData);
+                }   // This operation has been taken over by background thread.
+                return;
+            }
+        }
+        closeOperation(operationAndData);
+    }
+
     /**
      * @param operationAndData operation entry
      * @return true if the operation was actually queued, false if not
      */
     <DATA_TYPE> boolean queueOperation(OperationAndData<DATA_TYPE> operationAndData)
     {
-        if ( getState() == CuratorFrameworkState.STARTED )
+        synchronized (closeLock)
         {
-            backgroundOperations.offer(operationAndData);
-            return true;
+            if (getState() == CuratorFrameworkState.STARTED)

Review Comment:
   Hmm, `isStarted` is marked as `Deprecated`. CURATOR-147(#44) targeted removing invocation of its usages, though I guess we are only deprecating the API part. Keep it as is for now ? All other usages in Curator use this shape currently.



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1212586231


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -430,6 +448,8 @@ public void close()
             {
                 ensembleTracker.close();
             }
+            backgroundOperations.forEach(this::closeOperation);
+            backgroundOperations.clear();

Review Comment:
   Done. Added a fixup commit for this and correct a direct call to `backgroundOperations.offer(operationAndData)`.



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] tisonkun commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "tisonkun (via GitHub)" <gi...@apache.org>.
tisonkun commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1211770824


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -694,17 +714,46 @@ <DATA_TYPE> void processBackgroundOperation(OperationAndData<DATA_TYPE> operatio
         }
     }
 
+    private void closeOperation(OperationAndData<?> operation) {
+        if (operation.getCallback() == null) {
+            return;
+        }
+        CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.CLOSING, KeeperException.Code.SESSIONEXPIRED.intValue(), null, null, operation.getContext(), null, null, null, null, null, null);

Review Comment:
   ```suggestion
           CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.CLOSING, 0, null, null, operation.getContext(), null, null, null, null, null, null);
   ```
   
   At least we send `0` for listeners.



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1214056837


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -430,6 +448,8 @@ public void close()
             {
                 ensembleTracker.close();
             }
+            backgroundOperations.forEach(this::closeOperation);
+            backgroundOperations.clear();

Review Comment:
   Hmm, the `DelayQueue::drainTo` filters elements based on `Delayed::getDelay`. This is not what we want here. The test could not last 200 runs. I plan to resort to `toArray` for 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.

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1218023453


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -694,17 +714,46 @@ <DATA_TYPE> void processBackgroundOperation(OperationAndData<DATA_TYPE> operatio
         }
     }
 
+    private void closeOperation(OperationAndData<?> operation) {
+        if (operation.getCallback() == null) {
+            return;
+        }
+        CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.CLOSING, KeeperException.Code.SESSIONEXPIRED.intValue(), null, null, operation.getContext(), null, null, null, null, null, null);

Review Comment:
   I added `getBackgroundEventType` to `BackgroundOperation` in commit c18cb6fb6de30aeae4f8d04dcfff3d6e04d49112. `getBackgroundEventType` is somewhat an requirement for `abortOperation`, otherwise there is no suitable `CuratorEventType` for exception from no ZooKeeper world. Basically, event to `BackgroundCallback` has same event type as `BackgroundOperation::getBackgroundEventType`. This changes `CONNECTIONLOSS`'s event type from `uratorEventType.WATCHED` to its background operation's event type. I hope this does not hurt.



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] tisonkun commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "tisonkun (via GitHub)" <gi...@apache.org>.
tisonkun commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1211772703


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -694,17 +714,46 @@ <DATA_TYPE> void processBackgroundOperation(OperationAndData<DATA_TYPE> operatio
         }
     }
 
+    private void closeOperation(OperationAndData<?> operation) {
+        if (operation.getCallback() == null) {
+            return;
+        }
+        CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.CLOSING, KeeperException.Code.SESSIONEXPIRED.intValue(), null, null, operation.getContext(), null, null, null, null, null, null);

Review Comment:
   But a client permanent disconnect can be SESSIONEXPIRED.



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw merged pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

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


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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on PR #464:
URL: https://github.com/apache/curator/pull/464#issuecomment-1576746926

   @eolivelli @tisonkun Could you please take another look ?
   
   The code changed a bit due to last commit c18cb6fb6de30aeae4f8d04dcfff3d6e04d49112. It adds `abortOperation` to handle exception from background operation and `BackgroundOperation::getBackgroundEventType` for `abortOperation` to delivery event with correct event type.


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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on PR #464:
URL: https://github.com/apache/curator/pull/464#issuecomment-1571364009

   I missed code paths in `handleBackgroundOperationException(operationAndData, e)` from `performBackgroundOperation` could introduce. Let me check.


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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1218003423


##########
curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java:
##########
@@ -945,6 +949,58 @@ public void testGetSequentialChildren() throws Exception
         }
     }
 
+    private static class AlwaysRetry implements RetryPolicy {
+        private final int retryIntervalMs;
+
+        public AlwaysRetry(int retryIntervalMs) {
+            this.retryIntervalMs = retryIntervalMs;
+        }
+
+        @Override
+        public boolean allowRetry(Throwable exception) {
+            return true;
+        }
+
+        @Override
+        public boolean allowRetry(int retryCount, long elapsedTimeMs, RetrySleeper retrySleeper) {
+            try {
+                retrySleeper.sleepFor(retryIntervalMs, TimeUnit.MILLISECONDS);
+            } catch (InterruptedException ex) {
+                Thread.currentThread().interrupt();
+            }
+            return true;
+        }
+    }
+
+    @Test
+    public void testBackgroundClosing() throws Exception
+    {
+        AlwaysRetry alwaysRetry = new AlwaysRetry(2);
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), alwaysRetry);
+        client.start();
+        try {
+            // given: error background request with always-retry policy
+            CompletableFuture<CuratorEvent> future = new CompletableFuture<>();
+            client.delete().inBackground((ignored, event) -> future.complete(event)).forPath("/test");
+
+            // These chaos steps create chances to run into concurrent contentions.
+            // They could fail this test given enough runs if there are bugs.

Review Comment:
   I make this chaos configurable 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.

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on PR #464:
URL: https://github.com/apache/curator/pull/464#issuecomment-1576691764

   > I missed code paths in handleBackgroundOperationException(operationAndData, e) from performBackgroundOperation could introduce. Let me check.
   
   I have pushed one more commit to solve this. There are still chances for `BackgroundCallback` no called, and I have created [CURATOR-677][] for what I have found. I plan to solve it separately so we can keep this pr simple.
   
   [CURATOR-677]: https://issues.apache.org/jira/browse/CURATOR-677


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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] tisonkun commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "tisonkun (via GitHub)" <gi...@apache.org>.
tisonkun commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1211779013


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -694,17 +714,46 @@ <DATA_TYPE> void processBackgroundOperation(OperationAndData<DATA_TYPE> operatio
         }
     }
 
+    private void closeOperation(OperationAndData<?> operation) {
+        if (operation.getCallback() == null) {
+            return;
+        }
+        CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.CLOSING, KeeperException.Code.SESSIONEXPIRED.intValue(), null, null, operation.getContext(), null, null, null, null, null, null);
+        sendToBackgroundCallback(operation, event);
+    }
+
+    private void requeueSleepOperation(OperationAndData<?> operationAndData) {
+        operationAndData.clearSleep();
+        synchronized (closeLock)
+        {
+            if (getState() == CuratorFrameworkState.STARTED)
+            {
+                if (backgroundOperations.remove(operationAndData))
+                {
+                    // due to the internals of DelayQueue, operation must be removed/re-added so that re-sorting occurs
+                    backgroundOperations.offer(operationAndData);
+                }   // This operation has been taken over by background thread.
+                return;
+            }
+        }
+        closeOperation(operationAndData);
+    }
+
     /**
      * @param operationAndData operation entry
      * @return true if the operation was actually queued, false if not
      */
     <DATA_TYPE> boolean queueOperation(OperationAndData<DATA_TYPE> operationAndData)
     {
-        if ( getState() == CuratorFrameworkState.STARTED )
+        synchronized (closeLock)
         {
-            backgroundOperations.offer(operationAndData);
-            return true;
+            if (getState() == CuratorFrameworkState.STARTED)

Review Comment:
   ```suggestion
               if (isStarted())
   ```
   
   nit - a bit bikeshedding



##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -430,6 +448,8 @@ public void close()
             {
                 ensembleTracker.close();
             }
+            backgroundOperations.forEach(this::closeOperation);
+            backgroundOperations.clear();

Review Comment:
   Perhaps following these line https://github.com/apache/curator/pull/464/files#diff-1c44d20ca65904e08924d5b0fbc2e3e34d2a043f71f14aa1386e9f45a95ada52R1105-R1107
   
   ... to drain the queue first and then process ops one by one. I guess BlockingQueue.foreach is somehow a smell.



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] tisonkun commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "tisonkun (via GitHub)" <gi...@apache.org>.
tisonkun commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1223020774


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -694,17 +714,46 @@ <DATA_TYPE> void processBackgroundOperation(OperationAndData<DATA_TYPE> operatio
         }
     }
 
+    private void closeOperation(OperationAndData<?> operation) {
+        if (operation.getCallback() == null) {
+            return;
+        }
+        CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.CLOSING, KeeperException.Code.SESSIONEXPIRED.intValue(), null, null, operation.getContext(), null, null, null, null, null, null);
+        sendToBackgroundCallback(operation, event);
+    }
+
+    private void requeueSleepOperation(OperationAndData<?> operationAndData) {
+        operationAndData.clearSleep();
+        synchronized (closeLock)
+        {
+            if (getState() == CuratorFrameworkState.STARTED)
+            {
+                if (backgroundOperations.remove(operationAndData))
+                {
+                    // due to the internals of DelayQueue, operation must be removed/re-added so that re-sorting occurs
+                    backgroundOperations.offer(operationAndData);
+                }   // This operation has been taken over by background thread.
+                return;
+            }
+        }
+        closeOperation(operationAndData);
+    }
+
     /**
      * @param operationAndData operation entry
      * @return true if the operation was actually queued, false if not
      */
     <DATA_TYPE> boolean queueOperation(OperationAndData<DATA_TYPE> operationAndData)
     {
-        if ( getState() == CuratorFrameworkState.STARTED )
+        synchronized (closeLock)
         {
-            backgroundOperations.offer(operationAndData);
-            return true;
+            if (getState() == CuratorFrameworkState.STARTED)

Review Comment:
   Thanks for your explanation! SGTM.



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on PR #464:
URL: https://github.com/apache/curator/pull/464#issuecomment-1582991612

   @eolivelli @tisonkun Thank you for reviewing! Merged.


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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1218026939


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -694,17 +715,67 @@ <DATA_TYPE> void processBackgroundOperation(OperationAndData<DATA_TYPE> operatio
         }
     }
 
+    private void abortOperation(OperationAndData<?> operation, Throwable e)
+    {
+        if (operation.getCallback() == null) {
+            return;
+        }
+        CuratorEvent event;
+        if (e instanceof KeeperException)
+        {
+            event = new CuratorEventImpl(this, operation.getEventType(), ((KeeperException) e).code().intValue(), ((KeeperException) e).getPath(), null, operation.getContext(), null, null, null, null, null, null);
+        }
+        else if (getState() == CuratorFrameworkState.STARTED)
+        {
+            event = new CuratorEventImpl(this, operation.getEventType(), KeeperException.Code.SYSTEMERROR.intValue(), null, null, operation.getContext(), null, null, null, null, null, null);

Review Comment:
   This `SYSTEMERROR` matches `AsyncResultImpl`'s choice. I don't have any other candidates.



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1209685713


##########
curator-framework/src/test/java/org/apache/curator/framework/imps/TestFramework.java:
##########
@@ -945,6 +949,58 @@ public void testGetSequentialChildren() throws Exception
         }
     }
 
+    private static class AlwaysRetry implements RetryPolicy {
+        private final int retryIntervalMs;
+
+        public AlwaysRetry(int retryIntervalMs) {
+            this.retryIntervalMs = retryIntervalMs;
+        }
+
+        @Override
+        public boolean allowRetry(Throwable exception) {
+            return true;
+        }
+
+        @Override
+        public boolean allowRetry(int retryCount, long elapsedTimeMs, RetrySleeper retrySleeper) {
+            try {
+                retrySleeper.sleepFor(retryIntervalMs, TimeUnit.MILLISECONDS);
+            } catch (InterruptedException ex) {
+                Thread.currentThread().interrupt();
+            }
+            return true;
+        }
+    }
+
+    @Test
+    public void testBackgroundClosing() throws Exception
+    {
+        AlwaysRetry alwaysRetry = new AlwaysRetry(2);
+        CuratorFramework client = CuratorFrameworkFactory.newClient(server.getConnectString(), alwaysRetry);
+        client.start();
+        try {
+            // given: error background request with always-retry policy
+            CompletableFuture<CuratorEvent> future = new CompletableFuture<>();
+            client.delete().inBackground((ignored, event) -> future.complete(event)).forPath("/test");
+
+            // These chaos steps create chances to run into concurrent contentions.
+            // They could fail this test given enough runs if there are bugs.

Review Comment:
   I am reluctant and possibly incapable to mock internals here. And I have seen all paths routed to `CuratorFrameworkImpl::closeOperation` with these chaos steps in  IDEA's "Repeat until failure", there are `close`, `queueOperation` and `requeueSleepOperation`. That's it.



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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


[GitHub] [curator] kezhuw commented on a diff in pull request #464: CURATOR-673: Complete BackgroundCallback if CuratorFramework got closed

Posted by "kezhuw (via GitHub)" <gi...@apache.org>.
kezhuw commented on code in PR #464:
URL: https://github.com/apache/curator/pull/464#discussion_r1212550593


##########
curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java:
##########
@@ -694,17 +714,46 @@ <DATA_TYPE> void processBackgroundOperation(OperationAndData<DATA_TYPE> operatio
         }
     }
 
+    private void closeOperation(OperationAndData<?> operation) {
+        if (operation.getCallback() == null) {
+            return;
+        }
+        CuratorEvent event = new CuratorEventImpl(this, CuratorEventType.CLOSING, KeeperException.Code.SESSIONEXPIRED.intValue(), null, null, operation.getContext(), null, null, null, null, null, null);

Review Comment:
   Yeh, it is hard to choose 😮‍💨.  `performBackgroundOperation` uses `CONNECTIONLOSS` for callbacks when retry gave up in case of no active connection. For `BackgroundCallback`, I think it is more appropriate to treat `CuratorEvent::getResultCode` as authority for success or not and. From client side, no updates are required (at my best guess).
   
   Aside from this, `CuratorEvent::getType` is simple a "nice to match" for `BackgroundCallback`. It plays a significant role to differentiate events in `CuratorListener` which is catch-all fallback for events with no `BackgroundCallback`.
   
   https://github.com/apache/curator/blob/328f985304b07cec046e4b9e4244a9c962fc91f5/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java#L681-L687
   
   `WATCHED` is used in `CONNECTIONLOSS` case already. I would prefer to let `OperationAndData` carrying `CuratorEventType` from creation so we could delivery matching event type.  In case of closing, `CuratorEventType.CLOSING` is not a bad choice(actually there is no good candidate currently), but it is not worth us to treat `CuratorEventType.CLOSING` specially for `BackgroundCallback`. Maybe a separated issue to cover this ?
   
   https://github.com/apache/curator/blob/328f985304b07cec046e4b9e4244a9c962fc91f5/curator-framework/src/main/java/org/apache/curator/framework/imps/CuratorFrameworkImpl.java#L1019-L1024



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

To unsubscribe, e-mail: commits-unsubscribe@curator.apache.org

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