[ https://issues.apache.org/jira/browse/CURATOR-644?focusedWorklogId=808616&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-808616 ]
ASF GitHub Bot logged work on CURATOR-644: ------------------------------------------ Author: ASF GitHub Bot Created on: 14/Sep/22 10:07 Start Date: 14/Sep/22 10:07 Worklog Time Spent: 10m Work Description: XComp commented on code in PR #430: URL: https://github.com/apache/curator/pull/430#discussion_r970541350 ########## curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java: ########## @@ -586,6 +586,9 @@ private void checkLeadership(List<String> children) throws Exception final String localOurPath = ourPath.get(); List<String> sortedChildren = LockInternals.getSortedChildren(LOCK_NAME, sorter, children); int ourIndex = (localOurPath != null) ? sortedChildren.indexOf(ZKPaths.getNodeFromPath(localOurPath)) : -1; + + log.debug("checkLeadership with ourPath: {}, children: {}", localOurPath, sortedChildren); Review Comment: ```suggestion log.debug("[{}] checkLeadership with ourPath: {}, children: {}", id, localOurPath, sortedChildren); ``` ########## 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 are 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(); + + // no extra CountDownLatch needs to be set here because candidate #1 will rely on candidate #0 + latchCandidate1.start(); + timing.sleepABit(); Review Comment: You need to add `Awaitility` to the pom of the `curator-recipes` module: ``` <dependency> <groupId>org.awaitility</groupId> <artifactId>awaitility</artifactId> <scope>test</scope> </dependency> ``` ########## 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 are 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(); + + // no extra CountDownLatch needs to be set here because candidate #1 will rely on candidate #0 + latchCandidate1.start(); + timing.sleepABit(); Review Comment: ```suggestion // we want to make sure that the leader gets leadership before other instances are going to join 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(); final int expectedChildrenAfterCandidate0Joins = 2; Awaitility.await("There should be " + expectedChildrenAfterCandidate0Joins + " child nodes created after candidate #0 joins the leader election.") .pollInterval(Duration.ofMillis(100)) .pollInSameThread() .until(() -> client.getChildren().forPath(latchPath).size() == expectedChildrenAfterCandidate0Joins); // no extra CountDownLatch needs to be set here because candidate #1 will rely on candidate #0 latchCandidate1.start(); final int expectedChildrenAfterCandidate1Joins = 3; Awaitility.await("There should be " + expectedChildrenAfterCandidate1Joins + " child nodes created after candidate #1 joins the leader election.") .pollInterval(Duration.ofMillis(100)) .pollInSameThread() .until(() -> client.getChildren().forPath(latchPath).size() == expectedChildrenAfterCandidate1Joins); ``` Initially, I played around with checking the `ourPath` of each `LeaderLatch` instance. This didn't work, because that field is set in a `BackgroundCallback` (see [LeaderLatch:540](https://github.com/apache/curator/blob/2775855fc80fdea8433d5dedbb746f58ddd443c4/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java#L540)). The background tasks are handled in the `EventThread` of the ZK client. That one is already blocked by the `CountdownLatch` of candidate #0. But we could still use the `SendThread` which is used for outgoing commands. That's why calling `.getChildren` on the client itself still works and is sufficient enough to verify that the child nodes are actually created. ########## curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java: ########## @@ -667,9 +670,9 @@ protected void handleStateChange(ConnectionState newState) { try { - if ( client.getConnectionStateErrorPolicy().isErrorState(ConnectionState.SUSPENDED) || !hasLeadership.get() ) + if ( client.getConnectionStateErrorPolicy().isErrorState(ConnectionState.SUSPENDED) ) { - reset(); + getChildren(); Review Comment: I'm not convinced anymore that this change is correct. Here, we're covering the case where `ConnectionState.SUSPENDED` is considered an error state, i.e. the leadership is lost if the connection was suspended. In that case, we should do a reset because we're considering the leadership being lost already due to the connection instability having caused an error. Instead, we should add an `else` branch that does the `getChildren` call in case of a reconnect and the policy is allowing reconnects after suspension. Issue Time Tracking ------------------- Worklog Id: (was: 808616) Time Spent: 5.5h (was: 5h 20m) > 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: 5.5h > 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)