ahuang98 commented on code in PR #18987:
URL: https://github.com/apache/kafka/pull/18987#discussion_r1980407781


##########
raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java:
##########
@@ -117,40 +125,101 @@ 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 canElectInitialLeaderAndAddVoter(
+        @ForAll int seed,
+        @ForAll @IntRange(min = 1, max = 5) int numVoters,
+        @ForAll @IntRange(min = 1, max = 5) int numObservers
+    ) {
+        Random random = new Random(seed);
+        Cluster cluster = new Cluster(numVoters, numObservers, random, true);
+        MessageRouter router = new MessageRouter(cluster);
+        EventScheduler scheduler = schedulerWithKip853Invariants(cluster);
+
+        initializeClusterAndStartAppending(cluster, router, scheduler, 10);
+        int firstObserverId = numVoters;
+        scheduler.schedule(new AddVoterAction(cluster, 
cluster.running.get(firstObserverId)), 0, 5, 3);
+        scheduler.runUntil(() -> 
cluster.leaderWithMaxEpoch().get().client.partitionState().lastVoterSet().size()
 == numVoters + 1);
+        VoterSet latestVoterSet = 
cluster.leaderWithMaxEpoch().get().client.partitionState().lastVoterSet();
+        scheduler.runUntil(() -> 
cluster.allHaveLatestVoterSet(latestVoterSet));
+    }
+
+    @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY)
+    void canAddAndRemoveVoters(

Review Comment:
   > I remove a random node so that the tries property means we will remove a 
leader with probability = 1/numVoters on each try.
   
   which test will do this? this one just removes all observers and I see 
another which partitions a voter (would lose leadership) then removes it 



-- 
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