showuon commented on code in PR #20859:
URL: https://github.com/apache/kafka/pull/20859#discussion_r2548531651


##########
server/src/test/java/org/apache/kafka/server/ReconfigurableQuorumIntegrationTest.java:
##########
@@ -218,6 +218,74 @@ public void testControllersAutoJoinStandaloneVoter() 
throws Exception {
         }
     }
 
+    @Test
+    public void testRemovedControllerWontJoinAgain() throws Exception {
+        final var nodes = new TestKitNodes.Builder().
+                setNumBrokerNodes(1).
+                setNumControllerNodes(3).
+                build();
+
+        final Map<Integer, Uuid> initialVoters = new HashMap<>();
+        for (final var controllerNode : nodes.controllerNodes().values()) {
+            initialVoters.put(
+                controllerNode.id(), controllerNode.metadataDirectoryId()
+            );
+        }
+
+        try (KafkaClusterTestKit cluster = new 
KafkaClusterTestKit.Builder(nodes).
+                setConfigProp(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, 
true).
+                setInitialVoterSet(initialVoters).
+                build()
+        ) {
+            cluster.format();
+            cluster.startup();
+            try (var admin = Admin.create(cluster.clientProperties())) {
+                // Static voter set is initialized
+                TestUtils.retryOnExceptionWithTimeout(30_000, 100, () -> {
+                    Map<Integer, Uuid> voters = findVoterDirs(admin);
+                    assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
+                    for (int replicaId : new int[] {3000, 3001, 3002}) {
+                        
assertEquals(nodes.controllerNodes().get(replicaId).metadataDirectoryId(), 
voters.get(replicaId));
+                    }
+                });
+
+
+                // Remove 3002 from the voter set
+                TestUtils.retryOnExceptionWithTimeout(30_000, 100, () -> {
+                    Map<Integer, Uuid> voters = findVoterDirs(admin);
+                    if (!voters.containsKey(3002)) {
+                        // if there are no node 3002, it should be return
+                        return;
+                    }
+
+                    admin.removeRaftVoter(3002, voters.get(3002)).all().get();
+                    assertEquals(Set.of(3000, 3001), voters.keySet());
+                    for (int replicaId : new int[] {3000, 3001}) {
+                        
assertEquals(nodes.controllerNodes().get(replicaId).metadataDirectoryId(), 
voters.get(replicaId));
+                    }
+                });

Review Comment:
   This work, though, it's not clear because if the voters update takes some 
time, you could retry many times. And on each retry, you'll try to remove voter 
3002 again and again. We can do something similar like 
[this](https://github.com/apache/kafka/blob/trunk/server/src/test/java/org/apache/kafka/server/ReconfigurableQuorumIntegrationTest.java#L179-L186).
 It will remove voter once, and retry the validation until the voter updated.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to