chia7712 commented on code in PR #20318: URL: https://github.com/apache/kafka/pull/20318#discussion_r2297000920
########## raft/src/main/java/org/apache/kafka/raft/LeaderState.java: ########## @@ -188,6 +191,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> needSendBeginQuorumRequestReplicaKey(long currentTimeMs) { + Set<ReplicaKey> replicaKeys = new HashSet<>(); + beginQuorumEpochTimer.update(currentTimeMs); + for (Map.Entry<ReplicaKey, Long> entry : lastFetchRequestMs.entrySet()) { + if (beginQuorumEpochTimer.currentTimeMs() - entry.getValue() >= beginQuorumEpochTimeoutMs) { + replicaKeys.add(ReplicaKey.of(entry.getKey().id(), entry.getKey().directoryId().orElse(NO_DIRECTORY_ID))); Review Comment: how about `replicaKeys.add(entry.getKey());`? ########## raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java: ########## @@ -3043,12 +3047,17 @@ private long maybeSendBeginQuorumEpochRequests( ) ); - timeUntilNextBeginQuorumSend = maybeSendRequest( + Set<ReplicaKey> needToSendBeginQuorumRequest = state.needSendBeginQuorumRequestReplicaKey(currentTimeMs); + Set<Integer> needToSendBeginQuorumRequestNode = needToSendBeginQuorumRequest.stream(). + map(ReplicaKey::id).collect(Collectors.toSet()); + + timeUntilNextBeginQuorumSend = maybeSendRequests( currentTimeMs, voters .voterKeys() .stream() .filter(key -> key.id() != quorum.localIdOrThrow()) + .filter(key -> !needToSendBeginQuorumRequestNode.contains(key.id())) Review Comment: Why are we only checking `id`? The unique key is actually composed of both `id` and `directory id`, 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