diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml
index 1ca382883..98c357118 100644
--- a/.mvn/extensions.xml
+++ b/.mvn/extensions.xml
@@ -24,7 +24,7 @@
com.gradle
develocity-maven-extension
- 1.21.4
+ 1.23
com.gradle
diff --git a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
index a1cddde7e..4226e8819 100644
--- a/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
+++ b/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderLatch.java
@@ -624,18 +624,21 @@ public void processResult(CuratorFramework client, CuratorEvent event) throws Ex
client.getChildren().inBackground(callback).forPath(ZKPaths.makePath(latchPath, null));
}
+ @VisibleForTesting
+ volatile CountDownLatch debugHandleReconnectedLatch = null;
+
@VisibleForTesting
protected void handleStateChange(ConnectionState newState) {
switch (newState) {
case RECONNECTED: {
try {
- if (client.getConnectionStateErrorPolicy().isErrorState(ConnectionState.SUSPENDED)
- || !hasLeadership.get()) {
- getChildren();
+ if (debugHandleReconnectedLatch != null) {
+ debugHandleReconnectedLatch.await();
}
+ getChildren();
} catch (Exception e) {
ThreadUtils.checkInterrupted(e);
- log.error("failed to reset leader latch", e);
+ log.error("failed to recheck leadership on reconnected", e);
setLeadership(false);
}
break;
diff --git a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
index 528b317ff..0dc375d16 100644
--- a/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
+++ b/curator-recipes/src/test/java/org/apache/curator/framework/recipes/leader/TestLeaderLatch.java
@@ -179,6 +179,63 @@ public void testUncreatedPathGetLeader() throws Exception {
}
}
+ // @see https://issues.apache.org/jira/browse/CURATOR-724
+ @Test
+ public void testGetChildrenHitsNoNode() throws Exception {
+ final String latchPath = "/testGetChildrenHitsNoNode";
+ final Timing2 timing = new Timing2();
+ final BlockingQueue events0 = new LinkedBlockingQueue<>();
+ final BlockingQueue events1 = new LinkedBlockingQueue<>();
+
+ final List closeableResources = new ArrayList<>();
+ try {
+ final String id0 = "id0";
+ final CuratorFramework client0 = createAndStartClient(server.getConnectString(), timing, id0, null);
+ closeableResources.add(client0);
+ final LeaderLatch latch0 = createAndStartLeaderLatch(client0, latchPath, id0, events0);
+ closeableResources.add(latch0);
+
+ assertThat(events0.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS))
+ .isNotNull()
+ .isEqualTo(new TestEvent(id0, TestEventType.GAINED_LEADERSHIP));
+
+ final String id1 = "id1";
+ final CuratorFramework client1 = createAndStartClient(server.getConnectString(), timing, id1, null);
+ closeableResources.add(client1);
+ final LeaderLatch latch1 = createAndStartLeaderLatch(client1, latchPath, id1, events1);
+ closeableResources.add(latch1);
+
+ // wait for the non-leading LeaderLatch (i.e. latch1) instance to be done with its creation
+ // this call is time-consuming but necessary because we don't have a handle to detect the end of the reset
+ // call
+ timing.forWaiting().sleepABit();
+
+ assertTrue(latch0.hasLeadership());
+ assertFalse(latch1.hasLeadership());
+
+ // ensure we can observe the leadership transferred to latch1
+ latch0.debugHandleReconnectedLatch = new CountDownLatch(1);
+
+ // scale to zero - recreate the cluster
+ final int port = server.getPort();
+ server.close();
+ server = new TestingServer(port, true);
+
+ assertThat(events1.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS))
+ .isNotNull()
+ .isEqualTo(new TestEvent(id1, TestEventType.GAINED_LEADERSHIP));
+
+ latch0.debugHandleReconnectedLatch.countDown();
+ assertThat(events0.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS))
+ .isNotNull()
+ .isEqualTo(new TestEvent(id0, TestEventType.LOST_LEADERSHIP));
+ } finally {
+ // reverse is necessary for closing the LeaderLatch instances before closing the corresponding client
+ Collections.reverse(closeableResources);
+ closeableResources.forEach(CloseableUtils::closeQuietly);
+ }
+ }
+
@Test
public void testWatchedNodeDeletedOnReconnect() throws Exception {
final String latchPath = "/foo/bar";