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


##########
raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.apache.kafka.raft.internals.EpochElection;
+
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.OptionalInt;
+
+import static 
org.apache.kafka.raft.QuorumState.unattachedOrProspectiveCanGrantVote;
+
+public class ProspectiveState implements NomineeState {
+    private final int localId;
+    private final int epoch;
+    private final OptionalInt leaderId;
+    private final Optional<Endpoints> leaderEndpoints;
+    private final Optional<ReplicaKey> votedKey;
+    private final VoterSet voters;
+    private final EpochElection epochElection;
+    private final Optional<LogOffsetMetadata> highWatermark;
+    private final int retries;
+    private final long electionTimeoutMs;
+    private final Timer electionTimer;
+    private final Logger log;
+
+    /**
+     * The lifetime of a prospective state is the following.
+     *
+     * 1. Once started, it will send prevote requests and keep record of the 
received vote responses
+     * 2. If it receives a message denoting a leader with a higher epoch, it 
will transition to follower state.
+     * 3. If majority votes granted, it will transition to candidate state.
+     * 4. If majority votes rejected or election times out, it will transition 
to unattached or follower state
+     *    depending on if it knows the leader id and endpoints or not
+     */
+    public ProspectiveState(
+        Time time,
+        int localId,
+        int epoch,
+        OptionalInt leaderId,
+        Optional<Endpoints> leaderEndpoints,
+        Optional<ReplicaKey> votedKey,
+        VoterSet voters,
+        Optional<LogOffsetMetadata> highWatermark,
+        int retries,
+        int electionTimeoutMs,
+        LogContext logContext
+    ) {
+        this.localId = localId;
+        this.epoch = epoch;
+        this.leaderId = leaderId;
+        this.leaderEndpoints = leaderEndpoints;
+        this.votedKey = votedKey;
+        this.voters = voters;
+        this.highWatermark = highWatermark;
+        this.retries = retries;
+        this.electionTimeoutMs = electionTimeoutMs;
+        this.electionTimer = time.timer(electionTimeoutMs);
+        this.log = logContext.logger(ProspectiveState.class);
+
+        this.epochElection = new EpochElection(voters.voterKeys());
+        epochElection.recordVote(localId, true);
+    }
+
+    public Optional<ReplicaKey> votedKey() {
+        return votedKey;
+    }
+
+    @Override
+    public EpochElection epochElection() {
+        return epochElection;
+    }
+
+    public int retries() {
+        return retries;
+    }
+
+    @Override
+    public boolean recordGrantedVote(int remoteNodeId) {
+        return epochElection().recordVote(remoteNodeId, true);
+    }
+
+    @Override
+    public boolean recordRejectedVote(int remoteNodeId) {
+        if (remoteNodeId == localId) {
+            throw new IllegalArgumentException("Attempted to reject vote from 
ourselves");
+        }
+        return epochElection().recordVote(remoteNodeId, false);
+    }
+
+    @Override
+    public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, 
boolean isPreVote) {
+        return unattachedOrProspectiveCanGrantVote(
+            leaderId,
+            votedKey,
+            epoch,
+            replicaKey,
+            isLogUpToDate,
+            isPreVote,
+            log
+        );
+    }
+
+    @Override
+    public boolean hasElectionTimeoutExpired(long currentTimeMs) {
+        electionTimer.update(currentTimeMs);
+        return electionTimer.isExpired();
+    }
+
+    @Override
+    public long remainingElectionTimeMs(long currentTimeMs) {
+        electionTimer.update(currentTimeMs);
+        return electionTimer.remainingMs();
+    }
+
+    @Override
+    public ElectionState election() {
+        if (leaderId.isPresent()) {
+            return ElectionState.withElectedLeader(epoch, leaderId.getAsInt(), 
votedKey, voters.voterIds());
+        } else if (votedKey.isPresent()) {
+            return ElectionState.withVotedCandidate(epoch, votedKey.get(), 
voters.voterIds());
+        } else {
+            return ElectionState.withUnknownLeader(epoch, voters.voterIds());
+        }
+    }
+
+    @Override
+    public int epoch() {
+        return epoch;
+    }
+
+    @Override
+    public Endpoints leaderEndpoints() {
+        return leaderEndpoints.orElse(Endpoints.empty());
+    }

Review Comment:
   Given this implementation it is also correct to just store it as `Endpoints 
leaderEndpoints` and changing the constructor to accept an `Endpoints` instead 
of an `Optional<Endpoints>`.
   
   It looks like in the raft module we never use `Optional<Endpoints>` since 
`Endpoints.empty()` is a valid value.



##########
checkstyle/suppressions.xml:
##########
@@ -187,6 +187,9 @@
     <suppress checks="NPathComplexity"
               files="(DynamicVoter|RecordsIterator).java"/>
 
+    <suppress checks="JavaNCSS"
+              files="(KafkaRaftClientTest).java"/>
+

Review Comment:
   Can we make this an annotation that suppresses that check?



##########
raft/src/main/java/org/apache/kafka/raft/QuorumState.java:
##########
@@ -562,6 +656,25 @@ public void transitionToCandidate() {
         ));
     }
 
