Skip to content
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
13 changes: 8 additions & 5 deletions raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -358,7 +358,6 @@ private void updateLeaderEndOffsetAndTimestamp(
long currentTimeMs
) {
final LogOffsetMetadata endOffsetMetadata = log.endOffset();

if (state.updateLocalState(endOffsetMetadata, partitionState.lastVoterSet())) {
onUpdateLeaderHighWatermark(state, currentTimeMs);
}
Expand Down Expand Up @@ -1476,6 +1475,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 +1512,7 @@ private CompletableFuture<FetchResponseData> handleFetchRequest(
FetchRequest.replicaId(request),
fetchPartition.replicaDirectoryId()
);

FetchResponseData response = tryCompleteFetchRequest(
requestMetadata.listenerName(),
requestMetadata.apiVersion(),
Expand Down Expand Up @@ -2842,7 +2843,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 @@ -3029,13 +3030,15 @@ private long maybeSendBeginQuorumEpochRequests(
)
);

timeUntilNextBeginQuorumSend = maybeSendRequest(
Set<ReplicaKey> needToSendBeginQuorumRequests = state.needToSendBeginQuorumRequests(currentTimeMs);
timeUntilNextBeginQuorumSend = maybeSendRequests(
currentTimeMs,
voters
.voterKeys()
.stream()
.filter(key -> key.id() != quorum.localIdOrThrow())
.collect(Collectors.toSet()),
.filter(needToSendBeginQuorumRequests::contains)
.collect(Collectors.toUnmodifiableSet()),
nodeSupplier,
this::buildBeginQuorumEpochRequest
);
Expand Down Expand Up @@ -3117,7 +3120,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
12 changes: 12 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 @@ -188,6 +188,18 @@ public void resetBeginQuorumEpochTimer(long currentTimeMs) {
beginQuorumEpochTimer.reset(beginQuorumEpochTimeoutMs);
}

public Set<ReplicaKey> needToSendBeginQuorumRequests(long currentTimeMs) {
Set<ReplicaKey> replicaKeys = new HashSet<>();
beginQuorumEpochTimer.update(currentTimeMs);
for (ReplicaState state : voterStates.values()) {
if (beginQuorumEpochTimer.currentTimeMs() - state.lastFetchTimestamp >= beginQuorumEpochTimeoutMs
|| !state.hasAcknowledgedLeader) {
replicaKeys.add(state.replicaKey());
}
}
return replicaKeys;
}

/**
* Get the remaining time in milliseconds until the checkQuorumTimer expires.
*
Expand Down
42 changes: 42 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 @@ -39,6 +39,7 @@
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.raft.errors.BufferAllocationException;
import org.apache.kafka.raft.errors.NotLeaderException;
import org.apache.kafka.server.common.KRaftVersion;
import org.apache.kafka.test.TestUtils;

import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -633,6 +634,47 @@ public void testBeginQuorumEpochHeartbeat(boolean withKip853Rpc) throws Exceptio
context.assertSentBeginQuorumEpochRequest(epoch, Set.of(remoteId1, remoteId2));
}

@Test
public void testBeginQuorumShouldNotSendAfterFetchRequest() throws Exception {
ReplicaKey localId = replicaKey(randomReplicaId(), true);
int remoteId1 = localId.id() + 1;
int remoteId2 = localId.id() + 2;
ReplicaKey replicaKey1 = replicaKey(remoteId1, true);
ReplicaKey replicaKey2 = replicaKey(remoteId2, true);

RaftClientTestContext context = new RaftClientTestContext.Builder(localId.id(), localId.directoryId().get())
.withRaftProtocol(KIP_853_PROTOCOL)
.withStartingVoters(VoterSetTest.voterSet(Stream.of(localId, replicaKey1, replicaKey2)), KRaftVersion.KRAFT_VERSION_1)
.build();

context.unattachedToLeader();
int epoch = context.currentEpoch();
assertEquals(OptionalInt.of(localId.id()), 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));

long partialDelay = context.beginQuorumEpochTimeoutMs / 3;
context.time.sleep(context.beginQuorumEpochTimeoutMs / 3);
context.deliverRequest(context.fetchRequest(epoch, replicaKey1, 0, 0, 0));
context.pollUntilResponse();

context.time.sleep(context.beginQuorumEpochTimeoutMs - partialDelay);
context.client.poll();
// don't send BeginQuorumEpochRequest again for replicaKey1 since fetchRequest is sent.
context.assertSentBeginQuorumEpochRequest(epoch, Set.of(remoteId2));

context.deliverRequest(context.fetchRequest(epoch, replicaKey1, 0, 0, 0));
context.pollUntilResponse();
context.time.sleep(context.beginQuorumEpochTimeoutMs);
context.client.poll();
// should send BeginQuorumEpochRequest if sleep time equals beginQuorumEpochTimeoutMs
context.assertSentBeginQuorumEpochRequest(epoch, Set.of(remoteId1, remoteId2));
}


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