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


##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -500,11 +501,31 @@ public void initialize(
             kafkaRaftMetrics,
             externalKRaftMetrics
         );
+
+        // Set up listener to track voter set changes
+        partitionState.setVoterSetChangeListener((offset, voterSet) -> {
+            // We dont need to check high watermark here since it already 
check by
+            // hasJoined is not empty.
+            if (nodeId.isPresent() && hasJoined.isPresent()) {
+                ReplicaKey localReplicaKey = ReplicaKey.of(nodeId.getAsInt(), 
nodeDirectoryId);
+                if (voterSet.isVoter(localReplicaKey) && !hasJoined.get()) {
+                    logger.error("Detected that local node {} has been added 
to voter set at offset {}",
+                               localReplicaKey, offset);
+                    hasJoined = Optional.of(true);
+                }
+            }
+        });
+
         // Read the entire log
         logger.info("Reading KRaft snapshot and log as part of the 
initialization");
         partitionState.updateState();
         logger.info("Starting voters are {}", partitionState.lastVoterSet());
 
+        if (nodeId.isPresent() && canBecomeVoter && quorumConfig.autoJoin()
+                && isVoter(ReplicaKey.of(nodeId.getAsInt(), nodeDirectoryId))) 
{
+            hasJoined = Optional.of(true);
+        }
+

Review Comment:
   @TaiJuWu , about your comment:
   1. Even if this is the temporary solution for v4.2, we still need to make it 
right, otherwise, this is not a `good  solution` at all.
   2. If the change will fail the tests or make them flaky, we should:
     a. figure it out if it is expected or not
     b. update the tests to make it work as what we expected
   
   I appreciate you stay with us for this long time. Please let us know if you 
need any help. Thanks.



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