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


##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -882,7 +921,15 @@ private boolean handleVoteResponse(
         int remoteNodeId = responseMetadata.source().id();
         VoteResponseData response = (VoteResponseData) responseMetadata.data();
         Errors topLevelError = Errors.forCode(response.errorCode());
-        if (topLevelError != Errors.NONE) {
+        if (topLevelError == Errors.UNSUPPORTED_VERSION && 
quorum.isProspective()) {
+            logger.info(
+                "Prospective received unsupported version error in vote 
response in epoch {}, " +
+                "transitioning to Candidate state immediately since entire 
quorum may not support PreVote.",

Review Comment:
   english is hard 😛 I meant "not the entire quorum" vs "entire quorum does not"
   
   



##########
raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java:
##########
@@ -154,6 +155,163 @@ private ReplicaKey replicaKey(int id, boolean 
withDirectoryId) {
         return ReplicaKey.of(id, directoryId);
     }
 
+    @ParameterizedTest
+    @EnumSource(value = KRaftVersion.class)
+    public void testHasRemoteLeader(KRaftVersion kraftVersion) {
+        ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
+        VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, 
otherNodeKey));
+
+        QuorumState state = initializeEmptyState(voters, kraftVersion);
+        assertFalse(state.hasRemoteLeader());
+
+        state.transitionToProspective();
+        assertFalse(state.hasRemoteLeader());
+        state.transitionToCandidate();
+        assertFalse(state.hasRemoteLeader());
+
+        state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id());
+        state.transitionToLeader(0L, accumulator);
+        assertFalse(state.hasRemoteLeader());
+
+        state.transitionToUnattached(state.epoch() + 1, OptionalInt.empty());
+        assertFalse(state.hasRemoteLeader());
+
+        state.unattachedAddVotedState(state.epoch(), otherNodeKey);
+        assertFalse(state.hasRemoteLeader());
+
+        state.transitionToFollower(
+            state.epoch(),
+            otherNodeKey.id(),
+            voters.listeners(otherNodeKey.id())
+        );
+        assertTrue(state.hasRemoteLeader());
+    }
+
+    @ParameterizedTest
+    @EnumSource(value = KRaftVersion.class)
+    public void testHighWatermarkRetained(KRaftVersion kraftVersion) {
+        ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid());
+        VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, 
otherNodeKey));
+
+        QuorumState state = initializeEmptyState(voters, kraftVersion);
+        state.transitionToFollower(
+            5,
+            otherNodeKey.id(),
+            voters.listeners(otherNodeKey.id())
+        );
+
+        FollowerState followerState = state.followerStateOrThrow();
+        followerState.updateHighWatermark(OptionalLong.of(10L));
+
+        Optional<LogOffsetMetadata> highWatermark = Optional.of(new 
LogOffsetMetadata(10L));
+        assertEquals(highWatermark, state.highWatermark());
+
+        state.transitionToUnattached(6, OptionalInt.empty());
+        assertEquals(highWatermark, state.highWatermark());
+
+        state.unattachedAddVotedState(6, otherNodeKey);
+        assertEquals(highWatermark, state.highWatermark());
+
+        state.transitionToProspective();
+        assertEquals(highWatermark, state.highWatermark());
+
+        state.transitionToCandidate();
+        assertEquals(highWatermark, state.highWatermark());
+
+        CandidateState candidateState = state.candidateStateOrThrow();
+        candidateState.recordGrantedVote(otherNodeKey.id());
+        assertTrue(candidateState.epochElection().isVoteGranted());
+
+        state.transitionToLeader(10L, accumulator);
+        assertEquals(Optional.empty(), state.highWatermark());

Review Comment:
   I'll add more details to the Jira - we can decide if it's worth changing 
this behavior



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