showuon commented on code in PR #20859:
URL: https://github.com/apache/kafka/pull/20859#discussion_r2513056603


##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -3347,13 +3359,20 @@ private long pollFollowerAsVoter(FollowerState state, 
long currentTimeMs) {
         );
     }
 
+
     private boolean shouldSendAddOrRemoveVoterRequest(FollowerState state, 
long currentTimeMs) {
+        // When the node is bootstap, it should not send addVoterRequest 
immediately.
+        if (skipFirstAutoJoinAttempt) {
+            skipFirstAutoJoinAttempt = false;

Review Comment:
   If we reset `skipFirstAutoJoinAttempt` to false, will it send 
addVoterRequest in next `pollFollowerAsObserver`?



##########
raft/src/main/java/org/apache/kafka/raft/internals/RemoveVoterHandler.java:
##########
@@ -173,7 +173,7 @@ public void highWatermarkUpdated(LeaderState<?> 
leaderState) {
                         )
                     );
                     if (!voters.isVoter(localKey)) {
-                        logger.info(
+                        logger.warn(

Review Comment:
   Is this the required change?



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -222,6 +222,11 @@ public final class KafkaRaftClient<T> implements 
RaftClient<T> {
     private volatile RemoveVoterHandler removeVoterHandler;
     private volatile UpdateVoterHandler updateVoterHandler;
 
+    // When the node is a startup node, we need a flag to join the
+    // cluster once it is removed.
+    private volatile boolean skipFirstAutoJoinAttempt = false;
+    private volatile boolean hasJoin = false;

Review Comment:
   Why should we have 2 variables for the same purpose? Could we just use one 
variable to identify if we need to skip this auto join?



##########
raft/src/main/java/org/apache/kafka/raft/internals/RemoveVoterHandler.java:
##########
@@ -185,4 +185,11 @@ public void highWatermarkUpdated(LeaderState<?> 
leaderState) {
             })
         );
     }
+
+    public void printVoterSet(VoterSet voterSet) {

Review Comment:
   What's this for?



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -501,6 +506,10 @@ 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()) {
+            partitionState.setInitBootstrapNode(partitionState.lastVoterSet());
+            skipFirstAutoJoinAttempt = 
partitionState.initBootstrapNodes().contains(nodeId.getAsInt());

Review Comment:
   If the `initBootstrapNodes` is only used in this check, do we still need to 
store the initBootstrapNode in `setInitBootstrapNode`? Can we do this directly?
   `skipFirstAutoJoinAttempt = 
partitionState.lastVoterSet().contains(nodeId.getAsInt());`



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