Skip to content

KAFKA-16926: Optimize BeginQuorumEpoch heartbeat #20318

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 9 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 12 additions & 3 deletions raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -1476,6 +1476,7 @@ private boolean hasValidClusterId(String requestClusterId) {
* - {@link Errors#INVALID_REQUEST} if the request epoch is larger than the leader's current epoch
* or if either the fetch offset or the last fetched epoch is invalid
*/
@SuppressWarnings("CyclomaticComplexity")
private CompletableFuture<FetchResponseData> handleFetchRequest(
RaftRequest.Inbound requestMetadata,
long currentTimeMs
Expand Down Expand Up @@ -1512,6 +1513,9 @@ private CompletableFuture<FetchResponseData> handleFetchRequest(
FetchRequest.replicaId(request),
fetchPartition.replicaDirectoryId()
);
if (quorum.isLeader()) {
quorum.leaderStateOrThrow().updateLastReceivedFetchRequest(replicaKey, currentTimeMs);
}
FetchResponseData response = tryCompleteFetchRequest(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
Expand Down Expand Up @@ -2856,7 +2860,7 @@ private long maybeSendRequests(
return minBackoffMs;
}

private long maybeSendRequest(
private long maybeSendRequests(
long currentTimeMs,
Set<ReplicaKey> remoteVoters,
Function<Integer, Node> destinationSupplier,
Expand Down Expand Up @@ -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()))
.collect(Collectors.toSet()),
nodeSupplier,
this::buildBeginQuorumEpochRequest
Expand Down Expand Up @@ -3131,7 +3140,7 @@ private long maybeSendVoteRequests(
if (!state.epochElection().isVoteRejected()) {
VoterSet voters = partitionState.lastVoterSet();
boolean preVote = quorum.isProspective();
return maybeSendRequest(
return maybeSendRequests(
currentTimeMs,
state.epochElection().unrecordedVoters(),
voterId -> voters
Expand Down
19 changes: 19 additions & 0 deletions raft/src/main/java/org/apache/kafka/raft/LeaderState.java
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,8 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;

import static org.apache.kafka.raft.ReplicaKey.NO_DIRECTORY_ID;

/**
* In the context of LeaderState, an acknowledged voter means one who has acknowledged the current leader by either
* responding to a `BeginQuorumEpoch` request from the leader or by beginning to send `Fetch` requests.
Expand Down Expand Up @@ -88,6 +90,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<>();

// This is volatile because resignation can be requested from an external thread.
private volatile boolean resignRequested = false;
Expand Down Expand Up @@ -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)));
}
}
return replicaKeys;
}

/**
* Get the remaining time in milliseconds until the checkQuorumTimer expires.
*
Expand Down
33 changes: 33 additions & 0 deletions raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -633,6 +633,39 @@ public void testBeginQuorumEpochHeartbeat(boolean withKip853Rpc) throws Exceptio
context.assertSentBeginQuorumEpochRequest(epoch, Set.of(remoteId1, remoteId2));
}

@ParameterizedTest
@ValueSource(booleans = { true, false })
public void testBeginQuorumShouldNotSendAfterFetchRequest(boolean withKip853Rpc) throws Exception {
int localId = randomReplicaId();
int remoteId1 = localId + 1;
int remoteId2 = localId + 2;
Set<Integer> voters = Set.of(localId, remoteId1, remoteId2);
ReplicaKey replicaKey = replicaKey(localId + 1, withKip853Rpc);

RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.withKip853Rpc(withKip853Rpc)
.build();

context.unattachedToLeader();
int epoch = context.currentEpoch();
assertEquals(OptionalInt.of(localId), context.currentLeader());

// begin epoch requests should be sent out every beginQuorumEpochTimeoutMs
context.time.sleep(context.beginQuorumEpochTimeoutMs);
context.client.poll();
context.assertSentBeginQuorumEpochRequest(epoch, Set.of(remoteId1, remoteId2));

context.time.sleep(context.beginQuorumEpochTimeoutMs / 3);
context.deliverRequest(context.fetchRequest(epoch, replicaKey, 0, 0, 0));
context.pollUntilResponse();

context.time.sleep(context.beginQuorumEpochTimeoutMs);
context.client.poll();
// don't send BeginQuorumEpochRequest again if fetchRequest is sent.
context.assertSentBeginQuorumEpochRequest(epoch, Set.of(remoteId2));
}


@ParameterizedTest
@ValueSource(booleans = { true, false })
public void testLeaderShouldResignLeadershipIfNotGetFetchRequestFromMajorityVoters(boolean withKip853Rpc) throws Exception {
Expand Down