|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.kafka.clients.admin; |
| 19 | + |
| 20 | +import org.apache.kafka.common.Uuid; |
| 21 | +import org.apache.kafka.common.errors.InconsistentClusterIdException; |
| 22 | +import org.apache.kafka.common.test.KafkaClusterTestKit; |
| 23 | +import org.apache.kafka.common.test.TestKitNodes; |
| 24 | +import org.apache.kafka.test.TestUtils; |
| 25 | + |
| 26 | +import org.junit.jupiter.api.Tag; |
| 27 | +import org.junit.jupiter.api.Test; |
| 28 | + |
| 29 | +import java.util.HashMap; |
| 30 | +import java.util.Map; |
| 31 | +import java.util.Optional; |
| 32 | +import java.util.Set; |
| 33 | +import java.util.concurrent.ExecutionException; |
| 34 | +import java.util.stream.Collectors; |
| 35 | + |
| 36 | +import static org.junit.jupiter.api.Assertions.assertEquals; |
| 37 | + |
| 38 | +@Tag("integration") |
| 39 | +public class ReconfigurableQuorumIntegrationTest { |
| 40 | + |
| 41 | + static Map<Integer, Uuid> descVoterDirs(Admin admin) throws ExecutionException, InterruptedException { |
| 42 | + var quorumInfo = admin.describeMetadataQuorum().quorumInfo().get(); |
| 43 | + return quorumInfo.voters().stream().collect(Collectors.toMap(QuorumInfo.ReplicaState::replicaId, QuorumInfo.ReplicaState::replicaDirectoryId)); |
| 44 | + } |
| 45 | + |
| 46 | + @Test |
| 47 | + public void testRemoveAndAddVoterWithValidClusterId() throws Exception { |
| 48 | + final var nodes = new TestKitNodes.Builder() |
| 49 | + .setClusterId("test-cluster") |
| 50 | + .setNumBrokerNodes(1) |
| 51 | + .setNumControllerNodes(3) |
| 52 | + .build(); |
| 53 | + |
| 54 | + final Map<Integer, Uuid> initialVoters = new HashMap<>(); |
| 55 | + for (final var controllerNode : nodes.controllerNodes().values()) { |
| 56 | + initialVoters.put( |
| 57 | + controllerNode.id(), |
| 58 | + controllerNode.metadataDirectoryId() |
| 59 | + ); |
| 60 | + } |
| 61 | + |
| 62 | + try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) { |
| 63 | + cluster.format(); |
| 64 | + cluster.startup(); |
| 65 | + try (Admin admin = Admin.create(cluster.clientProperties())) { |
| 66 | + TestUtils.waitForCondition(() -> { |
| 67 | + Map<Integer, Uuid> voters = descVoterDirs(admin); |
| 68 | + assertEquals(Set.of(3000, 3001, 3002), voters.keySet()); |
| 69 | + return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID)); |
| 70 | + }, "Initial quorum voters should be {3000, 3001, 3002} and all should have non-zero directory IDs"); |
| 71 | + |
| 72 | + Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId(); |
| 73 | + admin.removeRaftVoter( |
| 74 | + 3000, |
| 75 | + dirId, |
| 76 | + new RemoveRaftVoterOptions().setClusterId(Optional.of("test-cluster")) |
| 77 | + ).all().get(); |
| 78 | + TestUtils.waitForCondition(() -> { |
| 79 | + Map<Integer, Uuid> voters = descVoterDirs(admin); |
| 80 | + assertEquals(Set.of(3001, 3002), voters.keySet()); |
| 81 | + return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID)); |
| 82 | + }, "After removing voter 3000, remaining voters should be {3001, 3002} with non-zero directory IDs"); |
| 83 | + |
| 84 | + admin.addRaftVoter( |
| 85 | + 3000, |
| 86 | + dirId, |
| 87 | + Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)), |
| 88 | + new AddRaftVoterOptions().setClusterId(Optional.of("test-cluster")) |
| 89 | + ).all().get(); |
| 90 | + } |
| 91 | + } |
| 92 | + } |
| 93 | + |
| 94 | + @Test |
| 95 | + public void testRemoveAndAddVoterWithInconsistentClusterId() throws Exception { |
| 96 | + final var nodes = new TestKitNodes.Builder() |
| 97 | + .setClusterId("test-cluster") |
| 98 | + .setNumBrokerNodes(1) |
| 99 | + .setNumControllerNodes(3) |
| 100 | + .build(); |
| 101 | + |
| 102 | + final Map<Integer, Uuid> initialVoters = new HashMap<>(); |
| 103 | + for (final var controllerNode : nodes.controllerNodes().values()) { |
| 104 | + initialVoters.put( |
| 105 | + controllerNode.id(), |
| 106 | + controllerNode.metadataDirectoryId() |
| 107 | + ); |
| 108 | + } |
| 109 | + |
| 110 | + try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) { |
| 111 | + cluster.format(); |
| 112 | + cluster.startup(); |
| 113 | + try (Admin admin = Admin.create(cluster.clientProperties())) { |
| 114 | + Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId(); |
| 115 | + var removeFuture = admin.removeRaftVoter( |
| 116 | + 3000, |
| 117 | + dirId, |
| 118 | + new RemoveRaftVoterOptions().setClusterId(Optional.of("inconsistent")) |
| 119 | + ).all(); |
| 120 | + TestUtils.assertFutureThrows(InconsistentClusterIdException.class, removeFuture); |
| 121 | + |
| 122 | + var addFuture = admin.addRaftVoter( |
| 123 | + 3000, |
| 124 | + dirId, |
| 125 | + Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)), |
| 126 | + new AddRaftVoterOptions().setClusterId(Optional.of("inconsistent")) |
| 127 | + ).all(); |
| 128 | + TestUtils.assertFutureThrows(InconsistentClusterIdException.class, addFuture); |
| 129 | + } |
| 130 | + } |
| 131 | + } |
| 132 | +} |
0 commit comments