kevin-wu24 commented on code in PR #18987:
URL: https://github.com/apache/kafka/pull/18987#discussion_r1980459283


##########
raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java:
##########
@@ -117,40 +126,58 @@ public class RaftEventSimulationTest {
     void canElectInitialLeader(
         @ForAll int seed,
         @ForAll @IntRange(min = 1, max = 5) int numVoters,
-        @ForAll @IntRange(min = 0, max = 5) int numObservers
+        @ForAll @IntRange(min = 0, max = 5) int numObservers,
+        @ForAll boolean withKip853
     ) {
         Random random = new Random(seed);
-        Cluster cluster = new Cluster(numVoters, numObservers, random);
+        Cluster cluster = new Cluster(numVoters, numObservers, random, 
withKip853);
         MessageRouter router = new MessageRouter(cluster);
         EventScheduler scheduler = schedulerWithDefaultInvariants(cluster);
 
-        cluster.startAll();
-        schedulePolling(scheduler, cluster, 3, 5);
-        scheduler.schedule(router::deliverAll, 0, 2, 1);
-        scheduler.schedule(new SequentialAppendAction(cluster), 0, 2, 3);
-        scheduler.runUntil(cluster::hasConsistentLeader);
-        scheduler.runUntil(() -> cluster.allReachedHighWatermark(10));
+        initializeClusterAndStartAppending(cluster, router, scheduler, 2);
+    }
+
+    @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY)
+    void canAddAndRemoveVoters(
+        @ForAll int seed
+    ) {
+        Random random = new Random(seed);
+        int numVoters = 3;
+        int numObservers = 2;
+        Cluster cluster = new Cluster(numVoters, numObservers, random, true);
+        MessageRouter router = new MessageRouter(cluster);
+        EventScheduler scheduler = schedulerWithDefaultInvariants(cluster);
+        Set<Integer> expectedVoterIds = new 
HashSet<>(cluster.initialVoters.keySet());
+
+        initializeClusterAndStartAppending(cluster, router, scheduler, 10);
+
+        // Add all observers to voter set one by one
+        for (int voterIdToAdd = numVoters; voterIdToAdd < numVoters + 
numObservers; voterIdToAdd++) {
+            addVoter(cluster, scheduler, voterIdToAdd, expectedVoterIds);
+        }
+        runUntilNewHighWatermark(cluster, scheduler);

Review Comment:
   I agree it is arbitrary. However, other tests do end with lines like 
   ```
   scheduler.runUntil(() -> cluster.allReachedHighWatermark(highWatermark + 
10));
   ```
   It's mainly just to actually continue delivering messages and polling a bit 
on the raft nodes after we commit the VotersRecord from calling 
`addVoter/removeVoter`. Otherwise, this test and others like it would just end 
after we commit the VotersRecord from the last call to `addVoter`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to