kevin-wu24 commented on code in PR #20859:
URL: https://github.com/apache/kafka/pull/20859#discussion_r2565548226


##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -501,6 +503,14 @@ public void initialize(
         logger.info("Reading KRaft snapshot and log as part of the 
initialization");
         partitionState.updateState();
         logger.info("Starting voters are {}", partitionState.lastVoterSet());
+        if (nodeId.isPresent()) {
+            // if the the VotersRecord with the highest offset contain the 
node id of this node,
+            // mark it as already joined because it is already in the voter 
set.
+            // Check using ReplicaKey (id + directoryId) to handle KIP-853 
properly
+            hasJoined = partitionState.lastVoterSet().isVoter(
+                ReplicaKey.of(nodeId.getAsInt(), nodeDirectoryId)
+            );
+        }

Review Comment:
   Hmmmm. Actually, we need to set `hasJoined = Optional.of(true)` IFF 
`hasJoined.isEmpty()` for the first leader elected, even if it is not a 
standalone leader. Otherwise, if we remove the leader, it can auto-join back 
because the state can go from UNKNOWN -> HAS_NOT_JOINED if the user manually 
removes the leader.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to