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


##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -779,39 +781,43 @@ private VoteResponseData handleVoteRequest(
         VoteRequestData.PartitionData partitionRequest =
             request.topics().get(0).partitions().get(0);
 
-        int candidateId = partitionRequest.candidateId();
-        int candidateEpoch = partitionRequest.candidateEpoch();
+        int replicaId = partitionRequest.replicaId();
+        int replicaEpoch = partitionRequest.replicaEpoch();
+        boolean preVote = partitionRequest.preVote();
 
         int lastEpoch = partitionRequest.lastOffsetEpoch();
         long lastEpochEndOffset = partitionRequest.lastOffset();
-        if (lastEpochEndOffset < 0 || lastEpoch < 0 || lastEpoch >= 
candidateEpoch) {
+        boolean isIllegalEpoch = preVote ? lastEpoch > replicaEpoch : 
lastEpoch >= replicaEpoch;

Review Comment:
   A good comment should explain why not what. This code is simple enough to 
tells us what it is doing. What may not be clear to a lot of readers is why the 
code is doing this. How about this:
   ```java
   /* Validate the replica epoch and the log's last epoch.
    *
    * For a normal vote, the candidate replica increases the epoch before 
sending a vote request. So we expect the   
    * replicaEpoch to be strictly greater than the log's last epoch. This is 
always true because the candidate has
    * never seen a leader at replicaEpoch.
    *
    * For pre-vote, the prospective replica doesn't increase the epoch so it is 
possible for there to be a leader and a 
    * record in the log at the prospective candidate's replicaEpoch.
    */
   ```



##########
raft/src/main/java/org/apache/kafka/raft/EpochState.java:
##########
@@ -26,16 +26,18 @@ default Optional<LogOffsetMetadata> highWatermark() {
     }
 
     /**
-     * Decide whether to grant a vote to a candidate.
+     * Decide whether to grant a vote to a replica.
      *
      * It is the responsibility of the caller to invoke
      * {@link QuorumState#transitionToUnattachedVotedState(int, ReplicaKey)} 
if vote is granted.
      *
-     * @param candidateKey the id and directory of the candidate
-     * @param isLogUpToDate whether the candidate’s log is at least as 
up-to-date as receiver’s log
+     * @param replicaKey the id and directory of the replica requesting the 
vote
+     * @param isLogUpToDate whether the replica's log is at least as 
up-to-date as receiver’s log
      * @return true if it can grant the vote, false otherwise
      */
-    boolean canGrantVote(ReplicaKey candidateKey, boolean isLogUpToDate);
+    boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate);
+
+    boolean canGrantPreVote(ReplicaKey replicaKey, boolean isLogUpToDate);

Review Comment:
   > but that just seemed like an extra hoop for the sake of not creating an 
extra method for the EpochState interface
   
   We do have this method call. It is in `QuorumState::canGrantVote`. If we 
didn't have this method, you would have the same `if` statement at every call 
site. The other concern I have is which is easier to unittest? For example, it 
doesn't look like we have a unittest for `QuorumState::canGrantVote`. 



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