jsancio commented on code in PR #19589:
URL: https://github.com/apache/kafka/pull/19589#discussion_r2254491540


##########
core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java:
##########
@@ -164,4 +169,71 @@ public void testRemoveAndAddSameController() throws 
Exception {
             }
         }
     }
+
+    @Test
+    public void testControllersAutoJoinStandaloneVoter() throws Exception {
+        try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder(
+            new TestKitNodes.Builder().
+                setNumBrokerNodes(1).
+                setNumControllerNodes(3).
+                setFeature(KRaftVersion.FEATURE_NAME, 
KRaftVersion.KRAFT_VERSION_1.featureLevel()).
+                build()
+            ).
+            setConfigProp(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, true).
+            setStandalone(true).
+            build()
+        ) {
+            cluster.format();
+            cluster.startup();
+            try (Admin admin = Admin.create(cluster.clientProperties())) {
+                TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
+                    Map<Integer, Uuid> voters = findVoterDirs(admin);
+                    assertEquals(new HashSet<>(List.of(3000, 3001, 3002)), 
voters.keySet());
+                    for (int replicaId : new int[] {3000, 3001, 3002}) {
+                        assertNotEquals(Uuid.ZERO_UUID, voters.get(replicaId));

Review Comment:
   Okay. Is there a way to get the exact directory id (UUID) and compare 
against that instead?



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -3329,6 +3408,23 @@ private UpdateRaftVoterRequestData 
buildUpdateVoterRequest() {
         );
     }
 
+    private AddRaftVoterRequestData buildAddVoterRequest() {
+        return RaftUtil.addVoterRequest(
+            clusterId,
+            quorumConfig.requestTimeoutMs(),
+            quorum.localReplicaKeyOrThrow(),
+            localListeners,
+            !quorumConfig.autoJoin()

Review Comment:
   Why is this the negative of auto join? Shouldn't it always be false? If 
KRaft send an "add voter" request, it should always be version 1 and return 
before committing.



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -3337,6 +3433,29 @@ private RequestSendResult 
maybeSendUpdateVoterRequest(FollowerState state, long
         );
     }
 
+    private RequestSendResult maybeSendAddVoterRequest(
+        FollowerState state,
+        long currentTimeMs
+    ) {
+        return maybeSendRequest(
+            currentTimeMs,
+            state.leaderNode(channel.listenerName()),
+            this::buildAddVoterRequest
+        );
+    }
+
+    private RequestSendResult maybeSendRemoveVoterRequest(
+        FollowerState state,
+        long currentTimeMs,
+        ReplicaKey replicaKey

Review Comment:
   Flip the order of these parameters. The kraft module has a pattern of using 
the last parameter as the current time when needed.



##########
raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientAutoJoinTest.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.utils.BufferSupplier;
+import org.apache.kafka.server.common.KRaftVersion;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey;
+import static 
org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_595_PROTOCOL;
+import static 
org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL;
+
+public class KafkaRaftClientAutoJoinTest {
+    @Test
+    public void testAutoRemoveOldVoter() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var oldFollower = replicaKey(leader.id() + 1, true);
+        final var newFollowerKey = replicaKey(oldFollower.id(), true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newFollowerKey.id(),
+            newFollowerKey.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, oldFollower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be a remove voter request
+        pollAndDeliverRemoveVoter(context, oldFollower);
+
+        // after sending a remove voter the next request should be a fetch
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the replica should send remove voter again because the fetch did 
not update the voter set
+        pollAndDeliverRemoveVoter(context, oldFollower);
+    }
+
+    @Test
+    public void testAutoAddNewVoter() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var follower = replicaKey(leader.id() + 1, true);
+        final var newVoter = replicaKey(follower.id() + 1, true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newVoter.id(),
+            newVoter.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, follower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be an add voter request
+        pollAndSendAddVoter(context, newVoter);
+
+        // expire the add voter request, the next request should be a fetch
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the replica should send add voter again because the completed fetch
+        // did not update the voter set, and its timer has expired
+        final var addVoterRequest = pollAndSendAddVoter(context, newVoter);
+
+        // deliver the add voter response, this is possible before a completed 
fetch because of KIP-1186
+        context.deliverResponse(
+            addVoterRequest.correlationId(),
+            addVoterRequest.destination(),
+            RaftUtil.addVoterResponse(Errors.NONE, Errors.NONE.message())
+        );
+
+        // verify the replica can perform a fetch to commit the new voter set 
while the add
+        // voter request is still in-flight
+        pollAndDeliverFetchToUpdateVoterSet(context, epoch,
+            VoterSetTest.voterSet(Stream.of(leader, newVoter)));
+    }
+
+    @Test
+    public void testObserverRemovesOldVoterAndAutoJoins() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var oldFollower = replicaKey(leader.id() + 1, true);
+        final var newFollowerKey = replicaKey(oldFollower.id(), true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newFollowerKey.id(),
+            newFollowerKey.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, oldFollower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        // advance time and complete a fetch to trigger the remove voter 
request
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be a remove voter request
+        pollAndDeliverRemoveVoter(context, oldFollower);
+
+        // after sending a remove voter the next request should be a fetch
+        // this fetch will remove the old follower from the voter set
+        pollAndDeliverFetchToUpdateVoterSet(
+            context,
+            epoch,
+            VoterSetTest.voterSet(Stream.of(leader))
+        );
+
+        // advance time and complete a fetch to trigger the add voter request
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be an add voter request
+        final var addVoterRequest = pollAndSendAddVoter(context, 
newFollowerKey);
+
+        // deliver the add voter response, this is possible before a completed 
fetch because of KIP-1186
+        context.deliverResponse(
+            addVoterRequest.correlationId(),
+            addVoterRequest.destination(),
+            RaftUtil.addVoterResponse(Errors.NONE, Errors.NONE.message())
+        );
+
+        // verify the replica can perform a fetch to commit the new voter set 
while the add
+        // voter request is still in-flight
+        pollAndDeliverFetchToUpdateVoterSet(
+            context,
+            epoch,
+            VoterSetTest.voterSet(Stream.of(leader, newFollowerKey))
+        );
+
+        // advance time and complete a fetch and expire the update voter set 
timer
+        // the next request should be a fetch because the log voter 
configuration is up-to-date
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+        context.pollUntilRequest();
+        context.assertSentFetchRequest();
+    }
+
+
+    @Test
+    public void testObserversDoNotAutoJoin() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var follower = replicaKey(leader.id() + 1, true);
+        final var newObserver = replicaKey(follower.id() + 1, true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newObserver.id(),
+            newObserver.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, follower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(false)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        context.time.sleep(context.fetchTimeoutMs - 1);
+        context.pollUntilRequest();
+
+        // When canBecomeVoter == false, the replica should not send an add 
voter request
+        final var fetchRequest = context.assertSentFetchRequest();
+        context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, 
context.client.highWatermark());
+    }
+
+    @Test
+    public void testObserverDoesNotAddItselfWhenAutoJoinDisabled() throws 
Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var follower = replicaKey(leader.id() + 1, true);
+        final var observer = replicaKey(follower.id() + 1, true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            observer.id(),
+            observer.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, follower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(false)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        context.time.sleep(context.fetchTimeoutMs - 1);
+        context.pollUntilRequest();
+
+        // When autoJoin == false, the replica should not send an add voter 
request
+        final var fetchRequest = context.assertSentFetchRequest();
+        context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, 
context.client.highWatermark());
+    }
+
+    @Test
+    public void testObserverDoesNotAutoJoinWithKRaftVersion0() throws 
Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var follower = replicaKey(leader.id() + 1, true);
+        final var observer = replicaKey(follower.id() + 1, true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            observer.id(),
+            observer.directoryId().get()
+        )
+            .withRaftProtocol(KIP_595_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, follower)), 
KRaftVersion.KRAFT_VERSION_0
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        context.time.sleep(context.fetchTimeoutMs - 1);
+        context.pollUntilRequest();
+
+        // When kraft.version == 0, the replica should not send an add voter 
request
+        final var fetchRequest = context.assertSentFetchRequest();
+
+        context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, 
context.client.highWatermark());
+    }
+
+    private void pollAndDeliverRemoveVoter(
+        RaftClientTestContext context,
+        ReplicaKey oldFollower
+    ) throws Exception {
+        context.pollUntilRequest();
+        final var removeRequest = 
context.assertSentRemoveVoterRequest(oldFollower);
+        context.deliverResponse(
+            removeRequest.correlationId(),
+            removeRequest.destination(),
+            RaftUtil.removeVoterResponse(Errors.NONE, Errors.NONE.message())
+        );
+    }
+
+    private RaftRequest.Outbound pollAndSendAddVoter(
+        RaftClientTestContext context,
+        ReplicaKey newVoter
+    ) throws Exception {
+        context.pollUntilRequest();
+        return context.assertSentAddVoterRequest(
+            newVoter,
+            context.client.quorum().localVoterNodeOrThrow().listeners(),
+            false
+        );
+    }
+
+    private void pollAndDeliverFetchToUpdateVoterSet(
+        RaftClientTestContext context,
+        int epoch,
+        VoterSet newVoterSet
+    ) throws Exception {
+        context.pollUntilRequest();
+        final var fetchRequest = context.assertSentFetchRequest();
+        context.assertFetchRequestData(
+            fetchRequest,
+            epoch,
+            context.log.endOffset().offset(),
+            context.log.lastFetchedEpoch(),
+            context.client.highWatermark()
+        );
+        // deliver the fetch response with the updated voter set
+        context.deliverResponse(
+            fetchRequest.correlationId(),
+            fetchRequest.destination(),
+            context.fetchResponse(
+                epoch,
+                fetchRequest.destination().id(),
+                MemoryRecords.withVotersRecord(
+                    context.log.endOffset().offset(),
+                    0,

Review Comment:
   Hmm. Maybe we can use `context.time.milliseconds()`.



##########
raft/src/main/java/org/apache/kafka/raft/RaftUtil.java:
##########
@@ -524,14 +526,16 @@ public static AddRaftVoterRequestData addVoterRequest(
         String clusterId,
         int timeoutMs,
         ReplicaKey voter,
-        Endpoints listeners
+        Endpoints listeners,
+        boolean ackWhenCommitted

Review Comment:
   This is the add voter request specific for the kraft implementation. "Ack 
when committed" should always be false. If that's true then let's remove this 
parameter and not give the caller the option to set it. In the implementation, 
the method should always call `setAckWhenCommitted(false)`.



##########
raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientAutoJoinTest.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.utils.BufferSupplier;
+import org.apache.kafka.server.common.KRaftVersion;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey;
+import static 
org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_595_PROTOCOL;
+import static 
org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL;
+
+public class KafkaRaftClientAutoJoinTest {
+    @Test
+    public void testAutoRemoveOldVoter() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var oldFollower = replicaKey(leader.id() + 1, true);
+        final var newFollowerKey = replicaKey(oldFollower.id(), true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newFollowerKey.id(),
+            newFollowerKey.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, oldFollower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be a remove voter request
+        pollAndDeliverRemoveVoter(context, oldFollower);
+
+        // after sending a remove voter the next request should be a fetch
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the replica should send remove voter again because the fetch did 
not update the voter set
+        pollAndDeliverRemoveVoter(context, oldFollower);
+    }
+
+    @Test
+    public void testAutoAddNewVoter() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var follower = replicaKey(leader.id() + 1, true);
+        final var newVoter = replicaKey(follower.id() + 1, true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newVoter.id(),
+            newVoter.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, follower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be an add voter request
+        pollAndSendAddVoter(context, newVoter);
+
+        // expire the add voter request, the next request should be a fetch
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the replica should send add voter again because the completed fetch
+        // did not update the voter set, and its timer has expired
+        final var addVoterRequest = pollAndSendAddVoter(context, newVoter);
+
+        // deliver the add voter response, this is possible before a completed 
fetch because of KIP-1186
+        context.deliverResponse(
+            addVoterRequest.correlationId(),
+            addVoterRequest.destination(),
+            RaftUtil.addVoterResponse(Errors.NONE, Errors.NONE.message())
+        );
+
+        // verify the replica can perform a fetch to commit the new voter set 
while the add
+        // voter request is still in-flight
+        pollAndDeliverFetchToUpdateVoterSet(context, epoch,
+            VoterSetTest.voterSet(Stream.of(leader, newVoter)));
+    }
+
+    @Test
+    public void testObserverRemovesOldVoterAndAutoJoins() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var oldFollower = replicaKey(leader.id() + 1, true);
+        final var newFollowerKey = replicaKey(oldFollower.id(), true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newFollowerKey.id(),
+            newFollowerKey.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, oldFollower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        // advance time and complete a fetch to trigger the remove voter 
request
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be a remove voter request
+        pollAndDeliverRemoveVoter(context, oldFollower);
+
+        // after sending a remove voter the next request should be a fetch
+        // this fetch will remove the old follower from the voter set
+        pollAndDeliverFetchToUpdateVoterSet(
+            context,
+            epoch,
+            VoterSetTest.voterSet(Stream.of(leader))
+        );
+
+        // advance time and complete a fetch to trigger the add voter request
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be an add voter request
+        final var addVoterRequest = pollAndSendAddVoter(context, 
newFollowerKey);
+
+        // deliver the add voter response, this is possible before a completed 
fetch because of KIP-1186
+        context.deliverResponse(
+            addVoterRequest.correlationId(),
+            addVoterRequest.destination(),
+            RaftUtil.addVoterResponse(Errors.NONE, Errors.NONE.message())
+        );
+
+        // verify the replica can perform a fetch to commit the new voter set 
while the add
+        // voter request is still in-flight
+        pollAndDeliverFetchToUpdateVoterSet(
+            context,
+            epoch,
+            VoterSetTest.voterSet(Stream.of(leader, newFollowerKey))
+        );
+
+        // advance time and complete a fetch and expire the update voter set 
timer
+        // the next request should be a fetch because the log voter 
configuration is up-to-date
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+        context.pollUntilRequest();
+        context.assertSentFetchRequest();
+    }
+
+
+    @Test
+    public void testObserversDoNotAutoJoin() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var follower = replicaKey(leader.id() + 1, true);
+        final var newObserver = replicaKey(follower.id() + 1, true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newObserver.id(),
+            newObserver.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, follower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(false)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        context.time.sleep(context.fetchTimeoutMs - 1);
+        context.pollUntilRequest();
+
+        // When canBecomeVoter == false, the replica should not send an add 
voter request
+        final var fetchRequest = context.assertSentFetchRequest();
+        context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, 
context.client.highWatermark());
+    }
+
+    @Test
+    public void testObserverDoesNotAddItselfWhenAutoJoinDisabled() throws 
Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var follower = replicaKey(leader.id() + 1, true);
+        final var observer = replicaKey(follower.id() + 1, true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            observer.id(),
+            observer.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, follower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(false)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        context.time.sleep(context.fetchTimeoutMs - 1);
+        context.pollUntilRequest();
+
+        // When autoJoin == false, the replica should not send an add voter 
request
+        final var fetchRequest = context.assertSentFetchRequest();
+        context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, 
context.client.highWatermark());
+    }
+
+    @Test
+    public void testObserverDoesNotAutoJoinWithKRaftVersion0() throws 
Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var follower = replicaKey(leader.id() + 1, true);
+        final var observer = replicaKey(follower.id() + 1, true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            observer.id(),
+            observer.directoryId().get()
+        )
+            .withRaftProtocol(KIP_595_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, follower)), 
KRaftVersion.KRAFT_VERSION_0
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        context.time.sleep(context.fetchTimeoutMs - 1);
+        context.pollUntilRequest();
+
+        // When kraft.version == 0, the replica should not send an add voter 
request
+        final var fetchRequest = context.assertSentFetchRequest();
+
+        context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, 
context.client.highWatermark());
+    }
+
+    private void pollAndDeliverRemoveVoter(
+        RaftClientTestContext context,
+        ReplicaKey oldFollower
+    ) throws Exception {
+        context.pollUntilRequest();
+        final var removeRequest = 
context.assertSentRemoveVoterRequest(oldFollower);
+        context.deliverResponse(
+            removeRequest.correlationId(),
+            removeRequest.destination(),
+            RaftUtil.removeVoterResponse(Errors.NONE, Errors.NONE.message())
+        );
+    }
+
+    private RaftRequest.Outbound pollAndSendAddVoter(
+        RaftClientTestContext context,
+        ReplicaKey newVoter
+    ) throws Exception {
+        context.pollUntilRequest();
+        return context.assertSentAddVoterRequest(
+            newVoter,
+            context.client.quorum().localVoterNodeOrThrow().listeners(),
+            false
+        );
+    }
+
+    private void pollAndDeliverFetchToUpdateVoterSet(
+        RaftClientTestContext context,
+        int epoch,
+        VoterSet newVoterSet
+    ) throws Exception {
+        context.pollUntilRequest();
+        final var fetchRequest = context.assertSentFetchRequest();
+        context.assertFetchRequestData(
+            fetchRequest,
+            epoch,
+            context.log.endOffset().offset(),
+            context.log.lastFetchedEpoch(),
+            context.client.highWatermark()
+        );
+        // deliver the fetch response with the updated voter set
+        context.deliverResponse(
+            fetchRequest.correlationId(),
+            fetchRequest.destination(),
+            context.fetchResponse(
+                epoch,
+                fetchRequest.destination().id(),
+                MemoryRecords.withVotersRecord(
+                    context.log.endOffset().offset(),
+                    0,
+                    epoch,
+                    BufferSupplier.NO_CACHING.get(300),
+                    newVoterSet.toVotersRecord((short) 0)
+                ),
+                context.log.endOffset().offset() + 1,

Review Comment:
   This assumes that the voter set only has one voter hence one record.



##########
raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientAutoJoinTest.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.utils.BufferSupplier;
+import org.apache.kafka.server.common.KRaftVersion;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey;
+import static 
org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_595_PROTOCOL;
+import static 
org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL;
+
+public class KafkaRaftClientAutoJoinTest {
+    @Test
+    public void testAutoRemoveOldVoter() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var oldFollower = replicaKey(leader.id() + 1, true);
+        final var newFollowerKey = replicaKey(oldFollower.id(), true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newFollowerKey.id(),
+            newFollowerKey.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, oldFollower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be a remove voter request
+        pollAndDeliverRemoveVoter(context, oldFollower);
+
+        // after sending a remove voter the next request should be a fetch
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the replica should send remove voter again because the fetch did 
not update the voter set
+        pollAndDeliverRemoveVoter(context, oldFollower);
+    }
+
+    @Test
+    public void testAutoAddNewVoter() throws Exception {
+        final var leader = replicaKey(randomReplicaId(), true);
+        final var follower = replicaKey(leader.id() + 1, true);
+        final var newVoter = replicaKey(follower.id() + 1, true);
+        final int epoch = 1;
+        final var context = new RaftClientTestContext.Builder(
+            newVoter.id(),
+            newVoter.directoryId().get()
+        )
+            .withRaftProtocol(KIP_853_PROTOCOL)
+            .withStartingVoters(
+                VoterSetTest.voterSet(Stream.of(leader, follower)), 
KRaftVersion.KRAFT_VERSION_1
+            )
+            .withElectedLeader(epoch, leader.id())
+            .withAutoJoin(true)
+            .withCanBecomeVoter(true)
+            .build();
+
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the next request should be an add voter request
+        pollAndSendAddVoter(context, newVoter);
+
+        // expire the add voter request, the next request should be a fetch
+        context.advanceTimeAndCompleteFetch(epoch, leader.id(), true);
+
+        // the replica should send add voter again because the completed fetch
+        // did not update the voter set, and its timer has expired
+        final var addVoterRequest = pollAndSendAddVoter(context, newVoter);
+
+        // deliver the add voter response, this is possible before a completed 
fetch because of KIP-1186
+        context.deliverResponse(
+            addVoterRequest.correlationId(),
+            addVoterRequest.destination(),
+            RaftUtil.addVoterResponse(Errors.NONE, Errors.NONE.message())
+        );
+
+        // verify the replica can perform a fetch to commit the new voter set 
while the add
+        // voter request is still in-flight
+        pollAndDeliverFetchToUpdateVoterSet(context, epoch,
+            VoterSetTest.voterSet(Stream.of(leader, newVoter)));

Review Comment:
   Let's fix this indentation. How about:
   ```java
           pollAndDeliverFetchToUpdateVoterSet(
               context,
               epoch,
               VoterSetTest.voterSet(Stream.of(leader, newVoter))
           );
   ```



##########
raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java:
##########
@@ -1259,6 +1313,27 @@ RaftRequest.Outbound assertSentApiVersionsRequest() {
         return sentRequests.get(0);
     }
 
+    RaftRequest.Outbound assertSentAddVoterRequest(
+        ReplicaKey replicaKey,
+        Endpoints endpoints,
+        boolean expectedAckWhenCommitted

Review Comment:
   Let's not give the caller the option to override this. This value should 
always be false and this method should just check for that explicitly.



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