+    private void checkValidTransitionToCandidate() {
+        if (isObserver()) {
+            throw new IllegalStateException(
+                String.format(
+                    "Cannot transition to Candidate since the local id (%s) 
and directory id (%s) " +
+                        "is not one of the voters %s",

Review Comment:
   I think we use this formatting in this case:
   ```suggestion
                       "Cannot transition to Candidate since the local id (%s) 
and directory id (%s) " +
                        "is not one of the voters %s",
   ```



##########
raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.internals;
+
+import org.apache.kafka.raft.ReplicaKey;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ *  Tracks the votes cast by voters in an election held by a Nominee.
+ */
+public class EpochElection {
+    private Map<Integer, VoterState> voterStates;
+
+    public EpochElection(Set<ReplicaKey> voters) {
+        this.voterStates = voters.stream()
+            .collect(
+                Collectors.toMap(
+                    ReplicaKey::id,
+                    VoterState::new
+                )
+            );
+    }
+
+    /**
+     * Record a vote from a voter.
+     *
+     * @param voterId The id of the voter
+     * @param isGranted true if the vote is granted, false if it is rejected
+     * @return true if the voter had not been previously recorded
+     */
+    public boolean recordVote(int voterId, boolean isGranted) {
+        VoterState voterState = getVoterStateOrThrow(voterId);
+        boolean wasUnrecorded = voterState.state == 
VoterState.State.UNRECORDED;
+        if (isGranted) {
+            voterState.setState(VoterState.State.GRANTED);
+        } else {
+            voterState.setState(VoterState.State.REJECTED);
+        }

Review Comment:
   Some code duplication can be removed with:
   ```java
           VoterState.State state = isGranted ? VoterState.State.GRANTED : 
VoterState.State.REJECTED;
           voterState.setState(state);
   ```
   or
   ```java
           voterState.setState(
               isGranted ? VoterState.State.GRANTED : VoterState.State.REJECTED
           );
   ```
   



##########
raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java:
##########
@@ -86,186 +100,249 @@ public void 
testHandlePreVoteRequestAsFollowerWithElectedLeader(boolean hasFetch
         assertTrue(context.client.quorum().isUnattachedNotVoted());
     }
 
-    @Test
-    public void testHandlePreVoteRequestAsCandidate() throws Exception {
+    @ParameterizedTest
+    @EnumSource(value = KRaftVersion.class)
+    public void 
testHandlePreVoteRequestAsFollowerWithVotedCandidate(KRaftVersion kraftVersion) 
throws Exception {
         int localId = randomReplicaId();
+        int epoch = 2;
+        ReplicaKey localKey = replicaKey(localId, true);
+        ReplicaKey otherNodeKey = replicaKey(localId + 1, true);
+        ReplicaKey votedCandidateKey = replicaKey(localId + 2, true);
+        VoterSet voters = VoterSetTest.voterSet(Stream.of(localKey, 
otherNodeKey, votedCandidateKey));
+
+        RaftClientTestContext context = contextBuilder(localKey, voters, 
kraftVersion.isReconfigSupported())
+            .withVotedCandidate(epoch, votedCandidateKey)
+            .withRaftProtocol(KIP_996_PROTOCOL)
+            .build();
+
+        context.deliverRequest(context.beginEpochRequest(epoch, 
votedCandidateKey.id(), voters.listeners(votedCandidateKey.id())));
+        context.pollUntilResponse();
+        context.assertSentFetchRequest();
+        context.assertSentBeginQuorumEpochResponse(Errors.NONE);
+        assertTrue(context.client.quorum().isFollower());
+
+        // follower can grant PreVotes if it has not fetched successfully from 
leader yet
+        context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, 
epoch, 1));
+        context.pollUntilResponse();
+        context.assertSentVoteResponse(Errors.NONE, epoch, 
OptionalInt.of(votedCandidateKey.id()), true);
+
+        // after fetching from leader, follower should reject PreVote requests
+        context.pollUntilRequest();
+        RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();

Review Comment:
   Why would the replica send another request since it already sent a request 
[here](https://github.com/apache/kafka/pull/18240/files#diff-7e685d565d946f882cb6bd79d85bec3ca3fae64c137ab93684d939f1371196aeR120)?
   
   Or is the issue that the replica change state from unattached (with voted) 
to follower 
[here](https://github.com/apache/kafka/pull/18240/files#diff-7e685d565d946f882cb6bd79d85bec3ca3fae64c137ab93684d939f1371196aeR121)
 and reset its connection and request manager?
   
   Which means that it will send another FETCH request when it becomes a 
follower? If this is the case, maybe the test structure you had earlier is 
better where you assert a FETCH request is sent while in the unattached stated.



##########
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:
   > since entire quorum may not support PreVote.
   
   If just one of the voter doesn't support pre-vote this replica need to 
transition to candidate. That because that voter that doesn't support pre-vote 
may be need to establish quorum with the majority.  I would change this working 
to:
   ```java
                   "transitioning to Candidate state immediately since at least 
one voter doesn't support PreVote.",
   ```



##########
raft/src/main/java/org/apache/kafka/raft/QuorumState.java:
##########
@@ -562,6 +656,25 @@ public void transitionToCandidate() {
         ));
     }
 
+    private void checkValidTransitionToCandidate() {
+        if (isObserver()) {
+            throw new IllegalStateException(
+                String.format(
+                    "Cannot transition to Candidate since the local id (%s) 
and directory id (%s) " +
+                        "is not one of the voters %s",
+                    localId,
+                    localDirectoryId,
+                    partitionState.lastVoterSet()
+                )
+            );
+        }
+        // Only Prospective is allowed to transition to Candidate
+        if (!isProspective()) {
+            throw new IllegalStateException("Cannot transition to Candidate 
since the local broker.id=" + localId +
+                " is state " + state);

Review Comment:
   Okay but I would like us to standardize on using `String.format`.
   
   I think his should be formatted as:
   ```java
               throw new IllegalStateException(
                   "Cannot transition to Candidate since the local broker.id=" 
+ localId + " is state " + state
               );
   ```



##########
raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.apache.kafka.raft.internals.EpochElection;
+
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.OptionalInt;
+
+import static 
org.apache.kafka.raft.QuorumState.unattachedOrProspectiveCanGrantVote;
+
+public class ProspectiveState implements NomineeState {
+    private final int localId;
+    private final int epoch;
+    private final OptionalInt leaderId;
+    private final Optional<Endpoints> leaderEndpoints;
+    private final Optional<ReplicaKey> votedKey;
+    private final VoterSet voters;
+    private final EpochElection epochElection;
+    private final Optional<LogOffsetMetadata> highWatermark;
+    private final int retries;
+    private final long electionTimeoutMs;
+    private final Timer electionTimer;
+    private final Logger log;
+
+    /**
+     * The lifetime of a prospective state is the following.
+     *
+     * 1. Once started, it will send prevote requests and keep record of the 
received vote responses
+     * 2. If it receives a message denoting a leader with a higher epoch, it 
will transition to follower state.
+     * 3. If majority votes granted, it will transition to candidate state.
+     * 4. If majority votes rejected or election times out, it will transition 
to unattached or follower state
+     *    depending on if it knows the leader id and endpoints or not
+     */
+    public ProspectiveState(
+        Time time,
+        int localId,
+        int epoch,
+        OptionalInt leaderId,
+        Optional<Endpoints> leaderEndpoints,
+        Optional<ReplicaKey> votedKey,
+        VoterSet voters,
+        Optional<LogOffsetMetadata> highWatermark,
+        int retries,
+        int electionTimeoutMs,
+        LogContext logContext

Review Comment:
   This type doesn't write any log messages. We don't need to pass the log 
context to the object.



##########
raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java:
##########
@@ -594,12 +594,35 @@ void expectAndGrantVotes(int epoch) throws Exception {
         assertElectedLeader(epoch, localIdOrThrow());
     }
 
+    void expectAndGrantPreVotes(int epoch) throws Exception {
+        pollUntilRequest();
+
+        List<RaftRequest.Outbound> voteRequests = collectPreVoteRequests(epoch,
+            log.lastFetchedEpoch(), log.endOffset().offset());

Review Comment:
   Let's use this formatting:
   ```java
           List<RaftRequest.Outbound> voteRequests = collectPreVoteRequests(
               epoch,
               log.lastFetchedEpoch(),
               log.endOffset().offset()
           );
   ```



##########
raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java:
##########
@@ -629,40 +652,67 @@ void pollUntilRequest() throws InterruptedException {
     }
 
     void assertVotedCandidate(int epoch, int candidateId) {
+        ReplicaKey candidateKey = ReplicaKey.of(candidateId, 
ReplicaKey.NO_DIRECTORY_ID);
+        assertVotedCandidate(epoch, candidateKey);
+    }
+
+    void assertVotedCandidate(int epoch, ReplicaKey candidateKey) {
         assertEquals(
             ElectionState.withVotedCandidate(
                 epoch,
-                ReplicaKey.of(candidateId, ReplicaKey.NO_DIRECTORY_ID),
-                startingVoters.voterIds()
+                persistedVotedKey(candidateKey, kraftVersion),
+                expectedVoters()
             ),
             quorumStateStore.readElectionState().get()
         );
     }
 
     public void assertElectedLeader(int epoch, int leaderId) {
-        Set<Integer> voters = kraftVersion.isReconfigSupported() ?
-                Collections.emptySet() : startingVoters.voterIds();
         assertEquals(
-            ElectionState.withElectedLeader(epoch, leaderId, voters),
+            ElectionState.withElectedLeader(epoch, leaderId, Optional.empty(), 
expectedVoters()),
             quorumStateStore.readElectionState().get()
         );
     }
 
-    void assertUnknownLeader(int epoch) {
+    public void assertElectedLeaderAndVotedKey(int epoch, int leaderId, 
ReplicaKey candidateKey) {
         assertEquals(
-            ElectionState.withUnknownLeader(epoch, startingVoters.voterIds()),
+            ElectionState.withElectedLeader(
+                epoch,
+                leaderId,
+                Optional.of(persistedVotedKey(candidateKey, kraftVersion)),
+                expectedVoters()
+            ),
             quorumStateStore.readElectionState().get()
         );
     }
 
+    private static ReplicaKey persistedVotedKey(ReplicaKey replicaKey, 
KRaftVersion kraftVersion) {
+        if (kraftVersion.isReconfigSupported()) {
+            return replicaKey;
+        }
+
+        return ReplicaKey.of(replicaKey.id(), ReplicaKey.NO_DIRECTORY_ID);
+    }
+
+    void assertUnknownLeaderAndNoVotedCandidate(int epoch) {
+        assertEquals(
+            ElectionState.withUnknownLeader(epoch, expectedVoters()),
+            quorumStateStore.readElectionState().get());

Review Comment:
   Missing newline character.
   ```java
               quorumStateStore.readElectionState().get()
           );
   ```



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -657,14 +658,39 @@ private void flushLeaderLog(LeaderState<T> state, long 
currentTimeMs) {
     }
 
     private boolean maybeTransitionToLeader(CandidateState state, long 
currentTimeMs) {
-        if (state.isVoteGranted()) {
+        if (state.epochElection().isVoteGranted()) {
             onBecomeLeader(currentTimeMs);
             return true;
         } else {
             return false;
         }
     }
 
+    private boolean maybeTransitionToCandidate(ProspectiveState state, long 
currentTimeMs) {
+        if (state.epochElection().isVoteGranted()) {
+            transitionToCandidate(currentTimeMs);
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Only applies to NomineeStates (Prospective or Candidate). If enough 
votes were granted
+     * then this method is called to transition the state forward - either 
from Prospective to Candidate
+     * or from Candidate to Leader.
+     */
+    private void maybeTransitionForward(NomineeState state, long 
currentTimeMs) {
+        if (state instanceof ProspectiveState prospective) {
+            maybeTransitionToCandidate(prospective, currentTimeMs);
+        } else if (state instanceof CandidateState candidate) {
+            maybeTransitionToLeader(candidate, currentTimeMs);
+        } else {
+            throw new IllegalStateException("Expected to be a NomineeState 
(Prospective or Candidate), " +
+                "but current state is " + state);

Review Comment:
   Let's use this formatting:
   ```java
               throw new IllegalStateException(
                   "Expected to be a NomineeState (Prospective or Candidate), " 
+
                   "but current state is " + state
               );
   ```



##########
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:
   FYI, this shows the issue you highlighted in the metrics test. The known HWM 
is lost when transitioning to leader. This is odd from the clients (users of 
RaftClient) point of view.
   
   This semantic turns out to be correct because the new HWM established by the 
leader is guarantee to be greater than the previous HWM. This is true because 
the leader first commits the current epoch before establishing the new HWM.



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