showuon commented on code in PR #20859:
URL: https://github.com/apache/kafka/pull/20859#discussion_r2544638061
##########
server/src/test/java/org/apache/kafka/server/ReconfigurableQuorumIntegrationTest.java:
##########
@@ -218,6 +218,82 @@ public void testControllersAutoJoinStandaloneVoter()
throws Exception {
}
}
+ @Test
+ public void testRemovedControllerWontJoinAgain() throws Exception {
+ final var nodes = new TestKitNodes.Builder().
+ setNumBrokerNodes(1).
+ setNumControllerNodes(3).
+ build();
+
+ // Configure the initial voters with one voter having a different
directory ID.
+ // This simulates the case where the controller failed and is brought
back up with a different directory ID.
+ final Map<Integer, Uuid> initialVoters = new HashMap<>();
+ final var oldDirectoryId = Uuid.randomUuid();
+ for (final var controllerNode : nodes.controllerNodes().values()) {
+ initialVoters.put(
+ controllerNode.id(),
+ controllerNode.id() == TestKitDefaults.CONTROLLER_ID_OFFSET ?
+ oldDirectoryId : 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 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 removed
+ 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));
+ }
+ });
+
+
+ // do not join the voter set in the next twenty seconds
+ for (int i = 0; i < 20; ++i) {
+ TestUtils.retryOnExceptionWithTimeout(30_000, 100, () -> {
+ TestUtils.retryOnExceptionWithTimeout(30_000, 100, ()
-> {
+ Map<Integer, Uuid> voters = findVoterDirs(admin);
+ if (!voters.containsKey(3002)) {
+ // if there are no node 3002, it should be
removed
+ 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));
+ }
+ });
+ });
+ Thread.sleep(1000);
+ }
Review Comment:
We can test won't auto join the removed node by:
1. Remove a controller node, ex: 3002.
2. Verify the highwatermark in controller metadata log is advancing by:
`cluster.raftManagers().get(3002)..client().highWatermark()`
The high watermark increasing means the follower is fetching the data from
the leader, and because we auto join the controller node before fetching from
the leader, if the node can be auto joined, it'll be already added.
3. Verify the voters are still 3000, 3001, without 3002.
So, we don't need the mysterious 20 times loop to do the voter removal.
What do you think?
--
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]