[ 
https://issues.apache.org/jira/browse/CURATOR-644?focusedWorklogId=808233&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-808233
 ]

ASF GitHub Bot logged work on CURATOR-644:
------------------------------------------

                Author: ASF GitHub Bot
            Created on: 13/Sep/22 11:51
            Start Date: 13/Sep/22 11:51
    Worklog Time Spent: 10m 
      Work Description: XComp commented on code in PR #430:
URL: https://github.com/apache/curator/pull/430#discussion_r969519573


##########
curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java:
##########
@@ -218,6 +218,56 @@ public void testWatchedNodeDeletedOnReconnect() throws 
Exception
         }
     }
 
+    @Test
+    public void 
testLeadershipElectionWhenNodeDisappearsAfterChildrenAreRetrieved() throws 
Exception
+    {
+        final String latchPath = "/foo/bar";
+        final Timing2 timing = new Timing2();
+        try (CuratorFramework client = 
CuratorFrameworkFactory.newClient(server.getConnectString(), timing.session(), 
timing.connection(), new RetryOneTime(1)))
+        {
+            client.start();
+            LeaderLatch latchInitialLeader = new LeaderLatch(client, 
latchPath, "initial-leader");
+            LeaderLatch latchCandidate0 = new LeaderLatch(client, latchPath, 
"candidate-0");
+            LeaderLatch latchCandidate1 = new LeaderLatch(client, latchPath, 
"candidate-1");
+
+            try
+            {
+                latchInitialLeader.start();
+
+                // we want to make sure that the leader gets leadership before 
other instances joining the party
+                waitForALeader(Collections.singletonList(latchInitialLeader), 
new Timing());
+
+                // candidate #0 will wait for the leader to go away - this 
should happen after the child nodes are retrieved by candidate #0
+                latchCandidate0.debugCheckLeaderShipLatch = new 
CountDownLatch(1);
+
+                latchCandidate0.start();
+                timing.sleepABit();

Review Comment:
   tbh, I'm not really happy with the sleep here and in [line 
248](https://github.com/apache/curator/pull/430/files#diff-75966280cab1f9788b771d244e889731ba35c7918d365c070565e070d5b801ebR248)
 because they are a cause for instabilities: The `close` in [line 
251](https://github.com/apache/curator/pull/430/files#diff-75966280cab1f9788b771d244e889731ba35c7918d365c070565e070d5b801ebR251)
 has to happen after the child nodes for `candidate #0` and `candidate #1` are 
created. AFAIU, we cannot ensure that with the sleep calls due to the 
asynchronous nature of the `start` command that is triggered right before each 
sleep.
   I tried to add a `waitForCondition` instead, when coming up with this test, 
initially, that would wait for corresponding child to be created. 
Unfortunately, this resulted in the test blocking forever because (I guess) the 
await on the `latchCandidate0.debugCheckLeaderShipLatch` is executed in the 
main thread which makes any subsequent operation (including the check for 
children nodes) being blocked.
   
   I hoped that somebody else could come up with a better approach here. 
:innocent: 





Issue Time Tracking
-------------------

    Worklog Id:     (was: 808233)
    Time Spent: 4h 10m  (was: 4h)

> CLONE - Race conditions in LeaderLatch after reconnecting to ensemble
> ---------------------------------------------------------------------
>
>                 Key: CURATOR-644
>                 URL: https://issues.apache.org/jira/browse/CURATOR-644
>             Project: Apache Curator
>          Issue Type: Bug
>    Affects Versions: 4.2.0
>            Reporter: Ken Huang
>            Assignee: Jordan Zimmerman
>            Priority: Minor
>          Time Spent: 4h 10m
>  Remaining Estimate: 0h
>
> Clone from CURATOR-504.
> We use LeaderLatch in a lot of places in our system and when ZooKeeper 
> ensemble is unstable and clients are reconnecting to logs are full of 
> messages like the following:
> {{{}[2017-08-31 
> 19:18:34,562][ERROR][org.apache.curator.framework.recipes.leader.LeaderLatch] 
> Can't find our node. Resetting. Index: -1 {{}}}}
> According to the 
> [implementation|https://github.com/apache/curator/blob/4251fe328908e5fca37af034fabc190aa452c73f/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java#L529-L536],
>  this can happen in two cases:
>  * When internal state `ourPath` is null
>  * When the list of latches does not have the expected one.
> I believe we hit the first condition because of races that occur after client 
> reconnects to ZooKeeper.
>  * Client reconnects to ZooKeeper and LeaderLatch gets the event and calls 
> reset method which set the internal state (`ourPath`) to null, removes old 
> latch and creates a new one. This happens in thread 
> "Curator-ConnectionStateManager-0".
>  * Almost simultaneously, LeaderLatch gets another even NodeDeleted 
> ([here|https://github.com/apache/curator/blob/4251fe328908e5fca37af034fabc190aa452c73f/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java#L543-L554])
>  and tries to re-read the list of latches and check leadership. This happens 
> in the thread "main-EventThread".
> Therefore, sometimes there is a situation when method `checkLeadership` is 
> called when `ourPath` is null.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to