-
Notifications
You must be signed in to change notification settings - Fork 444
RATIS-2548. Stabilize timing-sensitive Ratis tests #1475
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 3 commits
ee03331
83627dc
0a56010
82718bc
51b60c3
fb3ee4e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -51,6 +51,7 @@ | |
| import org.slf4j.event.Level; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
| import java.util.Optional; | ||
| import java.util.Set; | ||
|
|
@@ -127,46 +128,41 @@ static void runTestBasicAppendEntries( | |
|
|
||
| final CompletableFuture<Void> killAndRestartFollower = killAndRestartServer( | ||
| cluster.getFollowers().get(0).getId(), 0, 1000, cluster, log); | ||
| final CompletableFuture<Void> killAndRestartLeader; | ||
| if (killLeader) { | ||
| log.info("killAndRestart leader " + leader.getId()); | ||
| killAndRestartLeader = killAndRestartServer(leader.getId(), 2000, 4000, cluster, log); | ||
| } else { | ||
| killAndRestartLeader = CompletableFuture.completedFuture(null); | ||
| } | ||
|
|
||
| log.info(cluster.printServers()); | ||
| CompletableFuture<Void> killAndRestartLeader = CompletableFuture.completedFuture(null); | ||
|
|
||
| final SimpleMessage[] messages = SimpleMessage.create(numMessages); | ||
|
|
||
| try (final RaftClient client = cluster.createClient()) { | ||
| final AtomicInteger asyncReplyCount = new AtomicInteger(); | ||
| final CompletableFuture<Void> f = new CompletableFuture<>(); | ||
| try { | ||
| log.info(cluster.printServers()); | ||
|
|
||
| try (final RaftClient client = cluster.createClient()) { | ||
| final List<CompletableFuture<RaftClientReply>> asyncReplies = new ArrayList<>(); | ||
|
|
||
| for (SimpleMessage message : messages) { | ||
| for (SimpleMessage message : messages) { | ||
| if (async) { | ||
| asyncReplies.add(client.async().send(message)); | ||
| } else { | ||
| final RaftClientReply reply = client.io().send(message); | ||
| Assertions.assertTrue(reply.isSuccess()); | ||
| } | ||
| } | ||
| if (async) { | ||
| client.async().send(message).thenAcceptAsync(reply -> { | ||
| if (!reply.isSuccess()) { | ||
| f.completeExceptionally( | ||
| new AssertionError("Failed with reply " + reply)); | ||
| } else if (asyncReplyCount.incrementAndGet() == messages.length) { | ||
| f.complete(null); | ||
| } | ||
| CompletableFuture.allOf(asyncReplies.toArray(new CompletableFuture<?>[0])).join(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since join() is called below. This allOf is not needed. Let's remove it. BTW, changing final AtomicInteger asyncReplyCount = new AtomicInteger();
final CompletableFuture<Void> f = new CompletableFuture<>();to final List<CompletableFuture<RaftClientReply>> asyncReplies = new ArrayList<>();does make the code easier to understand (although the original code is also correct.)
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. Removed the redundant |
||
| asyncReplies.forEach(f -> { | ||
| final RaftClientReply reply = f.join(); | ||
| Assertions.assertTrue(reply.isSuccess(), () -> "Failed with reply " + reply); | ||
| }); | ||
| } else { | ||
| final RaftClientReply reply = client.io().send(message); | ||
| Assertions.assertTrue(reply.isSuccess()); | ||
| } | ||
| } | ||
| if (async) { | ||
| f.join(); | ||
| Assertions.assertEquals(messages.length, asyncReplyCount.get()); | ||
| if (killLeader) { | ||
| log.info("killAndRestart leader " + leader.getId()); | ||
| killAndRestartLeader = killAndRestartServer(leader.getId(), 0, 4000, cluster, log); | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Before this change, killLeader is in the beginning. This change moves it to the end. It makes the test easier to pass but not fixing a bug. It is good to test killLeader before client sending messages. So, let's don't make this change?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. Restored the kill-leader timing so the leader restart is scheduled before client messages are sent. The final kill-leader assertion now checks that the expected messages appear in order, while allowing an extra state-machine entry produced by retry/failover. |
||
| Thread.sleep(cluster.getTimeoutMax().toIntExact(TimeUnit.MILLISECONDS) + 100); | ||
| } finally { | ||
| CompletableFuture.allOf(killAndRestartFollower, killAndRestartLeader).join(); | ||
| } | ||
| Thread.sleep(cluster.getTimeoutMax().toIntExact(TimeUnit.MILLISECONDS) + 100); | ||
| log.info(cluster.printAllLogs()); | ||
| killAndRestartFollower.join(); | ||
| killAndRestartLeader.join(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
You are right that we should join before printing the log. How about we simply move cluster.printAllLogs() up? The try-finally make the code harder to read. Thread.sleep(cluster.getTimeoutMax().toIntExact(TimeUnit.MILLISECONDS) + 100);
- log.info(cluster.printAllLogs());
killAndRestartFollower.join();
killAndRestartLeader.join();
+ log.info(cluster.printAllLogs());
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. Removed the try/finally restructuring and now join the restart futures before |
||
|
|
||
|
|
||
| final List<RaftServer.Division> divisions = cluster.getServerAliveStream().collect(Collectors.toList()); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -150,13 +150,16 @@ public void testElectionStepDownCommand() throws Exception { | |
| void runTestElectionStepDownCommand(MiniRaftCluster cluster) throws Exception { | ||
| final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); | ||
| String sb = getClusterAddress(cluster); | ||
| RaftServer.Division newLeader = cluster.getFollowers().get(0); | ||
| final StringPrintStream out = new StringPrintStream(); | ||
| RatisShell shell = new RatisShell(out.getPrintStream()); | ||
| Assertions.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); | ||
| Assertions.assertEquals(2, cluster.getFollowers().size()); | ||
| int ret = shell.run("election", "stepDown", "-peers", sb.toString()); | ||
| int ret = shell.run("election", "pause", "-peers", sb.toString(), "-address", | ||
| leader.getPeer().getAddress()); | ||
| Assertions.assertEquals(0, ret); | ||
|
|
||
| ret = shell.run("election", "stepDown", "-peers", sb.toString()); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This change is good. Could you also remove the redundant toString() calls? int ret = shell.run("election", "pause", "-peers", sb, "-address", leader.getPeer().getAddress());
Assertions.assertEquals(0, ret);
ret = shell.run("election", "stepDown", "-peers", sb);
Assertions.assertEquals(0, ret);
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. Removed the redundant |
||
| Assertions.assertEquals(0, ret); | ||
| Assertions.assertEquals(3, cluster.getFollowers().size()); | ||
| JavaUtils.attempt(() -> Assertions.assertNotEquals(leader.getId(), RaftTestUtil.waitForLeader(cluster).getId()), | ||
| 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionStepDownCommand", LOG); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's do this change separately. Then, this PR changes only the test code.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done. Removed the
LeaderStateImplchange from this PR, so the current PR diff is test-only now. I will handle that leadership check separately.