Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -540,10 +540,17 @@ public String getLastPathIsLeader()
@VisibleForTesting
volatile CountDownLatch debugResetWaitLatch = null;

@VisibleForTesting
volatile CountDownLatch debugRestWaitBeforeNodeDelete = null;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
volatile CountDownLatch debugRestWaitBeforeNodeDelete = null;
volatile CountDownLatch debugResetWaitBeforeNodeDeleteLatch = null;

There's a typo in the name. Additionally, we might want to add Latch at the end to reflect the purpose of this member analogously to the other latches.


@VisibleForTesting
void reset() throws Exception
{
setLeadership(false);
if ( debugRestWaitBeforeNodeDelete != null )
{
debugRestWaitBeforeNodeDelete.await();
}
setNode(null);

BackgroundCallback callback = new BackgroundCallback()
Expand Down Expand Up @@ -623,6 +630,7 @@ else if ( ourIndex == 0 )
}
else
{
setLeadership(false);
String watchPath = sortedChildren.get(ourIndex - 1);
Watcher watcher = new Watcher()
{
Expand Down Expand Up @@ -726,7 +734,6 @@ protected void handleStateChange(ConnectionState newState)
private synchronized void setLeadership(boolean newValue)
{
boolean oldValue = hasLeadership.getAndSet(newValue);

if ( oldValue && !newValue )
{ // Lost leadership, was true, now false
listeners.forEach(LeaderLatchListener::notLeader);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import com.google.common.collect.Queues;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.time.Duration;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ForkJoinPool;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.framework.imps.TestCleanState;
Expand All @@ -48,6 +50,7 @@
import org.apache.curator.test.compatibility.Timing2;
import org.apache.curator.utils.CloseableUtils;
import org.awaitility.Awaitility;
import org.awaitility.core.ThrowingRunnable;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;

Expand Down Expand Up @@ -220,6 +223,108 @@ public void testWatchedNodeDeletedOnReconnect() throws Exception
}
}

@Test
public void testCheckLeaderShipTiming() throws Exception
{
final String latchPath = "/test";
Timing timing = new Timing();
List<LeaderLatch> latches = Lists.newArrayList();
List<CuratorFramework> clients = Lists.newArrayList();
final BlockingQueue<String> states = Queues.newLinkedBlockingQueue();
ExecutorService executorService = Executors.newFixedThreadPool(2);
for ( int i = 0; i < 2; ++i ) {
try {
CuratorFramework client = CuratorFrameworkFactory.builder()
.connectString(server.getConnectString())
.connectionTimeoutMs(10000)
.sessionTimeoutMs(60000)
.retryPolicy(new RetryOneTime(1))
.connectionStateErrorPolicy(new StandardConnectionStateErrorPolicy())
.build();
ConnectionStateListener stateListener = new ConnectionStateListener()
{
@Override
public void stateChanged(CuratorFramework client, ConnectionState newState)
{
if (newState == ConnectionState.CONNECTED) {
states.add(newState.name());
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Adding the LeaderLatch ID (we could use for loop i) here might help further down in the test understanding the state of the queue.

}
}
};
client.getConnectionStateListenable().addListener(stateListener);
client.start();
clients.add(client);
LeaderLatch latch = new LeaderLatch(client, latchPath, String.valueOf(i));
LeaderLatchListener listener = new LeaderLatchListener() {
@Override
public void isLeader() {
states.add("true");
}

@Override
public void notLeader() {
states.add("false");
}
};
Comment on lines +258 to +268
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Using the LeaderLatch ID in the event labels (as mentioned above) might help when evaluating the queue later on in the test. But to be fair: doing the asserts on hasLeadership like it's already done below (lines 303-304) serves the same purpose. I just mention it as another idea here. 🤷

latch.addListener(listener);
latch.start();
latches.add(latch);
assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), ConnectionState.CONNECTED.name());
if (i == 0) {
assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "true");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "true");
assertEquals("The first LeaderLatch instance should acquire leadership.", states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "true");

nit: maybe adding a bit more context to this polling here to describe the test case

}
}
catch (Exception e){
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why are we hiding thrown exceptions here? Shouldn't we expose it as part of the test run if something went wrong? 🤔 The test would succeed if the Exception is thrown in this block and caught here.

return;
}
}
timing.forWaiting().sleepABit();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the purpose of waiting here? 🤔

// now latch1 is leader, latch2 is not leader. latch2 listens to the ephemeral node created by latch1
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Moving comments into assert messages improves the test output and still works as some kind of comment. This comment could be added to the assertTrue and assertFalse in line 284-285 below describing the currently expected state.

LeaderLatch latch1 = latches.get(0);
LeaderLatch latch2 = latches.get(1);
Comment on lines +283 to +284
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
LeaderLatch latch1 = latches.get(0);
LeaderLatch latch2 = latches.get(1);
LeaderLatch initialLeaderLatch = latches.get(0);
LeaderLatch initialNonLeaderLatch = latches.get(1);

nit: maybe, making the variable more descriptive to avoid confusion. Especially because we're switching the order here in comparison to what is described in the PR description and the corresponding ticket.

assertTrue(latch1.hasLeadership());
assertFalse(latch2.hasLeadership());
try {
latch2.debugRestWaitBeforeNodeDelete = new CountDownLatch(1);
latch2.debugResetWaitLatch = new CountDownLatch(1);
latch1.debugResetWaitLatch = new CountDownLatch(1);

// force latch1 and latch2 reset
latch1.reset();
ForkJoinPool.commonPool().submit(() -> {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we add a comment here on why we're calling latch2.reset() in a separate thread? AFAIU, it's done to not make the test's thread block due to latch2.debugRestWaitBeforeNodeDelete. It might help readers if this is reflected in a comment here. WDYT?

latch2.reset();
return null;
});

// latch1 set itself is not the leader state and will delete old path and create new path then wait before getChildren
// latch2 wait before delete its old path and receive nodeDeleteEvent and then getChildren find itself is leader
assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "false"); //latch1 is not leader
assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "true"); //latch2 is leader
assertTrue(latch2.hasLeadership());
assertFalse(latch1.hasLeadership());
// latch1 continue and getChildren and find itself is not the leader and listen to the node created by latch2
latch1.debugResetWaitLatch.countDown();
timing.sleepABit();
// latch2 continue and delete old path and create new path then wait before getChildren
latch2.debugRestWaitBeforeNodeDelete.countDown();
// latch1 receive nodeDeleteEvent and then getChildren find itself is leader
assertEquals(states.poll(timing.forWaiting().milliseconds(), TimeUnit.MILLISECONDS), "true");
assertTrue(latch1.hasLeadership());
latch2.debugResetWaitLatch.countDown(); // latch2 continue and getChildren find itself is not leader
timing.forWaiting().sleepABit();

assertTrue(latch1.hasLeadership());
assertFalse(latch2.hasLeadership());
}
finally {
for(int i = 0; i < clients.size(); ++i) {
CloseableUtils.closeQuietly(latches.get(i));
CloseableUtils.closeQuietly(clients.get(i));
}
executorService.shutdown();
}
}

@Test
public void testLeadershipElectionWhenNodeDisappearsAfterChildrenAreRetrieved() throws Exception
{
Expand Down