brandboat commented on code in PR #20318:
URL: https://github.com/apache/kafka/pull/20318#discussion_r2301597904


##########
raft/src/main/java/org/apache/kafka/raft/LeaderState.java:
##########
@@ -188,6 +189,22 @@ public void resetBeginQuorumEpochTimer(long currentTimeMs) 
{
         beginQuorumEpochTimer.reset(beginQuorumEpochTimeoutMs);
     }
 
+    public void updateLastReceivedFetchRequest(ReplicaKey replicaKey, long 
currentTimeMs) {
+        beginQuorumEpochTimer.update(currentTimeMs);
+        lastFetchRequestMs.put(replicaKey, 
beginQuorumEpochTimer.currentTimeMs());
+    }
+
+    public Set<ReplicaKey> needSendBeginQuorumRequestNodes(long currentTimeMs) 
{
+        Set<ReplicaKey> replicaKeys = new HashSet<>();
+        beginQuorumEpochTimer.update(currentTimeMs);
+        for (Map.Entry<ReplicaKey, Long> entry : 
lastFetchRequestMs.entrySet()) {
+            if (beginQuorumEpochTimer.currentTimeMs() - entry.getValue() >= 
beginQuorumEpochTimeoutMs) {

Review Comment:
   From the code above at KafkaRaftClient#L3059,
   
   ```java
   .filter(key -> !needToSendBeginQuorumRequest.contains(key))
   ```
   
    I think the condition should be changed to:
   ```suggestion
               if (beginQuorumEpochTimer.currentTimeMs() - entry.getValue() < 
beginQuorumEpochTimeoutMs) {
   ```
   
   and we should rename the method to `skipSendBeginQuorumRequestNodes`.



##########
raft/src/main/java/org/apache/kafka/raft/LeaderState.java:
##########
@@ -88,6 +88,7 @@ public class LeaderState<T> implements EpochState {
     private final Timer beginQuorumEpochTimer;
     private final int beginQuorumEpochTimeoutMs;
     private final KafkaRaftMetrics kafkaRaftMetrics;
+    private final Map<ReplicaKey, Long> lastFetchRequestMs = new HashMap<>();

Review Comment:
   This could be replaced with ReplicaState#lastFetchTimestamp, right?



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