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


##########
raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java:
##########
@@ -0,0 +1,543 @@
+/*
+ * 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.Uuid;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.MemoryRecords;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Collections;
+import java.util.OptionalInt;
+import java.util.Set;
+
+import static org.apache.kafka.raft.KafkaRaftClientTest.randomReplicaId;
+import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class KafkaRaftClientPreVoteTest {
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testHandlePreVoteRequestAsFollowerWithElectedLeader(boolean 
hasFetchedFromLeader) throws Exception {
+        int localId = randomReplicaId();
+        int epoch = 2;
+        ReplicaKey otherNodeKey = replicaKey(localId + 1, true);
+        int electedLeaderId = localId + 2;
+        Set<Integer> voters = Set.of(localId, otherNodeKey.id(), 
electedLeaderId);
+
+        RaftClientTestContext context = new 
RaftClientTestContext.Builder(localId, voters)
+            .withElectedLeader(epoch, electedLeaderId)
+            .withKip853Rpc(true)
+            .build();
+
+        if (hasFetchedFromLeader) {
+            context.pollUntilRequest();
+            RaftRequest.Outbound fetchRequest = 
context.assertSentFetchRequest();
+            context.assertFetchRequestData(fetchRequest, epoch, 0L, 0);
+
+            context.deliverResponse(
+                fetchRequest.correlationId(),
+                fetchRequest.destination(),
+                context.fetchResponse(epoch, electedLeaderId, 
MemoryRecords.EMPTY, 0L, Errors.NONE)
+            );
+        }
+
+        // follower should reject pre-vote requests with the same epoch if it 
has successfully fetched from the leader
+        context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, 
epoch, 1));
+        context.pollUntilResponse();
+
+        boolean voteGranted = !hasFetchedFromLeader;
+        context.assertSentVoteResponse(Errors.NONE, epoch, 
OptionalInt.of(electedLeaderId), voteGranted);
+        context.assertElectedLeader(epoch, electedLeaderId);
+
+        // follower will transition to unattached if pre-vote request has a 
higher epoch
+        context.deliverRequest(context.preVoteRequest(epoch + 1, otherNodeKey, 
epoch + 1, 1));
+        context.pollUntilResponse();
+
+        context.assertSentVoteResponse(Errors.NONE, epoch + 1, 
OptionalInt.of(-1), true);
+        assertEquals(context.currentEpoch(), epoch + 1);
+        assertTrue(context.client.quorum().isUnattachedNotVoted());
+    }
+
+    @Test
+    public void testHandlePreVoteRequestAsFollowerWithVotedCandidate() throws 
Exception {

Review Comment:
   hm, we don't violate any raft invariants today with just persisting either 
leaderId or votedKey, but I think this may change with the introduction of 
ProspectiveState
   
   the situation I was thinking of was the following:
   ```
   UnattachedVoted (votedKey=node3, epoch=10) -> Prospective (votedKey=node3, 
epoch=10) 
   - election timeout, transition from unattached to prospective
   
   Prospective (votedKey=node3, epoch=10) -> Follower (leader=node1, 
votedKey=node3, epoch=10)
   - received PreVote response denoting leader=node1 in epoch 10, should it 
retain the votedKey or just clear it?
   ```
   I think we need to retain the votedKey, because if Follower then transitions 
to Prospective again, it will be able to incorrectly grant another standard 
(persisted) vote unless it has a votedKey. 
   



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