From 16217d0e7db7b45308bab234325a715dbbf58989 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Tue, 3 Dec 2024 16:18:51 -0800 Subject: [PATCH 01/23] initial commit --- checkstyle/suppressions.xml | 10 +- .../kafka/common/requests/VoteRequest.java | 6 +- .../unit/kafka/server/RequestQuotaTest.scala | 2 +- .../org/apache/kafka/raft/CandidateState.java | 162 +++++----- .../apache/kafka/raft/KafkaRaftClient.java | 257 +++++++++++++--- .../apache/kafka/raft/ProspectiveState.java | 261 ++++++++++++++++ .../org/apache/kafka/raft/QuorumState.java | 95 +++++- .../org/apache/kafka/raft/VotingState.java | 157 ++++++++++ .../kafka/raft/KafkaNetworkChannelTest.java | 2 +- .../raft/KafkaRaftClientPreVoteTest.java | 266 +++++++++++++++- .../kafka/raft/KafkaRaftClientTest.java | 115 +++++-- .../kafka/raft/ProspectiveStateTest.java | 286 ++++++++++++++++++ .../kafka/raft/RaftClientTestContext.java | 68 ++++- 13 files changed, 1505 insertions(+), 182 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java create mode 100644 raft/src/main/java/org/apache/kafka/raft/VotingState.java create mode 100644 raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 2d05fac1e7052..7362d9f586beb 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -91,7 +91,7 @@ files="ClientUtils.java"/> + files="(KafkaConsumer|ConsumerCoordinator|AbstractFetch|KafkaProducer|AbstractRequest|AbstractResponse|TransactionManager|Admin|KafkaAdminClient|MockAdminClient|KafkaNetworkChannelTest).java"/> @@ -102,7 +102,7 @@ files="(AbstractFetch|ClientTelemetryReporter|ConsumerCoordinator|CommitRequestManager|FetchCollector|OffsetFetcherUtils|KafkaProducer|Sender|ConfigDef|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer|RecordAccumulator|MemoryRecords|FetchSessionHandler|MockAdminClient).java"/> + files="(AbstractRequest|AbstractResponse|KerberosLogin|WorkerSinkTaskTest|TransactionManagerTest|SenderTest|KafkaAdminClient|ConsumerCoordinatorTest|KafkaAdminClientTest).java"/> @@ -187,6 +187,12 @@ + + + + diff --git a/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java index 4acf8d31ca5ec..619b5bd78e50c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/VoteRequest.java @@ -74,7 +74,8 @@ public static VoteRequestData singletonRequest(TopicPartition topicPartition, int replicaEpoch, int replicaId, int lastEpoch, - long lastEpochEndOffset) { + long lastEpochEndOffset, + boolean preVote) { return new VoteRequestData() .setClusterId(clusterId) .setTopics(Collections.singletonList( @@ -86,7 +87,8 @@ public static VoteRequestData singletonRequest(TopicPartition topicPartition, .setReplicaEpoch(replicaEpoch) .setReplicaId(replicaId) .setLastOffsetEpoch(lastEpoch) - .setLastOffset(lastEpochEndOffset)) + .setLastOffset(lastEpochEndOffset) + .setPreVote(preVote)) ))); } } diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 6ccc00a6c4adb..245e580c0da7c 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -624,7 +624,7 @@ class RequestQuotaTest extends BaseRequestTest { new AlterUserScramCredentialsRequest.Builder(new AlterUserScramCredentialsRequestData()) case ApiKeys.VOTE => - new VoteRequest.Builder(VoteRequest.singletonRequest(tp, null, 1, 2, 0, 10)) + new VoteRequest.Builder(VoteRequest.singletonRequest(tp, null, 1, 2, 0, 10, true)) case ApiKeys.BEGIN_QUORUM_EPOCH => new BeginQuorumEpochRequest.Builder(BeginQuorumEpochRequest.singletonRequest(tp, null, 2, 5)) diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 902d334ad792f..37fb32638e863 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -20,17 +20,16 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.raft.VotingState.VoterState.State; import org.slf4j.Logger; import java.util.HashMap; import java.util.Map; import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; import java.util.stream.Stream; -public class CandidateState implements EpochState { +public class CandidateState implements VotingState { private final int localId; private final Uuid localDirectoryId; private final int epoch; @@ -45,10 +44,10 @@ public class CandidateState implements EpochState { /** * The lifetime of a candidate state is the following. * - * 1. Once started, it would keep record of the received votes. - * 2. If majority votes granted, it can then end its life and will be replaced by a leader state; - * 3. If majority votes rejected or election timed out, it would transit into a backing off phase; - * after the backoff phase completes, it would end its left and be replaced by a new candidate state with bumped retry. + * 1. Once started, it will keep record of the received votes. + * 2. If majority votes granted, it will transition to leader state. + * 3. If majority votes rejected or election timed out, it will enter a backing off phase; + * after the backoff phase completes, it will be replaced by a new candidate state with bumped retry. */ private boolean isBackingOff; @@ -95,18 +94,23 @@ public int localId() { return localId; } - public int majoritySize() { - return voteStates.size() / 2 + 1; - } - - private long numGranted() { - return votersInState(State.GRANTED).count(); - } - - private long numUnrecorded() { - return votersInState(State.UNRECORDED).count(); + @Override + public Map voteStates() { + return voteStates; } +// public int majoritySize() { +// return voteStates.size() / 2 + 1; +// } +// +// public long numGranted() { +// return votersInState(State.GRANTED).count(); +// } +// +// public long numUnrecorded() { +// return votersInState(State.UNRECORDED).count(); +// } +// /** * Check if the candidate is backing off for the next election */ @@ -192,34 +196,34 @@ public void startBackingOff(long currentTimeMs, long backoffDurationMs) { this.isBackingOff = true; } - /** - * Get the set of voters which have not been counted as granted or rejected yet. - * - * @return The set of unrecorded voters - */ - public Set unrecordedVoters() { - return votersInState(State.UNRECORDED).collect(Collectors.toSet()); - } - - /** - * Get the set of voters that have granted our vote requests. - * - * @return The set of granting voters, which should always contain the ID of the candidate - */ - public Set grantingVoters() { - return votersInState(State.GRANTED).map(ReplicaKey::id).collect(Collectors.toSet()); - } - - /** - * Get the set of voters that have rejected our candidacy. - * - * @return The set of rejecting voters - */ - public Set rejectingVoters() { - return votersInState(State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet()); - } - - private Stream votersInState(State state) { +// /** +// * Get the set of voters which have not been counted as granted or rejected yet. +// * +// * @return The set of unrecorded voters +// */ +// public Set unrecordedVoters() { +// return votersInState(State.UNRECORDED).collect(Collectors.toSet()); +// } + +// /** +// * Get the set of voters that have granted our vote requests. +// * +// * @return The set of granting voters, which should always contain the ID of the candidate +// */ +// public Set grantingVoters() { +// return votersInState(State.GRANTED).map(ReplicaKey::id).collect(Collectors.toSet()); +// } +// +// /** +// * Get the set of voters that have rejected our candidacy. +// * +// * @return The set of rejecting voters +// */ +// public Set rejectingVoters() { +// return votersInState(State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet()); +// } + + public Stream votersInState(State state) { return voteStates .values() .stream() @@ -227,16 +231,25 @@ private Stream votersInState(State state) { .map(VoterState::replicaKey); } + @Override public boolean hasElectionTimeoutExpired(long currentTimeMs) { electionTimer.update(currentTimeMs); return electionTimer.isExpired(); } + @Override + public long remainingElectionTimeMs(long currentTimeMs) { + electionTimer.update(currentTimeMs); + return electionTimer.remainingMs(); + } + + @Override public boolean isBackoffComplete(long currentTimeMs) { backoffTimer.update(currentTimeMs); return backoffTimer.isExpired(); } + @Override public long remainingBackoffMs(long currentTimeMs) { if (!isBackingOff) { throw new IllegalStateException("Candidate is not currently backing off"); @@ -245,11 +258,6 @@ public long remainingBackoffMs(long currentTimeMs) { return backoffTimer.remainingMs(); } - public long remainingElectionTimeMs(long currentTimeMs) { - electionTimer.update(currentTimeMs); - return electionTimer.remainingMs(); - } - @Override public ElectionState election() { return ElectionState.withVotedCandidate( @@ -275,7 +283,7 @@ public Optional highWatermark() { } @Override - public boolean canGrantVote( + public boolean canGrantVote(// maybe this needs to grant prevotes ReplicaKey replicaKey, boolean isLogUpToDate, boolean isPreVote @@ -319,30 +327,30 @@ public String name() { @Override public void close() {} - private static final class VoterState { - private final ReplicaKey replicaKey; - private State state = State.UNRECORDED; - - private VoterState(ReplicaKey replicaKey) { - this.replicaKey = replicaKey; - } - - public State state() { - return state; - } - - public void setState(State state) { - this.state = state; - } - - public ReplicaKey replicaKey() { - return replicaKey; - } - } - - private enum State { - UNRECORDED, - GRANTED, - REJECTED - } +// private static final class VoterState { +// private final ReplicaKey replicaKey; +// private State state = State.UNRECORDED; +// +// private VoterState(ReplicaKey replicaKey) { +// this.replicaKey = replicaKey; +// } +// +// public State state() { +// return state; +// } +// +// public void setState(State state) { +// this.state = state; +// } +// +// public ReplicaKey replicaKey() { +// return replicaKey; +// } +// } +// +// private enum State { +// UNRECORDED, +// GRANTED, +// REJECTED +// } } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 41da22b4b36c4..c8b4a9b363ab1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -665,6 +665,32 @@ private boolean maybeTransitionToLeader(CandidateState state, long currentTimeMs } } + private boolean maybeTransitionToCandidate(ProspectiveState state, long currentTimeMs) { + if (state.isVoteGranted()) { + transitionToCandidate(currentTimeMs); + return true; + } else { + return false; + } + } + + /** + * Only applies to VotingStates (Prospective or Candidate). If enough votes were granted + * then this method is called to transition the state forward - either from Prospective to Candidate + * or from Candidate to Leader. + * @return true if the state transitioned forward, false otherwise + */ + private boolean maybeTransitionForward(VotingState state, long currentTimeMs) { + if (quorum.isProspective()) { + return maybeTransitionToCandidate(quorum.prospectiveStateOrThrow(), currentTimeMs); + } else if (quorum.isCandidate()) { + return maybeTransitionToLeader(quorum.candidateStateOrThrow(), currentTimeMs); + } else { + throw new IllegalStateException("Expected to be a VotingState (Prospective or Candidate), " + + "but current state is " + state); + } + } + private void onBecomeCandidate(long currentTimeMs) { CandidateState state = quorum.candidateStateOrThrow(); if (!maybeTransitionToLeader(state, currentTimeMs)) { @@ -679,6 +705,40 @@ private void transitionToCandidate(long currentTimeMs) { onBecomeCandidate(currentTimeMs); } + private void onBecomeProspective(long currentTimeMs) { + ProspectiveState state = quorum.prospectiveStateOrThrow(); + if (!maybeTransitionToCandidate(state, currentTimeMs)) { + resetConnections(); + kafkaRaftMetrics.updateElectionStartMs(currentTimeMs); + } + } + + private void transitionToProspective(long currentTimeMs) { + quorum.transitionToProspective(); + maybeFireLeaderChange(); + onBecomeProspective(currentTimeMs); + } + + private void transitionToVotingState(long currentTimeMs) { + if (partitionState.lastKraftVersion().isReconfigSupported()) { + transitionToProspective(currentTimeMs); + } else { + transitionToCandidate(currentTimeMs); + } + } + + private void refreshVotingState(long currentTimeMs) { + VotingState state = quorum.votingStateOrThrow(); + if (quorum.isProspective()) { + transitionToProspective(currentTimeMs); + } else if (quorum.isCandidate()) { + transitionToCandidate(currentTimeMs); + } else { + throw new IllegalStateException("Expected to be a VotingState (Prospective or Candidate) " + + "but current state is " + state); + } + } + private void transitionToUnattached(int epoch) { quorum.transitionToUnattached(epoch); maybeFireLeaderChange(); @@ -926,29 +986,16 @@ private boolean handleVoteResponse( if (quorum.isLeader()) { logger.debug("Ignoring vote response {} since we already became leader for epoch {}", partitionResponse, quorum.epoch()); - } else if (quorum.isCandidate()) { - CandidateState state = quorum.candidateStateOrThrow(); - if (partitionResponse.voteGranted()) { - state.recordGrantedVote(remoteNodeId); - maybeTransitionToLeader(state, currentTimeMs); - } else { - state.recordRejectedVote(remoteNodeId); - - // If our vote is rejected, we go immediately to the random backoff. This - // ensures that we are not stuck waiting for the election timeout when the - // vote has become gridlocked. - if (state.isVoteRejected() && !state.isBackingOff()) { - logger.info("Insufficient remaining votes to become leader (rejected by {}). " + - "We will backoff before retrying election again", state.rejectingVoters()); - - state.startBackingOff( - currentTimeMs, - binaryExponentialElectionBackoffMs(state.retries()) - ); - } - } + } else if (quorum.isVotingState()) { + VotingState state = quorum.votingStateOrThrow(); + handleVoteResponse( + state, + partitionResponse, + remoteNodeId, + currentTimeMs); } else { - logger.debug("Ignoring vote response {} since we are no longer a candidate in epoch {}", + logger.debug("Ignoring vote response {} since we are no longer a VotingState " + + "(Prospective or Candidate) in epoch {}", partitionResponse, quorum.epoch()); } return true; @@ -957,6 +1004,42 @@ private boolean handleVoteResponse( } } + private void handleVoteResponse(VotingState state, + VoteResponseData.PartitionData partitionResponse, + int remoteNodeId, + long currentTimeMs) { + if (quorum.isProspective() && !partitionResponse.preVote()) { + logger.info("Ignoring vote response {} since we sent a PreVote request but received a non-PreVote " + + "response in epoch {} from {}. Transitioning to from Prospective to Candidate since not " + + "all nodes in the quorum support PreVote", + partitionResponse, quorum.epoch(), remoteNodeId); + transitionToCandidate(currentTimeMs); + } else if (quorum.isCandidate() && partitionResponse.preVote()) { + logger.debug("Ignoring vote response {} since we already became a Candidate for epoch {} but " + + "received a PreVote response from {}", + partitionResponse, quorum.epoch(), remoteNodeId); + } + if (partitionResponse.voteGranted()) { + state.recordGrantedVote(remoteNodeId); + maybeTransitionForward(state, currentTimeMs); + } else { + state.recordRejectedVote(remoteNodeId); + + // If our vote is rejected, we go immediately to the random backoff. This + // ensures that we are not stuck waiting for the election timeout when the + // vote has become gridlocked. + if (state.isVoteRejected() && !state.isBackingOff()) { + logger.info("Insufficient remaining votes to win election (rejected by {}). " + + "We will backoff before retrying election again", state.rejectingVoters()); + + state.startBackingOff( + currentTimeMs, + binaryExponentialElectionBackoffMs(state.retries()) + ); + } + } + } + private int binaryExponentialElectionBackoffMs(int retries) { if (retries <= 0) { throw new IllegalArgumentException("Retries " + retries + " should be larger than zero"); @@ -2368,7 +2451,7 @@ private boolean hasConsistentLeader(int epoch, OptionalInt leaderId) { * - Optional.of(true) indicates that the response was successfully handled here and * the node can become ready * - Optional.of(false) indicates that the response was handled here, but that the - * node should got in to backoff + * node should go into backoff */ private Optional maybeHandleCommonResponse( Errors error, @@ -2752,6 +2835,7 @@ private BeginQuorumEpochRequestData buildBeginQuorumEpochRequest(ReplicaKey remo private VoteRequestData buildVoteRequest(ReplicaKey remoteVoter) { OffsetAndEpoch endOffset = endOffset(); + boolean isPreVote = quorum.isProspective(); return RaftUtil.singletonVoteRequest( log.topicPartition(), clusterId, @@ -2760,7 +2844,7 @@ private VoteRequestData buildVoteRequest(ReplicaKey remoteVoter) { remoteVoter, endOffset.epoch(), endOffset.offset(), - false + isPreVote ); } @@ -2936,7 +3020,7 @@ private long pollResigned(long currentTimeMs) { stateTimeoutMs = shutdown.remainingTimeMs(); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { if (quorum.isVoter()) { - transitionToCandidate(currentTimeMs); + transitionToVotingState(currentTimeMs); } else { // It is possible that the old leader is not a voter in the new voter set. // In that case increase the epoch and transition to unattached. The epoch needs @@ -2984,9 +3068,36 @@ private long pollLeader(long currentTimeMs) { ) ); } +// +// private long maybeSendVoteRequests( +// CandidateState state, +// long currentTimeMs +// ) { +// // Continue sending Vote requests as long as we still have a chance to win the election +// if (!state.isVoteRejected()) { +// VoterSet voters = partitionState.lastVoterSet(); +// return maybeSendRequest( +// currentTimeMs, +// state.unrecordedVoters(), +// voterId -> voters +// .voterNode(voterId, channel.listenerName()) +// .orElseThrow(() -> +// new IllegalStateException( +// String.format( +// "Unknown endpoint for voter id %d for listener name %s", +// voterId, +// channel.listenerName() +// ) +// ) +// ), +// this::buildVoteRequest +// ); +// } +// return Long.MAX_VALUE; +// } private long maybeSendVoteRequests( - CandidateState state, + VotingState state, long currentTimeMs ) { // Continue sending Vote requests as long as we still have a chance to win the election @@ -3012,12 +3123,50 @@ private long maybeSendVoteRequests( return Long.MAX_VALUE; } + /** + * pollVotingState is equivalent to pollCandidate and pollProspective + */ + private long pollVotingState(long currentTimeMs) { + VotingState state = quorum.votingStateOrThrow(); + GracefulShutdown shutdown = this.shutdown.get(); + + if (shutdown != null) { + if (quorum.isCandidate()) { + // If we happen to shut down while we are a candidate, we will continue + // with the current election until one of the following conditions is met: + // 1) we are elected as leader (which allows us to resign) + // 2) another leader is elected + // 3) the shutdown timer expires + long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); + return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); + } + return shutdown.remainingTimeMs(); + } else if (state.isBackingOff()) { + if (state.isBackoffComplete(currentTimeMs)) { + logger.info("Becoming {} again after election backoff has completed", state.name()); + refreshVotingState(currentTimeMs); + return 0L; + } + return state.remainingBackoffMs(currentTimeMs); + } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { + long backoffDurationMs = binaryExponentialElectionBackoffMs(state.retries()); + logger.info("Election has timed out, backing off for {}ms before becoming a {} again", + backoffDurationMs, state.name()); + state.startBackingOff(currentTimeMs, backoffDurationMs); +// long minFetchRequestBackoffMs = maybeSendAnyVoterFetch(currentTimeMs); + return backoffDurationMs; + } else { + long minVoteRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); + return Math.min(minVoteRequestBackoffMs, state.remainingElectionTimeMs(currentTimeMs)); + } + } + private long pollCandidate(long currentTimeMs) { - CandidateState state = quorum.candidateStateOrThrow(); + VotingState state = quorum.votingStateOrThrow(); GracefulShutdown shutdown = this.shutdown.get(); if (shutdown != null) { - // If we happen to shutdown while we are a candidate, we will continue + // If we happen to shut down while we are a candidate, we will continue // with the current election until one of the following conditions is met: // 1) we are elected as leader (which allows us to resign) // 2) another leader is elected @@ -3033,13 +3182,49 @@ private long pollCandidate(long currentTimeMs) { return state.remainingBackoffMs(currentTimeMs); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { long backoffDurationMs = binaryExponentialElectionBackoffMs(state.retries()); - logger.info("Election has timed out, backing off for {}ms before becoming a candidate again", - backoffDurationMs); + logger.info("Election has timed out, backing off for {}ms before becoming a {} again", + backoffDurationMs, state.name()); state.startBackingOff(currentTimeMs, backoffDurationMs); return backoffDurationMs; } else { + long minVoteRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); + return Math.min(minVoteRequestBackoffMs, state.remainingElectionTimeMs(currentTimeMs)); + } + } + + private long pollProspective(long currentTimeMs) { + VotingState state = quorum.votingStateOrThrow(); + GracefulShutdown shutdown = this.shutdown.get(); + + if (shutdown != null) { + // If we happen to shut down while we are a candidate, we will continue + // with the current election until one of the following conditions is met: + // 1) we are elected as leader (which allows us to resign) + // 2) another leader is elected + // 3) the shutdown timer expires long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); - return Math.min(minRequestBackoffMs, state.remainingElectionTimeMs(currentTimeMs)); + return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); + } else if (state.isBackingOff()) { + if (state.isBackoffComplete(currentTimeMs)) { + logger.info("Re-elect as prospective after election backoff has completed"); + transitionToProspective(currentTimeMs); + return 0L; + } + long minFetchRequestBackoffMs = maybeSendAnyVoterFetch(currentTimeMs); + return Math.min(minFetchRequestBackoffMs, state.remainingBackoffMs(currentTimeMs)); + } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { + // this hasElectionTimeoutExpired clause needs to follow isBackingOff clause + // because the backoff timer can be reset in this clause and we want to avoid + // resetting the timer on every fetch to bootstrap voters + long backoffDurationMs = binaryExponentialElectionBackoffMs(state.retries()); + logger.info("Election has timed out, backing off for {}ms before becoming a {} again", + backoffDurationMs, state.name()); + state.startBackingOff(currentTimeMs, backoffDurationMs); + long minFetchRequestBackoffMs = maybeSendAnyVoterFetch(currentTimeMs); + return Math.min(minFetchRequestBackoffMs, backoffDurationMs); + } else { + long minVoteRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); + return Math.min(minVoteRequestBackoffMs, state.remainingElectionTimeMs(currentTimeMs)); } } @@ -3060,8 +3245,8 @@ private long pollFollowerAsVoter(FollowerState state, long currentTimeMs) { // skip the transition to candidate in any case. backoffMs = 0; } else if (state.hasFetchTimeoutExpired(currentTimeMs)) { - logger.info("Become candidate due to fetch timeout"); - transitionToCandidate(currentTimeMs); + logger.info("Transitioning to VotingState (Prospective or Candidate) due to fetch timeout"); + transitionToVotingState(currentTimeMs); backoffMs = 0; } else if (state.hasUpdateVoterPeriodExpired(currentTimeMs)) { if (partitionState.lastKraftVersion().isReconfigSupported() && @@ -3165,7 +3350,7 @@ private long pollUnattachedAsVoter(UnattachedState state, long currentTimeMs) { // shutdown completes or an epoch bump forces another state transition return shutdown.remainingTimeMs(); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { - transitionToCandidate(currentTimeMs); + transitionToVotingState(currentTimeMs); return 0L; } else { return pollUnattachedCommon(state, currentTimeMs); @@ -3180,8 +3365,12 @@ private long pollUnattachedCommon(UnattachedState state, long currentTimeMs) { private long pollCurrentState(long currentTimeMs) { if (quorum.isLeader()) { return pollLeader(currentTimeMs); +// } else if (quorum.isVotingState()) { +// return pollVotingState(currentTimeMs); } else if (quorum.isCandidate()) { return pollCandidate(currentTimeMs); + } else if (quorum.isProspective()) { + return pollProspective(currentTimeMs); } else if (quorum.isFollower()) { return pollFollower(currentTimeMs); } else if (quorum.isUnattached()) { diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java new file mode 100644 index 0000000000000..0e11a79e58e72 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +public class ProspectiveState implements VotingState { + private final int epoch; + private final int localId; + private final Optional votedKey; + private final VoterSet voters; +// private final long electionTimeoutMs; +// private final Timer electionTimer; + private final Map voteStates = new HashMap<>(); + private final Optional highWatermark; + private int retries; + private final int electionTimeoutMs; + private final Timer electionTimer; + private final Timer backoffTimer; + private final Logger log; + + /** + * The lifetime of a prospective state is the following. + * + * 1. Once started, it will keep record of the received votes and continue to fetch from bootstrap voters. + * 2. If it receives a fetch response denoting a leader with a higher epoch, it will transition to follower state. + * 3. If majority votes granted, it will transition to leader state. + * 4. If majority votes rejected or election times out, it will enter a backing off phase; + * after the backoff phase completes, it will send out another round of PreVote requests. + */ + private boolean isBackingOff; + + public ProspectiveState( + Time time, + int localId, + int epoch, + Optional votedKey, + VoterSet voters, + Optional highWatermark, + int retries, + int electionTimeoutMs, + LogContext logContext + ) { + this.localId = localId; + this.epoch = epoch; + this.votedKey = votedKey; + this.voters = voters; + this.highWatermark = highWatermark; + this.retries = retries; + this.isBackingOff = false; + this.electionTimeoutMs = electionTimeoutMs; + this.electionTimer = time.timer(electionTimeoutMs); + this.backoffTimer = time.timer(0); + this.log = logContext.logger(ProspectiveState.class); + + for (ReplicaKey voter : voters.voterKeys()) { + voteStates.put(voter.id(), new VoterState(voter)); + } + voteStates.get(localId).setState(VoterState.State.GRANTED); + } + + public int localId() { + return localId; + } + + public Optional votedKey() { + return votedKey; + } + + @Override + public Map voteStates() { + return voteStates; + } + + @Override + public boolean isBackingOff() { + return isBackingOff; + } + + @Override + public int retries() { + return retries; + } + + /** + * Record a granted vote from one of the voters. + * + * @param remoteNodeId The id of the voter + * @return true if the voter had not been previously recorded + * @throws IllegalArgumentException if the remote node is not a voter + */ + public boolean recordGrantedVote(int remoteNodeId) { + VoterState voterState = voteStates.get(remoteNodeId); + if (voterState == null) { + throw new IllegalArgumentException("Attempt to grant vote to non-voter " + remoteNodeId); + } + + boolean recorded = voterState.state().equals(VoterState.State.UNRECORDED); + voterState.setState(VoterState.State.GRANTED); + + return recorded; + } + + /** + * Record a rejected vote from one of the voters. + * + * @param remoteNodeId The id of the voter + * @return true if the rejected vote had not been previously recorded + * @throws IllegalArgumentException if the remote node is not a voter + */ + public boolean recordRejectedVote(int remoteNodeId) { + VoterState voterState = voteStates.get(remoteNodeId); + if (voterState == null) { + throw new IllegalArgumentException("Attempt to reject vote to non-voter " + remoteNodeId); + } + if (remoteNodeId == localId) { + throw new IllegalStateException("Attempted to reject vote from ourselves"); + } + + boolean recorded = voterState.state().equals(VoterState.State.UNRECORDED); + voterState.setState(VoterState.State.REJECTED); + + return recorded; + } + +// /** +// * Restart the election timer since we've either received sufficient rejecting voters or election timed out +// */ +// public void restartElectionTimer(long currentTimeMs, long electionTimeoutMs) { +// this.electionTimer.update(currentTimeMs); +// this.electionTimer.reset(electionTimeoutMs); +// this.isBackingOff = true; +// } + + /** + * Record the current election has failed since we've either received sufficient rejecting voters or election timed out + */ + public void startBackingOff(long currentTimeMs, long backoffDurationMs) { + this.backoffTimer.update(currentTimeMs); + this.backoffTimer.reset(backoffDurationMs); + this.isBackingOff = true; + } + + @Override + public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolean isPreVote) { + if (votedKey.isPresent()) { + ReplicaKey votedReplicaKey = votedKey.get(); + if (votedReplicaKey.id() == replicaKey.id()) { + return votedReplicaKey.directoryId().isEmpty() || votedReplicaKey.directoryId().equals(replicaKey.directoryId()); + } + log.debug( + "Rejecting Vote request with PreVote={} from replica ({}), already have voted for another " + + "replica ({}) in epoch {}", + isPreVote, + replicaKey, + votedKey, + epoch + ); + return false; + } else if (!isLogUpToDate) { + log.debug( + "Rejecting Vote request with PreVote={} from replica ({}) since replica epoch/offset is not up to date with us", + isPreVote, + replicaKey + ); + } + + return isLogUpToDate; + } + + @Override + public boolean hasElectionTimeoutExpired(long currentTimeMs) { + electionTimer.update(currentTimeMs); + return electionTimer.isExpired(); + } + + @Override + public long remainingElectionTimeMs(long currentTimeMs) { + electionTimer.update(currentTimeMs); + return electionTimer.remainingMs(); + } + + @Override + public boolean isBackoffComplete(long currentTimeMs) { + backoffTimer.update(currentTimeMs); + return backoffTimer.isExpired(); + } + + @Override + public long remainingBackoffMs(long currentTimeMs) { + if (!isBackingOff) { + throw new IllegalStateException("Prospective is not currently backing off"); + } + backoffTimer.update(currentTimeMs); + return backoffTimer.remainingMs(); + } + + @Override + public ElectionState election() { + if (votedKey.isPresent()) { + return ElectionState.withVotedCandidate(epoch, votedKey().get(), voters.voterIds()); + } else { + return ElectionState.withUnknownLeader(epoch, voters.voterIds()); + } + } + + @Override + public int epoch() { + return epoch; + } + + @Override + public Endpoints leaderEndpoints() { + return Endpoints.empty(); + } + + @Override + public Optional highWatermark() { + return highWatermark; + } + + @Override + public String toString() { + return String.format( + "ProspectiveState(epoch=%d, votedKey=%s, voters=%s, highWatermark=%s)", + epoch, + votedKey, + voters, + highWatermark + ); + } + + @Override + public String name() { + return "Prospective"; + } + + @Override + public void close() {} +} diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 4e1ba679e359b..7728048d5f8bd 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -40,14 +40,21 @@ * * Resigned transitions to: * Unattached: After learning of a new election with a higher epoch - * Candidate: After expiration of the election timeout + * Prospective: After expiration of the election timeout * Follower: After discovering a leader with an equal or larger epoch * * Unattached transitions to: * Unattached: After learning of a new election with a higher epoch or after giving a binding vote - * Candidate: After expiration of the election timeout + * Prospective: After expiration of the election timeout * Follower: After discovering a leader with an equal or larger epoch * + * Prospective transitions to: + * Unattached: After learning of an election with a higher epoch, + * or node did not have last known leader and loses/times out election + * Candidate: After receiving a majority of PreVotes granted + * Follower: After discovering a leader with a larger epoch, + * or node had a last known leader and loses/times out election + * * Candidate transitions to: * Unattached: After learning of a new election with a higher epoch * Candidate: After expiration of the election timeout @@ -59,10 +66,10 @@ * * Follower transitions to: * Unattached: After learning of a new election with a higher epoch - * Candidate: After expiration of the fetch timeout + * Prospective: After expiration of the fetch timeout * Follower: After discovering a leader with a larger epoch * - * Observers follow a simpler state machine. The Voted/Candidate/Leader/Resigned + * Observers follow a simpler state machine. The Voted/Prospective/Candidate/Leader/Resigned * states are not possible for observers, so the only transitions that are possible * are between Unattached and Follower. * @@ -529,22 +536,44 @@ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { ); } - public void transitionToCandidate() { + public void transitionToProspective() { + if (!partitionState.lastKraftVersion().isReconfigSupported()) { + throw new IllegalStateException("Cannot transition to Prospective since the current version " + + partitionState.lastKraftVersion() + " does not support PreVote"); + } if (isObserver()) { throw new IllegalStateException( String.format( - "Cannot transition to Candidate since the local id (%s) and directory id (%s) " + - "is not one of the voters %s", + "Cannot transition to Prospective since the local id (%s) and directory id (%s) " + + "is not one of the voters %s", localId, localDirectoryId, partitionState.lastVoterSet() ) ); - } else if (isLeader()) { - throw new IllegalStateException("Cannot transition to Candidate since the local broker.id=" + localId + - " since this node is already a Leader with state " + state); + } else if (isLeader() || isCandidate()) { + throw new IllegalStateException("Cannot transition to Prospective since the local broker.id=" + localId + + " is state " + state); } + int retries = isProspective() ? prospectiveStateOrThrow().retries() + 1 : 1; + + durableTransitionTo(new ProspectiveState( + time, + localIdOrThrow(), + epoch(), + Optional.empty(), + partitionState.lastVoterSet(), + state.highWatermark(), + randomElectionTimeoutMs(), + retries, + logContext + )); + } + + public void transitionToCandidate() { + checkValidTransitionToCandidate(); + int retries = isCandidate() ? candidateStateOrThrow().retries() + 1 : 1; int newEpoch = epoch() + 1; int electionTimeoutMs = randomElectionTimeoutMs(); @@ -562,6 +591,31 @@ public void transitionToCandidate() { )); } + private void checkValidTransitionToCandidate() { + if (isObserver()) { + throw new IllegalStateException( + String.format( + "Cannot transition to Candidate since the local id (%s) and directory id (%s) " + + "is not one of the voters %s", + localId, + localDirectoryId, + partitionState.lastVoterSet() + ) + ); + } + if (partitionState.lastKraftVersion().isReconfigSupported()) { + if (!isCandidate() && !isProspective()) { + throw new IllegalStateException("Cannot transition to Candidate since the local broker.id=" + localId + + " is state " + state); + } + } else { + if (isLeader()) { + throw new IllegalStateException("Cannot transition to Candidate since the local broker.id=" + localId + + " since this node is already a Leader with state " + state); + } + } + } + public LeaderState transitionToLeader(long epochStartOffset, BatchAccumulator accumulator) { if (isObserver()) { throw new IllegalStateException( @@ -683,12 +737,25 @@ public ResignedState resignedStateOrThrow() { throw new IllegalStateException("Expected to be Resigned, but current state is " + state); } + public ProspectiveState prospectiveStateOrThrow() { + if (isProspective()) + return (ProspectiveState) state; + throw new IllegalStateException("Expected to be Prospective, but current state is " + state); + } + public CandidateState candidateStateOrThrow() { if (isCandidate()) return (CandidateState) state; throw new IllegalStateException("Expected to be Candidate, but current state is " + state); } + public VotingState votingStateOrThrow() { + if (isVotingState()) + return (VotingState) state; + throw new IllegalStateException("Expected to be a VotingState (Prospective or Candidate), " + + "but current state is " + state); + } + public LeaderAndEpoch leaderAndEpoch() { ElectionState election = state.election(); return new LeaderAndEpoch(election.optionalLeaderId(), election.epoch()); @@ -718,7 +785,15 @@ public boolean isResigned() { return state instanceof ResignedState; } + public boolean isProspective() { + return state instanceof ProspectiveState; + } + public boolean isCandidate() { return state instanceof CandidateState; } + + public boolean isVotingState() { + return state instanceof VotingState; + } } diff --git a/raft/src/main/java/org/apache/kafka/raft/VotingState.java b/raft/src/main/java/org/apache/kafka/raft/VotingState.java new file mode 100644 index 0000000000000..9fe327bf80c69 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/VotingState.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +interface VotingState extends EpochState { + Map voteStates(); + + default Stream votersInState(VoterState.State state) { + return voteStates() + .values() + .stream() + .filter(voterState -> voterState.state().equals(state)) + .map(VoterState::replicaKey); + } + + default int majoritySize() { + return voteStates().size() / 2 + 1; + } + + default long numGranted() { + return votersInState(VoterState.State.GRANTED).count(); + } + + default long numUnrecorded() { + return votersInState(VoterState.State.UNRECORDED).count(); + } + + /** + * Check if the candidate is backing off for the next election + */ + boolean isBackingOff(); + + int retries(); + + /** + * Check whether we have received enough votes to conclude the election and become leader. + * + * @return true if at least a majority of nodes have granted the vote + */ + default boolean isVoteGranted() { + return numGranted() >= majoritySize(); + } + + /** + * Check if we have received enough rejections that it is no longer possible to reach a + * majority of grants. + * + * @return true if the vote is rejected, false if the vote is already or can still be granted + */ + default boolean isVoteRejected() { + return numGranted() + numUnrecorded() < majoritySize(); + } + + /** + * Record a granted vote from one of the voters. + * + * @param remoteNodeId The id of the voter + * @return true if the voter had not been previously recorded + * @throws IllegalArgumentException + */ + boolean recordGrantedVote(int remoteNodeId); + + /** + * Record a rejected vote from one of the voters. + * + * @param remoteNodeId The id of the voter + * @return true if the rejected vote had not been previously recorded + * @throws IllegalArgumentException + */ + boolean recordRejectedVote(int remoteNodeId); + + /** + * Record the current election has failed since we've either received sufficient rejecting voters or election timed out + */ + void startBackingOff(long currentTimeMs, long backoffDurationMs); + + boolean hasElectionTimeoutExpired(long currentTimeMs); + + long remainingElectionTimeMs(long currentTimeMs); + + boolean isBackoffComplete(long currentTimeMs); + + long remainingBackoffMs(long currentTimeMs); + + /** + * Get the set of voters which have not been counted as granted or rejected yet. + * + * @return The set of unrecorded voters + */ + default Set unrecordedVoters() { + return votersInState(VoterState.State.UNRECORDED).collect(Collectors.toSet()); + } + + /** + * Get the set of voters that have granted our vote requests. + * + * @return The set of granting voters, which should always contain the localId + */ + default Set grantingVoters() { + return votersInState(VoterState.State.GRANTED).map(ReplicaKey::id).collect(Collectors.toSet()); + } + + /** + * Get the set of voters that have rejected our candidacy. + * + * @return The set of rejecting voters + */ + default Set rejectingVoters() { + return votersInState(VoterState.State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet()); + } + + final class VoterState { + private final ReplicaKey replicaKey; + private State state = State.UNRECORDED; + + VoterState(ReplicaKey replicaKey) { + this.replicaKey = replicaKey; + } + + public State state() { + return state; + } + + public void setState(State state) { + this.state = state; + } + + public ReplicaKey replicaKey() { + return replicaKey; + } + + enum State { + UNRECORDED, + GRANTED, + REJECTED + } + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java index af44137d061a3..ba49a9c14310b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java @@ -285,7 +285,7 @@ private ApiMessage buildTestRequest(ApiKeys key) { case VOTE: int lastEpoch = 4; - return VoteRequest.singletonRequest(topicPartition, clusterId, leaderEpoch, leaderId, lastEpoch, 329); + return VoteRequest.singletonRequest(topicPartition, clusterId, leaderEpoch, leaderId, lastEpoch, 329, true); case FETCH: FetchRequestData request = RaftUtil.singletonFetchRequest(topicPartition, topicId, fetchPartition -> diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index a27853ce64a94..020ae849d9fee 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; @@ -26,8 +27,11 @@ import java.util.Arrays; import java.util.Collections; +import java.util.List; +import java.util.Optional; import java.util.OptionalInt; import java.util.Set; +import java.util.stream.Stream; import static org.apache.kafka.raft.KafkaRaftClientTest.randomReplicaId; import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey; @@ -309,12 +313,13 @@ public void testHandleInvalidPreVoteRequestWithOlderEpoch() throws Exception { @Test public void testLeaderRejectPreVoteRequestOnSameEpoch() throws Exception { int localId = randomReplicaId(); + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) .withUnknownLeader(2) .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); context.becomeLeader(); @@ -331,11 +336,12 @@ public void testLeaderRejectPreVoteRequestOnSameEpoch() throws Exception { @Test public void testPreVoteRequestClusterIdValidation() throws Exception { int localId = randomReplicaId(); + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); context.becomeLeader(); @@ -365,11 +371,12 @@ public void testPreVoteRequestClusterIdValidation() throws Exception { @Test public void testInvalidVoterReplicaPreVoteRequest() throws Exception { int localId = randomReplicaId(); + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); context.becomeLeader(); @@ -409,12 +416,13 @@ public void testInvalidVoterReplicaPreVoteRequest() throws Exception { @Test public void testLeaderAcceptPreVoteFromObserver() throws Exception { int localId = randomReplicaId(); - int otherNodeId = localId + 1; - Set voters = Set.of(localId, otherNodeId); + ReplicaKey localKey = replicaKey(localId, true); + ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) .withUnknownLeader(4) .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); context.becomeLeader(); @@ -573,4 +581,244 @@ public void testRejectPreVoteIfRemoteLogIsNotUpToDate() throws Exception { assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), false); } + + @Test + public void testPreVoteResponseIgnoredAfterBecomingFollower() throws Exception { + int localId = randomReplicaId(); + ReplicaKey local = replicaKey(localId, true); + ReplicaKey voter2 = replicaKey(localId + 1, true); + ReplicaKey voter3 = replicaKey(localId + 2, true); + int epoch = 5; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + .withUnknownLeader(epoch) + .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, voter2, voter3)))) + .build(); + + context.assertUnknownLeader(epoch); + + // Sleep a little to ensure that we become a prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + + // Wait until the vote requests are inflight + context.pollUntilRequest(); + assertTrue(context.client.quorum().isProspective()); + List voteRequests = context.collectVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + + // While the vote requests are still inflight, we receive a BeginEpoch for the same epoch + context.deliverRequest(context.beginEpochRequest(epoch, voter3.id())); + context.client.poll(); + context.assertElectedLeader(epoch, voter3.id()); + + // If PreVote responses are received now they should be ignored + VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch); + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + voteResponse1 + ); + + VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3.id()), epoch); + context.deliverResponse( + voteRequests.get(1).correlationId(), + voteRequests.get(1).destination(), + voteResponse2 + ); + + context.client.poll(); + context.assertElectedLeader(epoch, voter3.id()); + } + + @Test + public void testStaticQuorumDoesNotSendPreVoteRequest() throws Exception { + int localId = randomReplicaId(); + ReplicaKey local = replicaKey(localId, true); + ReplicaKey voter2Bootstrap = replicaKey(localId + 1, true); + int epoch = 5; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + .withUnknownLeader(epoch) + .withStaticVoters(VoterSetTest.voterSet(Stream.of(local, voter2Bootstrap))) + .build(); + + context.assertUnknownLeader(epoch); + + // Sleep a little to ensure that we transition to a Voting state + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + + // We should transition to Candidate state + assertTrue(context.client.quorum().isCandidate()); + + // Candidate state should not send PreVote requests + List voteRequests = context.collectVoteRequests(epoch + 1, 0, 0); + assertEquals(1, voteRequests.size()); + context.assertVotedCandidate(epoch + 1, localId); + } + + @Test + public void testPreVoteNotSupportedByRemote() throws Exception { + int localId = randomReplicaId(); + ReplicaKey local = replicaKey(localId, true); + Uuid localDirectoryId = local.directoryId().orElse(Uuid.randomUuid()); + int voter2 = localId + 1; + ReplicaKey voter2Key = replicaKey(voter2, true); + int epoch = 5; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localDirectoryId) + .withUnknownLeader(epoch) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, voter2Key)))) + .build(); + + context.assertUnknownLeader(epoch); + + // Sleep a little to ensure that we transition to Prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + + assertTrue(context.client.quorum().isProspective()); + + List voteRequests = context.collectVoteRequests(epoch, 0, 0); + assertEquals(1, voteRequests.size()); + + // Simulate remote node not supporting PreVote and responding with Vote response with version 0 + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + context.voteResponse(false, OptionalInt.empty(), epoch, (short) 0) + ); + context.client.poll(); + + // Local should transition to Candidate since it sees a remote node does not support PreVote. It can tell this + // from responses with PreVote field set to false + assertEquals(epoch + 1, context.currentEpoch()); + context.client.quorum().isCandidate(); + } + + @Test + public void testProspectiveReceivesBeginQuorumRequest() throws Exception { + int localId = randomReplicaId(); + ReplicaKey local = replicaKey(localId, true); + ReplicaKey leader = replicaKey(localId + 1, true); + int epoch = 5; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + .withUnknownLeader(epoch) + .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, leader)))) + .build(); + + context.assertUnknownLeader(epoch); + + // Sleep a little to ensure that we transition to Prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + + assertTrue(context.client.quorum().isProspective()); + + context.deliverRequest(context.beginEpochRequest(epoch, leader.id())); + context.client.poll(); + + assertTrue(context.client.quorum().isFollower()); + context.assertElectedLeader(epoch, leader.id()); + } + + @Test + public void testProspectiveSendsFetchRequests() throws Exception { + int localId = randomReplicaId(); + ReplicaKey local = replicaKey(localId, true); + ReplicaKey otherNode = replicaKey(localId + 1, true); + int epoch = 5; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + .withUnknownLeader(epoch) + .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, otherNode)))) + .build(); + context.assertUnknownLeader(epoch); + + // Sleep a little to ensure that we transition to Prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + assertTrue(context.client.quorum().isProspective()); + RaftRequest.Outbound voteRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); + + // If election timeout expires, we should start backing off and sending fetch requests + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + context.assertSentFetchRequest(epoch, 0, 0); + + context.time.sleep(context.electionBackoffMaxMs); + + // After the backoff, we will transition back to Prospective and continue sending PreVote requests + context.pollUntilRequest(); + voteRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); + + // If we receive enough rejected votes, we also immediately start backing off and follow with fetch requests + context.deliverResponse( + voteRequest.correlationId(), + voteRequest.destination(), + context.voteResponse(false, OptionalInt.empty(), epoch, true)); + // handle vote response and update backoff timer + context.client.poll(); + // send fetch request while in backoff + context.client.poll(); + + assertTrue(context.client.quorum().isProspective()); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0, 0); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, -1, MemoryRecords.EMPTY, -1, Errors.BROKER_NOT_AVAILABLE)); + context.client.poll(); + + // We continue sending fetch requests until backoff timer expires + context.time.sleep(context.electionBackoffMaxMs / 2); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + context.assertSentFetchRequest(epoch, 0, 0); + } + + @Test + public void testPreVoteResponseIncludesLeaderId() throws Exception { + ReplicaKey local = replicaKey(randomReplicaId(), true); + ReplicaKey leader = replicaKey(local.id() + 1, true); + ReplicaKey follower = replicaKey(local.id() + 2, true); + int epoch = 5; + + VoterSet voters = VoterSetTest.voterSet(Stream.of(local, leader, follower)); + + RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(voters)) + .withUnknownLeader(epoch) + .build(); + + context.assertUnknownLeader(epoch); + + // Sleep a little to ensure that we transition to Prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + + assertTrue(context.client.quorum().isProspective()); + + List voteRequests = context.collectVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + + // Simulate PreVote response with granted=false and a leaderId + VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.of(leader.id()), epoch, false); + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + voteResponse1 + ); + + // Prospective should transition to Follower + context.client.poll(); + assertTrue(context.client.quorum().isFollower()); + assertEquals(leader.id(), context.client.quorum().leaderId().orElse(-1)); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index ab6b698cc380e..e0948177ade36 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -878,7 +878,7 @@ public void testInitializeAsCandidateAndBecomeLeader(boolean withKip853Rpc) thro context.deliverResponse( request.correlationId(), request.destination(), - context.voteResponse(true, OptionalInt.empty(), 1) + context.voteResponse(true, OptionalInt.empty(), 1, false) ); // Become leader after receiving the vote @@ -925,7 +925,7 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKi context.deliverResponse( request.correlationId(), request.destination(), - context.voteResponse(true, OptionalInt.empty(), 1) + context.voteResponse(true, OptionalInt.empty(), 1, false) ); VoteRequestData voteRequest = (VoteRequestData) request.data(); @@ -1141,9 +1141,9 @@ public void testEndQuorumStartsNewElectionImmediatelyIfFollowerUnattached( context.pollUntilResponse(); context.assertSentEndQuorumEpochResponse(Errors.NONE, epoch, OptionalInt.of(voter2)); - // Should become a candidate immediately + // Should become a prospective immediately context.client.poll(); - context.assertVotedCandidate(epoch + 1, localId); + context.client.quorum().isProspective(); } @ParameterizedTest @@ -1417,7 +1417,7 @@ public void testVoteRequestTimeout(boolean withKip853Rpc) throws Exception { context.deliverResponse( request.correlationId(), request.destination(), - context.voteResponse(true, OptionalInt.empty(), 1) + context.voteResponse(true, OptionalInt.empty(), 1, false) ); context.client.poll(); context.assertVotedCandidate(epoch, localId); @@ -1426,7 +1426,7 @@ public void testVoteRequestTimeout(boolean withKip853Rpc) throws Exception { context.deliverResponse( retryRequest.correlationId(), retryRequest.destination(), - context.voteResponse(true, OptionalInt.empty(), 1) + context.voteResponse(true, OptionalInt.empty(), 1, false) ); context.client.poll(); context.assertElectedLeader(epoch, localId); @@ -1684,7 +1684,7 @@ public void testRetryElection(boolean withKip853Rpc) throws Exception { context.deliverResponse( request.correlationId(), request.destination(), - context.voteResponse(false, OptionalInt.empty(), 1) + context.voteResponse(false, OptionalInt.empty(), 1, false) ); context.client.poll(); @@ -1987,7 +1987,7 @@ public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rp @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testObserverHandleRetryFetchtToBootstrapServer(boolean withKip853Rpc) throws Exception { + public void testObserverHandleRetryFetchToBootstrapServer(boolean withKip853Rpc) throws Exception { // This test tries to check that KRaft is able to handle a retrying Fetch request to // a boostrap server after a Fetch request to the leader. int localId = randomReplicaId(); @@ -2120,15 +2120,24 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testInvalidFetchRequest(boolean withKip853Rpc) throws Exception { + public void testInvalidFetchRequest(boolean withDynamicReconfig) throws Exception { int localId = randomReplicaId(); - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); - Set voters = Set.of(localId, otherNodeKey.id()); + ReplicaKey localKey = replicaKey(localId, withDynamicReconfig); + Uuid localDirectoryId = localKey.directoryId().orElse(Uuid.randomUuid()); + ReplicaKey otherNodeBootstrap = replicaKey(localId + 1, withDynamicReconfig); + ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeBootstrap.id(), otherNodeBootstrap.directoryId().orElse(Uuid.randomUuid())); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) .withUnknownLeader(4) - .withKip853Rpc(withKip853Rpc) - .build(); + .withKip853Rpc(withDynamicReconfig); + if (withDynamicReconfig) { + VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(localKey, otherNodeBootstrap)); + builder.withBootstrapSnapshot(Optional.of(bootstrapVoterSet)); + } else { + VoterSet staticVoterSet = VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)); + builder.withStaticVoters(staticVoterSet); + } + RaftClientTestContext context = builder.build(); context.becomeLeader(); int epoch = context.currentEpoch(); @@ -2674,42 +2683,84 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testVoteResponseIgnoredAfterBecomingFollower(boolean withKip853Rpc) throws Exception { + public void testVoteResponseIgnoredAfterBecomingFollower(boolean withDynamicReconfig) throws Exception { int localId = randomReplicaId(); + ReplicaKey local = replicaKey(localId, withDynamicReconfig); + Uuid localDirectoryId = local.directoryId().orElse(Uuid.randomUuid()); int voter2 = localId + 1; + ReplicaKey voter2Bootstrap = replicaKey(voter2, withDynamicReconfig); + Uuid voter2DirectoryId = voter2Bootstrap.directoryId().orElse(Uuid.randomUuid()); + ReplicaKey voter2Key = ReplicaKey.of(voter2, voter2DirectoryId); int voter3 = localId + 2; + ReplicaKey voter3Bootstrap = replicaKey(voter3, withDynamicReconfig); + Uuid voter3DirectoryId = voter3Bootstrap.directoryId().orElse(Uuid.randomUuid()); + ReplicaKey voter3Key = ReplicaKey.of(voter3, voter3DirectoryId); int epoch = 5; - Set voters = Set.of(localId, voter2, voter3); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withUnknownLeader(epoch - 1) - .withKip853Rpc(withKip853Rpc) - .build(); - context.assertUnknownLeader(epoch - 1); + RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) + .withUnknownLeader(epoch) + .withKip853Rpc(withDynamicReconfig); + if (withDynamicReconfig) { + VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(local, voter2Bootstrap, voter3Bootstrap)); + builder.withBootstrapSnapshot(Optional.of(bootstrapVoterSet)); + } else { + VoterSet staticVoterSet = VoterSetTest.voterSet(Stream.of(local, voter2Key, voter3Key)); + builder.withStaticVoters(staticVoterSet); + } + RaftClientTestContext context = builder.build(); + context.assertUnknownLeader(epoch); - // Sleep a little to ensure that we become a candidate - context.time.sleep(context.electionTimeoutMs() * 2L); + if (withDynamicReconfig) { + // Sleep a little to ensure that we become a prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + + // Wait until the PreVote requests are inflight + context.pollUntilRequest(); + assertTrue(context.client.quorum().isProspective()); + List voteRequests = context.collectVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + + // Become candidate after PreVote requests are granted + VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch, true); + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + voteResponse1 + ); + VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch, true); + context.deliverResponse( + voteRequests.get(1).correlationId(), + voteRequests.get(1).destination(), + voteResponse2 + ); + + context.client.poll(); + assertTrue(context.client.quorum().isCandidate()); + } else { + // Sleep a little to ensure that we become a candidate + context.time.sleep(context.electionTimeoutMs() * 2L); + } // Wait until the vote requests are inflight context.pollUntilRequest(); - context.assertVotedCandidate(epoch, localId); - List voteRequests = context.collectVoteRequests(epoch, 0, 0); + context.assertVotedCandidate(epoch + 1, local); + List voteRequests = context.collectVoteRequests(epoch + 1, 0, 0); assertEquals(2, voteRequests.size()); // While the vote requests are still inflight, we receive a BeginEpoch for the same epoch - context.deliverRequest(context.beginEpochRequest(epoch, voter3)); + context.deliverRequest(context.beginEpochRequest(epoch + 1, voter3)); context.client.poll(); - context.assertElectedLeader(epoch, voter3); + context.assertElectedLeader(epoch + 1, voter3); // The vote requests now return and should be ignored - VoteResponseData voteResponse1 = context.voteResponse(false, OptionalInt.empty(), epoch); + VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch + 1, false); context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), voteResponse1 ); - VoteResponseData voteResponse2 = context.voteResponse(false, OptionalInt.of(voter3), epoch); + VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch + 1, false); context.deliverResponse( voteRequests.get(1).correlationId(), voteRequests.get(1).destination(), @@ -2717,7 +2768,7 @@ public void testVoteResponseIgnoredAfterBecomingFollower(boolean withKip853Rpc) ); context.client.poll(); - context.assertElectedLeader(epoch, voter3); + context.assertElectedLeader(epoch + 1, voter3); } @ParameterizedTest @@ -3682,7 +3733,7 @@ public void testFetchShouldBeTreatedAsLeaderAcknowledgement(boolean withKip853Rp .build(); context.time.sleep(context.electionTimeoutMs()); - context.expectAndGrantVotes(epoch); + context.expectAndGrantVotes(epoch, false); context.pollUntilRequest(); @@ -3908,7 +3959,7 @@ public void testClusterAuthorizationFailedInBeginQuorumEpoch(boolean withKip853R .build(); context.time.sleep(context.electionTimeoutMs()); - context.expectAndGrantVotes(epoch); + context.expectAndGrantVotes(epoch, false); context.pollUntilRequest(); List requests = context.collectBeginEpochRequests(epoch); diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java new file mode 100644 index 0000000000000..7b30e5896362e --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -0,0 +1,286 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.Collections; +import java.util.Optional; +import java.util.Set; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class ProspectiveStateTest { + private final ReplicaKey localReplicaKey = ReplicaKey.of(0, Uuid.randomUuid()); + private final int epoch = 5; + private final MockTime time = new MockTime(); + private final int electionTimeoutMs = 5000; + private final LogContext logContext = new LogContext(); + + private ProspectiveState newProspectiveState(VoterSet voters) { + return new ProspectiveState( + time, + localReplicaKey.id(), + epoch, + Optional.empty(), + voters, + Optional.empty(), + 0, + electionTimeoutMs, + logContext + ); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testSingleNodeQuorum(boolean withDirectoryId) { + ProspectiveState state = newProspectiveState(voterSetWithLocal(IntStream.empty(), withDirectoryId)); + assertTrue(state.isVoteGranted()); + assertFalse(state.isVoteRejected()); + assertEquals(Collections.emptySet(), state.unrecordedVoters()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testTwoNodeQuorumVoteRejected(boolean withDirectoryId) { + ReplicaKey otherNode = replicaKey(1, withDirectoryId); + ProspectiveState state = newProspectiveState( + voterSetWithLocal(Stream.of(otherNode), withDirectoryId) + ); + assertFalse(state.isVoteGranted()); + assertFalse(state.isVoteRejected()); + assertEquals(Collections.singleton(otherNode), state.unrecordedVoters()); + assertTrue(state.recordRejectedVote(otherNode.id())); + assertFalse(state.isVoteGranted()); + assertTrue(state.isVoteRejected()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testTwoNodeQuorumVoteGranted(boolean withDirectoryId) { + ReplicaKey otherNode = replicaKey(1, withDirectoryId); + ProspectiveState state = newProspectiveState( + voterSetWithLocal(Stream.of(otherNode), withDirectoryId) + ); + assertFalse(state.isVoteGranted()); + assertFalse(state.isVoteRejected()); + assertEquals(Collections.singleton(otherNode), state.unrecordedVoters()); + assertTrue(state.recordGrantedVote(otherNode.id())); + assertEquals(Collections.emptySet(), state.unrecordedVoters()); + assertFalse(state.isVoteRejected()); + assertTrue(state.isVoteGranted()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testThreeNodeQuorumVoteGranted(boolean withDirectoryId) { + ReplicaKey node1 = replicaKey(1, withDirectoryId); + ReplicaKey node2 = replicaKey(2, withDirectoryId); + ProspectiveState state = newProspectiveState( + voterSetWithLocal(Stream.of(node1, node2), withDirectoryId) + ); + assertFalse(state.isVoteGranted()); + assertFalse(state.isVoteRejected()); + assertEquals(Set.of(node1, node2), state.unrecordedVoters()); + assertTrue(state.recordGrantedVote(node1.id())); + assertEquals(Collections.singleton(node2), state.unrecordedVoters()); + assertTrue(state.isVoteGranted()); + assertFalse(state.isVoteRejected()); + assertTrue(state.recordRejectedVote(node2.id())); + assertEquals(Collections.emptySet(), state.unrecordedVoters()); + assertTrue(state.isVoteGranted()); + assertFalse(state.isVoteRejected()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testThreeNodeQuorumVoteRejected(boolean withDirectoryId) { + ReplicaKey node1 = replicaKey(1, withDirectoryId); + ReplicaKey node2 = replicaKey(2, withDirectoryId); + ProspectiveState state = newProspectiveState( + voterSetWithLocal(Stream.of(node1, node2), withDirectoryId) + ); + assertFalse(state.isVoteGranted()); + assertFalse(state.isVoteRejected()); + assertEquals(Set.of(node1, node2), state.unrecordedVoters()); + assertTrue(state.recordRejectedVote(node1.id())); + assertEquals(Collections.singleton(node2), state.unrecordedVoters()); + assertFalse(state.isVoteGranted()); + assertFalse(state.isVoteRejected()); + assertTrue(state.recordRejectedVote(node2.id())); + assertEquals(Collections.emptySet(), state.unrecordedVoters()); + assertFalse(state.isVoteGranted()); + assertTrue(state.isVoteRejected()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testCannotRejectVoteFromLocalId(boolean withDirectoryId) { + int otherNodeId = 1; + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.of(otherNodeId), withDirectoryId) + ); + assertThrows( + IllegalStateException.class, + () -> state.recordRejectedVote(localReplicaKey.id()) + ); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testCannotChangeVoteGrantedToRejected(boolean withDirectoryId) { + int otherNodeId = 1; + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.of(otherNodeId), withDirectoryId) + ); + assertTrue(state.recordGrantedVote(otherNodeId)); + assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(otherNodeId)); + assertTrue(state.isVoteGranted()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testCannotChangeVoteRejectedToGranted(boolean withDirectoryId) { + int otherNodeId = 1; + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.of(otherNodeId), withDirectoryId) + ); + assertTrue(state.recordRejectedVote(otherNodeId)); + assertThrows(IllegalArgumentException.class, () -> state.recordGrantedVote(otherNodeId)); + assertTrue(state.isVoteRejected()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testCannotGrantOrRejectNonVoters(boolean withDirectoryId) { + int nonVoterId = 1; + ProspectiveState state = newProspectiveState(voterSetWithLocal(IntStream.empty(), withDirectoryId)); + assertThrows(IllegalArgumentException.class, () -> state.recordGrantedVote(nonVoterId)); + assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(nonVoterId)); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testIdempotentGrant(boolean withDirectoryId) { + int otherNodeId = 1; + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.of(otherNodeId), withDirectoryId) + ); + assertTrue(state.recordGrantedVote(otherNodeId)); + assertFalse(state.recordGrantedVote(otherNodeId)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testIdempotentReject(boolean withDirectoryId) { + int otherNodeId = 1; + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.of(otherNodeId), withDirectoryId) + ); + assertTrue(state.recordRejectedVote(otherNodeId)); + assertFalse(state.recordRejectedVote(otherNodeId)); + } + + @ParameterizedTest + @CsvSource({ "true,true,true", "true,true,false", "true,false,true", "true,false,false", + "false,true,true", "false,true,false", "false,false,true", "false,false,false" }) + public void testGrantVote(boolean isLogUpToDate, boolean withDirectoryId, boolean isPreVote) { + ReplicaKey node0 = replicaKey(0, withDirectoryId); + ReplicaKey node1 = replicaKey(1, withDirectoryId); + ReplicaKey node2 = replicaKey(2, withDirectoryId); + ReplicaKey node3 = replicaKey(3, withDirectoryId); + + ProspectiveState state = newProspectiveState( + voterSetWithLocal(Stream.of(node1, node2, node3), withDirectoryId) + ); + + assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate, isPreVote)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, isPreVote)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, isPreVote)); + assertEquals(isLogUpToDate, state.canGrantVote(node3, isLogUpToDate, isPreVote)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testElectionState(boolean withDirectoryId) { + VoterSet voters = voterSetWithLocal(IntStream.of(1, 2, 3), withDirectoryId); + ProspectiveState state = newProspectiveState(voters); + assertEquals( + ElectionState.withVotedCandidate( + epoch, + localReplicaKey, + voters.voterIds() + ), + state.election() + ); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testInvalidVoterSet(boolean withDirectoryId) { + assertThrows( + IllegalArgumentException.class, + () -> newProspectiveState( + VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), withDirectoryId)) + ) + ); + } + + @Test + public void testLeaderEndpoints() { + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.of(1, 2, 3), true) + ); + + assertEquals(Endpoints.empty(), state.leaderEndpoints()); + } + + private ReplicaKey replicaKey(int id, boolean withDirectoryId) { + Uuid directoryId = withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID; + return ReplicaKey.of(id, directoryId); + } + + private VoterSet voterSetWithLocal(IntStream remoteVoterIds, boolean withDirectoryId) { + Stream remoteVoterKeys = remoteVoterIds + .boxed() + .map(id -> replicaKey(id, withDirectoryId)); + + return voterSetWithLocal(remoteVoterKeys, withDirectoryId); + } + + private VoterSet voterSetWithLocal(Stream remoteVoterKeys, boolean withDirectoryId) { + ReplicaKey actualLocalVoter = withDirectoryId ? + localReplicaKey : + ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID); + + return VoterSetTest.voterSet( + Stream.concat(Stream.of(actualLocalVoter), remoteVoterKeys) + ); + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 1343a57da1ce9..0a8650bf45290 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -562,7 +562,10 @@ static RaftClientTestContext initializeAsLeader(int localId, Set voters public void becomeLeader() throws Exception { int currentEpoch = currentEpoch(); time.sleep(electionTimeoutMs * 2L); - expectAndGrantVotes(currentEpoch + 1); + if (kraftVersion.isReconfigSupported()) { + expectAndGrantVotes(currentEpoch, true); + } + expectAndGrantVotes(currentEpoch + 1, false); expectBeginEpoch(currentEpoch + 1); } @@ -579,19 +582,23 @@ LeaderAndEpoch currentLeaderAndEpoch() { return new LeaderAndEpoch(election.optionalLeaderId(), election.epoch()); } - void expectAndGrantVotes(int epoch) throws Exception { + void expectAndGrantVotes(int epoch, boolean preVote) throws Exception { pollUntilRequest(); List voteRequests = collectVoteRequests(epoch, log.lastFetchedEpoch(), log.endOffset().offset()); for (RaftRequest.Outbound request : voteRequests) { - VoteResponseData voteResponse = voteResponse(true, OptionalInt.empty(), epoch); + VoteResponseData voteResponse = voteResponse(true, OptionalInt.empty(), epoch, preVote); deliverResponse(request.correlationId(), request.destination(), voteResponse); } client.poll(); - assertElectedLeader(epoch, localIdOrThrow()); + if (preVote) { + assertTrue(client.quorum().isCandidate()); + } else { + assertElectedLeader(epoch, localIdOrThrow()); + } } private int localIdOrThrow() { @@ -629,40 +636,47 @@ void pollUntilRequest() throws InterruptedException { } void assertVotedCandidate(int epoch, int candidateId) { + ReplicaKey candidateKey = ReplicaKey.of(candidateId, ReplicaKey.NO_DIRECTORY_ID); + assertVotedCandidate(epoch, candidateKey); + } + + void assertVotedCandidate(int epoch, ReplicaKey candidateKey) { assertEquals( ElectionState.withVotedCandidate( epoch, - ReplicaKey.of(candidateId, ReplicaKey.NO_DIRECTORY_ID), - startingVoters.voterIds() + candidateKey, + expectedVoters() ), quorumStateStore.readElectionState().get() ); } public void assertElectedLeader(int epoch, int leaderId) { - Set voters = kraftVersion.isReconfigSupported() ? - Collections.emptySet() : startingVoters.voterIds(); assertEquals( - ElectionState.withElectedLeader(epoch, leaderId, voters), + ElectionState.withElectedLeader(epoch, leaderId, expectedVoters()), quorumStateStore.readElectionState().get() ); } void assertUnknownLeader(int epoch) { assertEquals( - ElectionState.withUnknownLeader(epoch, startingVoters.voterIds()), - quorumStateStore.readElectionState().get() - ); + ElectionState.withUnknownLeader(epoch, expectedVoters()), + quorumStateStore.readElectionState().get()); } void assertResignedLeader(int epoch, int leaderId) { assertTrue(client.quorum().isResigned()); assertEquals( - ElectionState.withElectedLeader(epoch, leaderId, startingVoters.voterIds()), + ElectionState.withElectedLeader(epoch, leaderId, expectedVoters()), quorumStateStore.readElectionState().get() ); } + // Voters are only written to ElectionState in KRaftVersion 0 + private Set expectedVoters() { + return kraftVersion.isReconfigSupported() ? Collections.emptySet() : startingVoters.voterIds(); + } + DescribeQuorumResponseData collectDescribeQuorumResponse() { List sentMessages = drainSentResponses(ApiKeys.DESCRIBE_QUORUM); assertEquals(1, sentMessages.size()); @@ -786,6 +800,28 @@ void assertSentVoteResponse( } } + List collectPreVoteRequests( + int epoch, + int lastEpoch, + long lastEpochOffset + ) { + List voteRequests = new ArrayList<>(); + for (RaftRequest.Outbound raftMessage : channel.drainSendQueue()) { + if (raftMessage.data() instanceof VoteRequestData) { + VoteRequestData request = (VoteRequestData) raftMessage.data(); + VoteRequestData.PartitionData partitionRequest = unwrap(request); + + assertTrue(partitionRequest.preVote()); + assertEquals(epoch, partitionRequest.replicaEpoch()); + assertEquals(localIdOrThrow(), partitionRequest.replicaId()); + assertEquals(lastEpoch, partitionRequest.lastOffsetEpoch()); + assertEquals(lastEpochOffset, partitionRequest.lastOffset()); + voteRequests.add(raftMessage); + } + } + return voteRequests; + } + List collectVoteRequests( int epoch, int lastEpoch, @@ -1506,9 +1542,13 @@ VoteRequestData voteRequest( } VoteResponseData voteResponse(boolean voteGranted, OptionalInt leaderId, int epoch) { + return voteResponse(voteGranted, leaderId, epoch, voteRpcVersion()); + } + + VoteResponseData voteResponse(boolean voteGranted, OptionalInt leaderId, int epoch, short version) { return RaftUtil.singletonVoteResponse( channel.listenerName(), - voteRpcVersion(), + version, Errors.NONE, metadataPartition, Errors.NONE, From c19bda564ded9bb5f83d3bce7a078d61bdd540e4 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Wed, 4 Dec 2024 22:09:09 -0800 Subject: [PATCH 02/23] still need to add more test cases to QuorumStateTest --- .../apache/kafka/raft/ProspectiveState.java | 34 ++-- .../org/apache/kafka/raft/QuorumState.java | 2 +- .../kafka/raft/ProspectiveStateTest.java | 49 ++++- .../raft/ProspectiveStateWithVoteTest.java | 179 ++++++++++++++++++ .../apache/kafka/raft/QuorumStateTest.java | 61 +++++- 5 files changed, 304 insertions(+), 21 deletions(-) create mode 100644 raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index 0e11a79e58e72..7abddeacb01e3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -32,7 +32,7 @@ public class ProspectiveState implements VotingState { private final VoterSet voters; // private final long electionTimeoutMs; // private final Timer electionTimer; - private final Map voteStates = new HashMap<>(); + private final Map preVoteStates = new HashMap<>(); private final Optional highWatermark; private int retries; private final int electionTimeoutMs; @@ -75,9 +75,9 @@ public ProspectiveState( this.log = logContext.logger(ProspectiveState.class); for (ReplicaKey voter : voters.voterKeys()) { - voteStates.put(voter.id(), new VoterState(voter)); + preVoteStates.put(voter.id(), new VoterState(voter)); } - voteStates.get(localId).setState(VoterState.State.GRANTED); + preVoteStates.get(localId).setState(VoterState.State.GRANTED); } public int localId() { @@ -90,7 +90,7 @@ public Optional votedKey() { @Override public Map voteStates() { - return voteStates; + return preVoteStates; } @Override @@ -111,7 +111,7 @@ public int retries() { * @throws IllegalArgumentException if the remote node is not a voter */ public boolean recordGrantedVote(int remoteNodeId) { - VoterState voterState = voteStates.get(remoteNodeId); + VoterState voterState = preVoteStates.get(remoteNodeId); if (voterState == null) { throw new IllegalArgumentException("Attempt to grant vote to non-voter " + remoteNodeId); } @@ -130,7 +130,7 @@ public boolean recordGrantedVote(int remoteNodeId) { * @throws IllegalArgumentException if the remote node is not a voter */ public boolean recordRejectedVote(int remoteNodeId) { - VoterState voterState = voteStates.get(remoteNodeId); + VoterState voterState = preVoteStates.get(remoteNodeId); if (voterState == null) { throw new IllegalArgumentException("Attempt to reject vote to non-voter " + remoteNodeId); } @@ -163,16 +163,15 @@ public void startBackingOff(long currentTimeMs, long backoffDurationMs) { } @Override - public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolean isPreVote) { + public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate) { if (votedKey.isPresent()) { ReplicaKey votedReplicaKey = votedKey.get(); if (votedReplicaKey.id() == replicaKey.id()) { return votedReplicaKey.directoryId().isEmpty() || votedReplicaKey.directoryId().equals(replicaKey.directoryId()); } log.debug( - "Rejecting Vote request with PreVote={} from replica ({}), already have voted for another " + - "replica ({}) in epoch {}", - isPreVote, + "Rejecting Vote request from candidate ({}), already have voted for another " + + "candidate ({}) in epoch {}", replicaKey, votedKey, epoch @@ -180,8 +179,19 @@ public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolea return false; } else if (!isLogUpToDate) { log.debug( - "Rejecting Vote request with PreVote={} from replica ({}) since replica epoch/offset is not up to date with us", - isPreVote, + "Rejecting Vote request from candidate ({}) since replica epoch/offset is not up to date with us", + replicaKey + ); + } + + return isLogUpToDate; + } + + @Override + public boolean canGrantPreVote(ReplicaKey replicaKey, boolean isLogUpToDate) { + if (!isLogUpToDate) { + log.debug( + "Rejecting PreVote request from prospective ({}) since prospective epoch/offset is not up to date with us", replicaKey ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 7728048d5f8bd..72200820dcaf8 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -562,7 +562,7 @@ public void transitionToProspective() { time, localIdOrThrow(), epoch(), - Optional.empty(), + state.election().optionalVotedKey(), partitionState.lastVoterSet(), state.highWatermark(), randomElectionTimeoutMs(), diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java index 7b30e5896362e..01992b8567011 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -208,9 +208,8 @@ public void testIdempotentReject(boolean withDirectoryId) { } @ParameterizedTest - @CsvSource({ "true,true,true", "true,true,false", "true,false,true", "true,false,false", - "false,true,true", "false,true,false", "false,false,true", "false,false,false" }) - public void testGrantVote(boolean isLogUpToDate, boolean withDirectoryId, boolean isPreVote) { + @CsvSource({ "true,true", "true,false", "false,true", "false,false" }) + public void testGrantVote(boolean isLogUpToDate, boolean withDirectoryId) { ReplicaKey node0 = replicaKey(0, withDirectoryId); ReplicaKey node1 = replicaKey(1, withDirectoryId); ReplicaKey node2 = replicaKey(2, withDirectoryId); @@ -220,10 +219,46 @@ public void testGrantVote(boolean isLogUpToDate, boolean withDirectoryId, boolea voterSetWithLocal(Stream.of(node1, node2, node3), withDirectoryId) ); - assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate, isPreVote)); - assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, isPreVote)); - assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, isPreVote)); - assertEquals(isLogUpToDate, state.canGrantVote(node3, isLogUpToDate, isPreVote)); + assertEquals(isLogUpToDate, state.canGrantPreVote(node0, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantPreVote(node1, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantPreVote(node2, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantPreVote(node3, isLogUpToDate)); + + assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node3, isLogUpToDate)); + } + + @ParameterizedTest + @CsvSource({ "true,true", "true,false", "false,true", "false,false" }) + public void testGrantVoteWithVotedKey(boolean isLogUpToDate, boolean withDirectoryId) { + ReplicaKey node0 = replicaKey(0, withDirectoryId); + ReplicaKey node1 = replicaKey(1, withDirectoryId); + ReplicaKey node2 = replicaKey(2, withDirectoryId); + ReplicaKey node3 = replicaKey(3, withDirectoryId); + + ProspectiveState state = new ProspectiveState( + time, + localReplicaKey.id(), + epoch, + Optional.of(node1), + voterSetWithLocal(Stream.of(node1, node2, node3), withDirectoryId), + Optional.empty(), + 0, + electionTimeoutMs, + logContext + ); + + assertEquals(isLogUpToDate, state.canGrantPreVote(node0, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantPreVote(node1, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantPreVote(node2, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantPreVote(node3, isLogUpToDate)); + + assertFalse(state.canGrantVote(node0, isLogUpToDate)); + assertTrue(state.canGrantVote(node1, isLogUpToDate)); + assertFalse(state.canGrantVote(node2, isLogUpToDate)); + assertFalse(state.canGrantVote(node3, isLogUpToDate)); } @ParameterizedTest diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java new file mode 100644 index 0000000000000..a7362e0080861 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.Collections; +import java.util.Optional; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class ProspectiveStateWithVoteTest { + + private final MockTime time = new MockTime(); + private final LogContext logContext = new LogContext(); + private final int localId = 0; + private final ReplicaKey localReplicaKey = ReplicaKey.of(localId, Uuid.randomUuid()); + private final int epoch = 5; + private final int votedId = 1; + private final Uuid votedDirectoryId = Uuid.randomUuid(); + private final ReplicaKey votedKeyWithDirectoryId = ReplicaKey.of(votedId, votedDirectoryId); + private final ReplicaKey votedKeyWithoutDirectoryId = ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID); + private final int electionTimeoutMs = 10000; + + private ProspectiveState newProspectiveVotedState( + VoterSet voters, + Optional votedKey + ) { + return new ProspectiveState( + time, + localId, + epoch, + votedKey, + voters, + Optional.empty(), + 0, + electionTimeoutMs, + logContext + ); + } + + @Test + public void testElectionTimeout() { + ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithDirectoryId)); + + assertEquals(epoch, state.epoch()); + assertEquals(votedKeyWithDirectoryId, state.votedKey().get()); + assertEquals( + ElectionState.withVotedCandidate(epoch, votedKeyWithDirectoryId, Collections.singleton(localId)), + state.election() + ); + assertEquals(electionTimeoutMs, state.remainingElectionTimeMs(time.milliseconds())); + assertFalse(state.hasElectionTimeoutExpired(time.milliseconds())); + + time.sleep(5000); + assertEquals(electionTimeoutMs - 5000, state.remainingElectionTimeMs(time.milliseconds())); + assertFalse(state.hasElectionTimeoutExpired(time.milliseconds())); + + time.sleep(5000); + assertEquals(0, state.remainingElectionTimeMs(time.milliseconds())); + assertTrue(state.hasElectionTimeoutExpired(time.milliseconds())); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { + ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithoutDirectoryId)); + + assertTrue(state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertTrue(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + + assertTrue(state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); + assertTrue(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); + + // Can grant PreVote to other replicas even if we have granted a standard vote to another replica + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testCanGrantVoteWithDirectoryId(boolean isLogUpToDate) { + ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithDirectoryId)); + + // Same voterKey + // We will not grant PreVote for a replica we have already granted a standard vote to if their log is behind + assertEquals( + isLogUpToDate, + state.canGrantPreVote(votedKeyWithDirectoryId, isLogUpToDate) + ); + assertTrue(state.canGrantVote(votedKeyWithDirectoryId, isLogUpToDate)); + + // Different directoryId + // We can grant PreVote for a replica we have already granted a standard vote to if their log is up-to-date, + // even if the directoryId is different + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); + + // Missing directoryId + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + + // Different voterId + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId + 1, votedDirectoryId), isLogUpToDate) + ); + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), true)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), true)); + } + + @Test + void testLeaderEndpoints() { + ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithDirectoryId)); + + assertEquals(Endpoints.empty(), state.leaderEndpoints()); + } + + private ReplicaKey replicaKey(int id, boolean withDirectoryId) { + Uuid directoryId = withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID; + return ReplicaKey.of(id, directoryId); + } + + private VoterSet voterSetWithLocal(IntStream remoteVoterIds, boolean withDirectoryId) { + Stream remoteVoterKeys = remoteVoterIds + .boxed() + .map(id -> replicaKey(id, withDirectoryId)); + + return voterSetWithLocal(remoteVoterKeys, withDirectoryId); + } + + private VoterSet voterSetWithLocal(Stream remoteVoterKeys, boolean withDirectoryId) { + ReplicaKey actualLocalVoter = withDirectoryId ? + localReplicaKey : + ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID); + + return VoterSetTest.voterSet( + Stream.concat(Stream.of(actualLocalVoter), remoteVoterKeys) + ); + } +} + diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index 360c3bba62a80..3698d10928324 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.KRaftVersion; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; import org.mockito.Mockito; @@ -40,6 +41,7 @@ import java.util.stream.IntStream; import java.util.stream.Stream; +import static org.apache.kafka.server.common.KRaftVersion.KRAFT_VERSION_1; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -1493,7 +1495,7 @@ public void testCanBecomeFollowerOfNonVoter(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testObserverCannotBecomeCandidateOrLeader(KRaftVersion kraftVersion) { + public void testObserverCannotBecomeProspectiveOrCandidateOrLeader(KRaftVersion kraftVersion) { boolean withDirectoryId = kraftVersion.featureLevel() > 0; int otherNodeId = 1; VoterSet voters = VoterSetTest.voterSet( @@ -1502,6 +1504,7 @@ public void testObserverCannotBecomeCandidateOrLeader(KRaftVersion kraftVersion) QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isObserver()); + assertThrows(IllegalStateException.class, state::transitionToProspective); assertThrows(IllegalStateException.class, state::transitionToCandidate); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); } @@ -1710,4 +1713,60 @@ public void testNoLocalIdInitializationFailsIfElectionStateHasVotedCandidate(KRa QuorumState state2 = buildQuorumState(OptionalInt.empty(), voters, kraftVersion); assertThrows(IllegalStateException.class, () -> state2.initialize(new OffsetAndEpoch(0, 0))); } + + // testUnattachedVotedToProspective + @Test + public void testUnattachedVotedToProspective() { + int node1 = 1; + Uuid node1DirectoryId = Uuid.randomUuid(); + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); + QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, node1DirectoryId)); + + state.transitionToProspective(); + assertTrue(state.isProspective()); + ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); + assertEquals(5, prospectiveState.epoch()); + assertEquals(electionTimeoutMs, prospectiveState.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + 5, + persistedVotedKey(ReplicaKey.of(node1, node1DirectoryId), KRAFT_VERSION_1), + persistedVoters(voters.voterIds(), KRAFT_VERSION_1) + ) + ), + store.readElectionState() + ); + } + + // testUnattachedWithLeaderToProspective + + // testIllegalTransitionsToProspective + + // testIllegalTransitionsFromProspective + + // testProspectiveToProspective + + // testProspectiveVotedToProspective + + // testProspectiveWithLeaderToProspective + + // testProspectiveToCandidate + + // testProspectiveVotedToCandidate + + // testCandidateToProspective (can include votedKey which is erased) + + // testProspectiveToUnattached (can include votedkey which is erased) + + // testProspectiveVotedToProspective (votedkey needs to be kept) + + // testProspectiveToUnattachedSameEpoch + + // testProspectiveToProspectiveHigherEpoch + + // testProspectiveToFollowerHigherEpoch } From 8a28714971b1b67ee7d755595297d8cb9eb7cf51 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Sun, 8 Dec 2024 23:41:13 -0800 Subject: [PATCH 03/23] match latest version of kip --- .../org/apache/kafka/raft/CandidateState.java | 70 +-- .../org/apache/kafka/raft/EpochState.java | 2 +- .../apache/kafka/raft/KafkaRaftClient.java | 141 ++--- .../apache/kafka/raft/ProspectiveState.java | 46 +- .../org/apache/kafka/raft/QuorumState.java | 69 ++- .../org/apache/kafka/raft/VotingState.java | 4 - .../raft/internals/KafkaRaftMetrics.java | 4 +- .../raft/KafkaRaftClientPreVoteTest.java | 62 +-- .../kafka/raft/ProspectiveStateTest.java | 30 +- .../raft/ProspectiveStateWithVoteTest.java | 7 +- .../apache/kafka/raft/QuorumStateTest.java | 512 +++++++++++++++--- .../org/apache/kafka/raft/VoterSetTest.java | 12 +- .../raft/internals/KafkaRaftMetricsTest.java | 34 +- 13 files changed, 656 insertions(+), 337 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 37fb32638e863..50ac194c763b6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -99,18 +99,6 @@ public Map voteStates() { return voteStates; } -// public int majoritySize() { -// return voteStates.size() / 2 + 1; -// } -// -// public long numGranted() { -// return votersInState(State.GRANTED).count(); -// } -// -// public long numUnrecorded() { -// return votersInState(State.UNRECORDED).count(); -// } -// /** * Check if the candidate is backing off for the next election */ @@ -196,33 +184,6 @@ public void startBackingOff(long currentTimeMs, long backoffDurationMs) { this.isBackingOff = true; } -// /** -// * Get the set of voters which have not been counted as granted or rejected yet. -// * -// * @return The set of unrecorded voters -// */ -// public Set unrecordedVoters() { -// return votersInState(State.UNRECORDED).collect(Collectors.toSet()); -// } - -// /** -// * Get the set of voters that have granted our vote requests. -// * -// * @return The set of granting voters, which should always contain the ID of the candidate -// */ -// public Set grantingVoters() { -// return votersInState(State.GRANTED).map(ReplicaKey::id).collect(Collectors.toSet()); -// } -// -// /** -// * Get the set of voters that have rejected our candidacy. -// * -// * @return The set of rejecting voters -// */ -// public Set rejectingVoters() { -// return votersInState(State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet()); -// } - public Stream votersInState(State state) { return voteStates .values() @@ -243,13 +204,11 @@ public long remainingElectionTimeMs(long currentTimeMs) { return electionTimer.remainingMs(); } - @Override public boolean isBackoffComplete(long currentTimeMs) { backoffTimer.update(currentTimeMs); return backoffTimer.isExpired(); } - @Override public long remainingBackoffMs(long currentTimeMs) { if (!isBackingOff) { throw new IllegalStateException("Candidate is not currently backing off"); @@ -283,7 +242,7 @@ public Optional highWatermark() { } @Override - public boolean canGrantVote(// maybe this needs to grant prevotes + public boolean canGrantVote( ReplicaKey replicaKey, boolean isLogUpToDate, boolean isPreVote @@ -326,31 +285,4 @@ public String name() { @Override public void close() {} - -// private static final class VoterState { -// private final ReplicaKey replicaKey; -// private State state = State.UNRECORDED; -// -// private VoterState(ReplicaKey replicaKey) { -// this.replicaKey = replicaKey; -// } -// -// public State state() { -// return state; -// } -// -// public void setState(State state) { -// this.state = state; -// } -// -// public ReplicaKey replicaKey() { -// return replicaKey; -// } -// } -// -// private enum State { -// UNRECORDED, -// GRANTED, -// REJECTED -// } } diff --git a/raft/src/main/java/org/apache/kafka/raft/EpochState.java b/raft/src/main/java/org/apache/kafka/raft/EpochState.java index c47fc087f0d46..338f660318140 100644 --- a/raft/src/main/java/org/apache/kafka/raft/EpochState.java +++ b/raft/src/main/java/org/apache/kafka/raft/EpochState.java @@ -29,7 +29,7 @@ default Optional highWatermark() { * Decide whether to grant a vote to a replica. * * It is the responsibility of the caller to invoke - * {@link QuorumState#transitionToUnattachedVotedState(int, ReplicaKey)} if a standard vote is granted. + * {@link QuorumState#unattachedAddVotedState(int, ReplicaKey)} if a standard vote is granted. * * @param replicaKey the id and directory of the replica requesting the vote * @param isLogUpToDate whether the replica's log is at least as up-to-date as receiver’s log diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index c8b4a9b363ab1..b99b5d37cb718 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -727,24 +727,26 @@ private void transitionToVotingState(long currentTimeMs) { } } - private void refreshVotingState(long currentTimeMs) { - VotingState state = quorum.votingStateOrThrow(); - if (quorum.isProspective()) { - transitionToProspective(currentTimeMs); - } else if (quorum.isCandidate()) { - transitionToCandidate(currentTimeMs); - } else { - throw new IllegalStateException("Expected to be a VotingState (Prospective or Candidate) " + - "but current state is " + state); - } + private void transitionToUnattached(int epoch) { + transitionToUnattached(epoch, OptionalInt.empty()); } - private void transitionToUnattached(int epoch) { - quorum.transitionToUnattached(epoch); + private void transitionToUnattached(int epoch, OptionalInt leaderId) { + quorum.transitionToUnattached(epoch, leaderId); maybeFireLeaderChange(); resetConnections(); } + private void transitionToUnattachedVoted(int epoch, ReplicaKey candidateKey) { + if (quorum.isUnattachedNotVoted() && quorum.epoch() == epoch) { + quorum.unattachedAddVotedState(epoch, candidateKey); + } else { + quorum.transitionToUnattached(epoch, Optional.of(candidateKey)); + maybeFireLeaderChange(); + resetConnections(); + } + } + private void transitionToResigned(List preferredSuccessors) { fetchPurgatory.completeAllExceptionally( Errors.NOT_LEADER_OR_FOLLOWER.exception("Not handling request since this node is resigning")); @@ -752,10 +754,6 @@ private void transitionToResigned(List preferredSuccessors) { resetConnections(); } - private void transitionToUnattachedVoted(ReplicaKey candidateKey, int epoch) { - quorum.transitionToUnattachedVotedState(epoch, candidateKey); - } - private void onBecomeFollower(long currentTimeMs) { kafkaRaftMetrics.maybeUpdateElectionLatency(currentTimeMs); @@ -918,7 +916,7 @@ private VoteResponseData handleVoteRequest( ); if (!preVote && voteGranted && quorum.isUnattachedNotVoted()) { - transitionToUnattachedVoted(replicaKey, replicaEpoch); + transitionToUnattachedVoted(replicaEpoch, replicaKey); } logger.info( @@ -943,6 +941,13 @@ private boolean handleVoteResponse( VoteResponseData response = (VoteResponseData) responseMetadata.data(); Errors topLevelError = Errors.forCode(response.errorCode()); if (topLevelError != Errors.NONE) { + if (topLevelError == Errors.UNSUPPORTED_VERSION && quorum.isProspective()) { + logger.warn("Prospective received unsupported version error in vote response in epoch {}, " + + "transitioning to Candidate state immediately since entire quorum does not support PreVote.", + quorum.epoch()); + transitionToCandidate(currentTimeMs); + return true; + } return handleTopLevelError(topLevelError, responseMetadata); } @@ -1009,11 +1014,9 @@ private void handleVoteResponse(VotingState state, int remoteNodeId, long currentTimeMs) { if (quorum.isProspective() && !partitionResponse.preVote()) { - logger.info("Ignoring vote response {} since we sent a PreVote request but received a non-PreVote " + - "response in epoch {} from {}. Transitioning to from Prospective to Candidate since not " + - "all nodes in the quorum support PreVote", + logger.error("Ignoring vote response {} since we sent a PreVote request but received a non-PreVote " + + "response in epoch {} from {}.", partitionResponse, quorum.epoch(), remoteNodeId); - transitionToCandidate(currentTimeMs); } else if (quorum.isCandidate() && partitionResponse.preVote()) { logger.debug("Ignoring vote response {} since we already became a Candidate for epoch {} but " + "received a PreVote response from {}", @@ -1025,9 +1028,10 @@ private void handleVoteResponse(VotingState state, } else { state.recordRejectedVote(remoteNodeId); - // If our vote is rejected, we go immediately to the random backoff. This - // ensures that we are not stuck waiting for the election timeout when the - // vote has become gridlocked. + // If our vote is rejected, we go immediately to backoff phase. This ensures that we are not stuck + // waiting for the election timeout when the vote has become gridlocked. Note for Prospective nodes + // startBackingOff just signals to transition to Unattached and attempt discovering the leader again, and + // Unattached's election timeout acts as the backoff phase for Prospective if the leader is not found. if (state.isVoteRejected() && !state.isBackingOff()) { logger.info("Insufficient remaining votes to win election (rejected by {}). " + "We will backoff before retrying election again", state.rejectingVoters()); @@ -2525,10 +2529,10 @@ private void maybeTransition( " and epoch " + epoch + " which is inconsistent with current leader " + quorum.leaderId() + " and epoch " + quorum.epoch()); } else if (epoch > quorum.epoch()) { - if (leaderId.isPresent()) { + if (leaderId.isPresent() && !leaderEndpoints.isEmpty()) { transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs); } else { - transitionToUnattached(epoch); + transitionToUnattached(epoch, leaderId); } } else if ( leaderId.isPresent() && @@ -3123,46 +3127,8 @@ private long maybeSendVoteRequests( return Long.MAX_VALUE; } - /** - * pollVotingState is equivalent to pollCandidate and pollProspective - */ - private long pollVotingState(long currentTimeMs) { - VotingState state = quorum.votingStateOrThrow(); - GracefulShutdown shutdown = this.shutdown.get(); - - if (shutdown != null) { - if (quorum.isCandidate()) { - // If we happen to shut down while we are a candidate, we will continue - // with the current election until one of the following conditions is met: - // 1) we are elected as leader (which allows us to resign) - // 2) another leader is elected - // 3) the shutdown timer expires - long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); - return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); - } - return shutdown.remainingTimeMs(); - } else if (state.isBackingOff()) { - if (state.isBackoffComplete(currentTimeMs)) { - logger.info("Becoming {} again after election backoff has completed", state.name()); - refreshVotingState(currentTimeMs); - return 0L; - } - return state.remainingBackoffMs(currentTimeMs); - } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { - long backoffDurationMs = binaryExponentialElectionBackoffMs(state.retries()); - logger.info("Election has timed out, backing off for {}ms before becoming a {} again", - backoffDurationMs, state.name()); - state.startBackingOff(currentTimeMs, backoffDurationMs); -// long minFetchRequestBackoffMs = maybeSendAnyVoterFetch(currentTimeMs); - return backoffDurationMs; - } else { - long minVoteRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); - return Math.min(minVoteRequestBackoffMs, state.remainingElectionTimeMs(currentTimeMs)); - } - } - private long pollCandidate(long currentTimeMs) { - VotingState state = quorum.votingStateOrThrow(); + CandidateState state = quorum.candidateStateOrThrow(); GracefulShutdown shutdown = this.shutdown.get(); if (shutdown != null) { @@ -3193,35 +3159,32 @@ private long pollCandidate(long currentTimeMs) { } private long pollProspective(long currentTimeMs) { - VotingState state = quorum.votingStateOrThrow(); + ProspectiveState state = quorum.prospectiveStateOrThrow(); GracefulShutdown shutdown = this.shutdown.get(); if (shutdown != null) { - // If we happen to shut down while we are a candidate, we will continue - // with the current election until one of the following conditions is met: - // 1) we are elected as leader (which allows us to resign) - // 2) another leader is elected - // 3) the shutdown timer expires long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); - } else if (state.isBackingOff()) { - if (state.isBackoffComplete(currentTimeMs)) { - logger.info("Re-elect as prospective after election backoff has completed"); - transitionToProspective(currentTimeMs); - return 0L; + } else if (state.isBackingOff() || state.hasElectionTimeoutExpired(currentTimeMs)) { + if (state.election().hasLeader() && !state.leaderEndpoints().isEmpty()) { + logger.info( + "Election has timed out, transitioning to Follower of leader {}", + state.election().leaderId()); + transitionToFollower( + quorum().epoch(), + state.election().leaderId(), + state.leaderEndpoints(), + currentTimeMs); + } else if (state.votedKey().isPresent()) { + logger.info( + "Election has timed out, transitioning to Unattached with votedKey={} to attempt rediscovering leader", + state.votedKey().get()); + transitionToUnattachedVoted(quorum().epoch(), state.votedKey().get()); + } else { + logger.info("Election has timed out, transitioning to Unattached to attempt rediscovering leader"); + transitionToUnattached(quorum().epoch()); } - long minFetchRequestBackoffMs = maybeSendAnyVoterFetch(currentTimeMs); - return Math.min(minFetchRequestBackoffMs, state.remainingBackoffMs(currentTimeMs)); - } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { - // this hasElectionTimeoutExpired clause needs to follow isBackingOff clause - // because the backoff timer can be reset in this clause and we want to avoid - // resetting the timer on every fetch to bootstrap voters - long backoffDurationMs = binaryExponentialElectionBackoffMs(state.retries()); - logger.info("Election has timed out, backing off for {}ms before becoming a {} again", - backoffDurationMs, state.name()); - state.startBackingOff(currentTimeMs, backoffDurationMs); - long minFetchRequestBackoffMs = maybeSendAnyVoterFetch(currentTimeMs); - return Math.min(minFetchRequestBackoffMs, backoffDurationMs); + return 0L; } else { long minVoteRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); return Math.min(minVoteRequestBackoffMs, state.remainingElectionTimeMs(currentTimeMs)); @@ -3365,8 +3328,6 @@ private long pollUnattachedCommon(UnattachedState state, long currentTimeMs) { private long pollCurrentState(long currentTimeMs) { if (quorum.isLeader()) { return pollLeader(currentTimeMs); -// } else if (quorum.isVotingState()) { -// return pollVotingState(currentTimeMs); } else if (quorum.isCandidate()) { return pollCandidate(currentTimeMs); } else if (quorum.isProspective()) { diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index 7abddeacb01e3..ce86998b2f0a5 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -24,20 +24,21 @@ import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.OptionalInt; public class ProspectiveState implements VotingState { - private final int epoch; private final int localId; + private final int epoch; + private final OptionalInt leaderId; + private final Optional leaderEndpoints; private final Optional votedKey; private final VoterSet voters; // private final long electionTimeoutMs; // private final Timer electionTimer; private final Map preVoteStates = new HashMap<>(); private final Optional highWatermark; - private int retries; private final int electionTimeoutMs; private final Timer electionTimer; - private final Timer backoffTimer; private final Logger log; /** @@ -55,23 +56,24 @@ public ProspectiveState( Time time, int localId, int epoch, + OptionalInt leaderId, + Optional leaderEndpoints, Optional votedKey, VoterSet voters, Optional highWatermark, - int retries, int electionTimeoutMs, LogContext logContext ) { this.localId = localId; this.epoch = epoch; + this.leaderId = leaderId; + this.leaderEndpoints = leaderEndpoints; this.votedKey = votedKey; this.voters = voters; this.highWatermark = highWatermark; - this.retries = retries; this.isBackingOff = false; this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); - this.backoffTimer = time.timer(0); this.log = logContext.logger(ProspectiveState.class); for (ReplicaKey voter : voters.voterKeys()) { @@ -100,7 +102,7 @@ public boolean isBackingOff() { @Override public int retries() { - return retries; + return 1; } /** @@ -144,21 +146,10 @@ public boolean recordRejectedVote(int remoteNodeId) { return recorded; } -// /** -// * Restart the election timer since we've either received sufficient rejecting voters or election timed out -// */ -// public void restartElectionTimer(long currentTimeMs, long electionTimeoutMs) { -// this.electionTimer.update(currentTimeMs); -// this.electionTimer.reset(electionTimeoutMs); -// this.isBackingOff = true; -// } - /** * Record the current election has failed since we've either received sufficient rejecting voters or election timed out */ public void startBackingOff(long currentTimeMs, long backoffDurationMs) { - this.backoffTimer.update(currentTimeMs); - this.backoffTimer.reset(backoffDurationMs); this.isBackingOff = true; } @@ -211,25 +202,12 @@ public long remainingElectionTimeMs(long currentTimeMs) { return electionTimer.remainingMs(); } - @Override - public boolean isBackoffComplete(long currentTimeMs) { - backoffTimer.update(currentTimeMs); - return backoffTimer.isExpired(); - } - - @Override - public long remainingBackoffMs(long currentTimeMs) { - if (!isBackingOff) { - throw new IllegalStateException("Prospective is not currently backing off"); - } - backoffTimer.update(currentTimeMs); - return backoffTimer.remainingMs(); - } - @Override public ElectionState election() { if (votedKey.isPresent()) { return ElectionState.withVotedCandidate(epoch, votedKey().get(), voters.voterIds()); + } else if (leaderId.isPresent()) { + return ElectionState.withElectedLeader(epoch, leaderId.getAsInt(), voters.voterIds()); } else { return ElectionState.withUnknownLeader(epoch, voters.voterIds()); } @@ -242,7 +220,7 @@ public int epoch() { @Override public Endpoints leaderEndpoints() { - return Endpoints.empty(); + return leaderEndpoints.orElse(Endpoints.empty()); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 72200820dcaf8..b3662f8ea9128 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -379,10 +379,48 @@ public void transitionToResigned(List preferredSuccessors) { * but we do not yet know of the elected leader. */ public void transitionToUnattached(int epoch) { + transitionToUnattached(epoch, Optional.empty(), OptionalInt.empty()); + } + + /** + * Transition to the "unattached" state with votedKey. This means we have found an epoch greater than the + * current epoch, but we do not yet know of the elected leader. Note, if we are transitioning from unattached and + * no epoch change, we take the path of unattachedTransitionToUnattachedVotedState instead. + */ + public void transitionToUnattached(int epoch, Optional votedKey) { + transitionToUnattached(epoch, votedKey, OptionalInt.empty()); + } + + public void transitionToUnattached(int epoch, OptionalInt leaderId) { + transitionToUnattached(epoch, Optional.empty(), leaderId); + } + + /** + * Transition to the "unattached" state with votedKey. This means we have found an epoch greater than the + * current epoch, but we do not yet know of the elected leader. Note, if we are transitioning from unattached and + * no epoch change, we take the path of unattachedTransitionToUnattachedVotedState instead. + * It is invalid to have a votedKey AND leaderId in Unattached state (or any state). + */ + private void transitionToUnattached(int epoch, Optional votedKey, OptionalInt leaderId) { int currentEpoch = state.epoch(); - if (epoch <= currentEpoch) { - throw new IllegalStateException("Cannot transition to Unattached with epoch= " + epoch + - " from current state " + state); + if (votedKey.isPresent() && leaderId.isPresent()) { + throw new IllegalStateException( + String.format( + "Cannot transition to Unattached with epoch= %d with both votedKey= %s and leaderId= %d from current state %s", + currentEpoch, + votedKey.get(), + leaderId.getAsInt(), + state + ) + ); + } else if (epoch < currentEpoch || (epoch == currentEpoch && !isProspective())) { + throw new IllegalStateException( + String.format( + "Cannot transition to Unattached with epoch= %d from current state %s", + epoch, + state + ) + ); } final long electionTimeoutMs; @@ -392,6 +430,8 @@ public void transitionToUnattached(int epoch) { electionTimeoutMs = candidateStateOrThrow().remainingElectionTimeMs(time.milliseconds()); } else if (isUnattached()) { electionTimeoutMs = unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); + } else if (isProspective() && !prospectiveStateOrThrow().isBackingOff()) { + electionTimeoutMs = prospectiveStateOrThrow().remainingElectionTimeMs(time.milliseconds()); } else { electionTimeoutMs = randomElectionTimeoutMs(); } @@ -399,8 +439,8 @@ public void transitionToUnattached(int epoch) { durableTransitionTo(new UnattachedState( time, epoch, - OptionalInt.empty(), - Optional.empty(), + leaderId, + votedKey, partitionState.lastVoterSet().voterIds(), state.highWatermark(), electionTimeoutMs, @@ -409,12 +449,10 @@ public void transitionToUnattached(int epoch) { } /** - * Grant a vote to a candidate. We will transition/remain in Unattached - * state until either the election timeout expires or a leader is elected. In particular, - * we do not begin fetching until the election has concluded and - * {@link #transitionToFollower(int, int, Endpoints)} is invoked. + * Grant a vote to a candidate as Unattached. We will transition to Unattached with votedKey + * state and remain there until either the election timeout expires or we discover the leader. */ - public void transitionToUnattachedVotedState( + public void unattachedAddVotedState( int epoch, ReplicaKey candidateKey ) { @@ -457,7 +495,7 @@ public void transitionToUnattachedVotedState( new UnattachedState( time, epoch, - OptionalInt.empty(), + state.election().optionalLeaderId(), Optional.of(candidateKey), partitionState.lastVoterSet().voterIds(), state.highWatermark(), @@ -551,22 +589,21 @@ public void transitionToProspective() { partitionState.lastVoterSet() ) ); - } else if (isLeader() || isCandidate()) { + } else if (isLeader() || isProspective()) { throw new IllegalStateException("Cannot transition to Prospective since the local broker.id=" + localId + " is state " + state); } - int retries = isProspective() ? prospectiveStateOrThrow().retries() + 1 : 1; - durableTransitionTo(new ProspectiveState( time, localIdOrThrow(), epoch(), + leaderId(), + Optional.of(state.leaderEndpoints()), state.election().optionalVotedKey(), partitionState.lastVoterSet(), state.highWatermark(), randomElectionTimeoutMs(), - retries, logContext )); } @@ -604,7 +641,7 @@ private void checkValidTransitionToCandidate() { ); } if (partitionState.lastKraftVersion().isReconfigSupported()) { - if (!isCandidate() && !isProspective()) { + if (!isProspective()) { throw new IllegalStateException("Cannot transition to Candidate since the local broker.id=" + localId + " is state " + state); } diff --git a/raft/src/main/java/org/apache/kafka/raft/VotingState.java b/raft/src/main/java/org/apache/kafka/raft/VotingState.java index 9fe327bf80c69..0c4204a3157a7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/VotingState.java +++ b/raft/src/main/java/org/apache/kafka/raft/VotingState.java @@ -97,10 +97,6 @@ default boolean isVoteRejected() { long remainingElectionTimeMs(long currentTimeMs); - boolean isBackoffComplete(long currentTimeMs); - - long remainingBackoffMs(long currentTimeMs); - /** * Get the set of voters which have not been counted as granted or rejected yet. * diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java index 64230b45f7292..a63a169c8632d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java @@ -71,10 +71,12 @@ public KafkaRaftMetrics(Metrics metrics, String metricGrpPrefix, QuorumState sta Gauge stateProvider = (mConfig, currentTimeMs) -> { if (state.isLeader()) { return "leader"; + } else if (state.isProspective()) { + return "prospective"; } else if (state.isCandidate()) { return "candidate"; } else if (state.isUnattachedAndVoted()) { - return "voted"; + return "unattached-voted"; } else if (state.isFollower()) { // a broker is special kind of follower, as not being a voter, it's an observer if (state.isObserver()) { diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 020ae849d9fee..623bdd867753b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.VoteResponseData; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; @@ -256,18 +257,18 @@ public void testHandlePreVoteRequestAsUnattachedWithLeader() throws Exception { } @ParameterizedTest - @ValueSource(booleans = {true, false}) + @ValueSource(booleans = {false}) public void testHandlePreVoteRequestAsFollowerObserver(boolean hasFetchedFromLeader) throws Exception { int localId = randomReplicaId(); int epoch = 2; - int leaderId = localId + 1; - ReplicaKey leader = replicaKey(leaderId, true); + ReplicaKey localKey = replicaKey(localId, true); + ReplicaKey leader = replicaKey(localId + 1, true); ReplicaKey follower = replicaKey(localId + 2, true); - Set voters = Set.of(leader.id(), follower.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) .withElectedLeader(epoch, leader.id()) .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, leader, follower)))) .build(); context.assertElectedLeader(epoch, leader.id()); @@ -279,7 +280,7 @@ public void testHandlePreVoteRequestAsFollowerObserver(boolean hasFetchedFromLea context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, 0L, Errors.NONE) + context.fetchResponse(epoch, leader.id(), MemoryRecords.EMPTY, 0L, Errors.NONE) ); } @@ -287,7 +288,7 @@ public void testHandlePreVoteRequestAsFollowerObserver(boolean hasFetchedFromLea context.pollUntilResponse(); boolean voteGranted = !hasFetchedFromLeader; - context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leaderId), voteGranted); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), voteGranted); assertTrue(context.client.quorum().isFollower()); } @@ -680,19 +681,19 @@ public void testPreVoteNotSupportedByRemote() throws Exception { assertTrue(context.client.quorum().isProspective()); + // Simulate remote node not supporting PreVote with UNSUPPORTED_VERSION response. + // Note: with the mocked network client this is a bit different, in reality this response would be generated + // from the network client and not sent from the remote node. List voteRequests = context.collectVoteRequests(epoch, 0, 0); assertEquals(1, voteRequests.size()); - - // Simulate remote node not supporting PreVote and responding with Vote response with version 0 context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), - context.voteResponse(false, OptionalInt.empty(), epoch, (short) 0) + RaftUtil.errorResponse(ApiKeys.VOTE, Errors.UNSUPPORTED_VERSION) ); context.client.poll(); - // Local should transition to Candidate since it sees a remote node does not support PreVote. It can tell this - // from responses with PreVote field set to false + // Local should transition to Candidate since it realizes remote node does not support PreVote. assertEquals(epoch + 1, context.currentEpoch()); context.client.quorum().isCandidate(); } @@ -726,7 +727,7 @@ public void testProspectiveReceivesBeginQuorumRequest() throws Exception { } @Test - public void testProspectiveSendsFetchRequests() throws Exception { + public void testProspectiveTransitionsToUnattachedOnElectionFailure() throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey otherNode = replicaKey(localId + 1, true); @@ -743,43 +744,28 @@ public void testProspectiveSendsFetchRequests() throws Exception { context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); assertTrue(context.client.quorum().isProspective()); - RaftRequest.Outbound voteRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); + context.assertSentVoteRequest(epoch, 0, 0L, 1); - // If election timeout expires, we should start backing off and sending fetch requests + // If election timeout expires, we should transition to Unattached to attempt re-discovering leader context.time.sleep(context.electionTimeoutMs() * 2L); context.client.poll(); - assertTrue(context.client.quorum().isProspective()); - context.assertSentFetchRequest(epoch, 0, 0); - - context.time.sleep(context.electionBackoffMaxMs); + assertTrue(context.client.quorum().isUnattached()); - // After the backoff, we will transition back to Prospective and continue sending PreVote requests + // After election times out again, we will transition back to Prospective and continue sending PreVote requests + context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); - voteRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); + RaftRequest.Outbound voteRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); - // If we receive enough rejected votes, we also immediately start backing off and follow with fetch requests + // If we receive enough rejected votes, we also transition to Unattached immediately context.deliverResponse( voteRequest.correlationId(), voteRequest.destination(), context.voteResponse(false, OptionalInt.empty(), epoch, true)); - // handle vote response and update backoff timer + // handle vote response and mark we should transition out of prospective context.client.poll(); - // send fetch request while in backoff + // transition context.client.poll(); - - assertTrue(context.client.quorum().isProspective()); - RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0, 0); - context.deliverResponse( - fetchRequest.correlationId(), - fetchRequest.destination(), - context.fetchResponse(epoch, -1, MemoryRecords.EMPTY, -1, Errors.BROKER_NOT_AVAILABLE)); - context.client.poll(); - - // We continue sending fetch requests until backoff timer expires - context.time.sleep(context.electionBackoffMaxMs / 2); - context.client.poll(); - assertTrue(context.client.quorum().isProspective()); - context.assertSentFetchRequest(epoch, 0, 0); + assertTrue(context.client.quorum().isUnattached()); } @Test diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java index 01992b8567011..a55f4cf1eecfb 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; @@ -25,8 +26,10 @@ import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; +import java.net.InetSocketAddress; import java.util.Collections; import java.util.Optional; +import java.util.OptionalInt; import java.util.Set; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -38,6 +41,12 @@ public class ProspectiveStateTest { private final ReplicaKey localReplicaKey = ReplicaKey.of(0, Uuid.randomUuid()); + private final Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses( + Collections.singletonMap( + ListenerName.normalised("CONTROLLER"), + InetSocketAddress.createUnresolved("mock-host-3", 1234) + ) + ); private final int epoch = 5; private final MockTime time = new MockTime(); private final int electionTimeoutMs = 5000; @@ -48,10 +57,26 @@ private ProspectiveState newProspectiveState(VoterSet voters) { time, localReplicaKey.id(), epoch, + OptionalInt.of(3), + Optional.empty(), + Optional.empty(), + voters, + Optional.empty(), + electionTimeoutMs, + logContext + ); + } + + private ProspectiveState newProspectiveStateWithLeaderEndpoints(VoterSet voters) { + return new ProspectiveState( + time, + localReplicaKey.id(), + epoch, + OptionalInt.of(3), + Optional.of(leaderEndpoints), Optional.empty(), voters, Optional.empty(), - 0, electionTimeoutMs, logContext ); @@ -242,10 +267,11 @@ public void testGrantVoteWithVotedKey(boolean isLogUpToDate, boolean withDirecto time, localReplicaKey.id(), epoch, + OptionalInt.empty(), + Optional.empty(), Optional.of(node1), voterSetWithLocal(Stream.of(node1, node2, node3), withDirectoryId), Optional.empty(), - 0, electionTimeoutMs, logContext ); diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java index a7362e0080861..86c4e8771d838 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.Optional; +import java.util.OptionalInt; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -54,15 +55,19 @@ private ProspectiveState newProspectiveVotedState( time, localId, epoch, + OptionalInt.empty(), + Optional.empty(), votedKey, voters, Optional.empty(), - 0, electionTimeoutMs, logContext ); } + // todo: need with leader state + // todo: don't forget to fix quorum state keeping leaderid, and prospective to follower transition + @Test public void testElectionTimeout() { ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithDirectoryId)); diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index 3698d10928324..1d31d9f65e70f 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -34,6 +34,7 @@ import java.net.InetSocketAddress; import java.util.Collections; import java.util.HashMap; +import java.util.Map; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; @@ -41,6 +42,7 @@ import java.util.stream.IntStream; import java.util.stream.Stream; +import static org.apache.kafka.server.common.KRaftVersion.KRAFT_VERSION_0; import static org.apache.kafka.server.common.KRaftVersion.KRAFT_VERSION_1; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -165,6 +167,9 @@ public void testInitializePrimordialEpoch(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); assertTrue(state.isUnattached()); assertEquals(0, state.epoch()); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); CandidateState candidateState = state.candidateStateOrThrow(); assertTrue(candidateState.isVoteGranted()); @@ -334,15 +339,14 @@ public void testInitializeAsResignedLeader(KRaftVersion kraftVersion) { resignedState.remainingElectionTimeMs(time.milliseconds())); } - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testCandidateToCandidate(KRaftVersion kraftVersion) { + @Test + public void testCandidateToCandidate() { int node1 = 1; int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_0); assertEquals(Optional.empty(), store.readElectionState()); - QuorumState state = initializeEmptyState(voters, kraftVersion); + QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_0); state.transitionToCandidate(); assertTrue(state.isCandidate()); assertEquals(1, state.epoch()); @@ -387,6 +391,9 @@ public void testCandidateToResigned(KRaftVersion kraftVersion) { assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); assertTrue(state.isCandidate()); assertEquals(1, state.epoch()); @@ -405,6 +412,9 @@ public void testCandidateToLeader(KRaftVersion kraftVersion) { assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); assertTrue(state.isCandidate()); assertEquals(1, state.epoch()); @@ -423,6 +433,9 @@ public void testCandidateToLeaderWithoutGrantedVote(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); assertFalse(state.candidateStateOrThrow().isVoteGranted()); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); @@ -440,6 +453,9 @@ public void testCandidateToFollower(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId)); @@ -464,6 +480,9 @@ public void testCandidateToUnattached(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); state.transitionToUnattached(5); @@ -487,9 +506,12 @@ public void testCandidateToUnattachedVoted(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); - state.transitionToUnattachedVotedState(5, otherNodeKey); + state.transitionToUnattached(5, Optional.of(otherNodeKey)); assertEquals(5, state.epoch()); assertEquals(OptionalInt.empty(), state.leaderId()); @@ -515,9 +537,12 @@ public void testCandidateToAnyStateLowerEpoch(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(4, otherNodeKey)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, Optional.of(otherNodeKey))); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -547,6 +572,9 @@ public void testLeaderToLeader(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); @@ -565,6 +593,9 @@ public void testLeaderToResigned(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); @@ -589,6 +620,9 @@ public void testLeaderToCandidate(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); @@ -607,6 +641,9 @@ public void testLeaderToFollower(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); state.transitionToLeader(0L, accumulator); @@ -633,6 +670,9 @@ public void testLeaderToUnattached(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); state.transitionToLeader(0L, accumulator); @@ -657,10 +697,13 @@ public void testLeaderToUnattachedVoted(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); state.transitionToLeader(0L, accumulator); - state.transitionToUnattachedVotedState(5, otherNodeKey); + state.transitionToUnattached(5, Optional.of(otherNodeKey)); assertEquals(5, state.epoch()); assertEquals(OptionalInt.empty(), state.leaderId()); @@ -688,11 +731,14 @@ public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); state.transitionToLeader(0L, accumulator); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(4, otherNodeKey)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, Optional.of(otherNodeKey))); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -729,7 +775,7 @@ public void testCannotFollowOrVoteForSelf(KRaftVersion kraftVersion) { voters.listeners(localId) ) ); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(0, localVoterKey)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(0, Optional.of(localVoterKey))); } @ParameterizedTest @@ -742,9 +788,9 @@ public void testUnattachedToLeaderOrResigned(KRaftVersion kraftVersion) { ElectionState.withVotedCandidate(epoch, leaderKey, voters.voterIds()), kraftVersion ); - QuorumState state = initializeEmptyState(voters, kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - assertTrue(state.isUnattached()); + assertTrue(state.isUnattachedAndVoted()); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @@ -760,7 +806,7 @@ public void testUnattachedVotedSameEpoch(KRaftVersion kraftVersion) { int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToUnattachedVotedState(5, otherNodeKey); + state.unattachedAddVotedState(5, otherNodeKey); UnattachedState votedState = state.unattachedStateOrThrow(); assertEquals(5, votedState.epoch()); @@ -792,7 +838,7 @@ public void testUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { state.transitionToUnattached(5); assertTrue(state.isUnattachedNotVoted()); - state.transitionToUnattachedVotedState(8, otherNodeKey); + state.transitionToUnattached(8, Optional.of(otherNodeKey)); assertTrue(state.isUnattachedAndVoted()); UnattachedState votedState = state.unattachedStateOrThrow(); @@ -822,6 +868,9 @@ public void testUnattachedToCandidate(KRaftVersion kraftVersion) { int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } state.transitionToCandidate(); assertTrue(state.isCandidate()); @@ -911,7 +960,7 @@ public void testUnattachedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(4, otherNodeKey)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, Optional.of(otherNodeKey))); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -940,7 +989,7 @@ public void testUnattachedVotedToInvalidLeaderOrResigned(KRaftVersion kraftVersi VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @@ -953,8 +1002,11 @@ public void testUnattachedVotedToCandidate(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); state.transitionToCandidate(); @@ -973,7 +1025,7 @@ public void testObserverFromUnattachedVotedToCandidate(KRaftVersion kraftVersion VoterSet voters = withRemoteVoterSet(IntStream.of(voter1, voter2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID)); + state.transitionToUnattached(5, Optional.of(ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID))); assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); assertTrue(state.isUnattached()); @@ -988,14 +1040,14 @@ public void testUnattachedVotedToUnattachedVotedSameEpoch(KRaftVersion kraftVers QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); - state.transitionToUnattachedVotedState(8, ReplicaKey.of(node1, Uuid.randomUuid())); + state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node1, Uuid.randomUuid()))); assertThrows( IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) + () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))) ); assertThrows( IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) + () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID))) ); } @@ -1007,7 +1059,7 @@ public void testUnattachedVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); state.transitionToFollower( 5, node2, @@ -1040,7 +1092,7 @@ public void testUnattachedVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); state.transitionToFollower( 8, node2, @@ -1073,7 +1125,7 @@ public void testUnattachedVotedToUnattachedSameEpoch(KRaftVersion kraftVersion) VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5)); } @@ -1084,7 +1136,7 @@ public void testUnattachedVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID)); + state.transitionToUnattached(5, Optional.of(ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID))); long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); time.sleep(1000); @@ -1105,9 +1157,9 @@ public void testUnattachedVotedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattachedVotedState(5, otherNodeKey); + state.transitionToUnattached(5, Optional.of(otherNodeKey)); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(4, otherNodeKey)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, Optional.of(otherNodeKey))); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -1131,7 +1183,7 @@ public void testUnattachedVotedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testAllStatesToUnattachedFailInSameEpoch(KRaftVersion kraftVersion) { + public void testIllegalStateTransitionToUnattachedInSameEpoch(KRaftVersion kraftVersion) { ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); @@ -1140,30 +1192,51 @@ public void testAllStatesToUnattachedFailInSameEpoch(KRaftVersion kraftVersion) // unattached to unattached state.unattachedStateOrThrow(); - state.transitionToUnattachedVotedState(5, voter1); + state.transitionToUnattached(5, Optional.of(voter1)); // cannot vote for same or different node in same epoch - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(5, voter1)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(5, voter2)); - // can vote for same or different node in larger epoch - state.transitionToUnattachedVotedState(10, voter1); - state.transitionToUnattachedVotedState(15, voter2); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5, Optional.of(voter1))); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5, Optional.of(voter2))); // follower to unattached - state.transitionToFollower(20, voter1.id(), voters.listeners(voter1.id())); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(state.epoch(), voter1)); - state.transitionToUnattachedVotedState(state.epoch() + 1, voter1); + state.transitionToFollower(10, voter1.id(), voters.listeners(voter1.id())); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), Optional.of(voter1))); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } // candidate state.transitionToCandidate(); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(state.epoch(), voter1)); - state.transitionToUnattachedVotedState(state.epoch() + 1, voter1); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), Optional.of(voter1))); // leader - state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(voter1.id()); state.transitionToLeader(0L, accumulator); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(state.epoch(), voter1)); - state.transitionToUnattachedVotedState(state.epoch() + 1, voter1); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), Optional.of(voter1))); + } + + @Test + public void testProspectiveToUnattachedInSameEpoch() { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + + // with leader state + state.transitionToProspective(); + state.transitionToUnattached(state.epoch(), OptionalInt.of(voter1.id())); + + // with voted state + state = initializeEmptyState(voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToUnattached(state.epoch(), Optional.of(voter1)); + + // without voted or leader state + state = initializeEmptyState(voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToUnattached(state.epoch(), Optional.empty()); } @ParameterizedTest @@ -1314,6 +1387,9 @@ public void testFollowerToCandidate(KRaftVersion kraftVersion) { voters.listeners(node2) ); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + } int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); state.transitionToCandidate(); @@ -1380,15 +1456,15 @@ public void testFollowerToUnattachedVotedSameEpoch(KRaftVersion kraftVersion) { assertThrows( IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) + () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))) ); assertThrows( IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(8, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) + () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID))) ); assertThrows( IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) + () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID))) ); } @@ -1410,7 +1486,7 @@ public void testFollowerToUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToUnattachedVotedState(9, nodeKey1); + state.transitionToUnattached(9, Optional.of(nodeKey1)); assertTrue(state.isUnattachedAndVoted()); UnattachedState votedState = state.unattachedStateOrThrow(); @@ -1436,7 +1512,7 @@ public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); assertThrows( IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(4, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID)) + () -> state.transitionToUnattached(4, Optional.of(ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID))) ); assertThrows( IllegalStateException.class, @@ -1469,7 +1545,7 @@ public void testCanBecomeFollowerOfNonVoter(KRaftVersion kraftVersion) { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); // Add voted state - state.transitionToUnattachedVotedState(4, nonVoterKey); + state.transitionToUnattached(4, Optional.of(nonVoterKey)); assertTrue(state.isUnattachedAndVoted()); UnattachedState votedState = state.unattachedStateOrThrow(); @@ -1519,7 +1595,7 @@ public void testObserverWithIdCanVote(KRaftVersion kraftVersion) { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isObserver()); - state.transitionToUnattachedVotedState(5, otherNodeKey); + state.transitionToUnattached(5, Optional.of(otherNodeKey)); assertTrue(state.isUnattachedAndVoted()); UnattachedState votedState = state.unattachedStateOrThrow(); @@ -1607,6 +1683,10 @@ public void testHasRemoteLeader(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); assertFalse(state.hasRemoteLeader()); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + assertFalse(state.hasRemoteLeader()); + } state.transitionToCandidate(); assertFalse(state.hasRemoteLeader()); @@ -1617,7 +1697,7 @@ public void testHasRemoteLeader(KRaftVersion kraftVersion) { state.transitionToUnattached(state.epoch() + 1); assertFalse(state.hasRemoteLeader()); - state.transitionToUnattachedVotedState(state.epoch() + 1, otherNodeKey); + state.transitionToUnattached(state.epoch() + 1, Optional.of(otherNodeKey)); assertFalse(state.hasRemoteLeader()); state.transitionToFollower( @@ -1650,9 +1730,13 @@ public void testHighWatermarkRetained(KRaftVersion kraftVersion) { state.transitionToUnattached(6); assertEquals(highWatermark, state.highWatermark()); - state.transitionToUnattachedVotedState(7, otherNodeKey); + state.transitionToUnattached(7, Optional.of(otherNodeKey)); assertEquals(highWatermark, state.highWatermark()); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + assertEquals(highWatermark, state.highWatermark()); + } state.transitionToCandidate(); assertEquals(highWatermark, state.highWatermark()); @@ -1677,10 +1761,14 @@ public void testInitializeWithEmptyLocalId(KRaftVersion kraftVersion) { assertTrue(state.isObserver()); assertFalse(state.isVoter()); - assertThrows(IllegalStateException.class, state::transitionToCandidate); + if (kraftVersion.isReconfigSupported()) { + assertThrows(IllegalStateException.class, state::transitionToProspective); + } else { + assertThrows(IllegalStateException.class, state::transitionToCandidate); + } assertThrows( IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(1, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)) + () -> state.unattachedAddVotedState(1, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)) ); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); @@ -1714,59 +1802,335 @@ public void testNoLocalIdInitializationFailsIfElectionStateHasVotedCandidate(KRa assertThrows(IllegalStateException.class, () -> state2.initialize(new OffsetAndEpoch(0, 0))); } - // testUnattachedVotedToProspective @Test - public void testUnattachedVotedToProspective() { + public void testUnattachedWithLeaderToProspective() { + int node1 = 1; + int node2 = 2; + int epoch = 5; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); + store.writeElectionState(ElectionState.withElectedLeader(epoch, node1, voters.voterIds()), KRAFT_VERSION_1); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertEquals( + Optional.of(ElectionState.withElectedLeader(epoch, node1, persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), + store.readElectionState() + ); + } + + @Test + public void testIllegalTransitionsToCandidate() { + int node1 = 1; + int node2 = 2; + int epoch = 5; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); + QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, epoch)); + + // unattached + assertTrue(state.isUnattached()); + assertThrows(IllegalStateException.class, state::transitionToCandidate); + + // follower + state.transitionToFollower(epoch, node1, voters.listeners(node1)); + assertThrows(IllegalStateException.class, state::transitionToCandidate); + + // candidate + state.transitionToProspective(); + state.transitionToCandidate(); + assertThrows(IllegalStateException.class, state::transitionToCandidate); + + // leader + state.candidateStateOrThrow().recordGrantedVote(1); + state.transitionToLeader(0L, accumulator); + assertThrows(IllegalStateException.class, state::transitionToCandidate); + + // resigned + state.transitionToResigned(Collections.emptyList()); + assertThrows(IllegalStateException.class, state::transitionToCandidate); + } + + @Test + public void testIllegalTransitionsToProspective() { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); + QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + + // prospective + state.transitionToProspective(); + assertThrows(IllegalStateException.class, state::transitionToProspective); + + // leader + state.transitionToCandidate(); + state.candidateStateOrThrow().recordGrantedVote(1); + state.transitionToLeader(0L, accumulator); + assertThrows(IllegalStateException.class, state::transitionToProspective); + + // observer + voters = withRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); + state = initializeEmptyState(voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isUnattached()); + assertTrue(state.isObserver()); + assertThrows(IllegalStateException.class, state::transitionToProspective); + } + + @Test + public void testIllegalTransitionsFromProspective() { + int leaderId = 1; + int followerId = 2; + int epoch = 5; + Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses( + Collections.singletonMap( + ListenerName.normalised("CONTROLLER"), + InetSocketAddress.createUnresolved("host-1", 1234))); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), KRAFT_VERSION_1); + QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, epoch)); + assertTrue(state.isUnattached()); + state.transitionToProspective(); + + assertThrows(IllegalStateException.class, state::transitionToProspective); + assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(epoch - 1)); + assertThrows(IllegalStateException.class, () -> state.transitionToFollower(epoch - 1, leaderId, leaderEndpoints)); + assertThrows(IllegalArgumentException.class, () -> state.transitionToFollower(epoch, leaderId, Endpoints.empty())); + } + + @Test + public void testUnattachedToAndFromProspective() { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); + QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isUnattached()); + assertEquals(logEndEpoch, state.epoch()); + state.transitionToProspective(); + assertEquals(logEndEpoch, state.epoch()); + state.transitionToUnattached(5); + assertTrue(state.isUnattached()); + assertEquals(5, state.epoch()); + } + + @Test + public void testUnattachedVotedToAndFromProspectiveVoted() { int node1 = 1; Uuid node1DirectoryId = Uuid.randomUuid(); int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, node1DirectoryId)); + state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, node1DirectoryId))); state.transitionToProspective(); assertTrue(state.isProspective()); ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); assertEquals(5, prospectiveState.epoch()); - assertEquals(electionTimeoutMs, prospectiveState.remainingElectionTimeMs(time.milliseconds())); assertEquals( Optional.of( ElectionState.withVotedCandidate( 5, persistedVotedKey(ReplicaKey.of(node1, node1DirectoryId), KRAFT_VERSION_1), - persistedVoters(voters.voterIds(), KRAFT_VERSION_1) - ) - ), + persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), + store.readElectionState() + ); + + state.transitionToUnattached(prospectiveState.epoch(), prospectiveState.votedKey()); + assertTrue(state.isUnattachedAndVoted()); + assertEquals(prospectiveState.epoch(), state.epoch()); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + prospectiveState.epoch(), + persistedVotedKey(ReplicaKey.of(node1, node1DirectoryId), KRAFT_VERSION_1), + persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), store.readElectionState() ); } - // testUnattachedWithLeaderToProspective + @Test + public void testUnattachedWithLeaderNoEndpointToAndFromProspective() { + int leaderId = 1; + ReplicaKey leaderKey = ReplicaKey.of(leaderId, Uuid.randomUuid()); + int followerId = 2; + ReplicaKey followerKey = ReplicaKey.of(followerId, Uuid.randomUuid()); + int epoch = 5; + Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses( + Collections.singletonMap( + ListenerName.normalised("CONTROLLER"), + InetSocketAddress.createUnresolved("host-1", 1234))); + Map voterMap = new HashMap<>(); + voterMap.put(localId, VoterSetTest.voterNode(localVoterKey)); + voterMap.put(leaderId, VoterSetTest.voterNode(leaderKey, Endpoints.empty())); + voterMap.put(followerId, VoterSetTest.voterNode(followerKey, Endpoints.empty())); + VoterSet voters = VoterSetTest.voterSet(voterMap); + + store.writeElectionState(ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()), KRAFT_VERSION_1); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isUnattached()); + assertTrue(state.hasLeader()); - // testIllegalTransitionsToProspective + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 5, + leaderId, + persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), + store.readElectionState() + ); - // testIllegalTransitionsFromProspective + state.transitionToFollower(epoch, leaderId, leaderEndpoints); + assertTrue(state.isFollower()); + assertEquals(epoch, state.epoch()); + assertEquals(leaderEndpoints, state.leaderEndpoints()); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 5, + leaderId, + persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), + store.readElectionState() + ); + } - // testProspectiveToProspective + @Test + public void testFollowerToAndFromProspectiveWithLeader() { + int leaderId = 1; + int followerId = 2; + int epoch = 5; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), KRAFT_VERSION_1); - // testProspectiveVotedToProspective + store.writeElectionState(ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()), KRAFT_VERSION_1); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isFollower()); + assertTrue(state.hasLeader()); - // testProspectiveWithLeaderToProspective + state.transitionToProspective(); + assertTrue(state.isProspective()); + ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); + assertFalse(prospectiveState.leaderEndpoints().isEmpty()); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 5, + leaderId, + persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), + store.readElectionState() + ); - // testProspectiveToCandidate + state.transitionToFollower(epoch, leaderId, prospectiveState.leaderEndpoints()); + assertTrue(state.isFollower()); + assertEquals(epoch, state.epoch()); + assertFalse(state.leaderEndpoints().isEmpty()); + assertEquals(prospectiveState.leaderEndpoints(), state.leaderEndpoints()); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 5, + leaderId, + persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), + store.readElectionState() + ); + } - // testProspectiveVotedToCandidate + @Test + public void testProspectiveVotedToAndFromCandidate() { + int node1 = 1; + Uuid node1DirectoryId = Uuid.randomUuid(); + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); + store.writeElectionState( + ElectionState.withVotedCandidate( + logEndEpoch, + ReplicaKey.of(node1, node1DirectoryId), + voters.voterIds() + ), + KRAFT_VERSION_1 + ); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isUnattachedAndVoted()); - // testCandidateToProspective (can include votedKey which is erased) + state.transitionToProspective(); + ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); + assertTrue(prospectiveState.votedKey().isPresent()); - // testProspectiveToUnattached (can include votedkey which is erased) + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertEquals(logEndEpoch + 1, state.epoch()); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, KRAFT_VERSION_1), + persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), + store.readElectionState() + ); - // testProspectiveVotedToProspective (votedkey needs to be kept) + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertEquals(logEndEpoch + 1, state.epoch()); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, KRAFT_VERSION_1), + persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), + store.readElectionState() + ); + } - // testProspectiveToUnattachedSameEpoch + @Test + public void testProspectiveWithLeaderToCandidate() { + int leaderId = 1; + int followerId = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), KRAFT_VERSION_1); - // testProspectiveToProspectiveHigherEpoch + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, leaderId, voters.voterIds()), KRAFT_VERSION_1); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertTrue(state.hasLeader()); - // testProspectiveToFollowerHigherEpoch + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertEquals(logEndEpoch + 1, state.epoch()); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, KRAFT_VERSION_1), + persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), + store.readElectionState() + ); + } + + @Test + public void testProspectiveToUnattachedHigherEpoch() { + int leaderId = 1; + int followerId = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), KRAFT_VERSION_1); + + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, leaderId, voters.voterIds()), KRAFT_VERSION_1); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertTrue(state.hasLeader()); + + state.transitionToUnattached(logEndEpoch + 1); + assertTrue(state.isUnattachedNotVoted()); + assertFalse(state.hasLeader()); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java index 0c6b8bfb57ec8..08443a69b7e37 100644 --- a/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java @@ -359,8 +359,7 @@ public static VoterSet.VoterNode voterNode(int id, boolean withDirectoryId) { } public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) { - return new VoterSet.VoterNode( - replicaKey, + return voterNode(replicaKey, Endpoints.fromInetSocketAddresses( Collections.singletonMap( DEFAULT_LISTENER_NAME, @@ -369,7 +368,14 @@ public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) { 9990 + replicaKey.id() ) ) - ), + ) + ); + } + + public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey, Endpoints endpoints) { + return new VoterSet.VoterNode( + replicaKey, + endpoints, Feature.KRAFT_VERSION.supportedVersionRange() ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index fc632672f9849..5262405a1169c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -39,6 +39,7 @@ import java.util.Collections; import java.util.Map; +import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Random; @@ -141,6 +142,18 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + assertEquals("prospective", getMetric(metrics, "current-state").metricValue()); + assertEquals((double) -1L, getMetric(metrics, "current-leader").metricValue()); + assertEquals((double) -1L, getMetric(metrics, "current-vote").metricValue()); + assertEquals( + Uuid.ZERO_UUID.toString(), + getMetric(metrics, "current-vote-directory-id").metricValue() + ); + assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); + } state.transitionToCandidate(); assertEquals("candidate", getMetric(metrics, "current-state").metricValue()); assertEquals((double) -1L, getMetric(metrics, "current-leader").metricValue()); @@ -186,8 +199,21 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { state.followerStateOrThrow().updateHighWatermark(OptionalLong.of(10L)); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToUnattachedVotedState(3, ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID)); - assertEquals("voted", getMetric(metrics, "current-state").metricValue()); + if (kraftVersion.isReconfigSupported()) { + state.transitionToProspective(); + assertEquals("prospective", getMetric(metrics, "current-state").metricValue()); + assertEquals((double) 1L, getMetric(metrics, "current-leader").metricValue()); + assertEquals((double) -1L, getMetric(metrics, "current-vote").metricValue()); + assertEquals( + Uuid.ZERO_UUID.toString(), + getMetric(metrics, "current-vote-directory-id").metricValue() + ); + assertEquals((double) 2, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); + } + + state.transitionToUnattached(3, Optional.of(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID))); + assertEquals("unattached-voted", getMetric(metrics, "current-state").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) 2, getMetric(metrics, "current-vote").metricValue()); assertEquals( @@ -197,9 +223,9 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { assertEquals((double) 3, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToUnattached(4); + state.transitionToUnattached(4, OptionalInt.of(1)); assertEquals("unattached", getMetric(metrics, "current-state").metricValue()); - assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); + assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); assertEquals( Uuid.ZERO_UUID.toString(), From dbb5501ec8ca7ef02ca817a0a59ba71485eeb525 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Wed, 11 Dec 2024 11:05:59 -0800 Subject: [PATCH 04/23] fix quorumstate description --- .../org/apache/kafka/raft/QuorumState.java | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index b3662f8ea9128..cc23d4a8903fc 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -39,35 +39,35 @@ * how they are triggered: * * Resigned transitions to: - * Unattached: After learning of a new election with a higher epoch + * Unattached: After learning of a new election with a higher epoch * Prospective: After expiration of the election timeout - * Follower: After discovering a leader with an equal or larger epoch + * Follower: After discovering a leader with an equal or larger epoch * * Unattached transitions to: - * Unattached: After learning of a new election with a higher epoch or after giving a binding vote + * Unattached: After learning of a new election with a higher epoch or after giving a binding vote * Prospective: After expiration of the election timeout - * Follower: After discovering a leader with an equal or larger epoch + * Follower: After discovering a leader with an equal or larger epoch * * Prospective transitions to: - * Unattached: After learning of an election with a higher epoch, - * or node did not have last known leader and loses/times out election - * Candidate: After receiving a majority of PreVotes granted - * Follower: After discovering a leader with a larger epoch, - * or node had a last known leader and loses/times out election + * Unattached: After learning of an election with a higher epoch, or node did not have last + * known leader and loses/times out election + * Candidate: After receiving a majority of PreVotes granted + * Follower: After discovering a leader with a larger epoch, or node had a last known leader + * and loses/times out election * * Candidate transitions to: - * Unattached: After learning of a new election with a higher epoch - * Candidate: After expiration of the election timeout - * Leader: After receiving a majority of votes + * Unattached: After learning of a new election with a higher epoch + * Prospective: After expiration of the election timeout or loss of election + * Leader: After receiving a majority of votes * * Leader transitions to: - * Unattached: After learning of a new election with a higher epoch - * Resigned: When shutting down gracefully + * Unattached: After learning of a new election with a higher epoch + * Resigned: When shutting down gracefully * * Follower transitions to: - * Unattached: After learning of a new election with a higher epoch + * Unattached: After learning of a new election with a higher epoch * Prospective: After expiration of the fetch timeout - * Follower: After discovering a leader with a larger epoch + * Follower: After discovering a leader with a larger epoch * * Observers follow a simpler state machine. The Voted/Prospective/Candidate/Leader/Resigned * states are not possible for observers, so the only transitions that are possible @@ -75,11 +75,11 @@ * * Unattached transitions to: * Unattached: After learning of a new election with a higher epoch - * Follower: After discovering a leader with an equal or larger epoch + * Follower: After discovering a leader with an equal or larger epoch * * Follower transitions to: * Unattached: After learning of a new election with a higher epoch - * Follower: After discovering a leader with a larger epoch + * Follower: After discovering a leader with a larger epoch * */ public class QuorumState { From 98e64ebc43d5211dfba7fc8c68d26d4a06c19c7c Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Wed, 11 Dec 2024 11:24:27 -0800 Subject: [PATCH 05/23] fix compilation after rebase --- .../org/apache/kafka/raft/KafkaRaftClient.java | 9 --------- .../kafka/raft/KafkaRaftClientPreVoteTest.java | 4 ++-- .../apache/kafka/raft/KafkaRaftClientTest.java | 18 +++++++++--------- .../kafka/raft/RaftClientTestContext.java | 2 +- 4 files changed, 12 insertions(+), 21 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index b99b5d37cb718..7836112ab49f0 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -1013,15 +1013,6 @@ private void handleVoteResponse(VotingState state, VoteResponseData.PartitionData partitionResponse, int remoteNodeId, long currentTimeMs) { - if (quorum.isProspective() && !partitionResponse.preVote()) { - logger.error("Ignoring vote response {} since we sent a PreVote request but received a non-PreVote " + - "response in epoch {} from {}.", - partitionResponse, quorum.epoch(), remoteNodeId); - } else if (quorum.isCandidate() && partitionResponse.preVote()) { - logger.debug("Ignoring vote response {} since we already became a Candidate for epoch {} but " + - "received a PreVote response from {}", - partitionResponse, quorum.epoch(), remoteNodeId); - } if (partitionResponse.voteGranted()) { state.recordGrantedVote(remoteNodeId); maybeTransitionForward(state, currentTimeMs); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 623bdd867753b..3bb2fe00c2a16 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -760,7 +760,7 @@ public void testProspectiveTransitionsToUnattachedOnElectionFailure() throws Exc context.deliverResponse( voteRequest.correlationId(), voteRequest.destination(), - context.voteResponse(false, OptionalInt.empty(), epoch, true)); + context.voteResponse(false, OptionalInt.empty(), epoch)); // handle vote response and mark we should transition out of prospective context.client.poll(); // transition @@ -795,7 +795,7 @@ public void testPreVoteResponseIncludesLeaderId() throws Exception { assertEquals(2, voteRequests.size()); // Simulate PreVote response with granted=false and a leaderId - VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.of(leader.id()), epoch, false); + VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.of(leader.id()), epoch); context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index e0948177ade36..d45e9c20e149c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -878,7 +878,7 @@ public void testInitializeAsCandidateAndBecomeLeader(boolean withKip853Rpc) thro context.deliverResponse( request.correlationId(), request.destination(), - context.voteResponse(true, OptionalInt.empty(), 1, false) + context.voteResponse(true, OptionalInt.empty(), 1) ); // Become leader after receiving the vote @@ -925,7 +925,7 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKi context.deliverResponse( request.correlationId(), request.destination(), - context.voteResponse(true, OptionalInt.empty(), 1, false) + context.voteResponse(true, OptionalInt.empty(), 1) ); VoteRequestData voteRequest = (VoteRequestData) request.data(); @@ -1417,7 +1417,7 @@ public void testVoteRequestTimeout(boolean withKip853Rpc) throws Exception { context.deliverResponse( request.correlationId(), request.destination(), - context.voteResponse(true, OptionalInt.empty(), 1, false) + context.voteResponse(true, OptionalInt.empty(), 1) ); context.client.poll(); context.assertVotedCandidate(epoch, localId); @@ -1426,7 +1426,7 @@ public void testVoteRequestTimeout(boolean withKip853Rpc) throws Exception { context.deliverResponse( retryRequest.correlationId(), retryRequest.destination(), - context.voteResponse(true, OptionalInt.empty(), 1, false) + context.voteResponse(true, OptionalInt.empty(), 1) ); context.client.poll(); context.assertElectedLeader(epoch, localId); @@ -1684,7 +1684,7 @@ public void testRetryElection(boolean withKip853Rpc) throws Exception { context.deliverResponse( request.correlationId(), request.destination(), - context.voteResponse(false, OptionalInt.empty(), 1, false) + context.voteResponse(false, OptionalInt.empty(), 1) ); context.client.poll(); @@ -2721,13 +2721,13 @@ public void testVoteResponseIgnoredAfterBecomingFollower(boolean withDynamicReco assertEquals(2, voteRequests.size()); // Become candidate after PreVote requests are granted - VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch, true); + VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch); context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), voteResponse1 ); - VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch, true); + VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch); context.deliverResponse( voteRequests.get(1).correlationId(), voteRequests.get(1).destination(), @@ -2753,14 +2753,14 @@ public void testVoteResponseIgnoredAfterBecomingFollower(boolean withDynamicReco context.assertElectedLeader(epoch + 1, voter3); // The vote requests now return and should be ignored - VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch + 1, false); + VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch + 1); context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), voteResponse1 ); - VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch + 1, false); + VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch + 1); context.deliverResponse( voteRequests.get(1).correlationId(), voteRequests.get(1).destination(), diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 0a8650bf45290..e3b4e72703fdb 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -589,7 +589,7 @@ void expectAndGrantVotes(int epoch, boolean preVote) throws Exception { log.lastFetchedEpoch(), log.endOffset().offset()); for (RaftRequest.Outbound request : voteRequests) { - VoteResponseData voteResponse = voteResponse(true, OptionalInt.empty(), epoch, preVote); + VoteResponseData voteResponse = voteResponse(true, OptionalInt.empty(), epoch); deliverResponse(request.correlationId(), request.destination(), voteResponse); } From cc6fe6a9c5711adbbe34055ca26e80d6054f9b62 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Wed, 11 Dec 2024 22:26:06 -0800 Subject: [PATCH 06/23] add raftProtocol testing matrix, need to fix quorumstatetest next --- .../apache/kafka/raft/KafkaRaftClient.java | 18 +- .../org/apache/kafka/raft/QuorumState.java | 17 +- .../raft/KafkaRaftClientPreVoteTest.java | 312 +++++++++++++----- .../raft/KafkaRaftClientReconfigTest.java | 76 ++--- .../raft/KafkaRaftClientSnapshotTest.java | 42 +-- .../kafka/raft/KafkaRaftClientTest.java | 303 +++++++++-------- .../kafka/raft/ProspectiveStateTest.java | 222 ++++++++++--- .../raft/ProspectiveStateWithVoteTest.java | 13 +- .../kafka/raft/RaftClientTestContext.java | 166 +++++++--- 9 files changed, 747 insertions(+), 422 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 7836112ab49f0..73a7511d7e423 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -719,14 +719,6 @@ private void transitionToProspective(long currentTimeMs) { onBecomeProspective(currentTimeMs); } - private void transitionToVotingState(long currentTimeMs) { - if (partitionState.lastKraftVersion().isReconfigSupported()) { - transitionToProspective(currentTimeMs); - } else { - transitionToCandidate(currentTimeMs); - } - } - private void transitionToUnattached(int epoch) { transitionToUnattached(epoch, OptionalInt.empty()); } @@ -3015,7 +3007,7 @@ private long pollResigned(long currentTimeMs) { stateTimeoutMs = shutdown.remainingTimeMs(); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { if (quorum.isVoter()) { - transitionToVotingState(currentTimeMs); + transitionToProspective(currentTimeMs); } else { // It is possible that the old leader is not a voter in the new voter set. // In that case increase the epoch and transition to unattached. The epoch needs @@ -3132,8 +3124,8 @@ private long pollCandidate(long currentTimeMs) { return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); } else if (state.isBackingOff()) { if (state.isBackoffComplete(currentTimeMs)) { - logger.info("Re-elect as candidate after election backoff has completed"); - transitionToCandidate(currentTimeMs); + logger.info("Transition to prospective after election backoff has completed"); + transitionToProspective(currentTimeMs); return 0L; } return state.remainingBackoffMs(currentTimeMs); @@ -3200,7 +3192,7 @@ private long pollFollowerAsVoter(FollowerState state, long currentTimeMs) { backoffMs = 0; } else if (state.hasFetchTimeoutExpired(currentTimeMs)) { logger.info("Transitioning to VotingState (Prospective or Candidate) due to fetch timeout"); - transitionToVotingState(currentTimeMs); + transitionToProspective(currentTimeMs); backoffMs = 0; } else if (state.hasUpdateVoterPeriodExpired(currentTimeMs)) { if (partitionState.lastKraftVersion().isReconfigSupported() && @@ -3304,7 +3296,7 @@ private long pollUnattachedAsVoter(UnattachedState state, long currentTimeMs) { // shutdown completes or an epoch bump forces another state transition return shutdown.remainingTimeMs(); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { - transitionToVotingState(currentTimeMs); + transitionToProspective(currentTimeMs); return 0L; } else { return pollUnattachedCommon(state, currentTimeMs); diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index cc23d4a8903fc..9dac5faf3d56d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -575,10 +575,6 @@ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { } public void transitionToProspective() { - if (!partitionState.lastKraftVersion().isReconfigSupported()) { - throw new IllegalStateException("Cannot transition to Prospective since the current version " + - partitionState.lastKraftVersion() + " does not support PreVote"); - } if (isObserver()) { throw new IllegalStateException( String.format( @@ -640,16 +636,9 @@ private void checkValidTransitionToCandidate() { ) ); } - if (partitionState.lastKraftVersion().isReconfigSupported()) { - if (!isProspective()) { - throw new IllegalStateException("Cannot transition to Candidate since the local broker.id=" + localId + - " is state " + state); - } - } else { - if (isLeader()) { - throw new IllegalStateException("Cannot transition to Candidate since the local broker.id=" + localId + - " since this node is already a Leader with state " + state); - } + if (!isProspective() && !isOnlyVoter()) { + throw new IllegalStateException("Cannot transition to Candidate since the local broker.id=" + localId + + " is state " + state); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 3bb2fe00c2a16..eca716da85bbb 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -21,9 +21,11 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; - +import org.apache.kafka.raft.RaftClientTestContext.RaftProtocol; +import org.apache.kafka.server.common.KRaftVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.ValueSource; import java.util.Arrays; @@ -36,6 +38,7 @@ import static org.apache.kafka.raft.KafkaRaftClientTest.randomReplicaId; import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_996_PROTOCOL; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -52,7 +55,7 @@ public void testHandlePreVoteRequestAsFollowerWithElectedLeader(boolean hasFetch RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, electedLeaderId) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); if (hasFetchedFromLeader) { @@ -91,6 +94,27 @@ public void testHandlePreVoteRequestAsFollowerWithElectedLeader(boolean hasFetch assertTrue(context.client.quorum().isUnattachedNotVoted()); } + @Test + public void testHandlePreVoteRequestAsFollowerWithVotedCandidate() throws Exception { + int localId = randomReplicaId(); + int epoch = 2; + ReplicaKey otherNodeKey = replicaKey(localId + 1, true); + ReplicaKey votedCandidateKey = replicaKey(localId + 2, true); + Set voters = Set.of(localId, otherNodeKey.id(), votedCandidateKey.id()); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withVotedCandidate(epoch, votedCandidateKey) + .withRaftProtocol(KIP_996_PROTOCOL) + .build(); + + context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, epoch, 1)); + context.pollUntilResponse(); + + // follower can grant pre-votes if it has not fetched successfully from leader yet + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); + context.assertVotedCandidate(epoch, votedCandidateKey.id()); + } + @Test public void testHandlePreVoteRequestAsCandidate() throws Exception { int localId = randomReplicaId(); @@ -101,7 +125,7 @@ public void testHandlePreVoteRequestAsCandidate() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); assertTrue(context.client.quorum().isCandidate()); @@ -140,7 +164,7 @@ public void testHandlePreVoteRequestAsUnattachedObserver() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); context.deliverRequest(context.preVoteRequest(epoch, replica1, epoch, 1)); @@ -182,7 +206,7 @@ public void testHandlePreVoteRequestAsUnattachedVoted() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withVotedCandidate(epoch, replica2) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); context.deliverRequest(context.preVoteRequest(epoch, replica1, epoch, 1)); @@ -225,7 +249,7 @@ public void testHandlePreVoteRequestAsUnattachedWithLeader() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, leader.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); context.deliverRequest(context.preVoteRequest(epoch, replica1, epoch, 1)); @@ -267,7 +291,7 @@ public void testHandlePreVoteRequestAsFollowerObserver(boolean hasFetchedFromLea RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) .withElectedLeader(epoch, leader.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, leader, follower)))) .build(); context.assertElectedLeader(epoch, leader.id()); @@ -301,14 +325,14 @@ public void testHandleInvalidPreVoteRequestWithOlderEpoch() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); context.deliverRequest(context.preVoteRequest(epoch - 1, otherNodeKey, epoch - 1, 1)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.empty(), false); - context.assertUnknownLeader(epoch); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); } @Test @@ -319,11 +343,11 @@ public void testLeaderRejectPreVoteRequestOnSameEpoch() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) .withUnknownLeader(2) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int leaderEpoch = context.currentEpoch(); context.deliverRequest(context.preVoteRequest(leaderEpoch, otherNodeKey, leaderEpoch, 1)); @@ -341,11 +365,11 @@ public void testPreVoteRequestClusterIdValidation() throws Exception { ReplicaKey otherNodeKey = replicaKey(localId + 1, true); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -376,11 +400,11 @@ public void testInvalidVoterReplicaPreVoteRequest() throws Exception { ReplicaKey otherNodeKey = replicaKey(localId + 1, true); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // invalid voter id is rejected @@ -422,11 +446,11 @@ public void testLeaderAcceptPreVoteFromObserver() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) .withUnknownLeader(4) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey observerKey = replicaKey(localId + 2, true); @@ -446,9 +470,9 @@ public void testHandlePreVoteRequestAsResigned() throws Exception { Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.quorum().transitionToResigned(Collections.emptyList()); assertTrue(context.client.quorum().isResigned()); @@ -466,7 +490,7 @@ public void testHandlePreVoteRequestAsResigned() throws Exception { } @Test - public void testInvalidVoteRequest() throws Exception { + public void testInvalidPreVoteRequest() throws Exception { int localId = randomReplicaId(); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); int epoch = 5; @@ -474,7 +498,7 @@ public void testInvalidVoteRequest() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeKey.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); assertEquals(epoch, context.currentEpoch()); context.assertElectedLeader(epoch, otherNodeKey.id()); @@ -513,18 +537,26 @@ public void testInvalidVoteRequest() throws Exception { context.assertElectedLeader(epoch, otherNodeKey.id()); } - @Test - public void testFollowerGrantsPreVoteIfHasNotFetchedYet() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerGrantsPreVoteIfHasNotFetchedYet(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); - int epoch = 2; + ReplicaKey local = replicaKey(localId, true); ReplicaKey replica1 = replicaKey(localId + 1, true); ReplicaKey replica2 = replicaKey(localId + 2, true); - Set voters = Set.of(replica1.id(), replica2.id()); + int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(replica1, replica2))), + kraftVersion + ) .withElectedLeader(epoch, replica1.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); + + context.assertElectedLeader(epoch, replica1.id()); + assertTrue(context.client.quorum().isFollower()); // We will grant PreVotes before fetching successfully from the leader, it will NOT contain the leaderId @@ -552,6 +584,7 @@ public void testFollowerGrantsPreVoteIfHasNotFetchedYet() throws Exception { assertTrue(context.client.quorum().isFollower()); } +<<<<<<< HEAD @Test public void testRejectPreVoteIfRemoteLogIsNotUpToDate() throws Exception { int localId = randomReplicaId(); @@ -583,21 +616,25 @@ public void testRejectPreVoteIfRemoteLogIsNotUpToDate() throws Exception { context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), false); } - @Test - public void testPreVoteResponseIgnoredAfterBecomingFollower() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testPreVoteResponseIgnoredAfterBecomingFollower(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey voter2 = replicaKey(localId + 1, true); ReplicaKey voter3 = replicaKey(localId + 2, true); int epoch = 5; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(local, voter2, voter3))), + kraftVersion + ) .withUnknownLeader(epoch) - .withKip853Rpc(true) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, voter2, voter3)))) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - context.assertUnknownLeader(epoch); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); // Sleep a little to ensure that we become a prospective context.time.sleep(context.electionTimeoutMs() * 2L); @@ -632,74 +669,94 @@ public void testPreVoteResponseIgnoredAfterBecomingFollower() throws Exception { context.assertElectedLeader(epoch, voter3.id()); } - @Test - public void testStaticQuorumDoesNotSendPreVoteRequest() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testPreVoteNotSupportedByRemote(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); - ReplicaKey voter2Bootstrap = replicaKey(localId + 1, true); + ReplicaKey voter2Key = replicaKey(localId + 1, true); + ReplicaKey voter3Key = replicaKey(localId + 2, true); int epoch = 5; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(local, voter2Key, voter3Key))), + kraftVersion + ) .withUnknownLeader(epoch) - .withStaticVoters(VoterSetTest.voterSet(Stream.of(local, voter2Bootstrap))) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - context.assertUnknownLeader(epoch); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); - // Sleep a little to ensure that we transition to a Voting state + // Sleep a little to ensure that we transition to Prospective context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); + assertEquals(epoch, context.currentEpoch()); + assertTrue(context.client.quorum().isProspective()); - // We should transition to Candidate state - assertTrue(context.client.quorum().isCandidate()); - - // Candidate state should not send PreVote requests - List voteRequests = context.collectVoteRequests(epoch + 1, 0, 0); - assertEquals(1, voteRequests.size()); - context.assertVotedCandidate(epoch + 1, localId); - } - - @Test - public void testPreVoteNotSupportedByRemote() throws Exception { - int localId = randomReplicaId(); - ReplicaKey local = replicaKey(localId, true); - Uuid localDirectoryId = local.directoryId().orElse(Uuid.randomUuid()); - int voter2 = localId + 1; - ReplicaKey voter2Key = replicaKey(voter2, true); - int epoch = 5; - - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localDirectoryId) - .withUnknownLeader(epoch) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, voter2Key)))) - .build(); + // Simulate one remote node not supporting PreVote with UNSUPPORTED_VERSION response. + // Note: with the mocked network client we simulate this is a bit differently, in reality this response would + // be generated from the network client and not sent from the remote node. + List voteRequests = context.collectPreVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + RaftUtil.errorResponse(ApiKeys.VOTE, Errors.UNSUPPORTED_VERSION) + ); - context.assertUnknownLeader(epoch); + // Local should transition to Candidate since it realizes remote node does not support PreVote. + context.client.poll(); + assertEquals(epoch + 1, context.currentEpoch()); + context.client.quorum().isCandidate(); - // Sleep a little to ensure that we transition to Prospective - context.time.sleep(context.electionTimeoutMs() * 2L); - context.pollUntilRequest(); + // Any further PreVote requests should be ignored + context.deliverResponse( + voteRequests.get(1).correlationId(), + voteRequests.get(1).destination(), + context.voteResponse(true, OptionalInt.empty(), epoch) + ); + context.client.poll(); + assertEquals(epoch + 1, context.currentEpoch()); + context.client.quorum().isCandidate(); + // Sleep to transition back to Prospective + context.time.sleep(context.client.quorum().candidateStateOrThrow().remainingElectionTimeMs(context.time.milliseconds())); + context.client.poll(); + context.time.sleep(context.client.quorum().candidateStateOrThrow().remainingBackoffMs(context.time.milliseconds())); + context.collectVoteRequests(epoch + 1, 0, 0); + context.client.poll(); + assertEquals(epoch + 1, context.currentEpoch()); assertTrue(context.client.quorum().isProspective()); - // Simulate remote node not supporting PreVote with UNSUPPORTED_VERSION response. - // Note: with the mocked network client this is a bit different, in reality this response would be generated - // from the network client and not sent from the remote node. - List voteRequests = context.collectVoteRequests(epoch, 0, 0); - assertEquals(1, voteRequests.size()); + // Simulate receiving enough valid PreVote responses for election to succeed + context.pollUntilRequest(); + voteRequests = context.collectPreVoteRequests(epoch + 1, 0, 0); + assertEquals(2, voteRequests.size()); context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), - RaftUtil.errorResponse(ApiKeys.VOTE, Errors.UNSUPPORTED_VERSION) + context.voteResponse(true, OptionalInt.empty(), epoch + 1) ); context.client.poll(); + assertEquals(epoch + 2, context.currentEpoch()); + context.client.quorum().isCandidate(); - // Local should transition to Candidate since it realizes remote node does not support PreVote. - assertEquals(epoch + 1, context.currentEpoch()); + // Any further PreVote requests should be ignored + context.deliverResponse( + voteRequests.get(1).correlationId(), + voteRequests.get(1).destination(), + RaftUtil.errorResponse(ApiKeys.VOTE, Errors.UNSUPPORTED_VERSION) + ); + context.client.poll(); + assertEquals(epoch + 2, context.currentEpoch()); context.client.quorum().isCandidate(); } - @Test - public void testProspectiveReceivesBeginQuorumRequest() throws Exception { + @ParameterizedTest + @EnumSource(value = RaftProtocol.class) + public void testProspectiveReceivesBeginQuorumRequest(RaftProtocol raftProtocol) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey leader = replicaKey(localId + 1, true); @@ -707,11 +764,11 @@ public void testProspectiveReceivesBeginQuorumRequest() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) .withUnknownLeader(epoch) - .withKip853Rpc(true) + .withRaftProtocol(raftProtocol) .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, leader)))) .build(); - context.assertUnknownLeader(epoch); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); // Sleep a little to ensure that we transition to Prospective context.time.sleep(context.electionTimeoutMs() * 2L); @@ -735,10 +792,10 @@ public void testProspectiveTransitionsToUnattachedOnElectionFailure() throws Exc RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) .withUnknownLeader(epoch) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, otherNode)))) .build(); - context.assertUnknownLeader(epoch); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); // Sleep a little to ensure that we transition to Prospective context.time.sleep(context.electionTimeoutMs() * 2L); @@ -766,10 +823,97 @@ public void testProspectiveTransitionsToUnattachedOnElectionFailure() throws Exc // transition context.client.poll(); assertTrue(context.client.quorum().isUnattached()); + + // After election times out again, we will transition back to Prospective and continue sending PreVote requests + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + voteRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); + + // If we receive vote response with different leaderId but empty leader endpoint, we will transition to + // unattached with leader immediately + context.deliverResponse( + voteRequest.correlationId(), + voteRequest.destination(), + context.voteResponse(false, OptionalInt.of(localId + 2), epoch + 1)); + context.client.poll(); + assertTrue(context.client.quorum().isUnattached()); + assertEquals(epoch + 1, context.currentEpoch()); + context.assertElectedLeader(epoch + 1, localId + 2); } @Test - public void testPreVoteResponseIncludesLeaderId() throws Exception { + public void testProspectiveWithLeaderTransitionsToFollower() throws Exception { + int localId = randomReplicaId(); + ReplicaKey local = replicaKey(localId, true); + ReplicaKey replica1 = replicaKey(localId + 1, true); + ReplicaKey replica2 = replicaKey(localId + 2, true); + int epoch = 5; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + .withElectedLeader(epoch, replica1.id()) + .withRaftProtocol(KIP_996_PROTOCOL) + .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, replica1, replica2)))) + .build(); + context.assertElectedLeader(epoch, replica1.id()); + assertTrue(context.client.quorum().isFollower()); + + // Sleep a little to ensure that we transition to Prospective + context.time.sleep(context.fetchTimeoutMs); + context.pollUntilRequest(); + assertTrue(context.client.quorum().isProspective()); + context.assertSentVoteRequest(epoch, 0, 0L, 2); + + // If election timeout expires, we should transition back to Follower if we haven't found new leader yet + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + context.assertSentFetchRequest(); + assertTrue(context.client.quorum().isFollower()); + context.assertElectedLeader(epoch, replica1.id()); + + // After election times out again, we will transition back to Prospective and continue sending PreVote requests + context.time.sleep(context.fetchTimeoutMs); + context.pollUntilRequest(); + List voteRequests = context.collectVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + assertTrue(context.client.quorum().isProspective()); + context.assertElectedLeader(epoch, replica1.id()); + + // If we receive enough rejected votes without leaderId, we also transition to Follower immediately + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + context.voteResponse(false, OptionalInt.empty(), epoch)); + context.deliverResponse( + voteRequests.get(1).correlationId(), + voteRequests.get(1).destination(), + context.voteResponse(false, OptionalInt.empty(), epoch)); + // handle vote response and mark we should transition out of prospective + context.client.poll(); + // transition + context.pollUntilRequest(); + assertTrue(context.client.quorum().isFollower()); + context.assertSentFetchRequest(); + + // After election times out again, we will transition back to Prospective and continue sending PreVote requests + context.time.sleep(context.fetchTimeoutMs); + context.pollUntilRequest(); + voteRequests = context.collectVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + assertTrue(context.client.quorum().isProspective()); + context.assertElectedLeader(epoch, replica1.id()); + + // If we receive vote response with different leaderId, we will transition to follower immediately + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + context.voteResponse(false, OptionalInt.of(replica2.id()), epoch + 1)); + context.client.poll(); + assertTrue(context.client.quorum().isFollower()); + context.assertElectedLeader(epoch + 1, replica2.id()); + } + + @Test + public void testProspectiveWithoutLeaderTransitionsToFollower() throws Exception { ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey leader = replicaKey(local.id() + 1, true); ReplicaKey follower = replicaKey(local.id() + 2, true); @@ -778,12 +922,12 @@ public void testPreVoteResponseIncludesLeaderId() throws Exception { VoterSet voters = VoterSetTest.voterSet(Stream.of(local, leader, follower)); RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .withBootstrapSnapshot(Optional.of(voters)) .withUnknownLeader(epoch) .build(); - context.assertUnknownLeader(epoch); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); // Sleep a little to ensure that we transition to Prospective context.time.sleep(context.electionTimeoutMs() * 2L); @@ -794,7 +938,7 @@ public void testPreVoteResponseIncludesLeaderId() throws Exception { List voteRequests = context.collectVoteRequests(epoch, 0, 0); assertEquals(2, voteRequests.size()); - // Simulate PreVote response with granted=false and a leaderId + // Simulate PreVote response with granted=true and a leaderId VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.of(leader.id()), epoch); context.deliverResponse( voteRequests.get(0).correlationId(), diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index 8dddba6a10dc0..493083831d1de 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -121,7 +121,7 @@ public void testLeaderWritesBootstrapRecords() throws Exception { SnapshotWriterReaderTest.assertControlSnapshot(expectedBootstrapRecords, reader); } - context.becomeLeader(); + context.unattachedToLeader(); // check if leader writes 3 bootstrap records to the log Records records = context.log.read(0, Isolation.UNCOMMITTED).records; @@ -155,7 +155,7 @@ public void testBootstrapCheckpointIsNotReturnedOnFetch() throws Exception { .withUnknownLeader(0) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // check that leader does not respond with bootstrap snapshot id when follower fetches offset 0 @@ -203,7 +203,7 @@ public void testLeaderDoesNotBootstrapRecordsWithKraftVersion0() throws Exceptio ); // check leader does not write bootstrap records to log - context.becomeLeader(); + context.unattachedToLeader(); Records records = context.log.read(0, Isolation.UNCOMMITTED).records; RecordBatch batch = records.batches().iterator().next(); @@ -333,7 +333,7 @@ public void testAddVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -412,7 +412,7 @@ void testAddVoterInvalidClusterId() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); InetSocketAddress newAddress = InetSocketAddress.createUnresolved( @@ -477,7 +477,7 @@ void testAddVoterWithMissingDefaultListener() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); InetSocketAddress newAddress = InetSocketAddress.createUnresolved( @@ -507,7 +507,7 @@ void testAddVoterWithPendingAddVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -563,7 +563,7 @@ void testAddVoterWithoutFencedPreviousLeaders() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -601,7 +601,7 @@ void testAddVoterWithKraftVersion0() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -646,7 +646,7 @@ void testAddVoterWithExistingVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(follower.id(), true); @@ -691,7 +691,7 @@ void testAddVoterTimeout() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -763,7 +763,7 @@ void testAddVoterWithApiVersionsFromIncorrectNode() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -823,7 +823,7 @@ void testAddVoterInvalidFeatureVersion() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -883,7 +883,7 @@ void testAddVoterWithLaggingNewVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -936,7 +936,7 @@ void testAddVoterFailsWhenLosingLeadership() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -985,7 +985,7 @@ void testAddVoterWithMissingDirectoryId() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, false); @@ -1024,7 +1024,7 @@ public void testRemoveVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertTrue(context.client.quorum().isVoter(follower2)); @@ -1073,7 +1073,7 @@ public void testRemoveVoterIsLeader() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1145,7 +1145,7 @@ public void testRemoveVoterInvalidClusterId() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); // empty cluster id is rejected context.deliverRequest(context.removeVoterRequest("", follower1)); @@ -1194,7 +1194,7 @@ void testRemoveVoterWithPendingRemoveVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1232,7 +1232,7 @@ void testRemoveVoterWithoutFencedPreviousLeaders() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); // Attempt to remove follower2 context.deliverRequest(context.removeVoterRequest(follower2)); @@ -1254,7 +1254,7 @@ void testRemoveVoterWithKraftVersion0() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1284,7 +1284,7 @@ void testRemoveVoterWithNoneVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1314,7 +1314,7 @@ void testRemoveVoterWithNoneVoterId() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1366,7 +1366,7 @@ void testRemoveVoterTimedOut() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1410,7 +1410,7 @@ void testRemoveVoterFailsWhenLosingLeadership() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1452,7 +1452,7 @@ void testAddVoterWithPendingRemoveVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1497,7 +1497,7 @@ void testRemoveVoterWithPendingAddVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -1545,7 +1545,7 @@ void testUpdateVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertTrue(context.client.quorum().isVoter(follower)); @@ -1615,7 +1615,7 @@ void testLeaderUpdatesVoter() throws Exception { .withLocalListeners(localListeners) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertTrue(context.client.quorum().isVoter(follower)); @@ -1650,7 +1650,7 @@ public void testUpdateVoterInvalidClusterId() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // empty cluster id is rejected @@ -1701,7 +1701,7 @@ void testUpdateVoterOldEpoch() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -1734,7 +1734,7 @@ void testUpdateVoterNewEpoch() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -1796,7 +1796,7 @@ void testUpdateVoterWithoutFencedPreviousLeaders() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Attempt to update the follower @@ -1841,7 +1841,7 @@ void testUpdateVoterWithKraftVersion0() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1892,7 +1892,7 @@ void testUpdateVoterWithNoneVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1943,7 +1943,7 @@ void testUpdateVoterWithNoneVoterId() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1994,7 +1994,7 @@ void testUpdateVoterWithPendingAddVoter() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 6ef692229af1b..66b83b0035f89 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -114,7 +114,7 @@ public void testLeaderListenerNotified(boolean entireLog, boolean withKip853Rpc) RaftClientTestContext context = contextBuilder.build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Advance the highWatermark @@ -236,7 +236,7 @@ public void testListenerRenotified(boolean withKip853Rpc) throws Exception { .deleteBeforeSnapshot(snapshotId) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Stop the listener from reading commit batches @@ -293,7 +293,7 @@ public void testLeaderImmediatelySendsSnapshotId(boolean withKip853Rpc) throws E .deleteBeforeSnapshot(snapshotId) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Send a fetch request for an end offset and epoch which has been snapshotted @@ -321,7 +321,7 @@ public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); List appendRecords = Arrays.asList("a", "b", "c"); @@ -371,7 +371,7 @@ public void testFetchRequestOffsetAtZero(boolean withKip853Rpc) throws Exception .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); List appendRecords = Arrays.asList("a", "b", "c"); @@ -423,7 +423,7 @@ public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) th .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 1, epoch); @@ -465,7 +465,7 @@ public void testFetchRequestTruncateToLogStart(boolean withKip853Rpc) throws Exc .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch); @@ -516,7 +516,7 @@ public void testFetchRequestAtLogStartOffsetWithValidEpoch(boolean withKip853Rpc .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch); @@ -562,7 +562,7 @@ public void testFetchRequestAtLogStartOffsetWithInvalidEpoch(boolean withKip853R .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch); @@ -616,7 +616,7 @@ public void testFetchRequestWithLastFetchedEpochLessThanOldestSnapshot( .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch); @@ -660,7 +660,7 @@ public void testFetchSnapshotRequestMissingSnapshot(boolean withKip853Rpc) throw .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -693,7 +693,7 @@ public void testFetchSnapshotRequestBootstrapSnapshot() throws Exception { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -724,7 +724,7 @@ public void testFetchSnapshotRequestUnknownPartition(boolean withKip853Rpc) thro .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -756,7 +756,7 @@ public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Excep .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.advanceLocalLeaderHighWatermarkToLogEndOffset(); @@ -813,7 +813,7 @@ public void testLeaderShouldResignLeadershipIfNotGetFetchSnapshotRequestFromMajo .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); FetchSnapshotRequestData voter1FetchSnapshotRequest = fetchSnapshotRequest( @@ -902,7 +902,7 @@ public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throw .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.advanceLocalLeaderHighWatermarkToLogEndOffset(); @@ -1015,7 +1015,7 @@ public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) t .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.advanceLocalLeaderHighWatermarkToLogEndOffset(); @@ -1074,7 +1074,7 @@ public void testFetchSnapshotRequestWithOlderEpoch(boolean withKip853Rpc) throws .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -1107,7 +1107,7 @@ public void testFetchSnapshotRequestWithNewerEpoch(boolean withKip853Rpc) throws .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -1828,7 +1828,7 @@ public void testFetchSnapshotRequestClusterIdValidation( .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -1909,7 +1909,7 @@ public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(boolean withKip853Rp .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int currentEpoch = context.currentEpoch(); // When leader creating snapshot: diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index d45e9c20e149c..d078350d04a5c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -46,6 +46,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; @@ -70,6 +71,7 @@ import static java.util.Collections.singletonList; import static org.apache.kafka.raft.RaftClientTestContext.Builder.DEFAULT_ELECTION_TIMEOUT_MS; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol; import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -270,7 +272,7 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Excepti .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Beginning shutdown @@ -323,7 +325,11 @@ public void testInitializeAsResignedAndBecomeCandidate(boolean withKip853Rpc) th context.time.sleep(context.electionTimeoutMs()); context.client.poll(); - // Become candidate in a new epoch + // Become prospective + assertTrue(context.client.quorum().isProspective()); + + // Receive granted PreVote and become candidate + context.expectAndGrantPreVotes(context.currentEpoch()); assertTrue(context.client.quorum().isCandidate()); context.assertVotedCandidate(epoch + 1, localId); } @@ -359,6 +365,7 @@ public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) context.client.poll(); context.time.sleep(context.electionTimeoutMs()); + context.expectAndGrantPreVotes(epoch); context.pollUntilRequest(); context.assertVotedCandidate(epoch + 1, localId); context.assertSentVoteRequest(epoch + 1, 0, 0L, 1); @@ -399,7 +406,7 @@ public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -418,7 +425,7 @@ public void testAppendFailedWithFencedEpoch(boolean withKip853Rpc) throws Except .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -439,7 +446,7 @@ public void testAppendFailedWithRecordBatchTooLargeException(boolean withKip853R .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -512,7 +519,7 @@ public void testResignWillCompleteFetchPurgatory(boolean withKip853Rpc) throws E .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); // send fetch request when become leader @@ -550,7 +557,7 @@ public void testResignInOlderEpochIgnored(boolean withKip853Rpc) throws Exceptio .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int currentEpoch = context.currentEpoch(); @@ -577,7 +584,7 @@ public void testHandleBeginQuorumEpochAfterUserInitiatedResign( .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int resignedEpoch = context.currentEpoch(); @@ -605,7 +612,7 @@ public void testBeginQuorumEpochHeartbeat(boolean withKip853Rpc) throws Exceptio .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(OptionalInt.of(localId), context.currentLeader()); @@ -641,7 +648,7 @@ public void testLeaderShouldResignLeadershipIfNotGetFetchRequestFromMajorityVote .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(OptionalInt.of(localId), context.currentLeader()); @@ -714,7 +721,7 @@ public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) t .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int resignedEpoch = context.currentEpoch(); @@ -748,11 +755,11 @@ public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) t OptionalInt.of(localId) ); - // After the election timer, we should become a candidate. + // After the election timer, we should become a prospective. context.time.sleep(2L * context.electionTimeoutMs()); - context.pollUntil(context.client.quorum()::isCandidate); - assertEquals(resignedEpoch + 1, context.currentEpoch()); - assertEquals(new LeaderAndEpoch(OptionalInt.empty(), resignedEpoch + 1), + context.pollUntil(context.client.quorum()::isProspective); + assertEquals(resignedEpoch, context.currentEpoch()); + assertEquals(new LeaderAndEpoch(OptionalInt.of(localId), resignedEpoch), context.listener.currentLeaderAndEpoch()); } @@ -766,7 +773,7 @@ public void testCannotResignWithLargerEpochThanCurrentEpoch(boolean withKip853Rp RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertThrows(IllegalArgumentException.class, () -> context.client.resign(context.currentEpoch() + 1)); @@ -839,7 +846,7 @@ public void testInitializeAsCandidateFromStateStore(boolean withKip853Rpc) throw @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testInitializeAsCandidateAndBecomeLeader(boolean withKip853Rpc) throws Exception { + public void testInitializeAsUnattachedAndBecomeLeader(boolean withKip853Rpc) throws Exception { final int localId = randomReplicaId(); final int otherNodeId = localId + 1; Set voters = Set.of(localId, otherNodeId); @@ -847,7 +854,7 @@ public void testInitializeAsCandidateAndBecomeLeader(boolean withKip853Rpc) thro .withKip853Rpc(withKip853Rpc) .build(); - context.assertUnknownLeader(0); + context.assertUnknownLeaderAndNoVotedCandidate(0); context.pollUntilRequest(); RaftRequest.Outbound request = context.assertSentFetchRequest(0, 0L, 0); assertTrue(context.client.quorum().isUnattached()); @@ -866,11 +873,13 @@ public void testInitializeAsCandidateAndBecomeLeader(boolean withKip853Rpc) thro assertTrue(context.client.quorum().isUnattached()); assertTrue(context.client.quorum().isVoter()); - // after election timeout should become candidate + // after election timeout should become prospective context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); - assertTrue(context.client.quorum().isCandidate()); + assertTrue(context.client.quorum().isProspective()); + // after receiving enough granted prevotes, should become candidate + context.expectAndGrantPreVotes(context.currentEpoch()); context.pollUntilRequest(); context.assertVotedCandidate(1, localId); @@ -915,9 +924,9 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKi .withKip853Rpc(withKip853Rpc) .build(); - context.assertUnknownLeader(0); - context.time.sleep(2L * context.electionTimeoutMs()); - + context.assertUnknownLeaderAndNoVotedCandidate(0); + assertTrue(context.client.quorum().isUnattached()); + context.unattachedToCandidate(); context.pollUntilRequest(); context.assertVotedCandidate(1, localId); @@ -1053,9 +1062,12 @@ public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) t .withKip853Rpc(withKip853Rpc) .build(); - // Sleep a little to ensure that we become a candidate + // Sleep a little to ensure that we become a prospective context.time.sleep(context.electionTimeoutMs() + jitterMs); context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + // Become candidate + context.expectAndGrantPreVotes(epoch - 1); context.assertVotedCandidate(epoch, localId); context.deliverRequest( @@ -1079,10 +1091,10 @@ public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) t context.client.poll(); context.assertVotedCandidate(epoch, localId); - // After backoff, we will become a candidate again + // After backoff, we will become prospective again context.time.sleep(context.electionBackoffMaxMs); context.client.poll(); - context.assertVotedCandidate(epoch + 1, localId); + assertTrue(context.client.quorum().isProspective()); } @ParameterizedTest @@ -1097,7 +1109,7 @@ public void testEndQuorumIgnoredAsLeaderIfOlderEpoch(boolean withKip853Rpc) thro .withUnknownLeader(6) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // One of the voters may have sent EndQuorumEpoch from an earlier epoch @@ -1165,7 +1177,7 @@ public void testAccumulatorClearedAfterBecomingFollower(boolean withKip853Rpc) t .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -1198,7 +1210,7 @@ public void testAccumulatorClearedAfterBecomingVoted(boolean withKip853Rpc) thro .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -1232,7 +1244,7 @@ public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -1241,7 +1253,7 @@ public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 0L)); context.pollUntilResponse(); - context.assertUnknownLeader(epoch + 1); + context.assertUnknownLeaderAndNoVotedCandidate(epoch + 1); // Expect two calls one for the leader change control batch and one for the data batch Mockito.verify(memoryPool, Mockito.times(2)).release(buffer); } @@ -1261,7 +1273,7 @@ public void testChannelWokenUpIfLingerTimeoutReachedWithoutAppend(boolean withKi .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); assertEquals(1L, context.log.endOffset().offset()); @@ -1297,7 +1309,7 @@ public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(boolean withKip .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); assertEquals(1L, context.log.endOffset().offset()); @@ -1344,7 +1356,8 @@ public void testHandleEndQuorumRequest(boolean withKip853Rpc) throws Exception { context.assertSentEndQuorumEpochResponse(Errors.NONE, leaderEpoch, OptionalInt.of(oldLeaderId)); context.client.poll(); - context.assertVotedCandidate(leaderEpoch + 1, localId); + assertTrue(context.client.quorum().isProspective()); + context.assertElectedLeader(leaderEpoch, oldLeaderId); } @ParameterizedTest @@ -1383,11 +1396,11 @@ public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(boolean wi context.pollUntilRequest(); - List voteRequests = context.collectVoteRequests(leaderEpoch + 1, 0, 0); + List voteRequests = context.collectVoteRequests(leaderEpoch, 0, 0); assertEquals(2, voteRequests.size()); - // Should have already done self-voting - context.assertVotedCandidate(leaderEpoch + 1, localId); + assertTrue(context.client.quorum().isProspective()); + assertEquals(leaderEpoch, context.currentEpoch()); } @ParameterizedTest @@ -1401,9 +1414,9 @@ public void testVoteRequestTimeout(boolean withKip853Rpc) throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withKip853Rpc(withKip853Rpc) .build(); - context.assertUnknownLeader(0); + context.assertUnknownLeaderAndNoVotedCandidate(0); - context.time.sleep(2L * context.electionTimeoutMs()); + context.unattachedToCandidate(); context.pollUntilRequest(); context.assertVotedCandidate(epoch, localId); @@ -1513,7 +1526,7 @@ public void testHandleInvalidVoteRequestWithOlderEpoch(boolean withKip853Rpc) th context.pollUntilResponse(); context.assertSentVoteResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.empty(), false); - context.assertUnknownLeader(epoch); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); } @ParameterizedTest @@ -1549,7 +1562,7 @@ public void testLeaderIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int leaderEpoch = context.currentEpoch(); context.deliverRequest(context.voteRequest(leaderEpoch, otherNodeKey, leaderEpoch - 1, 1)); @@ -1572,7 +1585,7 @@ public void testListenerCommitCallbackAfterLeaderWrite(boolean withKip853Rpc) th .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // First poll has no high watermark advance @@ -1622,7 +1635,7 @@ public void testLeaderImmediatelySendsDivergingEpoch(boolean withKip853Rpc) thro .build(); // Start off as the leader - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Send a fetch request for an end offset and epoch which has diverged @@ -1673,9 +1686,9 @@ public void testRetryElection(boolean withKip853Rpc) throws Exception { .withKip853Rpc(withKip853Rpc) .build(); - context.assertUnknownLeader(0); + context.assertUnknownLeaderAndNoVotedCandidate(0); - context.time.sleep(2L * context.electionTimeoutMs()); + context.unattachedToCandidate(); context.pollUntilRequest(); context.assertVotedCandidate(epoch, localId); @@ -1693,17 +1706,17 @@ public void testRetryElection(boolean withKip853Rpc) throws Exception { context.assertVotedCandidate(epoch, localId); // Even though our candidacy was rejected, we will backoff for jitter period - // before we bump the epoch and start a new election. + // before we transition to prospective and start a new election. context.time.sleep(context.electionBackoffMaxMs - 1); context.client.poll(); context.assertVotedCandidate(epoch, localId); - // After jitter expires, we become a candidate again + // After jitter expires, we become a prospective again context.time.sleep(1); context.client.poll(); + assertTrue(context.client.quorum().isProspective()); context.pollUntilRequest(); - context.assertVotedCandidate(epoch + 1, localId); - context.assertSentVoteRequest(epoch + 1, 0, 0L, 1); + context.assertSentVoteRequest(epoch, 0, 0L, 1); } @ParameterizedTest @@ -1767,6 +1780,7 @@ public void testVoterBecomeCandidateAfterFetchTimeout(boolean withKip853Rpc) thr context.assertSentFetchRequest(epoch, 1L, lastEpoch); context.time.sleep(context.fetchTimeoutMs); + context.expectAndGrantPreVotes(epoch); context.pollUntilRequest(); context.assertSentVoteRequest(epoch + 1, lastEpoch, 1L, 1); context.assertVotedCandidate(epoch + 1, localId); @@ -2119,27 +2133,20 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInvalidFetchRequest(boolean withDynamicReconfig) throws Exception { + @EnumSource(RaftProtocol.class) + public void testInvalidFetchRequest(RaftProtocol raftProtocol) throws Exception { + boolean withDynamicReconfig = raftProtocol.isReconfigSupported(); int localId = randomReplicaId(); - ReplicaKey localKey = replicaKey(localId, withDynamicReconfig); - Uuid localDirectoryId = localKey.directoryId().orElse(Uuid.randomUuid()); - ReplicaKey otherNodeBootstrap = replicaKey(localId + 1, withDynamicReconfig); - ReplicaKey otherNodeKey = ReplicaKey.of(otherNodeBootstrap.id(), otherNodeBootstrap.directoryId().orElse(Uuid.randomUuid())); + int otherNodeId = localId + 1; + ReplicaKey otherNodeKey = replicaKey(otherNodeId, withDynamicReconfig); + Set voters = Set.of(localId, otherNodeId); - RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withDynamicReconfig); - if (withDynamicReconfig) { - VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(localKey, otherNodeBootstrap)); - builder.withBootstrapSnapshot(Optional.of(bootstrapVoterSet)); - } else { - VoterSet staticVoterSet = VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)); - builder.withStaticVoters(staticVoterSet); - } - RaftClientTestContext context = builder.build(); + .withKip853Rpc(withDynamicReconfig) + .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, -5L, 0, 0)); @@ -2211,7 +2218,7 @@ public void testFetchRequestClusterIdValidation(boolean withKip853Rpc) throws Ex .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -2246,7 +2253,7 @@ public void testVoteRequestClusterIdValidation(boolean withKip853Rpc) throws Exc .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -2280,7 +2287,7 @@ public void testInvalidVoterReplicaVoteRequest() throws Exception { .withKip853Rpc(true) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // invalid voter id is rejected @@ -2326,7 +2333,7 @@ public void testInvalidVoterReplicaBeginQuorumEpochRequest() throws Exception { .withUnknownLeader(epoch - 1) .withKip853Rpc(true) .build(); - context.assertUnknownLeader(epoch - 1); + context.assertUnknownLeaderAndNoVotedCandidate(epoch - 1); // Leader voter3 sends a begin quorum epoch request with incorrect voter id context.deliverRequest( @@ -2379,7 +2386,7 @@ public void testBeginQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -2415,7 +2422,7 @@ public void testEndQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -2451,7 +2458,7 @@ public void testLeaderAcceptVoteFromObserver(boolean withKip853Rpc) throws Excep .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey observerKey = replicaKey(localId + 2, withKip853Rpc); @@ -2521,7 +2528,7 @@ public void testPurgatoryFetchTimeout(boolean withKip853Rpc) throws Exception { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Follower sends a fetch which cannot be satisfied immediately @@ -2549,7 +2556,7 @@ public void testPurgatoryFetchSatisfiedByWrite(boolean withKip853Rpc) throws Exc .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Follower sends a fetch which cannot be satisfied immediately @@ -2580,7 +2587,7 @@ public void testPurgatoryFetchCompletedByFollowerTransition(boolean withKip853Rp .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Follower sends a fetch which cannot be satisfied immediately @@ -2622,9 +2629,11 @@ public void testFetchResponseIgnoredAfterBecomingCandidate(boolean withKip853Rpc context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0L, 0); - // Now await the fetch timeout and become a candidate + // Now await the fetch timeout and become prospective, then candidate context.time.sleep(context.fetchTimeoutMs); context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + context.expectAndGrantPreVotes(epoch); context.assertVotedCandidate(epoch + 1, localId); // The fetch response from the old leader returns, but it should be ignored @@ -2682,69 +2691,52 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( } @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testVoteResponseIgnoredAfterBecomingFollower(boolean withDynamicReconfig) throws Exception { + @EnumSource(RaftProtocol.class) + public void testVoteResponseIgnoredAfterBecomingFollower(RaftProtocol raftProtocol) throws Exception { + boolean withDynamicReconfig = raftProtocol.isReconfigSupported(); int localId = randomReplicaId(); - ReplicaKey local = replicaKey(localId, withDynamicReconfig); - Uuid localDirectoryId = local.directoryId().orElse(Uuid.randomUuid()); int voter2 = localId + 1; - ReplicaKey voter2Bootstrap = replicaKey(voter2, withDynamicReconfig); - Uuid voter2DirectoryId = voter2Bootstrap.directoryId().orElse(Uuid.randomUuid()); - ReplicaKey voter2Key = ReplicaKey.of(voter2, voter2DirectoryId); int voter3 = localId + 2; - ReplicaKey voter3Bootstrap = replicaKey(voter3, withDynamicReconfig); - Uuid voter3DirectoryId = voter3Bootstrap.directoryId().orElse(Uuid.randomUuid()); - ReplicaKey voter3Key = ReplicaKey.of(voter3, voter3DirectoryId); int epoch = 5; - RaftClientTestContext.Builder builder = new RaftClientTestContext.Builder(localId, localDirectoryId) + Set voters = Set.of(localId, voter2, voter3); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withDynamicReconfig); - if (withDynamicReconfig) { - VoterSet bootstrapVoterSet = VoterSetTest.voterSet(Stream.of(local, voter2Bootstrap, voter3Bootstrap)); - builder.withBootstrapSnapshot(Optional.of(bootstrapVoterSet)); - } else { - VoterSet staticVoterSet = VoterSetTest.voterSet(Stream.of(local, voter2Key, voter3Key)); - builder.withStaticVoters(staticVoterSet); - } - RaftClientTestContext context = builder.build(); - context.assertUnknownLeader(epoch); - - if (withDynamicReconfig) { - // Sleep a little to ensure that we become a prospective - context.time.sleep(context.electionTimeoutMs() * 2L); - - // Wait until the PreVote requests are inflight - context.pollUntilRequest(); - assertTrue(context.client.quorum().isProspective()); - List voteRequests = context.collectVoteRequests(epoch, 0, 0); - assertEquals(2, voteRequests.size()); - - // Become candidate after PreVote requests are granted - VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch); - context.deliverResponse( - voteRequests.get(0).correlationId(), - voteRequests.get(0).destination(), - voteResponse1 - ); - VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch); - context.deliverResponse( - voteRequests.get(1).correlationId(), - voteRequests.get(1).destination(), - voteResponse2 - ); + .withKip853Rpc(withDynamicReconfig) + .build(); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); - context.client.poll(); - assertTrue(context.client.quorum().isCandidate()); - } else { - // Sleep a little to ensure that we become a candidate - context.time.sleep(context.electionTimeoutMs() * 2L); - } + // Sleep a little to ensure that we become a prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + + // Wait until the PreVote requests are inflight + context.pollUntilRequest(); + assertTrue(context.client.quorum().isProspective()); + List voteRequests = context.collectVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + + // Become candidate after PreVote requests are granted + VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch); + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + voteResponse1 + ); + VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch); + context.deliverResponse( + voteRequests.get(1).correlationId(), + voteRequests.get(1).destination(), + voteResponse2 + ); + + context.client.poll(); + assertTrue(context.client.quorum().isCandidate()); // Wait until the vote requests are inflight context.pollUntilRequest(); - context.assertVotedCandidate(epoch + 1, local); - List voteRequests = context.collectVoteRequests(epoch + 1, 0, 0); + context.assertVotedCandidate(epoch + 1, localId); + voteRequests = context.collectVoteRequests(epoch + 1, 0, 0); assertEquals(2, voteRequests.size()); // While the vote requests are still inflight, we receive a BeginEpoch for the same epoch @@ -2753,14 +2745,14 @@ public void testVoteResponseIgnoredAfterBecomingFollower(boolean withDynamicReco context.assertElectedLeader(epoch + 1, voter3); // The vote requests now return and should be ignored - VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch + 1); + voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch + 1); context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), voteResponse1 ); - VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch + 1); + voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch + 1); context.deliverResponse( voteRequests.get(1).correlationId(), voteRequests.get(1).destination(), @@ -2950,7 +2942,7 @@ public void testLeaderGracefulShutdown(boolean withKip853Rpc) throws Exception { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Now shutdown @@ -2998,7 +2990,7 @@ public void testEndQuorumEpochSentBasedOnFetchOffset(boolean withKip853Rpc) thro .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // The lagging follower fetches first @@ -3081,7 +3073,7 @@ public void testDescribeQuorumWithOnlyStaticVoters(boolean withKip853Rpc) throws .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Describe quorum response will not include directory ids @@ -3132,7 +3124,7 @@ public void testDescribeQuorumWithFollowers(boolean withKip853Rpc, boolean withB } RaftClientTestContext context = builder.build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Describe quorum response before any fetches made @@ -3238,7 +3230,7 @@ public void testDescribeQuorumWithObserver(boolean withKip853Rpc, boolean withBo } RaftClientTestContext context = builder.build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Update HW to non-initial value @@ -3380,7 +3372,7 @@ public void testDescribeQuorumNonMonotonicFollowerFetch(boolean withKip853Rpc, b } RaftClientTestContext context = builder.build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Update HW to non-initial value @@ -3451,7 +3443,7 @@ public void testStaticVotersIgnoredWithBootstrapSnapshot(boolean withKip853Rpc) .withBootstrapSnapshot(Optional.of(voterSet)) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // check describe quorum response has both followers context.deliverRequest(context.describeQuorumRequest()); @@ -3491,7 +3483,7 @@ public void testLeaderGracefulShutdownTimeout(boolean withKip853Rpc) throws Exce .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Now shutdown @@ -3557,7 +3549,7 @@ public void testObserverGracefulShutdown(boolean withKip853Rpc) throws Exception .withKip853Rpc(withKip853Rpc) .build(); context.client.poll(); - context.assertUnknownLeader(5); + context.assertUnknownLeaderAndNoVotedCandidate(5); // Observer shutdown should complete immediately even if the // current leader is unknown @@ -3732,8 +3724,8 @@ public void testFetchShouldBeTreatedAsLeaderAcknowledgement(boolean withKip853Rp .withKip853Rpc(withKip853Rpc) .build(); - context.time.sleep(context.electionTimeoutMs()); - context.expectAndGrantVotes(epoch, false); + context.unattachedToCandidate(); + context.expectAndGrantVotes(epoch); context.pollUntilRequest(); @@ -3959,7 +3951,8 @@ public void testClusterAuthorizationFailedInBeginQuorumEpoch(boolean withKip853R .build(); context.time.sleep(context.electionTimeoutMs()); - context.expectAndGrantVotes(epoch, false); + context.expectAndGrantPreVotes(epoch - 1); + context.expectAndGrantVotes(epoch); context.pollUntilRequest(); List requests = context.collectBeginEpochRequests(epoch); @@ -3986,8 +3979,8 @@ public void testClusterAuthorizationFailedInVote(boolean withKip853Rpc) throws E .withKip853Rpc(withKip853Rpc) .build(); - // Sleep a little to ensure that we become a candidate - context.time.sleep(context.electionTimeoutMs() * 2L); + // Become a candidate + context.unattachedToCandidate(); context.pollUntilRequest(); context.assertVotedCandidate(epoch, localId); @@ -4011,7 +4004,7 @@ public void testClusterAuthorizationFailedInEndQuorumEpoch(boolean withKip853Rpc .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.client.shutdown(5000); @@ -4038,7 +4031,7 @@ public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffsetOnEmp .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.poll(); int epoch = context.currentEpoch(); @@ -4095,7 +4088,7 @@ public void testHandleLeaderChangeFiresAfterListenerReachesEpochStartOffset( .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.poll(); // After becoming leader, we expect the `LeaderChange` record to be appended @@ -4161,7 +4154,7 @@ public void testLateRegisteredListenerCatchesUp(boolean withKip853Rpc) throws Ex .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.poll(); assertEquals(10L, context.log.endOffset().offset()); @@ -4204,7 +4197,7 @@ public void testReregistrationChangesListenerContext(boolean withKip853Rpc) thro .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.poll(); assertEquals(10L, context.log.endOffset().offset()); @@ -4306,7 +4299,7 @@ public void testHandleCommitCallbackFiresInVotedState(boolean withKip853Rpc) thr .build(); // Start off as the leader and receive a fetch to initialize the high watermark - context.becomeLeader(); + context.unattachedToLeader(); context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 10L, epoch, 500)); context.client.poll(); assertEquals(OptionalLong.of(10L), context.client.highWatermark()); @@ -4353,7 +4346,7 @@ public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) .build(); // Start off as the leader and receive a fetch to initialize the high watermark - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(10L, context.log.endOffset().offset()); context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 10L, epoch, 0)); @@ -4364,12 +4357,12 @@ public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) // Now we receive a vote request which transitions us to the 'unattached' state context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 9L)); context.pollUntilResponse(); - context.assertUnknownLeader(epoch + 1); + context.assertUnknownLeaderAndNoVotedCandidate(epoch + 1); assertEquals(OptionalLong.of(10L), context.client.highWatermark()); // Timeout the election and become candidate + context.unattachedToCandidate(); int candidateEpoch = epoch + 2; - context.time.sleep(context.electionTimeoutMs() * 2L); context.client.poll(); context.assertVotedCandidate(candidateEpoch, localId); diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java index a55f4cf1eecfb..b62445a25f82d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -51,15 +51,24 @@ public class ProspectiveStateTest { private final MockTime time = new MockTime(); private final int electionTimeoutMs = 5000; private final LogContext logContext = new LogContext(); - - private ProspectiveState newProspectiveState(VoterSet voters) { + private final int localId = 0; + private final int votedId = 1; + private final Uuid votedDirectoryId = Uuid.randomUuid(); + private final ReplicaKey votedKeyWithDirectoryId = ReplicaKey.of(votedId, votedDirectoryId); + private final ReplicaKey votedKeyWithoutDirectoryId = ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID); + + private ProspectiveState newProspectiveState( + VoterSet voters, + OptionalInt leaderId, + Optional votedKey + ) { return new ProspectiveState( time, localReplicaKey.id(), epoch, - OptionalInt.of(3), - Optional.empty(), - Optional.empty(), + leaderId, + leaderId.isPresent() ? Optional.of(leaderEndpoints) : Optional.empty(), + votedKey, voters, Optional.empty(), electionTimeoutMs, @@ -67,13 +76,13 @@ private ProspectiveState newProspectiveState(VoterSet voters) { ); } - private ProspectiveState newProspectiveStateWithLeaderEndpoints(VoterSet voters) { + private ProspectiveState newProspectiveState(VoterSet voters) { return new ProspectiveState( time, localReplicaKey.id(), epoch, - OptionalInt.of(3), - Optional.of(leaderEndpoints), + OptionalInt.empty(), + Optional.empty(), Optional.empty(), voters, Optional.empty(), @@ -179,28 +188,24 @@ public void testCannotRejectVoteFromLocalId(boolean withDirectoryId) { @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testCannotChangeVoteGrantedToRejected(boolean withDirectoryId) { - int otherNodeId = 1; + public void testCanChangePreVote(boolean withDirectoryId) { + int voter1 = 1; + int voter2 = 2; ProspectiveState state = newProspectiveState( - voterSetWithLocal(IntStream.of(otherNodeId), withDirectoryId) + voterSetWithLocal(IntStream.of(voter1, voter2), withDirectoryId) ); - assertTrue(state.recordGrantedVote(otherNodeId)); - assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(otherNodeId)); + assertTrue(state.recordGrantedVote(voter1)); assertTrue(state.isVoteGranted()); - } + assertFalse(state.recordRejectedVote(voter1)); + assertFalse(state.isVoteGranted()); - @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCannotChangeVoteRejectedToGranted(boolean withDirectoryId) { - int otherNodeId = 1; - ProspectiveState state = newProspectiveState( - voterSetWithLocal(IntStream.of(otherNodeId), withDirectoryId) - ); - assertTrue(state.recordRejectedVote(otherNodeId)); - assertThrows(IllegalArgumentException.class, () -> state.recordGrantedVote(otherNodeId)); + assertTrue(state.recordRejectedVote(voter2)); assertTrue(state.isVoteRejected()); + assertFalse(state.recordGrantedVote(voter2)); + assertFalse(state.isVoteRejected()); } + @ParameterizedTest @ValueSource(booleans = { true, false }) public void testCannotGrantOrRejectNonVoters(boolean withDirectoryId) { @@ -238,21 +243,18 @@ public void testGrantVote(boolean isLogUpToDate, boolean withDirectoryId) { ReplicaKey node0 = replicaKey(0, withDirectoryId); ReplicaKey node1 = replicaKey(1, withDirectoryId); ReplicaKey node2 = replicaKey(2, withDirectoryId); - ReplicaKey node3 = replicaKey(3, withDirectoryId); ProspectiveState state = newProspectiveState( - voterSetWithLocal(Stream.of(node1, node2, node3), withDirectoryId) + voterSetWithLocal(Stream.of(node1, node2), withDirectoryId) ); assertEquals(isLogUpToDate, state.canGrantPreVote(node0, isLogUpToDate)); assertEquals(isLogUpToDate, state.canGrantPreVote(node1, isLogUpToDate)); assertEquals(isLogUpToDate, state.canGrantPreVote(node2, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantPreVote(node3, isLogUpToDate)); assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate)); assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate)); assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantVote(node3, isLogUpToDate)); } @ParameterizedTest @@ -261,30 +263,42 @@ public void testGrantVoteWithVotedKey(boolean isLogUpToDate, boolean withDirecto ReplicaKey node0 = replicaKey(0, withDirectoryId); ReplicaKey node1 = replicaKey(1, withDirectoryId); ReplicaKey node2 = replicaKey(2, withDirectoryId); - ReplicaKey node3 = replicaKey(3, withDirectoryId); - ProspectiveState state = new ProspectiveState( - time, - localReplicaKey.id(), - epoch, + ProspectiveState state = newProspectiveState( + voterSetWithLocal(Stream.of(node1, node2), withDirectoryId), OptionalInt.empty(), - Optional.empty(), - Optional.of(node1), - voterSetWithLocal(Stream.of(node1, node2, node3), withDirectoryId), - Optional.empty(), - electionTimeoutMs, - logContext + Optional.of(node1) ); assertEquals(isLogUpToDate, state.canGrantPreVote(node0, isLogUpToDate)); assertEquals(isLogUpToDate, state.canGrantPreVote(node1, isLogUpToDate)); assertEquals(isLogUpToDate, state.canGrantPreVote(node2, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantPreVote(node3, isLogUpToDate)); assertFalse(state.canGrantVote(node0, isLogUpToDate)); assertTrue(state.canGrantVote(node1, isLogUpToDate)); assertFalse(state.canGrantVote(node2, isLogUpToDate)); - assertFalse(state.canGrantVote(node3, isLogUpToDate)); + } + + @ParameterizedTest + @CsvSource({ "true,true", "true,false", "false,true", "false,false" }) + public void testGrantVoteWithLeader(boolean isLogUpToDate, boolean withDirectoryId) { + ReplicaKey node0 = replicaKey(0, withDirectoryId); + ReplicaKey node1 = replicaKey(1, withDirectoryId); + ReplicaKey node2 = replicaKey(2, withDirectoryId); + + ProspectiveState state = newProspectiveState( + voterSetWithLocal(Stream.of(node1, node2), withDirectoryId), + OptionalInt.of(node1.id()), + Optional.empty() + ); + + assertEquals(isLogUpToDate, state.canGrantPreVote(node0, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantPreVote(node1, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantPreVote(node2, isLogUpToDate)); + + assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate)); } @ParameterizedTest @@ -292,31 +306,143 @@ public void testGrantVoteWithVotedKey(boolean isLogUpToDate, boolean withDirecto public void testElectionState(boolean withDirectoryId) { VoterSet voters = voterSetWithLocal(IntStream.of(1, 2, 3), withDirectoryId); ProspectiveState state = newProspectiveState(voters); + assertEquals( + ElectionState.withUnknownLeader( + epoch, + voters.voterIds() + ), + state.election() + ); + + // with leader + state = newProspectiveState(voters, OptionalInt.of(1), Optional.empty()); + assertEquals( + ElectionState.withElectedLeader( + epoch, + 1, + voters.voterIds() + ), + state.election() + ); + + // with voted key + ReplicaKey votedKey = replicaKey(1, withDirectoryId); + state = newProspectiveState(voters, OptionalInt.empty(), Optional.of(votedKey)); assertEquals( ElectionState.withVotedCandidate( epoch, - localReplicaKey, + votedKey, voters.voterIds() ), state.election() ); } + @Test + public void testElectionTimeout() { + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.empty(), true), + OptionalInt.empty(), + Optional.of(votedKeyWithDirectoryId) + ); + + assertEquals(epoch, state.epoch()); + assertEquals(votedKeyWithDirectoryId, state.votedKey().get()); + assertEquals( + ElectionState.withVotedCandidate(epoch, votedKeyWithDirectoryId, Collections.singleton(localId)), + state.election() + ); + assertEquals(electionTimeoutMs, state.remainingElectionTimeMs(time.milliseconds())); + assertFalse(state.hasElectionTimeoutExpired(time.milliseconds())); + + time.sleep(5000); + assertEquals(electionTimeoutMs - 5000, state.remainingElectionTimeMs(time.milliseconds())); + assertFalse(state.hasElectionTimeoutExpired(time.milliseconds())); + + time.sleep(5000); + assertEquals(0, state.remainingElectionTimeMs(time.milliseconds())); + assertTrue(state.hasElectionTimeoutExpired(time.milliseconds())); + } + @ParameterizedTest @ValueSource(booleans = {true, false}) - public void testInvalidVoterSet(boolean withDirectoryId) { - assertThrows( - IllegalArgumentException.class, - () -> newProspectiveState( - VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), withDirectoryId)) - ) + public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.empty(), true), + OptionalInt.empty(), + Optional.of(votedKeyWithoutDirectoryId)); + + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) ); + assertTrue(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate) + ); + assertTrue(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); + + // Can grant PreVote to other replicas even if we have granted a standard vote to another replica + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testCanGrantVoteWithDirectoryId(boolean isLogUpToDate) { + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.empty(), true), + OptionalInt.empty(), + Optional.of(votedKeyWithDirectoryId)); + + // Same voterKey + // We will not grant PreVote for a replica we have already granted a standard vote to if their log is behind + assertEquals( + isLogUpToDate, + state.canGrantPreVote(votedKeyWithDirectoryId, isLogUpToDate) + ); + assertTrue(state.canGrantVote(votedKeyWithDirectoryId, isLogUpToDate)); + + // Different directoryId + // We can grant PreVote for a replica we have already granted a standard vote to if their log is up-to-date, + // even if the directoryId is different + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); + + // Missing directoryId + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + + // Different voterId + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId + 1, votedDirectoryId), isLogUpToDate) + ); + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), true)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), true)); } @Test public void testLeaderEndpoints() { ProspectiveState state = newProspectiveState( - voterSetWithLocal(IntStream.of(1, 2, 3), true) + voterSetWithLocal(IntStream.of(1, 2, 3), true), + OptionalInt.empty(), + Optional.of(ReplicaKey.of(1, Uuid.randomUuid())) ); assertEquals(Endpoints.empty(), state.leaderEndpoints()); diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java index 86c4e8771d838..af169891e0690 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java @@ -65,9 +65,6 @@ private ProspectiveState newProspectiveVotedState( ); } - // todo: need with leader state - // todo: don't forget to fix quorum state keeping leaderid, and prospective to follower transition - @Test public void testElectionTimeout() { ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithDirectoryId)); @@ -95,10 +92,16 @@ public void testElectionTimeout() { public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithoutDirectoryId)); - assertTrue(state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + ); assertTrue(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); - assertTrue(state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); + assertEquals( + isLogUpToDate, + state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate) + ); assertTrue(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); // Can grant PreVote to other replicas even if we have granted a standard vote to another replica diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index e3b4e72703fdb..419a36f7f32bd 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -99,6 +99,7 @@ import java.util.stream.Stream; import static org.apache.kafka.raft.LeaderState.CHECK_QUORUM_TIMEOUT_FACTOR; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL; import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -137,7 +138,7 @@ public final class RaftClientTestContext { final VoterSet startingVoters; final Set bootstrapIds; // Used to determine which RPC request and response to construct - final boolean kip853Rpc; + final RaftProtocol raftProtocol; // Used to determine if the local kraft client was configured to always flush final boolean alwaysFlush; @@ -172,7 +173,7 @@ public static final class Builder { private int appendLingerMs = DEFAULT_APPEND_LINGER_MS; private MemoryPool memoryPool = MemoryPool.NONE; private Optional> bootstrapServers = Optional.empty(); - private boolean kip853Rpc = false; + private RaftProtocol raftProtocol = RaftProtocol.KIP_595_PROTOCOL; private boolean alwaysFlush = false; private VoterSet startingVoters = VoterSet.empty(); private Endpoints localListeners = Endpoints.empty(); @@ -197,6 +198,26 @@ public Builder(OptionalInt localId, Uuid localDirectoryId) { this.localDirectoryId = localDirectoryId; } + public Builder( + Optional localKey, + Optional voters, + KRaftVersion kraftVersion + ) { + this.localId = localKey.map(replicaKey -> + OptionalInt.of(replicaKey.id())).orElseGet(OptionalInt::empty); + this.localDirectoryId = localKey.map(replicaKey -> + replicaKey.directoryId().orElse(Uuid.randomUuid())).orElse(Uuid.randomUuid()); + this.kraftVersion = kraftVersion; + if (kraftVersion == KRaftVersion.KRAFT_VERSION_0) { + withStaticVoters( + voters.orElseThrow(() -> + new IllegalArgumentException("Static voters must be provided for KRaft version 0")) + .voterIds()); + } else { + withBootstrapSnapshot(voters); + } + } + Builder withElectedLeader(int epoch, int leaderId) { quorumStateStore.writeElectionState( ElectionState.withElectedLeader(epoch, leaderId, startingVoters.voterIds()), @@ -292,8 +313,16 @@ Builder withBootstrapServers(Optional> bootstrapServers) return this; } - Builder withKip853Rpc(boolean kip853Rpc) { - this.kip853Rpc = kip853Rpc; + // deprecated, use withRpc instead + Builder withKip853Rpc(boolean withKip853Rpc) { + if (withKip853Rpc) { + this.raftProtocol = KIP_853_PROTOCOL; + } + return this; + } + + Builder withRaftProtocol(RaftProtocol raftProtocol) { + this.raftProtocol = raftProtocol; return this; } @@ -330,7 +359,7 @@ Builder withBootstrapSnapshot(Optional voters) { isStartingVotersStatic = false; if (voters.isPresent()) { - kraftVersion = KRaftVersion.KRAFT_VERSION_1; + kraftVersion = KRaftVersion.LATEST_PRODUCTION; RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() .setRawSnapshotWriter( @@ -450,7 +479,7 @@ public RaftClientTestContext build() throws IOException { .limit(bootstrapServers.map(List::size).orElse(0)) .boxed() .collect(Collectors.toSet()), - kip853Rpc, + raftProtocol, alwaysFlush, metrics, listener @@ -478,7 +507,7 @@ private RaftClientTestContext( QuorumStateStore quorumStateStore, VoterSet startingVoters, Set bootstrapIds, - boolean kip853Rpc, + RaftProtocol raftProtocol, boolean alwaysFlush, Metrics metrics, MockListener listener @@ -495,7 +524,7 @@ private RaftClientTestContext( this.quorumStateStore = quorumStateStore; this.startingVoters = startingVoters; this.bootstrapIds = bootstrapIds; - this.kip853Rpc = kip853Rpc; + this.raftProtocol = raftProtocol; this.alwaysFlush = alwaysFlush; this.metrics = metrics; this.listener = listener; @@ -554,18 +583,20 @@ static RaftClientTestContext initializeAsLeader(int localId, Set voters .withUnknownLeader(epoch - 1) .build(); - context.assertUnknownLeader(epoch - 1); - context.becomeLeader(); + context.assertUnknownLeaderAndNoVotedCandidate(epoch - 1); + context.unattachedToLeader(); return context; } - public void becomeLeader() throws Exception { - int currentEpoch = currentEpoch(); + public void unattachedToCandidate() throws Exception { time.sleep(electionTimeoutMs * 2L); - if (kraftVersion.isReconfigSupported()) { - expectAndGrantVotes(currentEpoch, true); - } - expectAndGrantVotes(currentEpoch + 1, false); + expectAndGrantPreVotes(currentEpoch()); + } + + public void unattachedToLeader() throws Exception { + int currentEpoch = currentEpoch(); + unattachedToCandidate(); + expectAndGrantVotes(currentEpoch + 1); expectBeginEpoch(currentEpoch + 1); } @@ -582,7 +613,7 @@ LeaderAndEpoch currentLeaderAndEpoch() { return new LeaderAndEpoch(election.optionalLeaderId(), election.epoch()); } - void expectAndGrantVotes(int epoch, boolean preVote) throws Exception { + void expectAndGrantVotes(int epoch) throws Exception { pollUntilRequest(); List voteRequests = collectVoteRequests(epoch, @@ -594,11 +625,30 @@ void expectAndGrantVotes(int epoch, boolean preVote) throws Exception { } client.poll(); - if (preVote) { - assertTrue(client.quorum().isCandidate()); - } else { - assertElectedLeader(epoch, localIdOrThrow()); + assertElectedLeader(epoch, localIdOrThrow()); + } + + void expectAndGrantPreVotes(int epoch) throws Exception { + pollUntilRequest(); + + List voteRequests = collectPreVoteRequests(epoch, + log.lastFetchedEpoch(), log.endOffset().offset()); + + for (RaftRequest.Outbound request : voteRequests) { + if (!raftProtocol.isPreVoteSupported()) { + deliverResponse( + request.correlationId(), + request.destination(), + RaftUtil.errorResponse(ApiKeys.VOTE, Errors.UNSUPPORTED_VERSION) + ); + } else { + VoteResponseData voteResponse = voteResponse(true, OptionalInt.empty(), epoch); + deliverResponse(request.correlationId(), request.destination(), voteResponse); + } } + + client.poll(); + assertTrue(client.quorum().isCandidate()); } private int localIdOrThrow() { @@ -606,7 +656,7 @@ private int localIdOrThrow() { } public ReplicaKey localReplicaKey() { - return kip853Rpc ? + return raftProtocol.isReconfigSupported() ? ReplicaKey.of(localIdOrThrow(), localDirectoryId) : ReplicaKey.of(localIdOrThrow(), ReplicaKey.NO_DIRECTORY_ID); } @@ -658,7 +708,7 @@ public void assertElectedLeader(int epoch, int leaderId) { ); } - void assertUnknownLeader(int epoch) { + void assertUnknownLeaderAndNoVotedCandidate(int epoch) { assertEquals( ElectionState.withUnknownLeader(epoch, expectedVoters()), quorumStateStore.readElectionState().get()); @@ -789,7 +839,7 @@ void assertSentVoteResponse( assertEquals(leaderId.orElse(-1), partitionResponse.leaderId(), leaderIdDebugLog); assertEquals(epoch, partitionResponse.leaderEpoch()); - if (kip853Rpc && leaderId.isPresent()) { + if (raftProtocol.isReconfigSupported() && leaderId.isPresent()) { Endpoints expectedLeaderEndpoints = startingVoters.listeners(leaderId.getAsInt()); Endpoints responseEndpoints = Endpoints.fromVoteResponse( channel.listenerName(), @@ -927,7 +977,7 @@ void assertSentBeginQuorumEpochResponse( .get(0) .partitions() .get(0); - if (kip853Rpc && partitionResponse.leaderId() >= 0) { + if (raftProtocol.isReconfigSupported() && partitionResponse.leaderId() >= 0) { int leaderId = partitionResponse.leaderId(); Endpoints expectedLeaderEndpoints = startingVoters.listeners(leaderId); Endpoints responseEndpoints = Endpoints.fromBeginQuorumEpochResponse( @@ -964,7 +1014,7 @@ void assertSentBeginQuorumEpochResponse( " Partition response leader Id: " + partitionResponse.leaderId() ); - if (kip853Rpc && leaderId.isPresent()) { + if (raftProtocol.isReconfigSupported() && leaderId.isPresent()) { Endpoints expectedLeaderEndpoints = startingVoters.listeners(leaderId.getAsInt()); Endpoints responseEndpoints = Endpoints.fromBeginQuorumEpochResponse( channel.listenerName(), @@ -1001,7 +1051,7 @@ void assertSentEndQuorumEpochResponse( .get(0) .partitions() .get(0); - if (kip853Rpc && partitionResponse.leaderId() >= 0) { + if (raftProtocol.isReconfigSupported() && partitionResponse.leaderId() >= 0) { int leaderId = partitionResponse.leaderId(); Endpoints expectedLeaderEndpoints = startingVoters.listeners(leaderId); Endpoints responseEndpoints = Endpoints.fromEndQuorumEpochResponse( @@ -1033,7 +1083,7 @@ void assertSentEndQuorumEpochResponse( assertEquals(leaderId.orElse(-1), partitionResponse.leaderId()); assertEquals(partitionError, Errors.forCode(partitionResponse.errorCode())); - if (kip853Rpc && leaderId.isPresent()) { + if (raftProtocol.isReconfigSupported() && leaderId.isPresent()) { Endpoints expectedLeaderEndpoints = startingVoters.listeners(leaderId.getAsInt()); Endpoints responseEndpoints = Endpoints.fromEndQuorumEpochResponse( channel.listenerName(), @@ -1077,7 +1127,7 @@ FetchResponseData.PartitionData assertSentFetchPartitionResponse() { assertEquals(1, response.responses().get(0).partitions().size()); FetchResponseData.PartitionData partitionResponse = response.responses().get(0).partitions().get(0); - if (kip853Rpc && partitionResponse.currentLeader().leaderId() >= 0) { + if (raftProtocol.isReconfigSupported() && partitionResponse.currentLeader().leaderId() >= 0) { int leaderId = partitionResponse.currentLeader().leaderId(); Endpoints expectedLeaderEndpoints = startingVoters.listeners(leaderId); Endpoints responseEndpoints = Endpoints.fromFetchResponse( @@ -1167,7 +1217,7 @@ Optional assertSentFetchSnapshotRes Optional result = FetchSnapshotResponse.forTopicPartition(response, topicPartition); - if (result.isPresent() && kip853Rpc && result.get().currentLeader().leaderId() >= 0) { + if (result.isPresent() && raftProtocol.isReconfigSupported() && result.get().currentLeader().leaderId() >= 0) { int leaderId = result.get().currentLeader().leaderId(); Endpoints expectedLeaderEndpoints = startingVoters.listeners(leaderId); Endpoints responseEndpoints = Endpoints.fromFetchSnapshotResponse( @@ -1405,7 +1455,7 @@ BeginQuorumEpochRequestData beginEpochRequest(int epoch, int leaderId) { } BeginQuorumEpochRequestData beginEpochRequest(int epoch, int leaderId, Endpoints endpoints) { - ReplicaKey localReplicaKey = kip853Rpc ? + ReplicaKey localReplicaKey = raftProtocol.isReconfigSupported() ? ReplicaKey.of(localIdOrThrow(), localDirectoryId) : ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID); @@ -1413,7 +1463,7 @@ BeginQuorumEpochRequestData beginEpochRequest(int epoch, int leaderId, Endpoints } BeginQuorumEpochRequestData beginEpochRequest(String clusterId, int epoch, int leaderId) { - ReplicaKey localReplicaKey = kip853Rpc ? + ReplicaKey localReplicaKey = raftProtocol.isReconfigSupported() ? ReplicaKey.of(localIdOrThrow(), localDirectoryId) : ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID); @@ -1505,7 +1555,7 @@ VoteRequestData voteRequest( long lastEpochOffset, boolean preVote ) { - ReplicaKey localReplicaKey = kip853Rpc ? + ReplicaKey localReplicaKey = raftProtocol.isReconfigSupported() ? ReplicaKey.of(localIdOrThrow(), localDirectoryId) : ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID); @@ -1679,7 +1729,7 @@ FetchRequestData fetchRequest( .setCurrentLeaderEpoch(epoch) .setLastFetchedEpoch(lastFetchedEpoch) .setFetchOffset(fetchOffset); - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { fetchPartition .setReplicaDirectoryId(replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)); } @@ -1864,7 +1914,7 @@ UpdateRaftVoterResponseData updateVoterResponse( } private short fetchRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 17; } else { return 16; @@ -1872,7 +1922,7 @@ private short fetchRpcVersion() { } private short fetchSnapshotRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 1; } else { return 0; @@ -1880,15 +1930,18 @@ private short fetchSnapshotRpcVersion() { } short voteRpcVersion() { - if (kip853Rpc) { - return ApiKeys.VOTE.latestVersion(); + if (raftProtocol.isPreVoteSupported()) { + return 2; + } + if (raftProtocol.isReconfigSupported()) { + return 1; } else { return 0; } } private short beginQuorumEpochRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 1; } else { return 0; @@ -1896,7 +1949,7 @@ private short beginQuorumEpochRpcVersion() { } private short endQuorumEpochRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 1; } else { return 0; @@ -1904,7 +1957,7 @@ private short endQuorumEpochRpcVersion() { } private short describeQuorumRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 2; } else { return 1; @@ -1912,7 +1965,7 @@ private short describeQuorumRpcVersion() { } private short addVoterRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 0; } else { throw new IllegalStateException("Reconfiguration must be enabled by calling withKip853Rpc(true)"); @@ -1920,7 +1973,7 @@ private short addVoterRpcVersion() { } private short removeVoterRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 0; } else { throw new IllegalStateException("Reconfiguration must be enabled by calling withKip853Rpc(true)"); @@ -1928,7 +1981,7 @@ private short removeVoterRpcVersion() { } private short updateVoterRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 0; } else { throw new IllegalStateException("Reconfiguration must be enabled by calling withKip853Rpc(true)"); @@ -2155,4 +2208,29 @@ public void handleLoadSnapshot(SnapshotReader reader) { snapshot = Optional.of(reader); } } + + /** + * Determines what versions of RPCs are in use. Note, these are ordered from oldest to newest, and are + * cumulative. E.g. KIP_996_PROTOCOL includes KIP_853_PROTOCOL and KIP_595_PROTOCOL changes + */ + enum RaftProtocol { + // kraft support + KIP_595_PROTOCOL, + // dynamic quorum reconfiguration support + KIP_853_PROTOCOL, + // preVote support + KIP_996_PROTOCOL; + + boolean isReconfigSupported() { + return isAtLeast(KIP_853_PROTOCOL); + } + + boolean isPreVoteSupported() { + return isAtLeast(KIP_996_PROTOCOL); + } + + private boolean isAtLeast(RaftProtocol otherRpc) { + return this.compareTo(otherRpc) >= 0; + } + } } From 1839d7ca3911649907af88abae3516bee4914610 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Mon, 16 Dec 2024 10:38:18 -0800 Subject: [PATCH 07/23] wip, still needs more cleanup --- .../org/apache/kafka/raft/CandidateState.java | 101 ++------ .../org/apache/kafka/raft/EpochElection.java | 176 ++++++++++++++ .../apache/kafka/raft/KafkaRaftClient.java | 8 +- .../{VotingState.java => NomineeState.java} | 82 ++----- .../apache/kafka/raft/ProspectiveState.java | 74 ++---- .../org/apache/kafka/raft/QuorumState.java | 6 +- .../apache/kafka/raft/UnattachedState.java | 4 +- .../raft/KafkaRaftClientPreVoteTest.java | 230 ++++++++++++------ .../kafka/raft/ProspectiveStateTest.java | 68 +++--- .../raft/ProspectiveStateWithVoteTest.java | 187 -------------- 10 files changed, 418 insertions(+), 518 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/raft/EpochElection.java rename raft/src/main/java/org/apache/kafka/raft/{VotingState.java => NomineeState.java} (57%) delete mode 100644 raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 50ac194c763b6..6f3eb9e264cba 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -20,21 +20,19 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.raft.VotingState.VoterState.State; +import org.apache.kafka.raft.EpochElection.VoterState.State; import org.slf4j.Logger; -import java.util.HashMap; -import java.util.Map; import java.util.Optional; -import java.util.stream.Stream; +import java.util.Set; -public class CandidateState implements VotingState { +public class CandidateState implements NomineeState { private final int localId; private final Uuid localDirectoryId; private final int epoch; private final int retries; - private final Map voteStates = new HashMap<>(); + private final EpochElection epochElection; private final Optional highWatermark; private final int electionTimeoutMs; private final Timer electionTimer; @@ -84,10 +82,8 @@ protected CandidateState( this.backoffTimer = time.timer(0); this.log = logContext.logger(CandidateState.class); - for (ReplicaKey voter : voters.voterKeys()) { - voteStates.put(voter.id(), new VoterState(voter)); - } - voteStates.get(localId).setState(State.GRANTED); + this.epochElection = new EpochElection(localId, voters.voterKeys()); + epochElection.voterStates().get(localId).setState(State.GRANTED); } public int localId() { @@ -95,8 +91,8 @@ public int localId() { } @Override - public Map voteStates() { - return voteStates; + public EpochElection epochElection() { + return epochElection; } /** @@ -110,71 +106,6 @@ public int retries() { return retries; } - /** - * Check whether we have received enough votes to conclude the election and become leader. - * - * @return true if at least a majority of nodes have granted the vote - */ - public boolean isVoteGranted() { - return numGranted() >= majoritySize(); - } - - /** - * Check if we have received enough rejections that it is no longer possible to reach a - * majority of grants. - * - * @return true if the vote is rejected, false if the vote is already or can still be granted - */ - public boolean isVoteRejected() { - return numGranted() + numUnrecorded() < majoritySize(); - } - - /** - * Record a granted vote from one of the voters. - * - * @param remoteNodeId The id of the voter - * @return true if the voter had not been previously recorded - * @throws IllegalArgumentException if the remote node is not a voter or if the vote had already been - * rejected by this node - */ - public boolean recordGrantedVote(int remoteNodeId) { - VoterState voterState = voteStates.get(remoteNodeId); - if (voterState == null) { - throw new IllegalArgumentException("Attempt to grant vote to non-voter " + remoteNodeId); - } else if (voterState.state().equals(State.REJECTED)) { - throw new IllegalArgumentException("Attempt to grant vote from node " + remoteNodeId + - " which previously rejected our request"); - } - - boolean recorded = voterState.state().equals(State.UNRECORDED); - voterState.setState(State.GRANTED); - - return recorded; - } - - /** - * Record a rejected vote from one of the voters. - * - * @param remoteNodeId The id of the voter - * @return true if the rejected vote had not been previously recorded - * @throws IllegalArgumentException if the remote node is not a voter or if the vote had already been - * granted by this node - */ - public boolean recordRejectedVote(int remoteNodeId) { - VoterState voterState = voteStates.get(remoteNodeId); - if (voterState == null) { - throw new IllegalArgumentException("Attempt to reject vote to non-voter " + remoteNodeId); - } else if (voterState.state().equals(State.GRANTED)) { - throw new IllegalArgumentException("Attempt to reject vote from node " + remoteNodeId + - " which previously granted our request"); - } - - boolean recorded = voterState.state().equals(State.UNRECORDED); - voterState.setState(State.REJECTED); - - return recorded; - } - /** * Record the current election has failed since we've either received sufficient rejecting voters or election timed out */ @@ -184,14 +115,6 @@ public void startBackingOff(long currentTimeMs, long backoffDurationMs) { this.isBackingOff = true; } - public Stream votersInState(State state) { - return voteStates - .values() - .stream() - .filter(voterState -> voterState.state().equals(state)) - .map(VoterState::replicaKey); - } - @Override public boolean hasElectionTimeoutExpired(long currentTimeMs) { electionTimer.update(currentTimeMs); @@ -217,12 +140,16 @@ public long remainingBackoffMs(long currentTimeMs) { return backoffTimer.remainingMs(); } + Set grantingVoters() { + return epochElection().grantingVoters(); + } + @Override public ElectionState election() { return ElectionState.withVotedCandidate( epoch, ReplicaKey.of(localId, localDirectoryId), - voteStates.keySet() + epochElection.voterStates().keySet() ); } @@ -272,7 +199,7 @@ public String toString() { localDirectoryId, epoch, retries, - voteStates, + epochElection().voterStates(), highWatermark, electionTimeoutMs ); diff --git a/raft/src/main/java/org/apache/kafka/raft/EpochElection.java b/raft/src/main/java/org/apache/kafka/raft/EpochElection.java new file mode 100644 index 0000000000000..83d2d718260c0 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/EpochElection.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class EpochElection { + private final int localId; + private Map voterStates; + + EpochElection(int localId, Set voters) { + this.localId = localId; + this.voterStates = voters.stream() + .collect(Collectors.toMap( + ReplicaKey::id, + VoterState::new + )); + } + + Stream voters(VoterState.State state) { + return voterStates + .values() + .stream() + .filter(voterState -> voterState.state().equals(state)) + .map(VoterState::replicaKey); + } + + Map voterStates() { + return voterStates; + } + + int majoritySize() { + return voterStates.size() / 2 + 1; + } + + long numGranted() { + return voters(VoterState.State.GRANTED).count(); + } + + long numUnrecorded() { + return voters(VoterState.State.UNRECORDED).count(); + } + + /** + * Check whether we have received enough votes to conclude the election and become leader. + * + * @return true if at least a majority of nodes have granted the vote + */ + boolean isVoteGranted() { + return numGranted() >= majoritySize(); + } + + boolean isVoteRejected() { + return numGranted() + numUnrecorded() < majoritySize(); + } + + // this could be on prospective & candidate to check. this becomes grabbing the previous value and one generic record the vote (without checks) + /** + * Record a granted vote from one of the voters. + * + * @param remoteNodeId The id of the voter + * @param isPreVote Whether the vote is a PreVote (non-binding) or not (binding) + * @return true if the voter had not been previously recorded + * @throws IllegalArgumentException if the remote node is not a voter or if the vote had already been + * rejected by this node + */ + boolean recordGrantedVote(int remoteNodeId, boolean isPreVote) { + VoterState voterState = voterStates.get(remoteNodeId); + if (voterState == null) { + throw new IllegalArgumentException("Attempt to grant vote to non-voter " + remoteNodeId); + } else if (!isPreVote && voterState.state().equals(VoterState.State.REJECTED)) { + throw new IllegalArgumentException("Attempt to grant vote from node " + remoteNodeId + + " which previously rejected our request"); + } + boolean recorded = voterState.state().equals(VoterState.State.UNRECORDED); + voterState.setState(VoterState.State.GRANTED); + + return recorded; + } + + /** + * Record a rejected vote from one of the voters. + * + * @param remoteNodeId The id of the voter + * @param isPreVote Whether the vote is a PreVote (non-binding) or not (binding) + * @return true if the rejected vote had not been previously recorded + * @throws IllegalArgumentException if the remote node is not a voter or if the vote had already been + * granted by this node + */ + public boolean recordRejectedVote(int remoteNodeId, boolean isPreVote) { + VoterState voterState = voterStates.get(remoteNodeId); + if (voterState == null) { + throw new IllegalArgumentException("Attempt to reject vote to non-voter " + remoteNodeId); + } else if (isPreVote && remoteNodeId == localId) { + throw new IllegalStateException("Attempted to reject vote from ourselves"); + } else if (!isPreVote && voterState.state().equals(VoterState.State.GRANTED)) { + throw new IllegalArgumentException("Attempt to reject vote from node " + remoteNodeId + + " which previously granted our request"); + } + boolean recorded = voterState.state().equals(VoterState.State.UNRECORDED); + voterState.setState(VoterState.State.REJECTED); + + return recorded; + } + + /** + * Get the set of voters which have not been counted as granted or rejected yet. + * + * @return The set of unrecorded voters + */ + Set unrecordedVoters() { + return voters(VoterState.State.UNRECORDED).collect(Collectors.toSet()); + } + + /** + * Get the set of voters that have granted our vote requests. + * + * @return The set of granting voters, which should always contain the localId + */ + Set grantingVoters() { + return voters(VoterState.State.GRANTED).map(ReplicaKey::id).collect(Collectors.toSet()); + } + + /** + * Get the set of voters that have rejected our candidacy. + * + * @return The set of rejecting voters + */ + Set rejectingVoters() { + return voters(VoterState.State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet()); + } + + static final class VoterState { + private final ReplicaKey replicaKey; + private State state = State.UNRECORDED; + + VoterState(ReplicaKey replicaKey) { + this.replicaKey = replicaKey; + } + + public State state() { + return state; + } + + public void setState(State state) { + this.state = state; + } + + public ReplicaKey replicaKey() { + return replicaKey; + } + + enum State { + UNRECORDED, + GRANTED, + REJECTED + } + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 73a7511d7e423..88d37793edea2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -680,7 +680,7 @@ private boolean maybeTransitionToCandidate(ProspectiveState state, long currentT * or from Candidate to Leader. * @return true if the state transitioned forward, false otherwise */ - private boolean maybeTransitionForward(VotingState state, long currentTimeMs) { + private boolean maybeTransitionForward(NomineeState state, long currentTimeMs) { if (quorum.isProspective()) { return maybeTransitionToCandidate(quorum.prospectiveStateOrThrow(), currentTimeMs); } else if (quorum.isCandidate()) { @@ -984,7 +984,7 @@ private boolean handleVoteResponse( logger.debug("Ignoring vote response {} since we already became leader for epoch {}", partitionResponse, quorum.epoch()); } else if (quorum.isVotingState()) { - VotingState state = quorum.votingStateOrThrow(); + NomineeState state = quorum.votingStateOrThrow(); handleVoteResponse( state, partitionResponse, @@ -1001,7 +1001,7 @@ private boolean handleVoteResponse( } } - private void handleVoteResponse(VotingState state, + private void handleVoteResponse(NomineeState state, VoteResponseData.PartitionData partitionResponse, int remoteNodeId, long currentTimeMs) { @@ -3084,7 +3084,7 @@ private long pollLeader(long currentTimeMs) { // } private long maybeSendVoteRequests( - VotingState state, + NomineeState state, long currentTimeMs ) { // Continue sending Vote requests as long as we still have a chance to win the election diff --git a/raft/src/main/java/org/apache/kafka/raft/VotingState.java b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java similarity index 57% rename from raft/src/main/java/org/apache/kafka/raft/VotingState.java rename to raft/src/main/java/org/apache/kafka/raft/NomineeState.java index 0c4204a3157a7..73873ecb85f0a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/VotingState.java +++ b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java @@ -16,33 +16,10 @@ */ package org.apache.kafka.raft; -import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; -interface VotingState extends EpochState { - Map voteStates(); - - default Stream votersInState(VoterState.State state) { - return voteStates() - .values() - .stream() - .filter(voterState -> voterState.state().equals(state)) - .map(VoterState::replicaKey); - } - - default int majoritySize() { - return voteStates().size() / 2 + 1; - } - - default long numGranted() { - return votersInState(VoterState.State.GRANTED).count(); - } - - default long numUnrecorded() { - return votersInState(VoterState.State.UNRECORDED).count(); - } +interface NomineeState extends EpochState { + EpochElection epochElection(); /** * Check if the candidate is backing off for the next election @@ -57,7 +34,7 @@ default long numUnrecorded() { * @return true if at least a majority of nodes have granted the vote */ default boolean isVoteGranted() { - return numGranted() >= majoritySize(); + return epochElection().isVoteGranted(); } /** @@ -67,9 +44,10 @@ default boolean isVoteGranted() { * @return true if the vote is rejected, false if the vote is already or can still be granted */ default boolean isVoteRejected() { - return numGranted() + numUnrecorded() < majoritySize(); + return epochElection().isVoteRejected(); } + // override in prospective and candidate (they contain the validation for the vote now) /** * Record a granted vote from one of the voters. * @@ -77,7 +55,10 @@ default boolean isVoteRejected() { * @return true if the voter had not been previously recorded * @throws IllegalArgumentException */ - boolean recordGrantedVote(int remoteNodeId); + default boolean recordGrantedVote(int remoteNodeId) { + boolean isPreVote = this instanceof ProspectiveState; + return epochElection().recordGrantedVote(remoteNodeId, isPreVote); + } /** * Record a rejected vote from one of the voters. @@ -86,7 +67,10 @@ default boolean isVoteRejected() { * @return true if the rejected vote had not been previously recorded * @throws IllegalArgumentException */ - boolean recordRejectedVote(int remoteNodeId); + default boolean recordRejectedVote(int remoteNodeId) { + boolean isPreVote = this instanceof ProspectiveState; + return epochElection().recordRejectedVote(remoteNodeId, isPreVote); + } /** * Record the current election has failed since we've either received sufficient rejecting voters or election timed out @@ -103,16 +87,7 @@ default boolean isVoteRejected() { * @return The set of unrecorded voters */ default Set unrecordedVoters() { - return votersInState(VoterState.State.UNRECORDED).collect(Collectors.toSet()); - } - - /** - * Get the set of voters that have granted our vote requests. - * - * @return The set of granting voters, which should always contain the localId - */ - default Set grantingVoters() { - return votersInState(VoterState.State.GRANTED).map(ReplicaKey::id).collect(Collectors.toSet()); + return epochElection().unrecordedVoters(); } /** @@ -121,33 +96,6 @@ default Set grantingVoters() { * @return The set of rejecting voters */ default Set rejectingVoters() { - return votersInState(VoterState.State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet()); - } - - final class VoterState { - private final ReplicaKey replicaKey; - private State state = State.UNRECORDED; - - VoterState(ReplicaKey replicaKey) { - this.replicaKey = replicaKey; - } - - public State state() { - return state; - } - - public void setState(State state) { - this.state = state; - } - - public ReplicaKey replicaKey() { - return replicaKey; - } - - enum State { - UNRECORDED, - GRANTED, - REJECTED - } + return epochElection().rejectingVoters(); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index ce86998b2f0a5..7b86b423a7ce9 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -19,23 +19,20 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.raft.EpochElection.VoterState.State; import org.slf4j.Logger; -import java.util.HashMap; -import java.util.Map; import java.util.Optional; import java.util.OptionalInt; -public class ProspectiveState implements VotingState { +public class ProspectiveState implements NomineeState { private final int localId; private final int epoch; private final OptionalInt leaderId; private final Optional leaderEndpoints; private final Optional votedKey; private final VoterSet voters; -// private final long electionTimeoutMs; -// private final Timer electionTimer; - private final Map preVoteStates = new HashMap<>(); + private final EpochElection epochElection; private final Optional highWatermark; private final int electionTimeoutMs; private final Timer electionTimer; @@ -76,10 +73,8 @@ public ProspectiveState( this.electionTimer = time.timer(electionTimeoutMs); this.log = logContext.logger(ProspectiveState.class); - for (ReplicaKey voter : voters.voterKeys()) { - preVoteStates.put(voter.id(), new VoterState(voter)); - } - preVoteStates.get(localId).setState(VoterState.State.GRANTED); + this.epochElection = new EpochElection(localId, voters.voterKeys()); + epochElection.voterStates().get(localId).setState(State.GRANTED); } public int localId() { @@ -91,8 +86,8 @@ public Optional votedKey() { } @Override - public Map voteStates() { - return preVoteStates; + public EpochElection epochElection() { + return epochElection; } @Override @@ -105,47 +100,6 @@ public int retries() { return 1; } - /** - * Record a granted vote from one of the voters. - * - * @param remoteNodeId The id of the voter - * @return true if the voter had not been previously recorded - * @throws IllegalArgumentException if the remote node is not a voter - */ - public boolean recordGrantedVote(int remoteNodeId) { - VoterState voterState = preVoteStates.get(remoteNodeId); - if (voterState == null) { - throw new IllegalArgumentException("Attempt to grant vote to non-voter " + remoteNodeId); - } - - boolean recorded = voterState.state().equals(VoterState.State.UNRECORDED); - voterState.setState(VoterState.State.GRANTED); - - return recorded; - } - - /** - * Record a rejected vote from one of the voters. - * - * @param remoteNodeId The id of the voter - * @return true if the rejected vote had not been previously recorded - * @throws IllegalArgumentException if the remote node is not a voter - */ - public boolean recordRejectedVote(int remoteNodeId) { - VoterState voterState = preVoteStates.get(remoteNodeId); - if (voterState == null) { - throw new IllegalArgumentException("Attempt to reject vote to non-voter " + remoteNodeId); - } - if (remoteNodeId == localId) { - throw new IllegalStateException("Attempted to reject vote from ourselves"); - } - - boolean recorded = voterState.state().equals(VoterState.State.UNRECORDED); - voterState.setState(VoterState.State.REJECTED); - - return recorded; - } - /** * Record the current election has failed since we've either received sufficient rejecting voters or election timed out */ @@ -154,14 +108,17 @@ public void startBackingOff(long currentTimeMs, long backoffDurationMs) { } @Override - public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate) { + public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolean isPreVote) { + if (isPreVote) { + return canGrantPreVote(replicaKey, isLogUpToDate); + } if (votedKey.isPresent()) { ReplicaKey votedReplicaKey = votedKey.get(); if (votedReplicaKey.id() == replicaKey.id()) { return votedReplicaKey.directoryId().isEmpty() || votedReplicaKey.directoryId().equals(replicaKey.directoryId()); } log.debug( - "Rejecting Vote request from candidate ({}), already have voted for another " + + "Rejecting Vote request (preVote=false) from candidate ({}), already have voted for another " + "candidate ({}) in epoch {}", replicaKey, votedKey, @@ -170,7 +127,7 @@ public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate) { return false; } else if (!isLogUpToDate) { log.debug( - "Rejecting Vote request from candidate ({}) since replica epoch/offset is not up to date with us", + "Rejecting Vote request (preVote=false) from candidate ({}) since candidate's log is not up to date with us", replicaKey ); } @@ -178,11 +135,10 @@ public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate) { return isLogUpToDate; } - @Override - public boolean canGrantPreVote(ReplicaKey replicaKey, boolean isLogUpToDate) { + private boolean canGrantPreVote(ReplicaKey replicaKey, boolean isLogUpToDate) { if (!isLogUpToDate) { log.debug( - "Rejecting PreVote request from prospective ({}) since prospective epoch/offset is not up to date with us", + "Rejecting Vote request (preVote=true) from prospective ({}) since prospective's log is not up to date with us", replicaKey ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 9dac5faf3d56d..cd4d95234330d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -775,9 +775,9 @@ public CandidateState candidateStateOrThrow() { throw new IllegalStateException("Expected to be Candidate, but current state is " + state); } - public VotingState votingStateOrThrow() { + public NomineeState votingStateOrThrow() { if (isVotingState()) - return (VotingState) state; + return (NomineeState) state; throw new IllegalStateException("Expected to be a VotingState (Prospective or Candidate), " + "but current state is " + state); } @@ -820,6 +820,6 @@ public boolean isCandidate() { } public boolean isVotingState() { - return state instanceof VotingState; + return state instanceof NomineeState; } } diff --git a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java index 94a596d4115b4..9c856d0c0bc19 100644 --- a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java @@ -146,7 +146,7 @@ public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolea return false; } else if (!isLogUpToDate) { log.debug( - "Rejecting Vote request (preVote=false) from candidate ({}) since candidate epoch/offset is not up to date with us", + "Rejecting Vote request (preVote=false) from candidate ({}) since candidate's log is not up to date with us", replicaKey ); } @@ -157,7 +157,7 @@ public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolea private boolean canGrantPreVote(ReplicaKey replicaKey, boolean isLogUpToDate) { if (!isLogUpToDate) { log.debug( - "Rejecting Vote request (preVote=true) from replica ({}) since replica's log is not up to date with us", + "Rejecting Vote request (preVote=true) from replica ({}) since prospective's log is not up to date with us", replicaKey ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index eca716da85bbb..f93ec2c7f2a5f 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -23,10 +23,10 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.raft.RaftClientTestContext.RaftProtocol; import org.apache.kafka.server.common.KRaftVersion; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; import java.util.Collections; @@ -44,17 +44,24 @@ public class KafkaRaftClientPreVoteTest { @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testHandlePreVoteRequestAsFollowerWithElectedLeader(boolean hasFetchedFromLeader) throws Exception { + @MethodSource("kraftVersionHasFetchedCombinations") + public void testHandlePreVoteRequestAsFollowerWithElectedLeader( + KRaftVersion kraftVersion, + boolean hasFetchedFromLeader + ) throws Exception { int localId = randomReplicaId(); - int epoch = 2; + ReplicaKey local = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - ReplicaKey observer = replicaKey(localId + 2, true); - int electedLeaderId = localId + 2; - Set voters = Set.of(localId, otherNodeKey.id(), electedLeaderId); + ReplicaKey electedLeader = replicaKey(localId + 2, true); + ReplicaKey observer = replicaKey(localId + 3, true); + int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withElectedLeader(epoch, electedLeaderId) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(local, otherNodeKey, electedLeader))), + kraftVersion + ) + .withElectedLeader(epoch, electedLeader.id()) .withRaftProtocol(KIP_996_PROTOCOL) .build(); @@ -66,7 +73,7 @@ public void testHandlePreVoteRequestAsFollowerWithElectedLeader(boolean hasFetch context.deliverResponse( fetchRequest.correlationId(), fetchRequest.destination(), - context.fetchResponse(epoch, electedLeaderId, MemoryRecords.EMPTY, 0L, Errors.NONE) + context.fetchResponse(epoch, electedLeader.id(), MemoryRecords.EMPTY, 0L, Errors.NONE) ); } @@ -75,15 +82,15 @@ public void testHandlePreVoteRequestAsFollowerWithElectedLeader(boolean hasFetch context.pollUntilResponse(); boolean voteGranted = !hasFetchedFromLeader; - context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(electedLeaderId), voteGranted); - context.assertElectedLeader(epoch, electedLeaderId); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(electedLeader.id()), voteGranted); + context.assertElectedLeader(epoch, electedLeader.id()); // same with observers context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 1)); context.pollUntilResponse(); - context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(electedLeaderId), voteGranted); - context.assertElectedLeader(epoch, electedLeaderId); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(electedLeader.id()), voteGranted); + context.assertElectedLeader(epoch, electedLeader.id()); // follower will transition to unattached if pre-vote request has a higher epoch context.deliverRequest(context.preVoteRequest(epoch + 1, otherNodeKey, epoch + 1, 1)); @@ -94,15 +101,20 @@ public void testHandlePreVoteRequestAsFollowerWithElectedLeader(boolean hasFetch assertTrue(context.client.quorum().isUnattachedNotVoted()); } - @Test - public void testHandlePreVoteRequestAsFollowerWithVotedCandidate() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testHandlePreVoteRequestAsFollowerWithVotedCandidate(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); int epoch = 2; + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); ReplicaKey votedCandidateKey = replicaKey(localId + 2, true); - Set voters = Set.of(localId, otherNodeKey.id(), votedCandidateKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(localKey), + Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey, votedCandidateKey))), + kraftVersion + ) .withVotedCandidate(epoch, votedCandidateKey) .withRaftProtocol(KIP_996_PROTOCOL) .build(); @@ -115,7 +127,8 @@ public void testHandlePreVoteRequestAsFollowerWithVotedCandidate() throws Except context.assertVotedCandidate(epoch, votedCandidateKey.id()); } - @Test + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) public void testHandlePreVoteRequestAsCandidate() throws Exception { int localId = randomReplicaId(); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); @@ -153,16 +166,21 @@ public void testHandlePreVoteRequestAsCandidate() throws Exception { assertTrue(context.client.quorum().isUnattached()); } - @Test - public void testHandlePreVoteRequestAsUnattachedObserver() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testHandlePreVoteRequestAsUnattachedObserver(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); int epoch = 2; + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey replica1 = replicaKey(localId + 1, true); ReplicaKey replica2 = replicaKey(localId + 2, true); ReplicaKey observer = replicaKey(localId + 3, true); - Set voters = Set.of(replica1.id(), replica2.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(localKey), + Optional.of(VoterSetTest.voterSet(Stream.of(replica1, replica2))), + kraftVersion + ) .withUnknownLeader(epoch) .withRaftProtocol(KIP_996_PROTOCOL) .build(); @@ -195,16 +213,21 @@ public void testHandlePreVoteRequestAsUnattachedObserver() throws Exception { context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); } - @Test - public void testHandlePreVoteRequestAsUnattachedVoted() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testHandlePreVoteRequestAsUnattachedVoted(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); - int epoch = 2; + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey replica1 = replicaKey(localId + 1, true); ReplicaKey replica2 = replicaKey(localId + 2, true); ReplicaKey observer = replicaKey(localId + 3, true); - Set voters = Set.of(replica1.id(), replica2.id()); + int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(localKey), + Optional.of(VoterSetTest.voterSet(Stream.of(replica1, replica2))), + kraftVersion + ) .withVotedCandidate(epoch, replica2) .withRaftProtocol(KIP_996_PROTOCOL) .build(); @@ -237,17 +260,22 @@ public void testHandlePreVoteRequestAsUnattachedVoted() throws Exception { context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); } - @Test - public void testHandlePreVoteRequestAsUnattachedWithLeader() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testHandlePreVoteRequestAsUnattachedWithLeader(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); - int epoch = 2; + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey replica1 = replicaKey(localId + 1, true); ReplicaKey replica2 = replicaKey(localId + 2, true); ReplicaKey leader = replicaKey(localId + 3, true); ReplicaKey observer = replicaKey(localId + 4, true); - Set voters = Set.of(replica1.id(), replica2.id()); + int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(localKey), + Optional.of(VoterSetTest.voterSet(Stream.of(replica1, replica2))), + kraftVersion + ) .withElectedLeader(epoch, leader.id()) .withRaftProtocol(KIP_996_PROTOCOL) .build(); @@ -281,18 +309,24 @@ public void testHandlePreVoteRequestAsUnattachedWithLeader() throws Exception { } @ParameterizedTest - @ValueSource(booleans = {false}) - public void testHandlePreVoteRequestAsFollowerObserver(boolean hasFetchedFromLeader) throws Exception { + @MethodSource("kraftVersionHasFetchedCombinations") + public void testHandlePreVoteRequestAsFollowerObserverThatHasNotFetched( + KRaftVersion kraftVersion, + boolean hasFetchedFromLeader + ) throws Exception { int localId = randomReplicaId(); - int epoch = 2; ReplicaKey localKey = replicaKey(localId, true); ReplicaKey leader = replicaKey(localId + 1, true); ReplicaKey follower = replicaKey(localId + 2, true); + int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(localKey), + Optional.of(VoterSetTest.voterSet(Stream.of(localKey, leader, follower))), + kraftVersion + ) .withElectedLeader(epoch, leader.id()) .withRaftProtocol(KIP_996_PROTOCOL) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, leader, follower)))) .build(); context.assertElectedLeader(epoch, leader.id()); @@ -316,7 +350,8 @@ public void testHandlePreVoteRequestAsFollowerObserver(boolean hasFetchedFromLea assertTrue(context.client.quorum().isFollower()); } - @Test + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) public void testHandleInvalidPreVoteRequestWithOlderEpoch() throws Exception { int localId = randomReplicaId(); int epoch = 2; @@ -335,7 +370,8 @@ public void testHandleInvalidPreVoteRequestWithOlderEpoch() throws Exception { context.assertUnknownLeaderAndNoVotedCandidate(epoch); } - @Test + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) public void testLeaderRejectPreVoteRequestOnSameEpoch() throws Exception { int localId = randomReplicaId(); ReplicaKey localKey = replicaKey(localId, true); @@ -358,7 +394,8 @@ public void testLeaderRejectPreVoteRequestOnSameEpoch() throws Exception { context.assertElectedLeader(leaderEpoch, localId); } - @Test + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) public void testPreVoteRequestClusterIdValidation() throws Exception { int localId = randomReplicaId(); ReplicaKey localKey = replicaKey(localId, true); @@ -393,7 +430,8 @@ public void testPreVoteRequestClusterIdValidation() throws Exception { context.assertSentVoteResponse(Errors.INCONSISTENT_CLUSTER_ID); } - @Test + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) public void testInvalidVoterReplicaPreVoteRequest() throws Exception { int localId = randomReplicaId(); ReplicaKey localKey = replicaKey(localId, true); @@ -438,7 +476,8 @@ public void testInvalidVoterReplicaPreVoteRequest() throws Exception { context.assertSentVoteResponse(Errors.INVALID_VOTER_KEY, epoch, OptionalInt.of(localId), false); } - @Test + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) public void testLeaderAcceptPreVoteFromObserver() throws Exception { int localId = randomReplicaId(); ReplicaKey localKey = replicaKey(localId, true); @@ -463,7 +502,8 @@ public void testLeaderAcceptPreVoteFromObserver() throws Exception { context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(localId), false); } - @Test + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) public void testHandlePreVoteRequestAsResigned() throws Exception { int localId = randomReplicaId(); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); @@ -489,7 +529,8 @@ public void testHandlePreVoteRequestAsResigned() throws Exception { assertTrue(context.client.quorum().isUnattached()); } - @Test + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) public void testInvalidPreVoteRequest() throws Exception { int localId = randomReplicaId(); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); @@ -584,18 +625,21 @@ public void testFollowerGrantsPreVoteIfHasNotFetchedYet(KRaftVersion kraftVersio assertTrue(context.client.quorum().isFollower()); } -<<<<<<< HEAD - @Test - public void testRejectPreVoteIfRemoteLogIsNotUpToDate() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testRejectPreVoteIfRemoteLogIsNotUpToDate(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); int epoch = 2; + ReplicaKey local = replicaKey(localId, true); ReplicaKey replica1 = replicaKey(localId + 1, true); ReplicaKey replica2 = replicaKey(localId + 2, true); - Set voters = Set.of(localId, replica1.id(), replica2.id()); - - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(local, replica1, replica2))), + kraftVersion + ) .withUnknownLeader(epoch) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .appendToLog(epoch, Arrays.asList("a", "b", "c")) .build(); assertTrue(context.client.quorum().isUnattached()); @@ -755,17 +799,23 @@ public void testPreVoteNotSupportedByRemote(KRaftVersion kraftVersion) throws Ex } @ParameterizedTest - @EnumSource(value = RaftProtocol.class) - public void testProspectiveReceivesBeginQuorumRequest(RaftProtocol raftProtocol) throws Exception { + @MethodSource("kraftVersionRaftProtocolCombinations") + public void testProspectiveReceivesBeginQuorumRequest( + KRaftVersion kraftVersion, + RaftProtocol raftProtocol + ) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey leader = replicaKey(localId + 1, true); int epoch = 5; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(local, leader))), + kraftVersion + ) .withUnknownLeader(epoch) .withRaftProtocol(raftProtocol) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, leader)))) .build(); context.assertUnknownLeaderAndNoVotedCandidate(epoch); @@ -783,17 +833,24 @@ public void testProspectiveReceivesBeginQuorumRequest(RaftProtocol raftProtocol) context.assertElectedLeader(epoch, leader.id()); } - @Test - public void testProspectiveTransitionsToUnattachedOnElectionFailure() throws Exception { + @ParameterizedTest + @MethodSource("kraftVersionRaftProtocolCombinations") + public void testProspectiveTransitionsToUnattachedOnElectionFailure( + KRaftVersion kraftVersion, + RaftProtocol raftProtocol + ) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey otherNode = replicaKey(localId + 1, true); int epoch = 5; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(local, otherNode))), + kraftVersion + ) .withUnknownLeader(epoch) - .withRaftProtocol(KIP_996_PROTOCOL) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, otherNode)))) + .withRaftProtocol(raftProtocol) .build(); context.assertUnknownLeaderAndNoVotedCandidate(epoch); @@ -841,18 +898,25 @@ public void testProspectiveTransitionsToUnattachedOnElectionFailure() throws Exc context.assertElectedLeader(epoch + 1, localId + 2); } - @Test - public void testProspectiveWithLeaderTransitionsToFollower() throws Exception { + @ParameterizedTest + @MethodSource("kraftVersionRaftProtocolCombinations") + public void testProspectiveWithLeaderTransitionsToFollower( + KRaftVersion kraftVersion, + RaftProtocol raftProtocol + ) throws Exception { int localId = randomReplicaId(); ReplicaKey local = replicaKey(localId, true); ReplicaKey replica1 = replicaKey(localId + 1, true); ReplicaKey replica2 = replicaKey(localId + 2, true); int epoch = 5; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(local, replica1, replica2))), + kraftVersion + ) .withElectedLeader(epoch, replica1.id()) - .withRaftProtocol(KIP_996_PROTOCOL) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(local, replica1, replica2)))) + .withRaftProtocol(raftProtocol) .build(); context.assertElectedLeader(epoch, replica1.id()); assertTrue(context.client.quorum().isFollower()); @@ -912,21 +976,25 @@ public void testProspectiveWithLeaderTransitionsToFollower() throws Exception { context.assertElectedLeader(epoch + 1, replica2.id()); } - @Test - public void testProspectiveWithoutLeaderTransitionsToFollower() throws Exception { + @ParameterizedTest + @MethodSource("kraftVersionRaftProtocolCombinations") + public void testProspectiveWithoutLeaderTransitionsToFollower( + KRaftVersion kraftVersion, + RaftProtocol raftProtocol + ) throws Exception { ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey leader = replicaKey(local.id() + 1, true); ReplicaKey follower = replicaKey(local.id() + 2, true); int epoch = 5; - VoterSet voters = VoterSetTest.voterSet(Stream.of(local, leader, follower)); - - RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) - .withRaftProtocol(KIP_996_PROTOCOL) - .withBootstrapSnapshot(Optional.of(voters)) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(local, leader, follower))), + kraftVersion + ) .withUnknownLeader(epoch) + .withRaftProtocol(raftProtocol) .build(); - context.assertUnknownLeaderAndNoVotedCandidate(epoch); // Sleep a little to ensure that we transition to Prospective @@ -951,4 +1019,16 @@ public void testProspectiveWithoutLeaderTransitionsToFollower() throws Exception assertTrue(context.client.quorum().isFollower()); assertEquals(leader.id(), context.client.quorum().leaderId().orElse(-1)); } + + static Stream kraftVersionRaftProtocolCombinations() { + return Stream.of(KRaftVersion.values()) + .flatMap(enum1 -> Stream.of(RaftProtocol.values()) + .map(enum2 -> Arguments.of(enum1, enum2))); + } + + static Stream kraftVersionHasFetchedCombinations() { + return Stream.of(KRaftVersion.values()) + .flatMap(enum1 -> Stream.of(true, false) + .map(enum2 -> Arguments.of(enum1, enum2))); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java index b62445a25f82d..aa9398efcf187 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -248,13 +248,13 @@ public void testGrantVote(boolean isLogUpToDate, boolean withDirectoryId) { voterSetWithLocal(Stream.of(node1, node2), withDirectoryId) ); - assertEquals(isLogUpToDate, state.canGrantPreVote(node0, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantPreVote(node1, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantPreVote(node2, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, true)); - assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate, false)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, false)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, false)); } @ParameterizedTest @@ -270,13 +270,13 @@ public void testGrantVoteWithVotedKey(boolean isLogUpToDate, boolean withDirecto Optional.of(node1) ); - assertEquals(isLogUpToDate, state.canGrantPreVote(node0, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantPreVote(node1, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantPreVote(node2, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, true)); - assertFalse(state.canGrantVote(node0, isLogUpToDate)); - assertTrue(state.canGrantVote(node1, isLogUpToDate)); - assertFalse(state.canGrantVote(node2, isLogUpToDate)); + assertFalse(state.canGrantVote(node0, isLogUpToDate, false)); + assertTrue(state.canGrantVote(node1, isLogUpToDate, false)); + assertFalse(state.canGrantVote(node2, isLogUpToDate, false)); } @ParameterizedTest @@ -292,13 +292,13 @@ public void testGrantVoteWithLeader(boolean isLogUpToDate, boolean withDirectory Optional.empty() ); - assertEquals(isLogUpToDate, state.canGrantPreVote(node0, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantPreVote(node1, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantPreVote(node2, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, true)); - assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate)); - assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate, false)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, false)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, false)); } @ParameterizedTest @@ -374,22 +374,22 @@ public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { assertEquals( isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); - assertTrue(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertTrue(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); assertEquals( isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate, true) ); - assertTrue(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); + assertTrue(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate, false)); // Can grant PreVote to other replicas even if we have granted a standard vote to another replica assertEquals( isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); } @ParameterizedTest @@ -404,37 +404,37 @@ void testCanGrantVoteWithDirectoryId(boolean isLogUpToDate) { // We will not grant PreVote for a replica we have already granted a standard vote to if their log is behind assertEquals( isLogUpToDate, - state.canGrantPreVote(votedKeyWithDirectoryId, isLogUpToDate) + state.canGrantVote(votedKeyWithDirectoryId, isLogUpToDate, true) ); - assertTrue(state.canGrantVote(votedKeyWithDirectoryId, isLogUpToDate)); + assertTrue(state.canGrantVote(votedKeyWithDirectoryId, isLogUpToDate, false)); // Different directoryId // We can grant PreVote for a replica we have already granted a standard vote to if their log is up-to-date, // even if the directoryId is different assertEquals( isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate, true) ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate, false)); // Missing directoryId assertEquals( isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); // Different voterId assertEquals( isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId + 1, votedDirectoryId), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), isLogUpToDate, true) ); assertEquals( isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) + state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), true)); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), true)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), true, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), true, false)); } @Test diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java deleted file mode 100644 index af169891e0690..0000000000000 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateWithVoteTest.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.raft; - -import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.MockTime; - -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; - -import java.util.Collections; -import java.util.Optional; -import java.util.OptionalInt; -import java.util.stream.IntStream; -import java.util.stream.Stream; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - -class ProspectiveStateWithVoteTest { - - private final MockTime time = new MockTime(); - private final LogContext logContext = new LogContext(); - private final int localId = 0; - private final ReplicaKey localReplicaKey = ReplicaKey.of(localId, Uuid.randomUuid()); - private final int epoch = 5; - private final int votedId = 1; - private final Uuid votedDirectoryId = Uuid.randomUuid(); - private final ReplicaKey votedKeyWithDirectoryId = ReplicaKey.of(votedId, votedDirectoryId); - private final ReplicaKey votedKeyWithoutDirectoryId = ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID); - private final int electionTimeoutMs = 10000; - - private ProspectiveState newProspectiveVotedState( - VoterSet voters, - Optional votedKey - ) { - return new ProspectiveState( - time, - localId, - epoch, - OptionalInt.empty(), - Optional.empty(), - votedKey, - voters, - Optional.empty(), - electionTimeoutMs, - logContext - ); - } - - @Test - public void testElectionTimeout() { - ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithDirectoryId)); - - assertEquals(epoch, state.epoch()); - assertEquals(votedKeyWithDirectoryId, state.votedKey().get()); - assertEquals( - ElectionState.withVotedCandidate(epoch, votedKeyWithDirectoryId, Collections.singleton(localId)), - state.election() - ); - assertEquals(electionTimeoutMs, state.remainingElectionTimeMs(time.milliseconds())); - assertFalse(state.hasElectionTimeoutExpired(time.milliseconds())); - - time.sleep(5000); - assertEquals(electionTimeoutMs - 5000, state.remainingElectionTimeMs(time.milliseconds())); - assertFalse(state.hasElectionTimeoutExpired(time.milliseconds())); - - time.sleep(5000); - assertEquals(0, state.remainingElectionTimeMs(time.milliseconds())); - assertTrue(state.hasElectionTimeoutExpired(time.milliseconds())); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { - ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithoutDirectoryId)); - - assertEquals( - isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) - ); - assertTrue(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); - - assertEquals( - isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate) - ); - assertTrue(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); - - // Can grant PreVote to other replicas even if we have granted a standard vote to another replica - assertEquals( - isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) - ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - void testCanGrantVoteWithDirectoryId(boolean isLogUpToDate) { - ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithDirectoryId)); - - // Same voterKey - // We will not grant PreVote for a replica we have already granted a standard vote to if their log is behind - assertEquals( - isLogUpToDate, - state.canGrantPreVote(votedKeyWithDirectoryId, isLogUpToDate) - ); - assertTrue(state.canGrantVote(votedKeyWithDirectoryId, isLogUpToDate)); - - // Different directoryId - // We can grant PreVote for a replica we have already granted a standard vote to if their log is up-to-date, - // even if the directoryId is different - assertEquals( - isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate) - ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate)); - - // Missing directoryId - assertEquals( - isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) - ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate)); - - // Different voterId - assertEquals( - isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId + 1, votedDirectoryId), isLogUpToDate) - ); - assertEquals( - isLogUpToDate, - state.canGrantPreVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate) - ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), true)); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), true)); - } - - @Test - void testLeaderEndpoints() { - ProspectiveState state = newProspectiveVotedState(voterSetWithLocal(IntStream.empty(), true), Optional.of(votedKeyWithDirectoryId)); - - assertEquals(Endpoints.empty(), state.leaderEndpoints()); - } - - private ReplicaKey replicaKey(int id, boolean withDirectoryId) { - Uuid directoryId = withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID; - return ReplicaKey.of(id, directoryId); - } - - private VoterSet voterSetWithLocal(IntStream remoteVoterIds, boolean withDirectoryId) { - Stream remoteVoterKeys = remoteVoterIds - .boxed() - .map(id -> replicaKey(id, withDirectoryId)); - - return voterSetWithLocal(remoteVoterKeys, withDirectoryId); - } - - private VoterSet voterSetWithLocal(Stream remoteVoterKeys, boolean withDirectoryId) { - ReplicaKey actualLocalVoter = withDirectoryId ? - localReplicaKey : - ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID); - - return VoterSetTest.voterSet( - Stream.concat(Stream.of(actualLocalVoter), remoteVoterKeys) - ); - } -} - From b8a40025c2c153397dae67fb03323cc3de0f5816 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Mon, 16 Dec 2024 21:19:19 -0800 Subject: [PATCH 08/23] interface changes --- .../org/apache/kafka/raft/CandidateState.java | 30 ++-- .../org/apache/kafka/raft/EpochElection.java | 129 ++++++++--------- .../apache/kafka/raft/KafkaRaftClient.java | 62 +++----- .../org/apache/kafka/raft/NomineeState.java | 62 +------- .../apache/kafka/raft/ProspectiveState.java | 28 ++-- .../org/apache/kafka/raft/QuorumState.java | 6 +- .../apache/kafka/raft/CandidateStateTest.java | 68 ++++----- .../apache/kafka/raft/EpochElectionTest.java | 137 ++++++++++++++++++ .../kafka/raft/ProspectiveStateTest.java | 72 ++++----- .../apache/kafka/raft/QuorumStateTest.java | 18 +-- 10 files changed, 329 insertions(+), 283 deletions(-) create mode 100644 raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 6f3eb9e264cba..33c2b8932d2b6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -20,12 +20,10 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.raft.EpochElection.VoterState.State; import org.slf4j.Logger; import java.util.Optional; -import java.util.Set; public class CandidateState implements NomineeState { private final int localId; @@ -82,8 +80,8 @@ protected CandidateState( this.backoffTimer = time.timer(0); this.log = logContext.logger(CandidateState.class); - this.epochElection = new EpochElection(localId, voters.voterKeys()); - epochElection.voterStates().get(localId).setState(State.GRANTED); + this.epochElection = new EpochElection(voters.voterKeys()); + epochElection.recordVote(localId, true); //voterStates().get(localId).setState(State.GRANTED); } public int localId() { @@ -95,6 +93,24 @@ public EpochElection epochElection() { return epochElection; } + @Override + public boolean recordGrantedVote(int remoteNodeId) { + if (epochElection().isRejectedVoter(remoteNodeId)) { + throw new IllegalArgumentException("Attempt to grant vote from node " + remoteNodeId + + " which previously rejected our request"); + } + return epochElection().recordVote(remoteNodeId, true); + } + + @Override + public boolean recordRejectedVote(int remoteNodeId) { + if (epochElection().isGrantedVoter(remoteNodeId)) { + throw new IllegalArgumentException("Attempt to reject vote from node " + remoteNodeId + + " which previously granted our request"); + } + return epochElection().recordVote(remoteNodeId, false); + } + /** * Check if the candidate is backing off for the next election */ @@ -140,16 +156,12 @@ public long remainingBackoffMs(long currentTimeMs) { return backoffTimer.remainingMs(); } - Set grantingVoters() { - return epochElection().grantingVoters(); - } - @Override public ElectionState election() { return ElectionState.withVotedCandidate( epoch, ReplicaKey.of(localId, localDirectoryId), - epochElection.voterStates().keySet() + epochElection.voterIds() ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/EpochElection.java b/raft/src/main/java/org/apache/kafka/raft/EpochElection.java index 83d2d718260c0..3cbedf8988cc3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/EpochElection.java +++ b/raft/src/main/java/org/apache/kafka/raft/EpochElection.java @@ -16,17 +16,17 @@ */ package org.apache.kafka.raft; +import java.util.Collection; +import java.util.Collections; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; public class EpochElection { - private final int localId; private Map voterStates; - EpochElection(int localId, Set voters) { - this.localId = localId; + EpochElection(Set voters) { this.voterStates = voters.stream() .collect(Collectors.toMap( ReplicaKey::id, @@ -34,28 +34,42 @@ public class EpochElection { )); } - Stream voters(VoterState.State state) { - return voterStates - .values() - .stream() - .filter(voterState -> voterState.state().equals(state)) - .map(VoterState::replicaKey); + VoterState getVoterStateOrThrow(int voterId) { + VoterState voterState = voterStates.get(voterId); + if (voterState == null) { + throw new IllegalArgumentException("Attempt to access voter state of non-voter " + voterId); + } + return voterState; } - Map voterStates() { - return voterStates; + boolean recordVote(int voterId, boolean isGranted) { + boolean wasUnrecorded = false; + VoterState voterState = getVoterStateOrThrow(voterId); + if (voterState.state == VoterState.State.UNRECORDED) { + wasUnrecorded = true; + } + if (isGranted) { + voterState.setState(VoterState.State.GRANTED); + } else { + voterState.setState(VoterState.State.REJECTED); + } + return wasUnrecorded; } - int majoritySize() { - return voterStates.size() / 2 + 1; + boolean isGrantedVoter(int voterId) { + return getVoterStateOrThrow(voterId).state == VoterState.State.GRANTED; + } + + boolean isRejectedVoter(int voterId) { + return getVoterStateOrThrow(voterId).state == VoterState.State.REJECTED; } - long numGranted() { - return voters(VoterState.State.GRANTED).count(); + Set voterIds() { + return Collections.unmodifiableSet(voterStates.keySet()); } - long numUnrecorded() { - return voters(VoterState.State.UNRECORDED).count(); + Collection voterStates() { + return Collections.unmodifiableCollection(voterStates.values()); } /** @@ -67,57 +81,14 @@ boolean isVoteGranted() { return numGranted() >= majoritySize(); } - boolean isVoteRejected() { - return numGranted() + numUnrecorded() < majoritySize(); - } - - // this could be on prospective & candidate to check. this becomes grabbing the previous value and one generic record the vote (without checks) /** - * Record a granted vote from one of the voters. + * Check if we have received enough rejections that it is no longer possible to reach a + * majority of grants. * - * @param remoteNodeId The id of the voter - * @param isPreVote Whether the vote is a PreVote (non-binding) or not (binding) - * @return true if the voter had not been previously recorded - * @throws IllegalArgumentException if the remote node is not a voter or if the vote had already been - * rejected by this node + * @return true if the vote is rejected, false if the vote is already or can still be granted */ - boolean recordGrantedVote(int remoteNodeId, boolean isPreVote) { - VoterState voterState = voterStates.get(remoteNodeId); - if (voterState == null) { - throw new IllegalArgumentException("Attempt to grant vote to non-voter " + remoteNodeId); - } else if (!isPreVote && voterState.state().equals(VoterState.State.REJECTED)) { - throw new IllegalArgumentException("Attempt to grant vote from node " + remoteNodeId + - " which previously rejected our request"); - } - boolean recorded = voterState.state().equals(VoterState.State.UNRECORDED); - voterState.setState(VoterState.State.GRANTED); - - return recorded; - } - - /** - * Record a rejected vote from one of the voters. - * - * @param remoteNodeId The id of the voter - * @param isPreVote Whether the vote is a PreVote (non-binding) or not (binding) - * @return true if the rejected vote had not been previously recorded - * @throws IllegalArgumentException if the remote node is not a voter or if the vote had already been - * granted by this node - */ - public boolean recordRejectedVote(int remoteNodeId, boolean isPreVote) { - VoterState voterState = voterStates.get(remoteNodeId); - if (voterState == null) { - throw new IllegalArgumentException("Attempt to reject vote to non-voter " + remoteNodeId); - } else if (isPreVote && remoteNodeId == localId) { - throw new IllegalStateException("Attempted to reject vote from ourselves"); - } else if (!isPreVote && voterState.state().equals(VoterState.State.GRANTED)) { - throw new IllegalArgumentException("Attempt to reject vote from node " + remoteNodeId + - " which previously granted our request"); - } - boolean recorded = voterState.state().equals(VoterState.State.UNRECORDED); - voterState.setState(VoterState.State.REJECTED); - - return recorded; + boolean isVoteRejected() { + return numGranted() + numUnrecorded() < majoritySize(); } /** @@ -126,7 +97,7 @@ public boolean recordRejectedVote(int remoteNodeId, boolean isPreVote) { * @return The set of unrecorded voters */ Set unrecordedVoters() { - return voters(VoterState.State.UNRECORDED).collect(Collectors.toSet()); + return votersOfState(VoterState.State.UNRECORDED).collect(Collectors.toSet()); } /** @@ -135,7 +106,7 @@ Set unrecordedVoters() { * @return The set of granting voters, which should always contain the localId */ Set grantingVoters() { - return voters(VoterState.State.GRANTED).map(ReplicaKey::id).collect(Collectors.toSet()); + return votersOfState(VoterState.State.GRANTED).map(ReplicaKey::id).collect(Collectors.toSet()); } /** @@ -144,10 +115,30 @@ Set grantingVoters() { * @return The set of rejecting voters */ Set rejectingVoters() { - return voters(VoterState.State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet()); + return votersOfState(VoterState.State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet()); + } + + private Stream votersOfState(VoterState.State state) { + return voterStates + .values() + .stream() + .filter(voterState -> voterState.state().equals(state)) + .map(VoterState::replicaKey); + } + + private long numGranted() { + return votersOfState(VoterState.State.GRANTED).count(); + } + + private long numUnrecorded() { + return votersOfState(VoterState.State.UNRECORDED).count(); + } + + private int majoritySize() { + return voterStates.size() / 2 + 1; } - static final class VoterState { + private static final class VoterState { private final ReplicaKey replicaKey; private State state = State.UNRECORDED; diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 88d37793edea2..3851852645dff 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -657,7 +657,7 @@ private void flushLeaderLog(LeaderState state, long currentTimeMs) { } private boolean maybeTransitionToLeader(CandidateState state, long currentTimeMs) { - if (state.isVoteGranted()) { + if (state.epochElection().isVoteGranted()) { onBecomeLeader(currentTimeMs); return true; } else { @@ -666,7 +666,7 @@ private boolean maybeTransitionToLeader(CandidateState state, long currentTimeMs } private boolean maybeTransitionToCandidate(ProspectiveState state, long currentTimeMs) { - if (state.isVoteGranted()) { + if (state.epochElection().isVoteGranted()) { transitionToCandidate(currentTimeMs); return true; } else { @@ -1012,17 +1012,18 @@ private void handleVoteResponse(NomineeState state, state.recordRejectedVote(remoteNodeId); // If our vote is rejected, we go immediately to backoff phase. This ensures that we are not stuck - // waiting for the election timeout when the vote has become gridlocked. Note for Prospective nodes - // startBackingOff just signals to transition to Unattached and attempt discovering the leader again, and - // Unattached's election timeout acts as the backoff phase for Prospective if the leader is not found. - if (state.isVoteRejected() && !state.isBackingOff()) { - logger.info("Insufficient remaining votes to win election (rejected by {}). " + - "We will backoff before retrying election again", state.rejectingVoters()); - - state.startBackingOff( - currentTimeMs, - binaryExponentialElectionBackoffMs(state.retries()) - ); + // waiting for the election timeout when the vote has become gridlocked. + if (state.epochElection().isVoteRejected() && quorum.isCandidate()) { + CandidateState candidateState = quorum.candidateStateOrThrow(); + if (!candidateState.isBackingOff()) { + logger.info("Insufficient remaining votes to win election (rejected by {}). We will backoff " + + "before retrying election again", candidateState.epochElection().rejectingVoters()); + + candidateState.startBackingOff( + currentTimeMs, + binaryExponentialElectionBackoffMs(candidateState.retries()) + ); + } } } } @@ -3055,44 +3056,17 @@ private long pollLeader(long currentTimeMs) { ) ); } -// -// private long maybeSendVoteRequests( -// CandidateState state, -// long currentTimeMs -// ) { -// // Continue sending Vote requests as long as we still have a chance to win the election -// if (!state.isVoteRejected()) { -// VoterSet voters = partitionState.lastVoterSet(); -// return maybeSendRequest( -// currentTimeMs, -// state.unrecordedVoters(), -// voterId -> voters -// .voterNode(voterId, channel.listenerName()) -// .orElseThrow(() -> -// new IllegalStateException( -// String.format( -// "Unknown endpoint for voter id %d for listener name %s", -// voterId, -// channel.listenerName() -// ) -// ) -// ), -// this::buildVoteRequest -// ); -// } -// return Long.MAX_VALUE; -// } private long maybeSendVoteRequests( NomineeState state, long currentTimeMs ) { // Continue sending Vote requests as long as we still have a chance to win the election - if (!state.isVoteRejected()) { + if (!state.epochElection().isVoteRejected()) { VoterSet voters = partitionState.lastVoterSet(); return maybeSendRequest( currentTimeMs, - state.unrecordedVoters(), + state.epochElection().unrecordedVoters(), voterId -> voters .voterNode(voterId, channel.listenerName()) .orElseThrow(() -> @@ -3148,7 +3122,7 @@ private long pollProspective(long currentTimeMs) { if (shutdown != null) { long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); - } else if (state.isBackingOff() || state.hasElectionTimeoutExpired(currentTimeMs)) { + } else if (state.epochElection().isVoteRejected() || state.hasElectionTimeoutExpired(currentTimeMs)) { if (state.election().hasLeader() && !state.leaderEndpoints().isEmpty()) { logger.info( "Election has timed out, transitioning to Follower of leader {}", @@ -3191,7 +3165,7 @@ private long pollFollowerAsVoter(FollowerState state, long currentTimeMs) { // skip the transition to candidate in any case. backoffMs = 0; } else if (state.hasFetchTimeoutExpired(currentTimeMs)) { - logger.info("Transitioning to VotingState (Prospective or Candidate) due to fetch timeout"); + logger.info("Transitioning to Prospective state due to fetch timeout"); transitionToProspective(currentTimeMs); backoffMs = 0; } else if (state.hasUpdateVoterPeriodExpired(currentTimeMs)) { diff --git a/raft/src/main/java/org/apache/kafka/raft/NomineeState.java b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java index 73873ecb85f0a..07fd7cedbd095 100644 --- a/raft/src/main/java/org/apache/kafka/raft/NomineeState.java +++ b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java @@ -16,38 +16,9 @@ */ package org.apache.kafka.raft; -import java.util.Set; - interface NomineeState extends EpochState { EpochElection epochElection(); - /** - * Check if the candidate is backing off for the next election - */ - boolean isBackingOff(); - - int retries(); - - /** - * Check whether we have received enough votes to conclude the election and become leader. - * - * @return true if at least a majority of nodes have granted the vote - */ - default boolean isVoteGranted() { - return epochElection().isVoteGranted(); - } - - /** - * Check if we have received enough rejections that it is no longer possible to reach a - * majority of grants. - * - * @return true if the vote is rejected, false if the vote is already or can still be granted - */ - default boolean isVoteRejected() { - return epochElection().isVoteRejected(); - } - - // override in prospective and candidate (they contain the validation for the vote now) /** * Record a granted vote from one of the voters. * @@ -55,10 +26,7 @@ default boolean isVoteRejected() { * @return true if the voter had not been previously recorded * @throws IllegalArgumentException */ - default boolean recordGrantedVote(int remoteNodeId) { - boolean isPreVote = this instanceof ProspectiveState; - return epochElection().recordGrantedVote(remoteNodeId, isPreVote); - } + boolean recordGrantedVote(int remoteNodeId); /** * Record a rejected vote from one of the voters. @@ -67,35 +35,9 @@ default boolean recordGrantedVote(int remoteNodeId) { * @return true if the rejected vote had not been previously recorded * @throws IllegalArgumentException */ - default boolean recordRejectedVote(int remoteNodeId) { - boolean isPreVote = this instanceof ProspectiveState; - return epochElection().recordRejectedVote(remoteNodeId, isPreVote); - } - - /** - * Record the current election has failed since we've either received sufficient rejecting voters or election timed out - */ - void startBackingOff(long currentTimeMs, long backoffDurationMs); + boolean recordRejectedVote(int remoteNodeId); boolean hasElectionTimeoutExpired(long currentTimeMs); long remainingElectionTimeMs(long currentTimeMs); - - /** - * Get the set of voters which have not been counted as granted or rejected yet. - * - * @return The set of unrecorded voters - */ - default Set unrecordedVoters() { - return epochElection().unrecordedVoters(); - } - - /** - * Get the set of voters that have rejected our candidacy. - * - * @return The set of rejecting voters - */ - default Set rejectingVoters() { - return epochElection().rejectingVoters(); - } } diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index 7b86b423a7ce9..e984ea3b70460 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.raft.EpochElection.VoterState.State; import org.slf4j.Logger; import java.util.Optional; @@ -34,7 +33,6 @@ public class ProspectiveState implements NomineeState { private final VoterSet voters; private final EpochElection epochElection; private final Optional highWatermark; - private final int electionTimeoutMs; private final Timer electionTimer; private final Logger log; @@ -47,8 +45,6 @@ public class ProspectiveState implements NomineeState { * 4. If majority votes rejected or election times out, it will enter a backing off phase; * after the backoff phase completes, it will send out another round of PreVote requests. */ - private boolean isBackingOff; - public ProspectiveState( Time time, int localId, @@ -68,13 +64,11 @@ public ProspectiveState( this.votedKey = votedKey; this.voters = voters; this.highWatermark = highWatermark; - this.isBackingOff = false; - this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); this.log = logContext.logger(ProspectiveState.class); - this.epochElection = new EpochElection(localId, voters.voterKeys()); - epochElection.voterStates().get(localId).setState(State.GRANTED); + this.epochElection = new EpochElection(voters.voterKeys()); + epochElection.recordVote(localId, true); //voterStates().get(localId).setState(State.GRANTED); } public int localId() { @@ -91,20 +85,16 @@ public EpochElection epochElection() { } @Override - public boolean isBackingOff() { - return isBackingOff; + public boolean recordGrantedVote(int remoteNodeId) { + return epochElection().recordVote(remoteNodeId, true); } @Override - public int retries() { - return 1; - } - - /** - * Record the current election has failed since we've either received sufficient rejecting voters or election timed out - */ - public void startBackingOff(long currentTimeMs, long backoffDurationMs) { - this.isBackingOff = true; + public boolean recordRejectedVote(int remoteNodeId) { + if (remoteNodeId == localId) { + throw new IllegalStateException("Attempted to reject vote from ourselves"); + } + return epochElection().recordVote(remoteNodeId, false); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index cd4d95234330d..cb263a13449c6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -430,7 +430,7 @@ private void transitionToUnattached(int epoch, Optional votedKey, Op electionTimeoutMs = candidateStateOrThrow().remainingElectionTimeMs(time.milliseconds()); } else if (isUnattached()) { electionTimeoutMs = unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); - } else if (isProspective() && !prospectiveStateOrThrow().isBackingOff()) { + } else if (isProspective() && !prospectiveStateOrThrow().epochElection().isVoteRejected()) { electionTimeoutMs = prospectiveStateOrThrow().remainingElectionTimeMs(time.milliseconds()); } else { electionTimeoutMs = randomElectionTimeoutMs(); @@ -658,7 +658,7 @@ public LeaderState transitionToLeader(long epochStartOffset, BatchAccumul } CandidateState candidateState = candidateStateOrThrow(); - if (!candidateState.isVoteGranted()) + if (!candidateState.epochElection().isVoteGranted()) throw new IllegalStateException("Cannot become leader without majority votes granted"); // Note that the leader does not retain the high watermark that was known @@ -680,7 +680,7 @@ public LeaderState transitionToLeader(long epochStartOffset, BatchAccumul partitionState.lastVoterSet(), partitionState.lastVoterSetOffset(), partitionState.lastKraftVersion(), - candidateState.grantingVoters(), + candidateState.epochElection().grantingVoters(), accumulator, localListeners, fetchTimeoutMs, diff --git a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java index e494a306cad54..40263be7af4c2 100644 --- a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java @@ -61,9 +61,9 @@ private CandidateState newCandidateState(VoterSet voters) { @ValueSource(booleans = { true, false }) public void testSingleNodeQuorum(boolean withDirectoryId) { CandidateState state = newCandidateState(voterSetWithLocal(IntStream.empty(), withDirectoryId)); - assertTrue(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Collections.emptySet(), state.unrecordedVoters()); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); } @ParameterizedTest @@ -73,12 +73,12 @@ public void testTwoNodeQuorumVoteRejected(boolean withDirectoryId) { CandidateState state = newCandidateState( voterSetWithLocal(Stream.of(otherNode), withDirectoryId) ); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Collections.singleton(otherNode), state.unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Collections.singleton(otherNode), state.epochElection().unrecordedVoters()); assertTrue(state.recordRejectedVote(otherNode.id())); - assertFalse(state.isVoteGranted()); - assertTrue(state.isVoteRejected()); + assertFalse(state.epochElection().isVoteGranted()); + assertTrue(state.epochElection().isVoteRejected()); } @ParameterizedTest @@ -88,13 +88,13 @@ public void testTwoNodeQuorumVoteGranted(boolean withDirectoryId) { CandidateState state = newCandidateState( voterSetWithLocal(Stream.of(otherNode), withDirectoryId) ); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Collections.singleton(otherNode), state.unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Collections.singleton(otherNode), state.epochElection().unrecordedVoters()); assertTrue(state.recordGrantedVote(otherNode.id())); - assertEquals(Collections.emptySet(), state.unrecordedVoters()); - assertFalse(state.isVoteRejected()); - assertTrue(state.isVoteGranted()); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); + assertFalse(state.epochElection().isVoteRejected()); + assertTrue(state.epochElection().isVoteGranted()); } @ParameterizedTest @@ -105,17 +105,17 @@ public void testThreeNodeQuorumVoteGranted(boolean withDirectoryId) { CandidateState state = newCandidateState( voterSetWithLocal(Stream.of(node1, node2), withDirectoryId) ); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Set.of(node1, node2), state.unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Set.of(node1, node2), state.epochElection().unrecordedVoters()); assertTrue(state.recordGrantedVote(node1.id())); - assertEquals(Collections.singleton(node2), state.unrecordedVoters()); - assertTrue(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); + assertEquals(Collections.singleton(node2), state.epochElection().unrecordedVoters()); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); assertTrue(state.recordRejectedVote(node2.id())); - assertEquals(Collections.emptySet(), state.unrecordedVoters()); - assertTrue(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); } @ParameterizedTest @@ -126,17 +126,17 @@ public void testThreeNodeQuorumVoteRejected(boolean withDirectoryId) { CandidateState state = newCandidateState( voterSetWithLocal(Stream.of(node1, node2), withDirectoryId) ); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Set.of(node1, node2), state.unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Set.of(node1, node2), state.epochElection().unrecordedVoters()); assertTrue(state.recordRejectedVote(node1.id())); - assertEquals(Collections.singleton(node2), state.unrecordedVoters()); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); + assertEquals(Collections.singleton(node2), state.epochElection().unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); assertTrue(state.recordRejectedVote(node2.id())); - assertEquals(Collections.emptySet(), state.unrecordedVoters()); - assertFalse(state.isVoteGranted()); - assertTrue(state.isVoteRejected()); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertTrue(state.epochElection().isVoteRejected()); } @ParameterizedTest @@ -161,7 +161,7 @@ public void testCannotChangeVoteGrantedToRejected(boolean withDirectoryId) { ); assertTrue(state.recordGrantedVote(otherNodeId)); assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(otherNodeId)); - assertTrue(state.isVoteGranted()); + assertTrue(state.epochElection().isVoteGranted()); } @ParameterizedTest @@ -173,7 +173,7 @@ public void testCannotChangeVoteRejectedToGranted(boolean withDirectoryId) { ); assertTrue(state.recordRejectedVote(otherNodeId)); assertThrows(IllegalArgumentException.class, () -> state.recordGrantedVote(otherNodeId)); - assertTrue(state.isVoteRejected()); + assertTrue(state.epochElection().isVoteRejected()); } @ParameterizedTest diff --git a/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java b/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java new file mode 100644 index 0000000000000..ef37eb126b887 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import org.apache.kafka.common.Uuid; +import org.junit.jupiter.api.Test; + +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class EpochElectionTest { + static final int VOTER_1_ID = randomReplicaId(); + static final Set VOTERS = Set.of( + ReplicaKey.of(VOTER_1_ID, Uuid.randomUuid()), + ReplicaKey.of(VOTER_1_ID + 1, Uuid.randomUuid()), + ReplicaKey.of(VOTER_1_ID + 2, Uuid.randomUuid()) + ); + @Test + public void testStateOnInitialization() { + EpochElection epochElection = new EpochElection(VOTERS); + + assertEquals(VOTERS, epochElection.unrecordedVoters()); + assertTrue(epochElection.grantingVoters().isEmpty()); + assertTrue(epochElection.rejectingVoters().isEmpty()); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + assertFalse(epochElection.isGrantedVoter(VOTER_1_ID)); + assertFalse(epochElection.isRejectedVoter(VOTER_1_ID)); + } + + @Test + public void testRecordGrantedVote() { + EpochElection epochElection = new EpochElection(VOTERS); + + assertTrue(epochElection.recordVote(VOTER_1_ID, true)); + assertEquals(1, epochElection.grantingVoters().size()); + assertTrue(epochElection.grantingVoters().contains(VOTER_1_ID)); + assertEquals(0, epochElection.rejectingVoters().size()); + assertEquals(2, epochElection.unrecordedVoters().size()); + assertTrue(epochElection.isGrantedVoter(VOTER_1_ID)); + assertFalse(epochElection.isRejectedVoter(VOTER_1_ID)); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + + // recording same id as granted + assertFalse(epochElection.recordVote(VOTER_1_ID, true)); + assertTrue(epochElection.isGrantedVoter(VOTER_1_ID)); + assertFalse(epochElection.isVoteGranted()); + + // recording majority as granted + assertTrue(epochElection.recordVote(VOTER_1_ID + 1, true)); + assertEquals(2, epochElection.grantingVoters().size()); + assertEquals(0, epochElection.rejectingVoters().size()); + assertEquals(1, epochElection.unrecordedVoters().size()); + assertTrue(epochElection.isGrantedVoter(VOTER_1_ID + 1)); + assertFalse(epochElection.isRejectedVoter(VOTER_1_ID + 1)); + assertTrue(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + } + + @Test + public void testRecordRejectedVote() { + EpochElection epochElection = new EpochElection(VOTERS); + + assertTrue(epochElection.recordVote(VOTER_1_ID, false)); + assertEquals(0, epochElection.grantingVoters().size()); + assertEquals(1, epochElection.rejectingVoters().size()); + assertTrue(epochElection.rejectingVoters().contains(VOTER_1_ID)); + assertEquals(2, epochElection.unrecordedVoters().size()); + assertFalse(epochElection.isGrantedVoter(VOTER_1_ID)); + assertTrue(epochElection.isRejectedVoter(VOTER_1_ID)); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + + // recording same id as rejected + assertFalse(epochElection.recordVote(VOTER_1_ID, false)); + assertFalse(epochElection.isGrantedVoter(VOTER_1_ID)); + assertFalse(epochElection.isVoteRejected()); + + // recording majority as rejected + assertTrue(epochElection.recordVote(VOTER_1_ID + 1, false)); + assertEquals(0, epochElection.grantingVoters().size()); + assertEquals(2, epochElection.rejectingVoters().size()); + assertEquals(1, epochElection.unrecordedVoters().size()); + assertFalse(epochElection.isGrantedVoter(VOTER_1_ID + 1)); + assertTrue(epochElection.isRejectedVoter(VOTER_1_ID + 1)); + assertFalse(epochElection.isVoteGranted()); + assertTrue(epochElection.isVoteRejected()); + } + + @Test + public void testOverWritingVote() { + EpochElection epochElection = new EpochElection(VOTERS); + + assertTrue(epochElection.recordVote(VOTER_1_ID, true)); + assertFalse(epochElection.recordVote(VOTER_1_ID, false)); + assertEquals(0, epochElection.grantingVoters().size()); + assertEquals(1, epochElection.rejectingVoters().size()); + assertTrue(epochElection.rejectingVoters().contains(VOTER_1_ID)); + assertFalse(epochElection.isGrantedVoter(VOTER_1_ID)); + assertTrue(epochElection.isRejectedVoter(VOTER_1_ID)); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + + assertTrue(epochElection.recordVote(VOTER_1_ID + 2, false)); + assertFalse(epochElection.recordVote(VOTER_1_ID + 2, true)); + assertEquals(1, epochElection.grantingVoters().size()); + assertEquals(1, epochElection.rejectingVoters().size()); + assertTrue(epochElection.grantingVoters().contains(VOTER_1_ID + 2)); + assertTrue(epochElection.isGrantedVoter(VOTER_1_ID + 2)); + assertFalse(epochElection.isRejectedVoter(VOTER_1_ID + 2)); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + } + + static int randomReplicaId() { + return ThreadLocalRandom.current().nextInt(1025); + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java index aa9398efcf187..4e47a4f98e55e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -95,9 +95,9 @@ private ProspectiveState newProspectiveState(VoterSet voters) { @ValueSource(booleans = { true, false }) public void testSingleNodeQuorum(boolean withDirectoryId) { ProspectiveState state = newProspectiveState(voterSetWithLocal(IntStream.empty(), withDirectoryId)); - assertTrue(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Collections.emptySet(), state.unrecordedVoters()); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); } @ParameterizedTest @@ -107,12 +107,12 @@ public void testTwoNodeQuorumVoteRejected(boolean withDirectoryId) { ProspectiveState state = newProspectiveState( voterSetWithLocal(Stream.of(otherNode), withDirectoryId) ); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Collections.singleton(otherNode), state.unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Collections.singleton(otherNode), state.epochElection().unrecordedVoters()); assertTrue(state.recordRejectedVote(otherNode.id())); - assertFalse(state.isVoteGranted()); - assertTrue(state.isVoteRejected()); + assertFalse(state.epochElection().isVoteGranted()); + assertTrue(state.epochElection().isVoteRejected()); } @ParameterizedTest @@ -122,13 +122,13 @@ public void testTwoNodeQuorumVoteGranted(boolean withDirectoryId) { ProspectiveState state = newProspectiveState( voterSetWithLocal(Stream.of(otherNode), withDirectoryId) ); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Collections.singleton(otherNode), state.unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Collections.singleton(otherNode), state.epochElection().unrecordedVoters()); assertTrue(state.recordGrantedVote(otherNode.id())); - assertEquals(Collections.emptySet(), state.unrecordedVoters()); - assertFalse(state.isVoteRejected()); - assertTrue(state.isVoteGranted()); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); + assertFalse(state.epochElection().isVoteRejected()); + assertTrue(state.epochElection().isVoteGranted()); } @ParameterizedTest @@ -139,17 +139,17 @@ public void testThreeNodeQuorumVoteGranted(boolean withDirectoryId) { ProspectiveState state = newProspectiveState( voterSetWithLocal(Stream.of(node1, node2), withDirectoryId) ); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Set.of(node1, node2), state.unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Set.of(node1, node2), state.epochElection().unrecordedVoters()); assertTrue(state.recordGrantedVote(node1.id())); - assertEquals(Collections.singleton(node2), state.unrecordedVoters()); - assertTrue(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); + assertEquals(Collections.singleton(node2), state.epochElection().unrecordedVoters()); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); assertTrue(state.recordRejectedVote(node2.id())); - assertEquals(Collections.emptySet(), state.unrecordedVoters()); - assertTrue(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); } @ParameterizedTest @@ -160,17 +160,17 @@ public void testThreeNodeQuorumVoteRejected(boolean withDirectoryId) { ProspectiveState state = newProspectiveState( voterSetWithLocal(Stream.of(node1, node2), withDirectoryId) ); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); - assertEquals(Set.of(node1, node2), state.unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Set.of(node1, node2), state.epochElection().unrecordedVoters()); assertTrue(state.recordRejectedVote(node1.id())); - assertEquals(Collections.singleton(node2), state.unrecordedVoters()); - assertFalse(state.isVoteGranted()); - assertFalse(state.isVoteRejected()); + assertEquals(Collections.singleton(node2), state.epochElection().unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); assertTrue(state.recordRejectedVote(node2.id())); - assertEquals(Collections.emptySet(), state.unrecordedVoters()); - assertFalse(state.isVoteGranted()); - assertTrue(state.isVoteRejected()); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertTrue(state.epochElection().isVoteRejected()); } @ParameterizedTest @@ -195,14 +195,14 @@ public void testCanChangePreVote(boolean withDirectoryId) { voterSetWithLocal(IntStream.of(voter1, voter2), withDirectoryId) ); assertTrue(state.recordGrantedVote(voter1)); - assertTrue(state.isVoteGranted()); + assertTrue(state.epochElection().isVoteGranted()); assertFalse(state.recordRejectedVote(voter1)); - assertFalse(state.isVoteGranted()); + assertFalse(state.epochElection().isVoteGranted()); assertTrue(state.recordRejectedVote(voter2)); - assertTrue(state.isVoteRejected()); + assertTrue(state.epochElection().isVoteRejected()); assertFalse(state.recordGrantedVote(voter2)); - assertFalse(state.isVoteRejected()); + assertFalse(state.epochElection().isVoteRejected()); } diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index 1d31d9f65e70f..5b65b5ba059e3 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -172,7 +172,7 @@ public void testInitializePrimordialEpoch(KRaftVersion kraftVersion) { } state.transitionToCandidate(); CandidateState candidateState = state.candidateStateOrThrow(); - assertTrue(candidateState.isVoteGranted()); + assertTrue(candidateState.epochElection().isVoteGranted()); assertEquals(1, candidateState.epoch()); } @@ -299,9 +299,9 @@ public void testInitializeAsResignedCandidate(KRaftVersion kraftVersion) { ElectionState.withVotedCandidate(epoch, localVoterKey, voters.voterIds()), candidateState.election() ); - assertEquals(Set.of(node1, node2), candidateState.unrecordedVoters()); - assertEquals(Set.of(localId), candidateState.grantingVoters()); - assertEquals(Collections.emptySet(), candidateState.rejectingVoters()); + assertEquals(Set.of(node1, node2), candidateState.epochElection().unrecordedVoters()); + assertEquals(Set.of(localId), candidateState.epochElection().grantingVoters()); + assertEquals(Collections.emptySet(), candidateState.epochElection().rejectingVoters()); assertEquals( electionTimeoutMs + jitterMs, candidateState.remainingElectionTimeMs(time.milliseconds()) @@ -376,8 +376,8 @@ public void testCandidateToCandidate() { assertTrue(state.isCandidate()); CandidateState candidate2 = state.candidateStateOrThrow(); assertEquals(2, state.epoch()); - assertEquals(Collections.singleton(localId), candidate2.grantingVoters()); - assertEquals(Collections.emptySet(), candidate2.rejectingVoters()); + assertEquals(Collections.singleton(localId), candidate2.epochElection().grantingVoters()); + assertEquals(Collections.emptySet(), candidate2.epochElection().rejectingVoters()); assertEquals(electionTimeoutMs + jitterMs, candidate2.remainingElectionTimeMs(time.milliseconds())); } @@ -437,10 +437,10 @@ public void testCandidateToLeaderWithoutGrantedVote(KRaftVersion kraftVersion) { state.transitionToProspective(); } state.transitionToCandidate(); - assertFalse(state.candidateStateOrThrow().isVoteGranted()); + assertFalse(state.candidateStateOrThrow().epochElection().isVoteGranted()); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - assertTrue(state.candidateStateOrThrow().isVoteGranted()); + assertTrue(state.candidateStateOrThrow().epochElection().isVoteGranted()); state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); } @@ -1742,7 +1742,7 @@ public void testHighWatermarkRetained(KRaftVersion kraftVersion) { CandidateState candidateState = state.candidateStateOrThrow(); candidateState.recordGrantedVote(otherNodeKey.id()); - assertTrue(candidateState.isVoteGranted()); + assertTrue(candidateState.epochElection().isVoteGranted()); state.transitionToLeader(10L, accumulator); assertEquals(Optional.empty(), state.highWatermark()); From e621e7883183dc42cf9e894fba14bb97296bce5b Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Wed, 18 Dec 2024 01:09:54 -0800 Subject: [PATCH 09/23] needs additional revisions to voted states --- .../org/apache/kafka/raft/FollowerState.java | 7 ++ .../apache/kafka/raft/KafkaRaftClient.java | 17 +---- .../org/apache/kafka/raft/QuorumState.java | 70 ++++++------------- .../apache/kafka/raft/FollowerStateTest.java | 1 + .../raft/KafkaRaftClientPreVoteTest.java | 22 ++++-- .../apache/kafka/raft/QuorumStateTest.java | 67 ++++++++++++++++++ .../internals/FollowerStateWithVoteTest.java | 4 ++ 7 files changed, 118 insertions(+), 70 deletions(-) create mode 100644 raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index e3db67893bfc1..5443fc605d0ad 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -34,6 +34,7 @@ public class FollowerState implements EpochState { private final int epoch; private final int leaderId; private final Endpoints leaderEndpoints; + private final Optional votedKey; private final Set voters; // Used for tracking the expiration of both the Fetch and FetchSnapshot requests private final Timer fetchTimer; @@ -56,6 +57,7 @@ public FollowerState( int epoch, int leaderId, Endpoints leaderEndpoints, + Optional votedKey, Set voters, Optional highWatermark, int fetchTimeoutMs, @@ -65,6 +67,7 @@ public FollowerState( this.epoch = epoch; this.leaderId = leaderId; this.leaderEndpoints = leaderEndpoints; + this.votedKey = votedKey; this.voters = voters; this.fetchTimer = time.timer(fetchTimeoutMs); this.updateVoterPeriodTimer = time.timer(updateVoterPeriodMs()); @@ -118,6 +121,10 @@ public Node leaderNode(ListenerName listener) { ); } + public Optional votedKey() { + return votedKey; + } + public boolean hasFetchTimeoutExpired(long currentTimeMs) { fetchTimer.update(currentTimeMs); return fetchTimer.isExpired(); diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 3851852645dff..8eefffdad37e1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -729,16 +729,6 @@ private void transitionToUnattached(int epoch, OptionalInt leaderId) { resetConnections(); } - private void transitionToUnattachedVoted(int epoch, ReplicaKey candidateKey) { - if (quorum.isUnattachedNotVoted() && quorum.epoch() == epoch) { - quorum.unattachedAddVotedState(epoch, candidateKey); - } else { - quorum.transitionToUnattached(epoch, Optional.of(candidateKey)); - maybeFireLeaderChange(); - resetConnections(); - } - } - private void transitionToResigned(List preferredSuccessors) { fetchPurgatory.completeAllExceptionally( Errors.NOT_LEADER_OR_FOLLOWER.exception("Not handling request since this node is resigning")); @@ -908,7 +898,7 @@ private VoteResponseData handleVoteRequest( ); if (!preVote && voteGranted && quorum.isUnattachedNotVoted()) { - transitionToUnattachedVoted(replicaEpoch, replicaKey); + quorum.unattachedAddVotedState(replicaEpoch, replicaKey); } logger.info( @@ -3132,11 +3122,6 @@ private long pollProspective(long currentTimeMs) { state.election().leaderId(), state.leaderEndpoints(), currentTimeMs); - } else if (state.votedKey().isPresent()) { - logger.info( - "Election has timed out, transitioning to Unattached with votedKey={} to attempt rediscovering leader", - state.votedKey().get()); - transitionToUnattachedVoted(quorum().epoch(), state.votedKey().get()); } else { logger.info("Election has timed out, transitioning to Unattached to attempt rediscovering leader"); transitionToUnattached(quorum().epoch()); diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index cb263a13449c6..9582c179465eb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -245,6 +245,7 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateE election.epoch(), election.leaderId(), leaderEndpoints, + Optional.empty(), voters.voterIds(), Optional.empty(), fetchTimeoutMs, @@ -322,6 +323,10 @@ public OptionalInt leaderId() { return OptionalInt.empty(); } + public Optional votedKey() { + return state.election().optionalVotedKey(); + } + public boolean hasLeader() { return leaderId().isPresent(); } @@ -375,45 +380,23 @@ public void transitionToResigned(List preferredSuccessors) { } /** - * Transition to the "unattached" state. This means we have found an epoch greater than the current epoch, - * but we do not yet know of the elected leader. + * Transition to the "unattached" state. This means we have found an epoch greater than the current epoch + * and do not yet know of the elected leader, or we have transitioned from Prospective with the same epoch. + * Note, if we are transitioning from unattached and there is no epoch change, we take the path of + * unattachedAddVotedState instead. */ public void transitionToUnattached(int epoch) { - transitionToUnattached(epoch, Optional.empty(), OptionalInt.empty()); + transitionToUnattached(epoch, OptionalInt.empty()); } - /** - * Transition to the "unattached" state with votedKey. This means we have found an epoch greater than the - * current epoch, but we do not yet know of the elected leader. Note, if we are transitioning from unattached and - * no epoch change, we take the path of unattachedTransitionToUnattachedVotedState instead. - */ + // ahu todo: need to fix all tests which call this method. delete method afterwards public void transitionToUnattached(int epoch, Optional votedKey) { - transitionToUnattached(epoch, votedKey, OptionalInt.empty()); + transitionToUnattached(epoch, OptionalInt.empty()); } public void transitionToUnattached(int epoch, OptionalInt leaderId) { - transitionToUnattached(epoch, Optional.empty(), leaderId); - } - - /** - * Transition to the "unattached" state with votedKey. This means we have found an epoch greater than the - * current epoch, but we do not yet know of the elected leader. Note, if we are transitioning from unattached and - * no epoch change, we take the path of unattachedTransitionToUnattachedVotedState instead. - * It is invalid to have a votedKey AND leaderId in Unattached state (or any state). - */ - private void transitionToUnattached(int epoch, Optional votedKey, OptionalInt leaderId) { int currentEpoch = state.epoch(); - if (votedKey.isPresent() && leaderId.isPresent()) { - throw new IllegalStateException( - String.format( - "Cannot transition to Unattached with epoch= %d with both votedKey= %s and leaderId= %d from current state %s", - currentEpoch, - votedKey.get(), - leaderId.getAsInt(), - state - ) - ); - } else if (epoch < currentEpoch || (epoch == currentEpoch && !isProspective())) { + if (epoch < currentEpoch || (epoch == currentEpoch && !isProspective())) { throw new IllegalStateException( String.format( "Cannot transition to Unattached with epoch= %d from current state %s", @@ -440,7 +423,7 @@ private void transitionToUnattached(int epoch, Optional votedKey, Op time, epoch, leaderId, - votedKey, + epoch == currentEpoch ? votedKey() : Optional.empty(), partitionState.lastVoterSet().voterIds(), state.highWatermark(), electionTimeoutMs, @@ -467,24 +450,14 @@ public void unattachedAddVotedState( ) ); } else if (localId.isEmpty()) { - throw new IllegalStateException("Cannot transition to voted without a replica id"); - } else if (epoch < currentEpoch) { - throw new IllegalStateException( - String.format( - "Cannot transition to Voted for %s and epoch %d since the current epoch " + - "(%d) is larger", - candidateKey, - epoch, - currentEpoch - ) - ); - } else if (epoch == currentEpoch && !isUnattachedNotVoted()) { + throw new IllegalStateException("Cannot add voted state without a replica id"); + } else if (epoch != currentEpoch || !isUnattachedNotVoted()) { throw new IllegalStateException( String.format( - "Cannot transition to Voted for %s and epoch %d from the current state (%s)", + "Cannot add voted key (%s) to current state (%s) in epoch %d", candidateKey, - epoch, - state + state, + epoch ) ); } @@ -511,6 +484,7 @@ public void unattachedAddVotedState( */ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { int currentEpoch = state.epoch(); + boolean retainVotedKey = false; if (endpoints.isEmpty()) { throw new IllegalArgumentException( String.format( @@ -538,6 +512,7 @@ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { ) ); } else if (epoch == currentEpoch) { + retainVotedKey = true; if (isFollower() && state.leaderEndpoints().size() >= endpoints.size()) { throw new IllegalStateException( String.format( @@ -566,6 +541,7 @@ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { epoch, leaderId, endpoints, + retainVotedKey ? votedKey() : Optional.empty(), partitionState.lastVoterSet().voterIds(), state.highWatermark(), fetchTimeoutMs, @@ -596,7 +572,7 @@ public void transitionToProspective() { epoch(), leaderId(), Optional.of(state.leaderEndpoints()), - state.election().optionalVotedKey(), + votedKey(), partitionState.lastVoterSet(), state.highWatermark(), randomElectionTimeoutMs(), diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index 4c93cad065a32..c536bb7108bd5 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -57,6 +57,7 @@ private FollowerState newFollowerState( epoch, leaderId, leaderEndpoints, + Optional.empty(), voters, highWatermark, fetchTimeoutMs, diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index f93ec2c7f2a5f..27cab4de8a9a6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -124,20 +124,26 @@ public void testHandlePreVoteRequestAsFollowerWithVotedCandidate(KRaftVersion kr // follower can grant pre-votes if it has not fetched successfully from leader yet context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - context.assertVotedCandidate(epoch, votedCandidateKey.id()); + ReplicaKey expectedVotedKey = kraftVersion == KRaftVersion.KRAFT_VERSION_1 + ? votedCandidateKey : replicaKey(votedCandidateKey.id(), false); + context.assertVotedCandidate(epoch, expectedVotedKey); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testHandlePreVoteRequestAsCandidate() throws Exception { + public void testHandlePreVoteRequestAsCandidate(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); ReplicaKey observer = replicaKey(localId + 2, true); int leaderEpoch = 2; - Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(localKey), + Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey))), + kraftVersion + ) + .withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, localKey.directoryId().get())) .withRaftProtocol(KIP_996_PROTOCOL) .build(); assertTrue(context.client.quorum().isCandidate()); @@ -147,7 +153,9 @@ public void testHandlePreVoteRequestAsCandidate() throws Exception { context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, leaderEpoch, OptionalInt.empty(), true); - context.assertVotedCandidate(leaderEpoch, localId); + ReplicaKey expectedVotedKey = kraftVersion == KRaftVersion.KRAFT_VERSION_1 + ? localKey : replicaKey(localId, false); + context.assertVotedCandidate(leaderEpoch, expectedVotedKey); assertTrue(context.client.quorum().isCandidate()); // if an observer sends a pre-vote request for the same epoch, it should also be granted @@ -155,7 +163,7 @@ public void testHandlePreVoteRequestAsCandidate() throws Exception { context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, leaderEpoch, OptionalInt.empty(), true); - context.assertVotedCandidate(leaderEpoch, localId); + context.assertVotedCandidate(leaderEpoch, expectedVotedKey); assertTrue(context.client.quorum().isCandidate()); // candidate will transition to unattached if pre-vote request has a higher epoch diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index 5b65b5ba059e3..ed90c947b6834 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -2133,4 +2133,71 @@ public void testProspectiveToUnattachedHigherEpoch() { assertTrue(state.isUnattachedNotVoted()); assertFalse(state.hasLeader()); } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveVotedToAndFromFollower(KRaftVersion kraftVersion) { + int candidate = 1; + Uuid candidateDirectoryId = Uuid.randomUuid(); + int leader = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(candidate, leader), kraftVersion); + store.writeElectionState( + ElectionState.withVotedCandidate( + logEndEpoch, + ReplicaKey.of(candidate, candidateDirectoryId), + voters.voterIds() + ), kraftVersion + ); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isUnattachedAndVoted()); + + // transition to prospective with votedKey + state.transitionToProspective(); + + assertTrue(state.isProspective()); + assertEquals(logEndEpoch, state.epoch()); + Optional expectedElectionState = Optional.of( + ElectionState.withVotedCandidate( + logEndEpoch, + persistedVotedKey(ReplicaKey.of(candidate, candidateDirectoryId), kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ) + ); + assertEquals(expectedElectionState, store.readElectionState()); + + // transition to follower of leader with votedKey + state.transitionToFollower(logEndEpoch, leader, voters.listeners(leader)); + assertTrue(state.isFollower()); + assertEquals(logEndEpoch, state.epoch()); + assertEquals( + Optional.of(new ElectionState( + logEndEpoch, + OptionalInt.of(leader), + Optional.of(ReplicaKey.of(candidate, candidateDirectoryId)), + persistedVoters(voters.voterIds(), kraftVersion)) + ), + store.readElectionState() + ); + + // transition back to prospective with votedKey and leaderId + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertEquals(logEndEpoch, state.epoch()); + assertEquals( + Optional.of(new ElectionState( + logEndEpoch, + OptionalInt.of(leader), + Optional.of(ReplicaKey.of(candidate, candidateDirectoryId)), + persistedVoters(voters.voterIds(), kraftVersion)) + ), + store.readElectionState() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerVotedToUnattached(KRaftVersion kraftVersion) { + + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java new file mode 100644 index 0000000000000..d09209d201957 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java @@ -0,0 +1,4 @@ +package org.apache.kafka.raft.internals; + +public class FollowerStateWithVoteTest { +} From a213374acac934d52b03ade02dacb8204d8ed874 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Wed, 18 Dec 2024 23:32:11 -0800 Subject: [PATCH 10/23] addressing comments plus quorumstatetest --- .../org/apache/kafka/raft/CandidateState.java | 48 +- .../org/apache/kafka/raft/FollowerState.java | 3 +- .../apache/kafka/raft/KafkaRaftClient.java | 74 +-- .../org/apache/kafka/raft/NomineeState.java | 11 + .../apache/kafka/raft/ProspectiveState.java | 11 +- .../org/apache/kafka/raft/QuorumState.java | 20 +- .../apache/kafka/raft/UnattachedState.java | 8 +- .../raft/{ => internals}/EpochElection.java | 67 ++- .../apache/kafka/raft/CandidateStateTest.java | 1 - .../apache/kafka/raft/EpochElectionTest.java | 1 + .../raft/KafkaRaftClientPreVoteTest.java | 4 +- .../apache/kafka/raft/QuorumStateTest.java | 511 +++++++++--------- .../internals/FollowerStateWithVoteTest.java | 16 + .../raft/internals/KafkaRaftMetricsTest.java | 4 +- 14 files changed, 354 insertions(+), 425 deletions(-) rename raft/src/main/java/org/apache/kafka/raft/{ => internals}/EpochElection.java (76%) diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 33c2b8932d2b6..64c8144b94bea 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.raft.internals.EpochElection; import org.slf4j.Logger; import java.util.Optional; @@ -29,12 +30,10 @@ public class CandidateState implements NomineeState { private final int localId; private final Uuid localDirectoryId; private final int epoch; - private final int retries; private final EpochElection epochElection; private final Optional highWatermark; private final int electionTimeoutMs; private final Timer electionTimer; - private final Timer backoffTimer; private final Logger log; /** @@ -42,11 +41,8 @@ public class CandidateState implements NomineeState { * * 1. Once started, it will keep record of the received votes. * 2. If majority votes granted, it will transition to leader state. - * 3. If majority votes rejected or election timed out, it will enter a backing off phase; - * after the backoff phase completes, it will be replaced by a new candidate state with bumped retry. + * 3. If majority votes rejected or election timed out, it will transition to prospective. */ - private boolean isBackingOff; - protected CandidateState( Time time, int localId, @@ -54,7 +50,6 @@ protected CandidateState( int epoch, VoterSet voters, Optional highWatermark, - int retries, int electionTimeoutMs, LogContext logContext ) { @@ -73,11 +68,8 @@ protected CandidateState( this.localDirectoryId = localDirectoryId; this.epoch = epoch; this.highWatermark = highWatermark; - this.retries = retries; - this.isBackingOff = false; this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); - this.backoffTimer = time.timer(0); this.log = logContext.logger(CandidateState.class); this.epochElection = new EpochElection(voters.voterKeys()); @@ -111,26 +103,6 @@ public boolean recordRejectedVote(int remoteNodeId) { return epochElection().recordVote(remoteNodeId, false); } - /** - * Check if the candidate is backing off for the next election - */ - public boolean isBackingOff() { - return isBackingOff; - } - - public int retries() { - return retries; - } - - /** - * Record the current election has failed since we've either received sufficient rejecting voters or election timed out - */ - public void startBackingOff(long currentTimeMs, long backoffDurationMs) { - this.backoffTimer.update(currentTimeMs); - this.backoffTimer.reset(backoffDurationMs); - this.isBackingOff = true; - } - @Override public boolean hasElectionTimeoutExpired(long currentTimeMs) { electionTimer.update(currentTimeMs); @@ -143,19 +115,6 @@ public long remainingElectionTimeMs(long currentTimeMs) { return electionTimer.remainingMs(); } - public boolean isBackoffComplete(long currentTimeMs) { - backoffTimer.update(currentTimeMs); - return backoffTimer.isExpired(); - } - - public long remainingBackoffMs(long currentTimeMs) { - if (!isBackingOff) { - throw new IllegalStateException("Candidate is not currently backing off"); - } - backoffTimer.update(currentTimeMs); - return backoffTimer.remainingMs(); - } - @Override public ElectionState election() { return ElectionState.withVotedCandidate( @@ -205,12 +164,11 @@ public boolean canGrantVote( @Override public String toString() { return String.format( - "CandidateState(localId=%d, localDirectoryId=%s,epoch=%d, retries=%d, voteStates=%s, " + + "CandidateState(localId=%d, localDirectoryId=%s,epoch=%d, voteStates=%s, " + "highWatermark=%s, electionTimeoutMs=%d)", localId, localDirectoryId, epoch, - retries, epochElection().voterStates(), highWatermark, electionTimeoutMs diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index 5443fc605d0ad..576b8a59a4816 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -26,6 +26,7 @@ import org.slf4j.Logger; import java.util.Optional; +import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Set; @@ -78,7 +79,7 @@ public FollowerState( @Override public ElectionState election() { - return ElectionState.withElectedLeader(epoch, leaderId, voters); + return new ElectionState(epoch, OptionalInt.of(leaderId), votedKey, voters); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 8eefffdad37e1..dba80eb9dd75b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -924,7 +924,7 @@ private boolean handleVoteResponse( Errors topLevelError = Errors.forCode(response.errorCode()); if (topLevelError != Errors.NONE) { if (topLevelError == Errors.UNSUPPORTED_VERSION && quorum.isProspective()) { - logger.warn("Prospective received unsupported version error in vote response in epoch {}, " + + logger.info("Prospective received unsupported version error in vote response in epoch {}, " + "transitioning to Candidate state immediately since entire quorum does not support PreVote.", quorum.epoch()); transitionToCandidate(currentTimeMs); @@ -973,13 +973,14 @@ private boolean handleVoteResponse( if (quorum.isLeader()) { logger.debug("Ignoring vote response {} since we already became leader for epoch {}", partitionResponse, quorum.epoch()); - } else if (quorum.isVotingState()) { - NomineeState state = quorum.votingStateOrThrow(); - handleVoteResponse( - state, - partitionResponse, - remoteNodeId, - currentTimeMs); + } else if (quorum.isNomineeState()) { + NomineeState state = quorum.nomineeStateOrThrow(); + if (partitionResponse.voteGranted()) { + state.recordGrantedVote(remoteNodeId); + maybeTransitionForward(state, currentTimeMs); + } else { + state.recordRejectedVote(remoteNodeId); + } } else { logger.debug("Ignoring vote response {} since we are no longer a VotingState " + "(Prospective or Candidate) in epoch {}", @@ -991,42 +992,6 @@ private boolean handleVoteResponse( } } - private void handleVoteResponse(NomineeState state, - VoteResponseData.PartitionData partitionResponse, - int remoteNodeId, - long currentTimeMs) { - if (partitionResponse.voteGranted()) { - state.recordGrantedVote(remoteNodeId); - maybeTransitionForward(state, currentTimeMs); - } else { - state.recordRejectedVote(remoteNodeId); - - // If our vote is rejected, we go immediately to backoff phase. This ensures that we are not stuck - // waiting for the election timeout when the vote has become gridlocked. - if (state.epochElection().isVoteRejected() && quorum.isCandidate()) { - CandidateState candidateState = quorum.candidateStateOrThrow(); - if (!candidateState.isBackingOff()) { - logger.info("Insufficient remaining votes to win election (rejected by {}). We will backoff " + - "before retrying election again", candidateState.epochElection().rejectingVoters()); - - candidateState.startBackingOff( - currentTimeMs, - binaryExponentialElectionBackoffMs(candidateState.retries()) - ); - } - } - } - } - - private int binaryExponentialElectionBackoffMs(int retries) { - if (retries <= 0) { - throw new IllegalArgumentException("Retries " + retries + " should be larger than zero"); - } - // upper limit exponential co-efficients at 20 to avoid overflow - return Math.min(RETRY_BACKOFF_BASE_MS * random.nextInt(2 << Math.min(20, retries - 1)), - quorumConfig.electionBackoffMaxMs()); - } - private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) { if (positionInSuccessors == 0) { return 0; @@ -3086,19 +3051,10 @@ private long pollCandidate(long currentTimeMs) { // 3) the shutdown timer expires long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); - } else if (state.isBackingOff()) { - if (state.isBackoffComplete(currentTimeMs)) { - logger.info("Transition to prospective after election backoff has completed"); - transitionToProspective(currentTimeMs); - return 0L; - } - return state.remainingBackoffMs(currentTimeMs); - } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { - long backoffDurationMs = binaryExponentialElectionBackoffMs(state.retries()); - logger.info("Election has timed out, backing off for {}ms before becoming a {} again", - backoffDurationMs, state.name()); - state.startBackingOff(currentTimeMs, backoffDurationMs); - return backoffDurationMs; + } else if (state.epochElection().isVoteRejected() || state.hasElectionTimeoutExpired(currentTimeMs)) { + logger.info("Election was not granted, transitioning to prospective"); + transitionToProspective(currentTimeMs); + return 0L; } else { long minVoteRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); return Math.min(minVoteRequestBackoffMs, state.remainingElectionTimeMs(currentTimeMs)); @@ -3115,7 +3071,7 @@ private long pollProspective(long currentTimeMs) { } else if (state.epochElection().isVoteRejected() || state.hasElectionTimeoutExpired(currentTimeMs)) { if (state.election().hasLeader() && !state.leaderEndpoints().isEmpty()) { logger.info( - "Election has timed out, transitioning to Follower of leader {}", + "Election was not granted, transitioning to Follower of leader {}", state.election().leaderId()); transitionToFollower( quorum().epoch(), @@ -3123,7 +3079,7 @@ private long pollProspective(long currentTimeMs) { state.leaderEndpoints(), currentTimeMs); } else { - logger.info("Election has timed out, transitioning to Unattached to attempt rediscovering leader"); + logger.info("Election was not granted, transitioning to Unattached to attempt rediscovering leader"); transitionToUnattached(quorum().epoch()); } return 0L; diff --git a/raft/src/main/java/org/apache/kafka/raft/NomineeState.java b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java index 07fd7cedbd095..7c1b6e5abeeab 100644 --- a/raft/src/main/java/org/apache/kafka/raft/NomineeState.java +++ b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.raft.internals.EpochElection; + interface NomineeState extends EpochState { EpochElection epochElection(); @@ -37,7 +39,16 @@ interface NomineeState extends EpochState { */ boolean recordRejectedVote(int remoteNodeId); + /** + * Returns true if the election timeout has expired, false otherwise. + * @param currentTimeMs The current time in milliseconds + */ boolean hasElectionTimeoutExpired(long currentTimeMs); + /** + * Returns the remaining time in milliseconds until the election timeout expires. + * @param currentTimeMs + * @return + */ long remainingElectionTimeMs(long currentTimeMs); } diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index e984ea3b70460..7ec81813b088f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.raft.internals.EpochElection; import org.slf4j.Logger; import java.util.Optional; @@ -92,7 +93,7 @@ public boolean recordGrantedVote(int remoteNodeId) { @Override public boolean recordRejectedVote(int remoteNodeId) { if (remoteNodeId == localId) { - throw new IllegalStateException("Attempted to reject vote from ourselves"); + throw new IllegalArgumentException("Attempted to reject vote from ourselves"); } return epochElection().recordVote(remoteNodeId, false); } @@ -150,13 +151,7 @@ public long remainingElectionTimeMs(long currentTimeMs) { @Override public ElectionState election() { - if (votedKey.isPresent()) { - return ElectionState.withVotedCandidate(epoch, votedKey().get(), voters.voterIds()); - } else if (leaderId.isPresent()) { - return ElectionState.withElectedLeader(epoch, leaderId.getAsInt(), voters.voterIds()); - } else { - return ElectionState.withUnknownLeader(epoch, voters.voterIds()); - } + return new ElectionState(epoch, leaderId, votedKey, voters.voterIds()); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 9582c179465eb..b2cf03262cf13 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -196,7 +196,6 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateE election.epoch(), partitionState.lastVoterSet(), Optional.empty(), - 1, randomElectionTimeoutMs(), logContext ); @@ -389,11 +388,6 @@ public void transitionToUnattached(int epoch) { transitionToUnattached(epoch, OptionalInt.empty()); } - // ahu todo: need to fix all tests which call this method. delete method afterwards - public void transitionToUnattached(int epoch, Optional votedKey) { - transitionToUnattached(epoch, OptionalInt.empty()); - } - public void transitionToUnattached(int epoch, OptionalInt leaderId) { int currentEpoch = state.epoch(); if (epoch < currentEpoch || (epoch == currentEpoch && !isProspective())) { @@ -583,7 +577,6 @@ public void transitionToProspective() { public void transitionToCandidate() { checkValidTransitionToCandidate(); - int retries = isCandidate() ? candidateStateOrThrow().retries() + 1 : 1; int newEpoch = epoch() + 1; int electionTimeoutMs = randomElectionTimeoutMs(); @@ -594,7 +587,6 @@ public void transitionToCandidate() { newEpoch, partitionState.lastVoterSet(), state.highWatermark(), - retries, electionTimeoutMs, logContext )); @@ -612,7 +604,9 @@ private void checkValidTransitionToCandidate() { ) ); } - if (!isProspective() && !isOnlyVoter()) { + // Leader state can never transition to Candidate state + // Only Prospective is allowed to transition to Candidate, unless the local replica is the only voter + if (isLeader() || (!isProspective() && !isOnlyVoter())) { throw new IllegalStateException("Cannot transition to Candidate since the local broker.id=" + localId + " is state " + state); } @@ -751,10 +745,10 @@ public CandidateState candidateStateOrThrow() { throw new IllegalStateException("Expected to be Candidate, but current state is " + state); } - public NomineeState votingStateOrThrow() { - if (isVotingState()) + public NomineeState nomineeStateOrThrow() { + if (isNomineeState()) return (NomineeState) state; - throw new IllegalStateException("Expected to be a VotingState (Prospective or Candidate), " + + throw new IllegalStateException("Expected to be a NomineeState (Prospective or Candidate), " + "but current state is " + state); } @@ -795,7 +789,7 @@ public boolean isCandidate() { return state instanceof CandidateState; } - public boolean isVotingState() { + public boolean isNomineeState() { return state instanceof NomineeState; } } diff --git a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java index 9c856d0c0bc19..9e1aef7d63fe7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java @@ -71,13 +71,7 @@ public UnattachedState( @Override public ElectionState election() { - if (votedKey.isPresent()) { - return ElectionState.withVotedCandidate(epoch, votedKey().get(), voters); - } else if (leaderId.isPresent()) { - return ElectionState.withElectedLeader(epoch, leaderId.getAsInt(), voters); - } else { - return ElectionState.withUnknownLeader(epoch, voters); - } + return new ElectionState(epoch, leaderId, votedKey, voters); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/EpochElection.java b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java similarity index 76% rename from raft/src/main/java/org/apache/kafka/raft/EpochElection.java rename to raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java index 3cbedf8988cc3..19cf7715b4920 100644 --- a/raft/src/main/java/org/apache/kafka/raft/EpochElection.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.raft; +package org.apache.kafka.raft.internals; + +import org.apache.kafka.raft.ReplicaKey; import java.util.Collection; import java.util.Collections; @@ -23,10 +25,13 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +/** + * Tracks the votes cast by voters in an election held by a Nominee. + */ public class EpochElection { private Map voterStates; - EpochElection(Set voters) { + public EpochElection(Set voters) { this.voterStates = voters.stream() .collect(Collectors.toMap( ReplicaKey::id, @@ -34,15 +39,13 @@ public class EpochElection { )); } - VoterState getVoterStateOrThrow(int voterId) { - VoterState voterState = voterStates.get(voterId); - if (voterState == null) { - throw new IllegalArgumentException("Attempt to access voter state of non-voter " + voterId); - } - return voterState; - } - - boolean recordVote(int voterId, boolean isGranted) { + /** + * Record a vote from a voter. + * @param voterId The id of the voter + * @param isGranted true if the vote is granted, false if it is rejected + * @return true if the voter had not been previously recorded + */ + public boolean recordVote(int voterId, boolean isGranted) { boolean wasUnrecorded = false; VoterState voterState = getVoterStateOrThrow(voterId); if (voterState.state == VoterState.State.UNRECORDED) { @@ -56,19 +59,35 @@ boolean recordVote(int voterId, boolean isGranted) { return wasUnrecorded; } - boolean isGrantedVoter(int voterId) { + /** + * Returns if a voter has granted the vote. + * @param voterId The id of the voter + * @throws IllegalArgumentException if the voter is not in the set of voters + */ + public boolean isGrantedVoter(int voterId) { return getVoterStateOrThrow(voterId).state == VoterState.State.GRANTED; } - boolean isRejectedVoter(int voterId) { + /** + * Returns if a voter has rejected the vote. + * @param voterId The id of the voter + * @throws IllegalArgumentException if the voter is not in the set of voters + */ + public boolean isRejectedVoter(int voterId) { return getVoterStateOrThrow(voterId).state == VoterState.State.REJECTED; } - Set voterIds() { + /** + * The set of voter ids. + */ + public Set voterIds() { return Collections.unmodifiableSet(voterStates.keySet()); } - Collection voterStates() { + /** + * Get the collection of voter states. + */ + public Collection voterStates() { return Collections.unmodifiableCollection(voterStates.values()); } @@ -77,7 +96,7 @@ Collection voterStates() { * * @return true if at least a majority of nodes have granted the vote */ - boolean isVoteGranted() { + public boolean isVoteGranted() { return numGranted() >= majoritySize(); } @@ -87,7 +106,7 @@ boolean isVoteGranted() { * * @return true if the vote is rejected, false if the vote is already or can still be granted */ - boolean isVoteRejected() { + public boolean isVoteRejected() { return numGranted() + numUnrecorded() < majoritySize(); } @@ -96,7 +115,7 @@ boolean isVoteRejected() { * * @return The set of unrecorded voters */ - Set unrecordedVoters() { + public Set unrecordedVoters() { return votersOfState(VoterState.State.UNRECORDED).collect(Collectors.toSet()); } @@ -105,7 +124,7 @@ Set unrecordedVoters() { * * @return The set of granting voters, which should always contain the localId */ - Set grantingVoters() { + public Set grantingVoters() { return votersOfState(VoterState.State.GRANTED).map(ReplicaKey::id).collect(Collectors.toSet()); } @@ -114,10 +133,18 @@ Set grantingVoters() { * * @return The set of rejecting voters */ - Set rejectingVoters() { + public Set rejectingVoters() { return votersOfState(VoterState.State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet()); } + private VoterState getVoterStateOrThrow(int voterId) { + VoterState voterState = voterStates.get(voterId); + if (voterState == null) { + throw new IllegalArgumentException("Attempt to access voter state of non-voter " + voterId); + } + return voterState; + } + private Stream votersOfState(VoterState.State state) { return voterStates .values() diff --git a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java index 40263be7af4c2..235de20f75473 100644 --- a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java @@ -51,7 +51,6 @@ private CandidateState newCandidateState(VoterSet voters) { epoch, voters, Optional.empty(), - 0, electionTimeoutMs, logContext ); diff --git a/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java b/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java index ef37eb126b887..dd5d8314e4140 100644 --- a/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.Uuid; +import org.apache.kafka.raft.internals.EpochElection; import org.junit.jupiter.api.Test; import java.util.Set; diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 27cab4de8a9a6..758bb724466bb 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -772,13 +772,11 @@ public void testPreVoteNotSupportedByRemote(KRaftVersion kraftVersion) throws Ex context.client.poll(); assertEquals(epoch + 1, context.currentEpoch()); context.client.quorum().isCandidate(); + context.collectVoteRequests(epoch + 1, 0, 0); // Sleep to transition back to Prospective context.time.sleep(context.client.quorum().candidateStateOrThrow().remainingElectionTimeMs(context.time.milliseconds())); context.client.poll(); - context.time.sleep(context.client.quorum().candidateStateOrThrow().remainingBackoffMs(context.time.milliseconds())); - context.collectVoteRequests(epoch + 1, 0, 0); - context.client.poll(); assertEquals(epoch + 1, context.currentEpoch()); assertTrue(context.client.quorum().isProspective()); diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index ed90c947b6834..48a43199b364e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -167,9 +167,7 @@ public void testInitializePrimordialEpoch(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); assertTrue(state.isUnattached()); assertEquals(0, state.epoch()); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); CandidateState candidateState = state.candidateStateOrThrow(); assertTrue(candidateState.epochElection().isVoteGranted()); @@ -347,39 +345,34 @@ public void testCandidateToCandidate() { assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_0); + state.transitionToProspective(); + int jitterMs1 = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs1); state.transitionToCandidate(); assertTrue(state.isCandidate()); assertEquals(1, state.epoch()); - - CandidateState candidate1 = state.candidateStateOrThrow(); - candidate1.recordRejectedVote(node2); - - // Check backoff behavior before transitioning - int backoffMs = 500; - candidate1.startBackingOff(time.milliseconds(), backoffMs); - assertTrue(candidate1.isBackingOff()); - assertFalse(candidate1.isBackoffComplete(time.milliseconds())); - - time.sleep(backoffMs - 1); - assertTrue(candidate1.isBackingOff()); - assertFalse(candidate1.isBackoffComplete(time.milliseconds())); - - time.sleep(1); - assertTrue(candidate1.isBackingOff()); - assertTrue(candidate1.isBackoffComplete(time.milliseconds())); - - // The election timeout should be reset after we become a candidate again - int jitterMs = 2500; - random.mockNextInt(jitterMs); - + CandidateState candidate = state.candidateStateOrThrow(); + assertEquals(electionTimeoutMs + jitterMs1, + candidate.remainingElectionTimeMs(time.milliseconds())); + + // The election timeout should be reset after we transition to prospective + time.sleep(candidate.remainingElectionTimeMs(time.milliseconds())); + assertEquals(0, candidate.remainingElectionTimeMs(time.milliseconds())); + int jitterMs2 = 3000; + random.mockNextInt(electionTimeoutMs, jitterMs2); + state.transitionToProspective(); + ProspectiveState prospective = state.prospectiveStateOrThrow(); + assertEquals(electionTimeoutMs + jitterMs2, + prospective.remainingElectionTimeMs(time.milliseconds())); + + // The election timeout should be reset after we transition to candidate again + int jitterMs3 = 1000; + random.mockNextInt(electionTimeoutMs, jitterMs3); + prospective.recordGrantedVote(node1); state.transitionToCandidate(); - assertTrue(state.isCandidate()); - CandidateState candidate2 = state.candidateStateOrThrow(); - assertEquals(2, state.epoch()); - assertEquals(Collections.singleton(localId), candidate2.epochElection().grantingVoters()); - assertEquals(Collections.emptySet(), candidate2.epochElection().rejectingVoters()); - assertEquals(electionTimeoutMs + jitterMs, - candidate2.remainingElectionTimeMs(time.milliseconds())); + candidate = state.candidateStateOrThrow(); + assertEquals(electionTimeoutMs + jitterMs3, + candidate.remainingElectionTimeMs(time.milliseconds())); } @ParameterizedTest @@ -391,9 +384,7 @@ public void testCandidateToResigned(KRaftVersion kraftVersion) { assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); assertTrue(state.isCandidate()); assertEquals(1, state.epoch()); @@ -412,9 +403,7 @@ public void testCandidateToLeader(KRaftVersion kraftVersion) { assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); assertTrue(state.isCandidate()); assertEquals(1, state.epoch()); @@ -433,9 +422,7 @@ public void testCandidateToLeaderWithoutGrantedVote(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); assertFalse(state.candidateStateOrThrow().epochElection().isVoteGranted()); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); @@ -453,9 +440,7 @@ public void testCandidateToFollower(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId)); @@ -480,9 +465,7 @@ public void testCandidateToUnattached(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); state.transitionToUnattached(5); @@ -499,36 +482,6 @@ public void testCandidateToUnattached(KRaftVersion kraftVersion) { ); } - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testCandidateToUnattachedVoted(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } - state.transitionToCandidate(); - - state.transitionToUnattached(5, Optional.of(otherNodeKey)); - assertEquals(5, state.epoch()); - assertEquals(OptionalInt.empty(), state.leaderId()); - - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(otherNodeKey, votedState.votedKey().get()); - - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 5, - persistedVotedKey(otherNodeKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion)) - ), - store.readElectionState() - ); - } - @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testCandidateToAnyStateLowerEpoch(KRaftVersion kraftVersion) { @@ -537,12 +490,9 @@ public void testCandidateToAnyStateLowerEpoch(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, Optional.of(otherNodeKey))); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -572,9 +522,7 @@ public void testLeaderToLeader(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); @@ -593,9 +541,7 @@ public void testLeaderToResigned(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); @@ -620,9 +566,7 @@ public void testLeaderToCandidate(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); @@ -641,9 +585,7 @@ public void testLeaderToFollower(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); state.transitionToLeader(0L, accumulator); @@ -670,9 +612,7 @@ public void testLeaderToUnattached(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); state.transitionToLeader(0L, accumulator); @@ -690,39 +630,6 @@ public void testLeaderToUnattached(KRaftVersion kraftVersion) { ); } - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testLeaderToUnattachedVoted(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } - state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); - state.transitionToLeader(0L, accumulator); - state.transitionToUnattached(5, Optional.of(otherNodeKey)); - - assertEquals(5, state.epoch()); - assertEquals(OptionalInt.empty(), state.leaderId()); - - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(otherNodeKey, votedState.votedKey().get()); - - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 5, - persistedVotedKey(otherNodeKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() - ); - } - @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { @@ -731,14 +638,11 @@ public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); state.transitionToLeader(0L, accumulator); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, Optional.of(otherNodeKey))); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -775,7 +679,7 @@ public void testCannotFollowOrVoteForSelf(KRaftVersion kraftVersion) { voters.listeners(localId) ) ); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(0, Optional.of(localVoterKey))); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(0, localVoterKey)); } @ParameterizedTest @@ -838,23 +742,7 @@ public void testUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { state.transitionToUnattached(5); assertTrue(state.isUnattachedNotVoted()); - state.transitionToUnattached(8, Optional.of(otherNodeKey)); - assertTrue(state.isUnattachedAndVoted()); - - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(8, votedState.epoch()); - assertEquals(otherNodeKey, votedState.votedKey().get()); - - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 8, - persistedVotedKey(otherNodeKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() - ); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, otherNodeKey)); } @ParameterizedTest @@ -868,9 +756,7 @@ public void testUnattachedToCandidate(KRaftVersion kraftVersion) { int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); state.transitionToCandidate(); assertTrue(state.isCandidate()); @@ -960,7 +846,7 @@ public void testUnattachedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, Optional.of(otherNodeKey))); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -989,7 +875,7 @@ public void testUnattachedVotedToInvalidLeaderOrResigned(KRaftVersion kraftVersi VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); + state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @@ -1002,33 +888,36 @@ public void testUnattachedVotedToCandidate(KRaftVersion kraftVersion) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); - - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + state.transitionToProspective(); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); state.transitionToCandidate(); assertTrue(state.isCandidate()); CandidateState candidateState = state.candidateStateOrThrow(); - assertEquals(6, candidateState.epoch()); + assertEquals(logEndEpoch + 1, candidateState.epoch()); assertEquals(electionTimeoutMs + jitterMs, candidateState.remainingElectionTimeMs(time.milliseconds())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testObserverFromUnattachedVotedToCandidate(KRaftVersion kraftVersion) { + public void testObserverCannotTransitionToProspective(KRaftVersion kraftVersion) { int voter1 = 1; int voter2 = 2; VoterSet voters = withRemoteVoterSet(IntStream.of(voter1, voter2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5, Optional.of(ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID))); - assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); - assertTrue(state.isUnattached()); + assertTrue(state.isUnattachedNotVoted()); + assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); + + state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID)); + assertTrue(state.isUnattachedAndVoted()); + assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); + + state.transitionToFollower(logEndEpoch + 1, voter1, voters.listeners(voter1)); + assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); } @ParameterizedTest @@ -1038,16 +927,15 @@ public void testUnattachedVotedToUnattachedVotedSameEpoch(KRaftVersion kraftVers int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node1, Uuid.randomUuid()))); + state.initialize(new OffsetAndEpoch(0L, 8)); + state.unattachedAddVotedState(8, ReplicaKey.of(node1, Uuid.randomUuid())); assertThrows( IllegalStateException.class, - () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))) + () -> state.unattachedAddVotedState(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) ); assertThrows( IllegalStateException.class, - () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID))) + () -> state.unattachedAddVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) ); } @@ -1055,11 +943,12 @@ public void testUnattachedVotedToUnattachedVotedSameEpoch(KRaftVersion kraftVers @EnumSource(value = KRaftVersion.class) public void testUnattachedVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { int node1 = 1; + ReplicaKey node1Key = ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID); int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, node1Key); state.transitionToFollower( 5, node2, @@ -1074,9 +963,10 @@ public void testUnattachedVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { ); assertEquals( Optional.of( - ElectionState.withElectedLeader( + new ElectionState( 5, - node2, + OptionalInt.of(node2), + Optional.of(persistedVotedKey(node1Key, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -1091,8 +981,8 @@ public void testUnattachedVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); state.transitionToFollower( 8, node2, @@ -1125,8 +1015,8 @@ public void testUnattachedVotedToUnattachedSameEpoch(KRaftVersion kraftVersion) VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5)); + state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(logEndEpoch)); } @ParameterizedTest @@ -1135,8 +1025,8 @@ public void testUnattachedVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion int otherNodeId = 1; VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5, Optional.of(ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID))); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID)); long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); time.sleep(1000); @@ -1156,10 +1046,10 @@ public void testUnattachedVotedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5, Optional.of(otherNodeKey)); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, otherNodeKey); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, Optional.of(otherNodeKey))); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -1188,55 +1078,110 @@ public void testIllegalStateTransitionToUnattachedInSameEpoch(KRaftVersion kraft ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.initialize(new OffsetAndEpoch(0L, 5)); // unattached to unattached state.unattachedStateOrThrow(); - state.transitionToUnattached(5, Optional.of(voter1)); + state.unattachedAddVotedState(5, voter1); // cannot vote for same or different node in same epoch - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5, Optional.of(voter1))); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5, Optional.of(voter2))); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(5, voter1)); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(5, voter2)); // follower to unattached state.transitionToFollower(10, voter1.id(), voters.listeners(voter1.id())); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), Optional.of(voter1))); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + + state.transitionToProspective(); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } // candidate state.transitionToCandidate(); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), Optional.of(voter1))); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); // leader state.candidateStateOrThrow().recordGrantedVote(voter1.id()); state.transitionToLeader(0L, accumulator); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), Optional.of(voter1))); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); } - @Test - public void testProspectiveToUnattachedInSameEpoch() { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveToUnattachedInSameEpoch(KRaftVersion kraftVersion) { ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); + QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - - // with leader state state.transitionToProspective(); + + // transition to unattached with leader state state.transitionToUnattached(state.epoch(), OptionalInt.of(voter1.id())); + assertEquals( + ElectionState.withElectedLeader( + logEndEpoch, + voter1.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); - // with voted state - state = initializeEmptyState(voters, KRAFT_VERSION_1); + // without leader state + state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.transitionToUnattached(state.epoch(), Optional.of(voter1)); + state.transitionToUnattached(state.epoch()); + assertEquals( + ElectionState.withUnknownLeader( + logEndEpoch, + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } - // without voted or leader state - state = initializeEmptyState(voters, KRAFT_VERSION_1); + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveVotedToUnattachedVotedInSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState( + ElectionState.withVotedCandidate( + logEndEpoch, + voter1, + voters.voterIds() + ), kraftVersion + ); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + + // transition to unattached with leader state + state.transitionToUnattached(state.epoch(), OptionalInt.of(voter1.id())); + + // with voted state + assertEquals( + new ElectionState( + logEndEpoch, + OptionalInt.of(voter1.id()), + Optional.of(persistedVotedKey(voter1, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + + // without leader state + state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.transitionToUnattached(state.epoch(), Optional.empty()); + state.transitionToUnattached(state.epoch()); + assertEquals( + ElectionState.withVotedCandidate( + logEndEpoch, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); } @ParameterizedTest @@ -1386,10 +1331,7 @@ public void testFollowerToCandidate(KRaftVersion kraftVersion) { node2, voters.listeners(node2) ); - - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - } + state.transitionToProspective(); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); state.transitionToCandidate(); @@ -1442,59 +1384,96 @@ public void testFollowerToUnattachedHigherEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToUnattachedVotedSameEpoch(KRaftVersion kraftVersion) { + public void testIllegalCallsToUnattachedAddVotedState(KRaftVersion kraftVersion) { + boolean withDirectoryId = kraftVersion.featureLevel() > 0; + ReplicaKey node1 = replicaKey(1, withDirectoryId); + ReplicaKey node2 = replicaKey(2, withDirectoryId); + VoterSet voters = localWithRemoteVoterSet(Stream.of(node1, node2), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + // should fail on any state that isn't unattached and not voted + + // unattached and voted + state.unattachedAddVotedState(state.epoch(), node1); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); + + // prospective + state.transitionToUnattached(5); + state.transitionToProspective(); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); + + // prospective and voted + state.prospectiveStateOrThrow().recordGrantedVote(node1.id()); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); + + // candidate + state.transitionToCandidate(); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); + + // candidate and voted + state.candidateStateOrThrow().recordGrantedVote(node1.id()); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); + + // leader + state.transitionToLeader(6, accumulator); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); + + // should fail if epoch is not equal + state.transitionToUnattached(7); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(8, node1)); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerVotedToUnattachedSameEpoch(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.prospectiveStateOrThrow().recordGrantedVote(node1); state.transitionToFollower( - 8, + state.epoch(), node2, voters.listeners(node2) ); + assertEquals(0, state.epoch()); assertThrows( IllegalStateException.class, - () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID))) - ); - assertThrows( - IllegalStateException.class, - () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID))) - ); - assertThrows( - IllegalStateException.class, - () -> state.transitionToUnattached(8, Optional.of(ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID))) + () -> state.transitionToUnattached(0) ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { - ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid()); - ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); - - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, nodeKey1, nodeKey2)); + public void testFollowerVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.prospectiveStateOrThrow().recordGrantedVote(node1); state.transitionToFollower( - 8, - nodeKey2.id(), - voters.listeners(nodeKey2.id()) + state.epoch(), + node2, + voters.listeners(node2) ); + assertEquals(0, state.epoch()); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToUnattached(9, Optional.of(nodeKey1)); - assertTrue(state.isUnattachedAndVoted()); + state.transitionToUnattached(10); + assertTrue(state.isUnattachedNotVoted()); - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(9, votedState.epoch()); - assertEquals(nodeKey1, votedState.votedKey().get()); + UnattachedState unattached = state.unattachedStateOrThrow(); + assertEquals(10, unattached.epoch()); assertEquals(electionTimeoutMs + jitterMs, - votedState.remainingElectionTimeMs(time.milliseconds())); + unattached.remainingElectionTimeMs(time.milliseconds())); } @ParameterizedTest @@ -1510,10 +1489,6 @@ public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { voters.listeners(otherNodeId) ); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows( - IllegalStateException.class, - () -> state.transitionToUnattached(4, Optional.of(ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID))) - ); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -1522,6 +1497,7 @@ public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { voters.listeners(otherNodeId) ) ); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(4, accumulator)); assertEquals(5, state.epoch()); assertEquals( Optional.of( @@ -1542,10 +1518,10 @@ public void testCanBecomeFollowerOfNonVoter(KRaftVersion kraftVersion) { ReplicaKey nonVoterKey = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.initialize(new OffsetAndEpoch(0L, 4)); // Add voted state - state.transitionToUnattached(4, Optional.of(nonVoterKey)); + state.unattachedAddVotedState(4, nonVoterKey); assertTrue(state.isUnattachedAndVoted()); UnattachedState votedState = state.unattachedStateOrThrow(); @@ -1567,6 +1543,15 @@ public void testCanBecomeFollowerOfNonVoter(KRaftVersion kraftVersion) { new LeaderAndEpoch(OptionalInt.of(nonVoterKey.id()), 4), state.leaderAndEpoch() ); + assertEquals( + new ElectionState( + 4, + OptionalInt.of(nonVoterKey.id()), + Optional.of(persistedVotedKey(nonVoterKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); } @ParameterizedTest @@ -1592,10 +1577,10 @@ public void testObserverWithIdCanVote(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.initialize(new OffsetAndEpoch(0L, 5)); assertTrue(state.isObserver()); - state.transitionToUnattached(5, Optional.of(otherNodeKey)); + state.unattachedAddVotedState(5, otherNodeKey); assertTrue(state.isUnattachedAndVoted()); UnattachedState votedState = state.unattachedStateOrThrow(); @@ -1683,10 +1668,8 @@ public void testHasRemoteLeader(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); assertFalse(state.hasRemoteLeader()); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - assertFalse(state.hasRemoteLeader()); - } + state.transitionToProspective(); + assertFalse(state.hasRemoteLeader()); state.transitionToCandidate(); assertFalse(state.hasRemoteLeader()); @@ -1697,11 +1680,11 @@ public void testHasRemoteLeader(KRaftVersion kraftVersion) { state.transitionToUnattached(state.epoch() + 1); assertFalse(state.hasRemoteLeader()); - state.transitionToUnattached(state.epoch() + 1, Optional.of(otherNodeKey)); + state.unattachedAddVotedState(state.epoch(), otherNodeKey); assertFalse(state.hasRemoteLeader()); state.transitionToFollower( - state.epoch() + 1, + state.epoch(), otherNodeKey.id(), voters.listeners(otherNodeKey.id()) ); @@ -1730,13 +1713,12 @@ public void testHighWatermarkRetained(KRaftVersion kraftVersion) { state.transitionToUnattached(6); assertEquals(highWatermark, state.highWatermark()); - state.transitionToUnattached(7, Optional.of(otherNodeKey)); + state.unattachedAddVotedState(6, otherNodeKey); + assertEquals(highWatermark, state.highWatermark()); + + state.transitionToProspective(); assertEquals(highWatermark, state.highWatermark()); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - assertEquals(highWatermark, state.highWatermark()); - } state.transitionToCandidate(); assertEquals(highWatermark, state.highWatermark()); @@ -1761,11 +1743,8 @@ public void testInitializeWithEmptyLocalId(KRaftVersion kraftVersion) { assertTrue(state.isObserver()); assertFalse(state.isVoter()); - if (kraftVersion.isReconfigSupported()) { - assertThrows(IllegalStateException.class, state::transitionToProspective); - } else { - assertThrows(IllegalStateException.class, state::transitionToCandidate); - } + assertThrows(IllegalStateException.class, state::transitionToProspective); + assertThrows( IllegalStateException.class, () -> state.unattachedAddVotedState(1, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)) @@ -1925,8 +1904,8 @@ public void testUnattachedVotedToAndFromProspectiveVoted() { int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5, Optional.of(ReplicaKey.of(node1, node1DirectoryId))); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, ReplicaKey.of(node1, node1DirectoryId)); state.transitionToProspective(); assertTrue(state.isProspective()); @@ -1941,7 +1920,7 @@ public void testUnattachedVotedToAndFromProspectiveVoted() { store.readElectionState() ); - state.transitionToUnattached(prospectiveState.epoch(), prospectiveState.votedKey()); + state.transitionToUnattached(prospectiveState.epoch()); assertTrue(state.isUnattachedAndVoted()); assertEquals(prospectiveState.epoch(), state.epoch()); assertEquals( @@ -2138,13 +2117,13 @@ public void testProspectiveToUnattachedHigherEpoch() { @EnumSource(value = KRaftVersion.class) public void testProspectiveVotedToAndFromFollower(KRaftVersion kraftVersion) { int candidate = 1; - Uuid candidateDirectoryId = Uuid.randomUuid(); + ReplicaKey candidateKey = ReplicaKey.of(candidate, Uuid.randomUuid()); int leader = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(candidate, leader), kraftVersion); store.writeElectionState( ElectionState.withVotedCandidate( logEndEpoch, - ReplicaKey.of(candidate, candidateDirectoryId), + candidateKey, voters.voterIds() ), kraftVersion ); @@ -2160,7 +2139,7 @@ public void testProspectiveVotedToAndFromFollower(KRaftVersion kraftVersion) { Optional expectedElectionState = Optional.of( ElectionState.withVotedCandidate( logEndEpoch, - persistedVotedKey(ReplicaKey.of(candidate, candidateDirectoryId), kraftVersion), + persistedVotedKey(candidateKey, kraftVersion), persistedVoters(voters.voterIds(), kraftVersion) ) ); @@ -2174,7 +2153,7 @@ public void testProspectiveVotedToAndFromFollower(KRaftVersion kraftVersion) { Optional.of(new ElectionState( logEndEpoch, OptionalInt.of(leader), - Optional.of(ReplicaKey.of(candidate, candidateDirectoryId)), + Optional.of(persistedVotedKey(candidateKey, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion)) ), store.readElectionState() @@ -2188,7 +2167,7 @@ public void testProspectiveVotedToAndFromFollower(KRaftVersion kraftVersion) { Optional.of(new ElectionState( logEndEpoch, OptionalInt.of(leader), - Optional.of(ReplicaKey.of(candidate, candidateDirectoryId)), + Optional.of(persistedVotedKey(candidateKey, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion)) ), store.readElectionState() diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java index d09209d201957..6b6941b25bc5d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.raft.internals; public class FollowerStateWithVoteTest { diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index 5262405a1169c..f16066b90e908 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -39,7 +39,6 @@ import java.util.Collections; import java.util.Map; -import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Random; @@ -212,7 +211,8 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); } - state.transitionToUnattached(3, Optional.of(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID))); + state.transitionToUnattached(3); + state.unattachedAddVotedState(3, ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID)); assertEquals("unattached-voted", getMetric(metrics, "current-state").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) 2, getMetric(metrics, "current-vote").metricValue()); From 317026d4ab484167e3b7ac4cb5ad1c3c13486674 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Fri, 20 Dec 2024 08:07:09 -0800 Subject: [PATCH 11/23] more tests --- .../apache/kafka/raft/KafkaRaftClient.java | 10 +- .../org/apache/kafka/raft/QuorumState.java | 86 +- .../apache/kafka/raft/FollowerStateTest.java | 4 +- .../kafka/raft/FollowerStateWithVoteTest.java | 89 ++ .../kafka/raft/KafkaRaftClientTest.java | 35 + .../apache/kafka/raft/QuorumStateTest.java | 1138 ++++++++++++----- .../kafka/raft/UnattachedStateTest.java | 13 + .../raft/UnattachedStateWithVoteTest.java | 22 +- .../internals/FollowerStateWithVoteTest.java | 20 - 9 files changed, 1058 insertions(+), 359 deletions(-) create mode 100644 raft/src/test/java/org/apache/kafka/raft/FollowerStateWithVoteTest.java delete mode 100644 raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index dba80eb9dd75b..2666abcec42a8 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -897,8 +897,12 @@ private VoteResponseData handleVoteRequest( preVote ); - if (!preVote && voteGranted && quorum.isUnattachedNotVoted()) { - quorum.unattachedAddVotedState(replicaEpoch, replicaKey); + if (!preVote && voteGranted) { + if (quorum.isUnattachedNotVoted()) { + quorum.unattachedAddVotedState(replicaEpoch, replicaKey); + } else if (quorum.isProspectiveNotVoted()) { + quorum.prospectiveAddVotedState(replicaEpoch, replicaKey); + } } logger.info( @@ -3080,7 +3084,7 @@ private long pollProspective(long currentTimeMs) { currentTimeMs); } else { logger.info("Election was not granted, transitioning to Unattached to attempt rediscovering leader"); - transitionToUnattached(quorum().epoch()); + transitionToUnattached(quorum().epoch(), state.election().optionalLeaderId()); } return 0L; } else { diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index b2cf03262cf13..85c34e2b200a0 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -199,17 +199,6 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateE randomElectionTimeoutMs(), logContext ); - } else if (election.hasVoted()) { - initialState = new UnattachedState( - time, - election.epoch(), - OptionalInt.empty(), - Optional.of(election.votedKey()), - partitionState.lastVoterSet().voterIds(), - Optional.empty(), - randomElectionTimeoutMs(), - logContext - ); } else if (election.hasLeader()) { VoterSet voters = partitionState.lastVoterSet(); Endpoints leaderEndpoints = voters.listeners(election.leaderId()); @@ -232,7 +221,7 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateE time, election.epoch(), OptionalInt.of(election.leaderId()), - Optional.empty(), + election.optionalVotedKey(), partitionState.lastVoterSet().voterIds(), Optional.empty(), randomElectionTimeoutMs(), @@ -244,7 +233,7 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateE election.epoch(), election.leaderId(), leaderEndpoints, - Optional.empty(), + election.optionalVotedKey(), voters.voterIds(), Optional.empty(), fetchTimeoutMs, @@ -256,7 +245,7 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateE time, election.epoch(), OptionalInt.empty(), - Optional.empty(), + election.optionalVotedKey(), partitionState.lastVoterSet().voterIds(), Optional.empty(), randomElectionTimeoutMs(), @@ -445,7 +434,7 @@ public void unattachedAddVotedState( ); } else if (localId.isEmpty()) { throw new IllegalStateException("Cannot add voted state without a replica id"); - } else if (epoch != currentEpoch || !isUnattachedNotVoted()) { + } else if (epoch != currentEpoch || isUnattachedAndVoted()) { throw new IllegalStateException( String.format( "Cannot add voted key (%s) to current state (%s) in epoch %d", @@ -473,6 +462,56 @@ public void unattachedAddVotedState( log.debug("Voted for candidate {} in epoch {}", candidateKey, epoch); } + /** + * Grant a vote to a candidate as Prospective. We will transition to Prospective with votedKey + * state and remain there until either the election timeout expires or we discover the leader. + */ + public void prospectiveAddVotedState( + int epoch, + ReplicaKey candidateKey + ) { + int currentEpoch = state.epoch(); + if (localId.isPresent() && candidateKey.id() == localId.getAsInt()) { + throw new IllegalStateException( + String.format( + "Cannot transition to Voted for %s and epoch %d since it matches the local " + + "broker.id", + candidateKey, + epoch + ) + ); + } else if (localId.isEmpty()) { + throw new IllegalStateException("Cannot add voted state without a replica id"); + } else if (epoch != currentEpoch || isProspectiveAndVoted()) { + throw new IllegalStateException( + String.format( + "Cannot add voted key (%s) to current state (%s) in epoch %d", + candidateKey, + state, + epoch + ) + ); + } + + // Note that we reset the election timeout after voting for a candidate because we + // know that the candidate has at least as good of a chance of getting elected as us + durableTransitionTo( + new ProspectiveState( + time, + localIdOrThrow(), + epoch, + state.election().optionalLeaderId(), + Optional.of(state.leaderEndpoints()), + Optional.of(candidateKey), + partitionState.lastVoterSet(), + state.highWatermark(), + randomElectionTimeoutMs(), + logContext + ) + ); + log.debug("Voted for candidate {} in epoch {}", candidateKey, epoch); + } + /** * Become a follower of an elected leader so that we can begin fetching. */ @@ -733,12 +772,29 @@ public ResignedState resignedStateOrThrow() { throw new IllegalStateException("Expected to be Resigned, but current state is " + state); } + public Optional maybeProspectiveState() { + EpochState fixedState = state; + if (fixedState instanceof ProspectiveState) { + return Optional.of((ProspectiveState) fixedState); + } else { + return Optional.empty(); + } + } + public ProspectiveState prospectiveStateOrThrow() { if (isProspective()) return (ProspectiveState) state; throw new IllegalStateException("Expected to be Prospective, but current state is " + state); } + public boolean isProspectiveNotVoted() { + return maybeProspectiveState().filter(unattached -> unattached.votedKey().isEmpty()).isPresent(); + } + + public boolean isProspectiveAndVoted() { + return maybeUnattachedState().flatMap(UnattachedState::votedKey).isPresent(); + } + public CandidateState candidateStateOrThrow() { if (isCandidate()) return (CandidateState) state; diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index c536bb7108bd5..598b7bc0aea8e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -108,7 +108,7 @@ public void testPreVoteIfHasNotFetchedFromLeaderYet(boolean isLogUpToDate) { @ParameterizedTest @ValueSource(booleans = {true, false}) - public void testGrantVote(boolean isLogUpToDate) { + public void testGrantVoteAfterSuccessfulFetchFromLeader(boolean isLogUpToDate) { FollowerState state = newFollowerState( Set.of(1, 2, 3), Optional.empty() @@ -131,4 +131,4 @@ public void testLeaderIdAndEndpoint() { assertEquals(leaderId, state.leaderId()); assertEquals(leaderEndpoints, state.leaderEndpoints()); } -} +} \ No newline at end of file diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateWithVoteTest.java new file mode 100644 index 0000000000000..9f40d59c52c9b --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateWithVoteTest.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.net.InetSocketAddress; +import java.util.Collections; +import java.util.Optional; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +public class FollowerStateWithVoteTest { + private final MockTime time = new MockTime(); + private final LogContext logContext = new LogContext(); + private final int epoch = 5; + private final int leaderId = 1; + private final Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses( + Collections.singletonMap( + ListenerName.normalised("CONTROLLER"), + InetSocketAddress.createUnresolved("mock-host-1", 1234) + ) + ); + private final ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); + private final Set voters = Set.of(1, 2, 3); + private final int fetchTimeoutMs = 15000; + + private FollowerState newFollowerVotedState() { + return new FollowerState( + time, + epoch, + leaderId, + leaderEndpoints, + Optional.of(votedKey), + voters, + Optional.empty(), + fetchTimeoutMs, + logContext + ); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testPreVoteIfHasNotFetchedFromLeaderYet(boolean isLogUpToDate) { + FollowerState state = newFollowerVotedState(); + assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGrantVoteAfterSuccessfulFetchFromLeader(boolean isLogUpToDate) { + FollowerState state = newFollowerVotedState(); + state.resetFetchTimeoutForSuccessfulFetch(time.milliseconds()); + + assertFalse(state.canGrantVote(ReplicaKey.of(1, Uuid.randomUuid()), isLogUpToDate, true)); + assertFalse(state.canGrantVote(ReplicaKey.of(2, Uuid.randomUuid()), isLogUpToDate, true)); + assertFalse(state.canGrantVote(ReplicaKey.of(3, Uuid.randomUuid()), isLogUpToDate, true)); + assertFalse(state.canGrantVote(ReplicaKey.of(10, Uuid.randomUuid()), isLogUpToDate, true)); + + assertFalse(state.canGrantVote(ReplicaKey.of(1, Uuid.randomUuid()), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(2, Uuid.randomUuid()), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(3, Uuid.randomUuid()), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(10, Uuid.randomUuid()), isLogUpToDate, false)); + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index d078350d04a5c..1e03f591c5fe1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -1509,6 +1509,41 @@ public void testHandleVoteRequestAsFollowerWithVotedCandidate(boolean withKip853 context.assertVotedCandidate(epoch, votedCandidateKey.id()); } + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testProspectiveLosesElectionHasLeaderButMissingEndpoint(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + int epoch = 2; + ReplicaKey voter1 = replicaKey(localId + 1, withKip853Rpc); + int electedLeaderId = localId + 3; + Set voters = Set.of(localId, voter1.id()); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withElectedLeader(epoch, electedLeaderId) + .withKip853Rpc(withKip853Rpc) + .build(); + context.assertElectedLeader(epoch, electedLeaderId); + assertTrue(context.client.quorum().isUnattached()); + // Sleep a little to ensure that we become a prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + + // Sleep past election timeout + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + + // Prospective should transition to unattached + assertTrue(context.client.quorum().isUnattached()); + assertTrue(context.client.quorum().hasLeader()); + + // If election timeout expires again, it should transition back to prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + assertTrue(context.client.quorum().hasLeader()); + } + @ParameterizedTest @ValueSource(booleans = { true, false }) public void testHandleInvalidVoteRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception { diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index 48a43199b364e..c342888265ed6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -42,7 +42,6 @@ import java.util.stream.IntStream; import java.util.stream.Stream; -import static org.apache.kafka.server.common.KRaftVersion.KRAFT_VERSION_0; import static org.apache.kafka.server.common.KRaftVersion.KRAFT_VERSION_1; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -218,6 +217,73 @@ public void testInitializeAsFollower(KRaftVersion kraftVersion) { assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testInitializeVotedAndLeaderAreSame(KRaftVersion kraftVersion) { + int epoch = 5; + ReplicaKey leaderKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderKey.id(), 2), kraftVersion); + store.writeElectionState( + new ElectionState(epoch, OptionalInt.of(leaderKey.id()), Optional.of(leaderKey), voters.voterIds()), + kraftVersion + ); + + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isFollower()); + assertEquals(epoch, state.epoch()); + + FollowerState followerState = state.followerStateOrThrow(); + assertEquals(epoch, followerState.epoch()); + assertEquals(leaderKey.id(), followerState.leaderId()); + assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); + assertEquals( + Optional.of( + new ElectionState( + epoch, + OptionalInt.of(leaderKey.id()), + Optional.of(persistedVotedKey(leaderKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testInitializeVotedAndLeaderAreDifferent(KRaftVersion kraftVersion) { + int leader = 1; + int epoch = 5; + ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leader, votedKey.id()), kraftVersion); + store.writeElectionState( + new ElectionState(epoch, OptionalInt.of(leader), Optional.of(votedKey), voters.voterIds()), + kraftVersion + ); + + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isFollower()); + assertEquals(epoch, state.epoch()); + + FollowerState followerState = state.followerStateOrThrow(); + assertEquals(epoch, followerState.epoch()); + assertEquals(leader, followerState.leaderId()); + assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); + assertEquals( + Optional.of( + new ElectionState( + epoch, + OptionalInt.of(leader), + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); + } + @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testInitializeAsUnattachedWhenMissingEndpoints(KRaftVersion kraftVersion) { @@ -225,8 +291,12 @@ public void testInitializeAsUnattachedWhenMissingEndpoints(KRaftVersion kraftVer int node2 = 2; int leader = 3; int epoch = 5; + ReplicaKey votedKey = ReplicaKey.of(leader, Uuid.randomUuid()); VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - store.writeElectionState(ElectionState.withElectedLeader(epoch, leader, voters.voterIds()), kraftVersion); + store.writeElectionState( + new ElectionState(epoch, OptionalInt.of(leader), Optional.of(votedKey), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -235,11 +305,22 @@ public void testInitializeAsUnattachedWhenMissingEndpoints(KRaftVersion kraftVer UnattachedState unattachedState = state.unattachedStateOrThrow(); assertEquals(epoch, unattachedState.epoch()); + assertEquals( + Optional.of( + new ElectionState( + epoch, + OptionalInt.of(leader), + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testInitializeAsVoted(KRaftVersion kraftVersion) { + public void testInitializeAsVotedNoLeader(KRaftVersion kraftVersion) { ReplicaKey nodeKey1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); @@ -337,14 +418,29 @@ public void testInitializeAsResignedLeader(KRaftVersion kraftVersion) { resignedState.remainingElectionTimeMs(time.milliseconds())); } - @Test - public void testCandidateToCandidate() { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testInitializeAsOnlyVoter(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + store.writeElectionState(ElectionState.withUnknownLeader(0, voters.voterIds()), kraftVersion); + + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + // start as unattached, in KafkaRaftClient initialization the local replica then transitions to candidate + assertTrue(state.isUnattached()); + assertEquals(0, state.epoch()); + assertEquals(OptionalInt.empty(), state.leaderId()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToProspective(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_0); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); assertEquals(Optional.empty(), store.readElectionState()); - QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_0); + QuorumState state = initializeEmptyState(voters, kraftVersion); state.transitionToProspective(); int jitterMs1 = 2500; random.mockNextInt(electionTimeoutMs, jitterMs1); @@ -352,8 +448,10 @@ public void testCandidateToCandidate() { assertTrue(state.isCandidate()); assertEquals(1, state.epoch()); CandidateState candidate = state.candidateStateOrThrow(); - assertEquals(electionTimeoutMs + jitterMs1, - candidate.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + electionTimeoutMs + jitterMs1, + candidate.remainingElectionTimeMs(time.milliseconds()) + ); // The election timeout should be reset after we transition to prospective time.sleep(candidate.remainingElectionTimeMs(time.milliseconds())); @@ -362,17 +460,33 @@ public void testCandidateToCandidate() { random.mockNextInt(electionTimeoutMs, jitterMs2); state.transitionToProspective(); ProspectiveState prospective = state.prospectiveStateOrThrow(); - assertEquals(electionTimeoutMs + jitterMs2, - prospective.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + electionTimeoutMs + jitterMs2, + prospective.remainingElectionTimeMs(time.milliseconds()) + ); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion))), + store.readElectionState() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToCandidate(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + assertEquals(Optional.empty(), store.readElectionState()); - // The election timeout should be reset after we transition to candidate again - int jitterMs3 = 1000; - random.mockNextInt(electionTimeoutMs, jitterMs3); - prospective.recordGrantedVote(node1); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.transitionToProspective(); state.transitionToCandidate(); - candidate = state.candidateStateOrThrow(); - assertEquals(electionTimeoutMs + jitterMs3, - candidate.remainingElectionTimeMs(time.milliseconds())); + assertTrue(state.isCandidate()); + assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); } @ParameterizedTest @@ -434,7 +548,33 @@ public void testCandidateToLeaderWithoutGrantedVote(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToFollower(KRaftVersion kraftVersion) { + public void testCandidateToFollowerSameEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + + state.transitionToFollower(state.epoch(), otherNodeId, voters.listeners(otherNodeId)); + assertEquals(1, state.epoch()); + assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 1, + otherNodeId, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToFollowerHigherEpoch(KRaftVersion kraftVersion) { int otherNodeId = 1; VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); @@ -460,7 +600,20 @@ public void testCandidateToFollower(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToUnattached(KRaftVersion kraftVersion) { + public void testCandidateToUnattachedSameEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToUnattachedHigherEpoch(KRaftVersion kraftVersion) { int otherNodeId = 1; VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -558,6 +711,25 @@ public void testLeaderToResigned(KRaftVersion kraftVersion) { assertEquals(Collections.emptySet(), resignedState.unackedVoters()); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderToProspective(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + state.transitionToLeader(0L, accumulator); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); + + assertThrows(IllegalStateException.class, state::transitionToProspective); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); + } + @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testLeaderToCandidate(KRaftVersion kraftVersion) { @@ -579,7 +751,25 @@ public void testLeaderToCandidate(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderToFollower(KRaftVersion kraftVersion) { + public void testLeaderToFollowerSameEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + + state.transitionToProspective(); + state.transitionToCandidate(); + state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + state.transitionToLeader(0L, accumulator); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower(state.epoch(), otherNodeId, voters.listeners(otherNodeId)) + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderToFollowerHigherEpoch(KRaftVersion kraftVersion) { int otherNodeId = 1; VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); @@ -607,7 +797,21 @@ public void testLeaderToFollower(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderToUnattached(KRaftVersion kraftVersion) { + public void testLeaderToUnattachedSameEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + state.transitionToLeader(0L, accumulator); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderToUnattachedHigherEpoch(KRaftVersion kraftVersion) { int otherNodeId = 1; VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -666,7 +870,7 @@ public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCannotFollowOrVoteForSelf(KRaftVersion kraftVersion) { + public void testCannotFollowSelf(KRaftVersion kraftVersion) { VoterSet voters = localStandaloneVoterSet(); assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -682,6 +886,15 @@ public void testCannotFollowOrVoteForSelf(KRaftVersion kraftVersion) { assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(0, localVoterKey)); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCannotAddVotedStateForSelfAsUnattached(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); + QuorumState state = initializeEmptyState(voters, kraftVersion); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(0, localVoterKey)); + } + @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testUnattachedToLeaderOrResigned(KRaftVersion kraftVersion) { @@ -701,7 +914,7 @@ public void testUnattachedToLeaderOrResigned(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedSameEpoch(KRaftVersion kraftVersion) { + public void testUnattachedToUnattachedVotedSameEpoch(KRaftVersion kraftVersion) { ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -734,7 +947,7 @@ public void testUnattachedVotedSameEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { + public void testUnattachedToUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -747,7 +960,7 @@ public void testUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedToCandidate(KRaftVersion kraftVersion) { + public void testUnattachedToProspectiveToCandidate(KRaftVersion kraftVersion) { ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); @@ -766,6 +979,17 @@ public void testUnattachedToCandidate(KRaftVersion kraftVersion) { candidateState.remainingElectionTimeMs(time.milliseconds())); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testUnattachedToCandidate(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToUnattached(5); + assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); + } + @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testUnattachedToUnattached(KRaftVersion kraftVersion) { @@ -779,6 +1003,10 @@ public void testUnattachedToUnattached(KRaftVersion kraftVersion) { long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); time.sleep(1000); + // cannot transition to unattached in same epoch + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + + // can transition to unattached in higher epoch state.transitionToUnattached(6); assertTrue(state.isUnattachedNotVoted()); UnattachedState unattachedState = state.unattachedStateOrThrow(); @@ -869,20 +1097,21 @@ public void testUnattachedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToInvalidLeaderOrResigned(KRaftVersion kraftVersion) { + public void testUnattachedVotedToCandidateOrLeaderOrResigned(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToCandidate(KRaftVersion kraftVersion) { + public void testUnattachedVotedToProspectiveToCandidate(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); @@ -902,40 +1131,27 @@ public void testUnattachedVotedToCandidate(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testObserverCannotTransitionToProspective(KRaftVersion kraftVersion) { - int voter1 = 1; - int voter2 = 2; - VoterSet voters = withRemoteVoterSet(IntStream.of(voter1, voter2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - - assertTrue(state.isUnattachedNotVoted()); - assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); - - state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID)); - assertTrue(state.isUnattachedAndVoted()); - assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); - - state.transitionToFollower(logEndEpoch + 1, voter1, voters.listeners(voter1)); - assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToUnattachedVotedSameEpoch(KRaftVersion kraftVersion) { + public void testUnattachedVotedToUnattachedVoted(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, 8)); state.unattachedAddVotedState(8, ReplicaKey.of(node1, Uuid.randomUuid())); + // same epoch + assertThrows( + IllegalStateException.class, + () -> state.unattachedAddVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) + ); + // same votedKey assertThrows( IllegalStateException.class, () -> state.unattachedAddVotedState(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) ); + // higher epoch assertThrows( IllegalStateException.class, - () -> state.unattachedAddVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) + () -> state.unattachedAddVotedState(10, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) ); } @@ -1073,48 +1289,34 @@ public void testUnattachedVotedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testIllegalStateTransitionToUnattachedInSameEpoch(KRaftVersion kraftVersion) { + public void testProspectiveToUnattachedInSameEpoch(KRaftVersion kraftVersion) { ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, 5)); - - // unattached to unattached - state.unattachedStateOrThrow(); - state.unattachedAddVotedState(5, voter1); - // cannot vote for same or different node in same epoch - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(5, voter1)); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(5, voter2)); - - // follower to unattached - state.transitionToFollower(10, voter1.id(), voters.listeners(voter1.id())); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); - + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - // candidate - state.transitionToCandidate(); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); - - // leader - state.candidateStateOrThrow().recordGrantedVote(voter1.id()); - state.transitionToLeader(0L, accumulator); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + state.transitionToUnattached(state.epoch()); + assertEquals( + ElectionState.withUnknownLeader( + logEndEpoch, + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testProspectiveToUnattachedInSameEpoch(KRaftVersion kraftVersion) { + public void testProspectiveWithLeaderToUnattachedInSameEpoch(KRaftVersion kraftVersion) { ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - QuorumState state = initializeEmptyState(voters, kraftVersion); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - - // transition to unattached with leader state - state.transitionToUnattached(state.epoch(), OptionalInt.of(voter1.id())); assertEquals( ElectionState.withElectedLeader( logEndEpoch, @@ -1124,14 +1326,113 @@ public void testProspectiveToUnattachedInSameEpoch(KRaftVersion kraftVersion) { store.readElectionState().get() ); - // without leader state - state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToUnattached(state.epoch(), OptionalInt.of(voter1.id())); + assertEquals( + ElectionState.withElectedLeader( + logEndEpoch, + voter1.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveToUnattachedInHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + + state.transitionToUnattached(20); + assertEquals( + ElectionState.withUnknownLeader( + 20, + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveToUnattachedWithLeaderInHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + + state.transitionToUnattached(10, OptionalInt.of(voter1.id())); + assertEquals( + ElectionState.withElectedLeader( + 10, + voter1.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveWithLeaderToUnattachedInHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.transitionToUnattached(state.epoch()); + assertEquals( + ElectionState.withElectedLeader( + logEndEpoch, + voter1.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + + // transition to unattached + state.transitionToUnattached(10); assertEquals( ElectionState.withUnknownLeader( + 10, + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveWithLeaderToUnattachedWithLeaderInHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + assertEquals( + ElectionState.withElectedLeader( logEndEpoch, + voter1.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + + // transition to unattached with different leader state + state.transitionToUnattached(10, OptionalInt.of(voter2.id())); + assertEquals( + ElectionState.withElectedLeader( + 10, + voter2.id(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1140,48 +1441,297 @@ public void testProspectiveToUnattachedInSameEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testProspectiveVotedToUnattachedVotedInSameEpoch(KRaftVersion kraftVersion) { + public void testProspectiveToUnattachedVoted(KRaftVersion kraftVersion) { ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState( + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + + // in same epoch + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(logEndEpoch, voter1)); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(logEndEpoch, localVoterKey)); + + // in higher epoch + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, voter1)); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, localVoterKey)); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveToFollowerSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + + state.transitionToFollower(state.epoch(), voter2.id(), voters.listeners(voter2.id())); + assertEquals( + ElectionState.withElectedLeader( + logEndEpoch, + voter2.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveToFollowerHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + + state.transitionToFollower(10, voter2.id(), voters.listeners(voter2.id())); + assertEquals( + ElectionState.withElectedLeader( + 10, + voter2.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveToProspective(KRaftVersion kraftVersion) { + int leaderId = 1; + int followerId = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), kraftVersion); + + store.writeElectionState(ElectionState.withUnknownLeader(logEndEpoch, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertThrows(IllegalStateException.class, state::transitionToProspective); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveToCandidate(KRaftVersion kraftVersion) { + int leaderId = 1; + int followerId = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), kraftVersion); + + store.writeElectionState(ElectionState.withUnknownLeader(logEndEpoch, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + int jitterMs1 = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs1); + state.transitionToProspective(); + assertFalse(state.hasLeader()); + assertEquals( + electionTimeoutMs + jitterMs1, + state.prospectiveStateOrThrow().remainingElectionTimeMs(time.milliseconds()) + ); + + int jitterMs2 = 3000; + random.mockNextInt(electionTimeoutMs, jitterMs2); + state.transitionToCandidate(); + assertEquals( + electionTimeoutMs + jitterMs2, + state.candidateStateOrThrow().remainingElectionTimeMs(time.milliseconds()) + ); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion))), + store.readElectionState() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveWithLeaderToCandidate(KRaftVersion kraftVersion) { + int leaderId = 1; + int followerId = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), kraftVersion); + + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, leaderId, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertTrue(state.hasLeader()); + + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertEquals(logEndEpoch + 1, state.epoch()); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion))), + store.readElectionState() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveToLeaderOrResigned(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + + // in same epoch + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(logEndEpoch, accumulator)); + assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); + + // in higher epoch + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(10, accumulator)); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveVotedToUnattachedInSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withUnknownLeader(logEndEpoch, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.prospectiveAddVotedState(logEndEpoch, voter1); + + state.transitionToUnattached(logEndEpoch); + assertEquals( ElectionState.withVotedCandidate( logEndEpoch, - voter1, - voters.voterIds() - ), kraftVersion + persistedVotedKey(voter1, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveVotedWithLeaderToUnattachedInSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey leader = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey candidate = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leader, candidate)); + store.writeElectionState( + ElectionState.withElectedLeader(logEndEpoch, leader.id(), voters.voterIds()), + kraftVersion ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); + state.prospectiveAddVotedState(logEndEpoch, candidate); - // transition to unattached with leader state - state.transitionToUnattached(state.epoch(), OptionalInt.of(voter1.id())); + // transition to unattached with different leader state + state.transitionToUnattached(state.epoch(), OptionalInt.of(candidate.id())); - // with voted state assertEquals( new ElectionState( logEndEpoch, - OptionalInt.of(voter1.id()), - Optional.of(persistedVotedKey(voter1, kraftVersion)), + OptionalInt.of(candidate.id()), + Optional.of(persistedVotedKey(candidate, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() ); + } - // without leader state - state = initializeEmptyState(voters, kraftVersion); + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveVotedToAndFromFollowerSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withUnknownLeader(logEndEpoch, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.transitionToUnattached(state.epoch()); + state.prospectiveAddVotedState(logEndEpoch, voter1); + + // transition to follower of voter2 with votedKey voter1 + state.transitionToFollower(state.epoch(), voter2.id(), voters.listeners(voter2.id())); assertEquals( - ElectionState.withVotedCandidate( + new ElectionState( logEndEpoch, - persistedVotedKey(localVoterKey, kraftVersion), + OptionalInt.of(voter2.id()), + Optional.of(persistedVotedKey(voter1, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() ); + + // transition back to prospective + state.transitionToProspective(); + assertEquals( + Optional.of(new ElectionState( + logEndEpoch, + OptionalInt.of(voter2.id()), + Optional.of(persistedVotedKey(voter1, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion)) + ), + store.readElectionState() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveVotedToCandidate(KRaftVersion kraftVersion) { + int node1 = 1; + Uuid node1DirectoryId = Uuid.randomUuid(); + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + store.writeElectionState( + ElectionState.withVotedCandidate( + logEndEpoch, + ReplicaKey.of(node1, node1DirectoryId), + voters.voterIds() + ), + kraftVersion + ); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isUnattachedAndVoted()); + int jitterMs1 = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs1); + state.transitionToProspective(); + ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); + assertEquals( + electionTimeoutMs + jitterMs1, + prospectiveState.remainingElectionTimeMs(time.milliseconds()) + ); + assertTrue(prospectiveState.votedKey().isPresent()); + + int jitterMs2 = 3000; + random.mockNextInt(electionTimeoutMs, jitterMs2); + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertEquals( + electionTimeoutMs + jitterMs2, + state.candidateStateOrThrow().remainingElectionTimeMs(time.milliseconds()) + ); + assertEquals(logEndEpoch + 1, state.epoch()); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion))), + store.readElectionState() + ); } @ParameterizedTest @@ -1320,7 +1870,7 @@ public void testFollowerToLeaderOrResigned(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToCandidate(KRaftVersion kraftVersion) { + public void testFollowerToProspective(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); @@ -1331,15 +1881,29 @@ public void testFollowerToCandidate(KRaftVersion kraftVersion) { node2, voters.listeners(node2) ); - state.transitionToProspective(); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - CandidateState candidateState = state.candidateStateOrThrow(); - assertEquals(9, candidateState.epoch()); + state.transitionToProspective(); + ProspectiveState prospective = state.prospectiveStateOrThrow(); + assertEquals(8, prospective.epoch()); assertEquals(electionTimeoutMs + jitterMs, - candidateState.remainingElectionTimeMs(time.milliseconds())); + prospective.remainingElectionTimeMs(time.milliseconds())); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerToCandidate(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); + assertThrows(IllegalStateException.class, state::transitionToCandidate); } @ParameterizedTest @@ -1347,6 +1911,7 @@ public void testFollowerToCandidate(KRaftVersion kraftVersion) { public void testFollowerToUnattachedSameEpoch(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; + ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -1356,6 +1921,7 @@ public void testFollowerToUnattachedSameEpoch(KRaftVersion kraftVersion) { voters.listeners(node2) ); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8)); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(8, votedKey)); } @ParameterizedTest @@ -1384,43 +1950,88 @@ public void testFollowerToUnattachedHigherEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testIllegalCallsToUnattachedAddVotedState(KRaftVersion kraftVersion) { - boolean withDirectoryId = kraftVersion.featureLevel() > 0; - ReplicaKey node1 = replicaKey(1, withDirectoryId); - ReplicaKey node2 = replicaKey(2, withDirectoryId); - VoterSet voters = localWithRemoteVoterSet(Stream.of(node1, node2), kraftVersion); + public void testFollowerToUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + ReplicaKey votedKey = ReplicaKey.of(node2, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); + + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(9, votedKey)); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerToProspectiveSameEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + store.writeElectionState(ElectionState.withUnknownLeader(logEndEpoch, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); + state.transitionToProspective(); + ProspectiveState prospective = state.prospectiveStateOrThrow(); + assertEquals(8, prospective.epoch()); + assertEquals( + electionTimeoutMs + jitterMs, + prospective.remainingElectionTimeMs(time.milliseconds()) + ); + assertEquals( + ElectionState.withElectedLeader( + 8, + node2, + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - // should fail on any state that isn't unattached and not voted - - // unattached and voted - state.unattachedAddVotedState(state.epoch(), node1); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); - - // prospective - state.transitionToUnattached(5); - state.transitionToProspective(); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); - - // prospective and voted - state.prospectiveStateOrThrow().recordGrantedVote(node1.id()); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); - - // candidate - state.transitionToCandidate(); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); - - // candidate and voted - state.candidateStateOrThrow().recordGrantedVote(node1.id()); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); - - // leader - state.transitionToLeader(6, accumulator); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(state.epoch(), node1)); - - // should fail if epoch is not equal - state.transitionToUnattached(7); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(8, node1)); + state.transitionToFollower( + 5, + otherNodeId, + voters.listeners(otherNodeId) + ); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + otherNodeId, + voters.listeners(otherNodeId) + ) + ); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(4, accumulator)); + assertEquals(5, state.epoch()); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 5, + otherNodeId, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); } @ParameterizedTest @@ -1478,32 +2089,73 @@ public void testFollowerVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion) @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + public void testFollowerVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + ReplicaKey votedKey = ReplicaKey.of(node1, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.prospectiveAddVotedState(state.epoch(), votedKey); state.transitionToFollower( - 5, - otherNodeId, - voters.listeners(otherNodeId) + state.epoch(), + node2, + voters.listeners(node2) ); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertEquals( + Optional.of( + new ElectionState( + 0, + OptionalInt.of(node2), + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); + assertThrows( IllegalStateException.class, - () -> state.transitionToFollower( - 4, - otherNodeId, - voters.listeners(otherNodeId) - ) + () -> state.transitionToFollower(state.epoch(), node1, voters.listeners(node1)) ); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(4, accumulator)); - assertEquals(5, state.epoch()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + ReplicaKey votedKey = ReplicaKey.of(node1, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.prospectiveAddVotedState(state.epoch(), votedKey); + state.transitionToFollower( + state.epoch(), + node2, + voters.listeners(node2) + ); + assertEquals( + Optional.of( + new ElectionState( + 0, + OptionalInt.of(node2), + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); + + state.transitionToFollower(state.epoch() + 1, node1, voters.listeners(node1)); + assertEquals(1, state.epoch()); assertEquals( Optional.of( ElectionState.withElectedLeader( - 5, - otherNodeId, + 1, + node1, persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -1640,6 +2292,26 @@ public void testObserverUnattachedToFollower(KRaftVersion kraftVersion) { assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testObserverUnattachedToProspective(KRaftVersion kraftVersion) { + int voter1 = 1; + int voter2 = 2; + VoterSet voters = withRemoteVoterSet(IntStream.of(voter1, voter2), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + + assertTrue(state.isUnattachedNotVoted()); + assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); + + state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID)); + assertTrue(state.isUnattachedAndVoted()); + assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); + + state.transitionToFollower(logEndEpoch + 1, voter1, voters.listeners(voter1)); + assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); + } + @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testInitializeWithCorruptedStore(KRaftVersion kraftVersion) { @@ -2021,162 +2693,4 @@ public void testFollowerToAndFromProspectiveWithLeader() { store.readElectionState() ); } - - @Test - public void testProspectiveVotedToAndFromCandidate() { - int node1 = 1; - Uuid node1DirectoryId = Uuid.randomUuid(); - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); - store.writeElectionState( - ElectionState.withVotedCandidate( - logEndEpoch, - ReplicaKey.of(node1, node1DirectoryId), - voters.voterIds() - ), - KRAFT_VERSION_1 - ); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - assertTrue(state.isUnattachedAndVoted()); - - state.transitionToProspective(); - ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); - assertTrue(prospectiveState.votedKey().isPresent()); - - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - assertEquals(logEndEpoch + 1, state.epoch()); - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - logEndEpoch + 1, - persistedVotedKey(localVoterKey, KRAFT_VERSION_1), - persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() - ); - - state.transitionToProspective(); - assertTrue(state.isProspective()); - assertEquals(logEndEpoch + 1, state.epoch()); - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - logEndEpoch + 1, - persistedVotedKey(localVoterKey, KRAFT_VERSION_1), - persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() - ); - } - - @Test - public void testProspectiveWithLeaderToCandidate() { - int leaderId = 1; - int followerId = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), KRAFT_VERSION_1); - - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, leaderId, voters.voterIds()), KRAFT_VERSION_1); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - assertTrue(state.isProspective()); - assertTrue(state.hasLeader()); - - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - assertEquals(logEndEpoch + 1, state.epoch()); - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - logEndEpoch + 1, - persistedVotedKey(localVoterKey, KRAFT_VERSION_1), - persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() - ); - } - - @Test - public void testProspectiveToUnattachedHigherEpoch() { - int leaderId = 1; - int followerId = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), KRAFT_VERSION_1); - - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, leaderId, voters.voterIds()), KRAFT_VERSION_1); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - assertTrue(state.isProspective()); - assertTrue(state.hasLeader()); - - state.transitionToUnattached(logEndEpoch + 1); - assertTrue(state.isUnattachedNotVoted()); - assertFalse(state.hasLeader()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testProspectiveVotedToAndFromFollower(KRaftVersion kraftVersion) { - int candidate = 1; - ReplicaKey candidateKey = ReplicaKey.of(candidate, Uuid.randomUuid()); - int leader = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(candidate, leader), kraftVersion); - store.writeElectionState( - ElectionState.withVotedCandidate( - logEndEpoch, - candidateKey, - voters.voterIds() - ), kraftVersion - ); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - assertTrue(state.isUnattachedAndVoted()); - - // transition to prospective with votedKey - state.transitionToProspective(); - - assertTrue(state.isProspective()); - assertEquals(logEndEpoch, state.epoch()); - Optional expectedElectionState = Optional.of( - ElectionState.withVotedCandidate( - logEndEpoch, - persistedVotedKey(candidateKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion) - ) - ); - assertEquals(expectedElectionState, store.readElectionState()); - - // transition to follower of leader with votedKey - state.transitionToFollower(logEndEpoch, leader, voters.listeners(leader)); - assertTrue(state.isFollower()); - assertEquals(logEndEpoch, state.epoch()); - assertEquals( - Optional.of(new ElectionState( - logEndEpoch, - OptionalInt.of(leader), - Optional.of(persistedVotedKey(candidateKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion)) - ), - store.readElectionState() - ); - - // transition back to prospective with votedKey and leaderId - state.transitionToProspective(); - assertTrue(state.isProspective()); - assertEquals(logEndEpoch, state.epoch()); - assertEquals( - Optional.of(new ElectionState( - logEndEpoch, - OptionalInt.of(leader), - Optional.of(persistedVotedKey(candidateKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion)) - ), - store.readElectionState() - ); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testFollowerVotedToUnattached(KRaftVersion kraftVersion) { - - } } diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java index d4131e77c05b0..883bb1f17f9bb 100644 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java @@ -92,6 +92,10 @@ public void testGrantVote(boolean isLogUpToDate) { isLogUpToDate, state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); assertEquals( isLogUpToDate, @@ -105,6 +109,10 @@ public void testGrantVote(boolean isLogUpToDate) { isLogUpToDate, state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) ); + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) + ); } @Test @@ -139,10 +147,15 @@ void testUnattachedWithLeader(boolean isLogUpToDate) { isLogUpToDate, state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); // Check that the replica rejects all standard votes request if the leader is known assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java index c7ec0292e4061..008706abf3064 100644 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java @@ -24,9 +24,9 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Collections; import java.util.Optional; import java.util.OptionalInt; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -36,6 +36,7 @@ class UnattachedStateWithVoteTest { private final MockTime time = new MockTime(); private final LogContext logContext = new LogContext(); + private final Set voters = Set.of(1, 2, 3); private final int epoch = 5; private final int votedId = 1; private final int electionTimeoutMs = 10000; @@ -48,7 +49,7 @@ private UnattachedState newUnattachedVotedState( epoch, OptionalInt.empty(), Optional.of(ReplicaKey.of(votedId, votedDirectoryId)), - Collections.emptySet(), + voters, Optional.empty(), electionTimeoutMs, logContext @@ -63,7 +64,7 @@ public void testElectionTimeout() { assertEquals(epoch, state.epoch()); assertEquals(votedKey, state.votedKey().get()); assertEquals( - ElectionState.withVotedCandidate(epoch, votedKey, Collections.emptySet()), + ElectionState.withVotedCandidate(epoch, votedKey, voters), state.election() ); assertEquals(electionTimeoutMs, state.remainingElectionTimeMs(time.milliseconds())); @@ -105,7 +106,7 @@ public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { @ParameterizedTest @ValueSource(booleans = {true, false}) - void testCanGrantVoteWithDirectoryId(boolean isLogUpToDate) { + public void testCanGrantVoteWithDirectoryId(boolean isLogUpToDate) { Uuid votedDirectoryId = Uuid.randomUuid(); UnattachedState state = newUnattachedVotedState(votedDirectoryId); @@ -142,12 +143,19 @@ void testCanGrantVoteWithDirectoryId(boolean isLogUpToDate) { isLogUpToDate, state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), true, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), true, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + + // Observer + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); } @Test - void testLeaderEndpoints() { + public void testLeaderEndpoints() { Uuid votedDirectoryId = Uuid.randomUuid(); UnattachedState state = newUnattachedVotedState(votedDirectoryId); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java deleted file mode 100644 index 6b6941b25bc5d..0000000000000 --- a/raft/src/test/java/org/apache/kafka/raft/internals/FollowerStateWithVoteTest.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.raft.internals; - -public class FollowerStateWithVoteTest { -} From 309783211f0b51657312b2eb04c6508498b04906 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Mon, 23 Dec 2024 07:46:07 -0800 Subject: [PATCH 12/23] more comments and more test fixes --- checkstyle/suppressions.xml | 11 +- .../org/apache/kafka/raft/CandidateState.java | 2 +- .../apache/kafka/raft/KafkaRaftClient.java | 2 +- .../apache/kafka/raft/ProspectiveState.java | 10 + .../org/apache/kafka/raft/QuorumState.java | 12 +- .../raft/internals/KafkaRaftMetrics.java | 13 +- .../apache/kafka/raft/EpochElectionTest.java | 1 + .../apache/kafka/raft/FollowerStateTest.java | 7 +- .../kafka/raft/FollowerStateWithVoteTest.java | 89 - .../raft/KafkaRaftClientPreVoteTest.java | 64 +- .../kafka/raft/KafkaRaftClientTest.java | 148 +- .../kafka/raft/ProspectiveStateTest.java | 21 +- .../apache/kafka/raft/QuorumStateTest.java | 2761 ++++++++--------- .../kafka/raft/RaftClientTestContext.java | 5 +- .../kafka/raft/UnattachedStateTest.java | 128 +- .../raft/internals/KafkaRaftMetricsTest.java | 120 +- 16 files changed, 1729 insertions(+), 1665 deletions(-) delete mode 100644 raft/src/test/java/org/apache/kafka/raft/FollowerStateWithVoteTest.java diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 7362d9f586beb..f6724d35c9bd2 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -58,7 +58,7 @@ + files="(AbstractFetch|Sender|SenderTest|ConsumerCoordinator|KafkaConsumer|KafkaProducer|Utils|TransactionManager|TransactionManagerTest|KafkaAdminClient|NetworkClient|Admin|RaftClientTestContext|TestingMetricsInterceptingAdminClient).java"/> + files="(NetworkClient|FieldSpec|KafkaProducer).java"/> + files="(ConsumerCoordinator|BufferPool|MetricName|Node|ConfigDef|RecordBatch|SslFactory|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|TokenInformation|Agent|PluginUtils|MiniTrogdorCluster|TasksRequest|KafkaProducer|AbstractStickyAssignor|Authorizer|FetchSessionHandler|RecordAccumulator|Shell).java"/> @@ -188,10 +188,7 @@ files="(DynamicVoter|RecordsIterator).java"/> - - + files="(KafkaRaftClientTest).java"/> implements RaftClient { - private static final int RETRY_BACKOFF_BASE_MS = 100; private static final int MAX_NUMBER_OF_BATCHES = 10; public static final int MAX_FETCH_WAIT_MS = 500; public static final int MAX_BATCH_SIZE_BYTES = 8 * 1024 * 1024; diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index 7ec81813b088f..7f6a0e5d69248 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.raft.internals.EpochElection; + import org.slf4j.Logger; import java.util.Optional; @@ -116,6 +117,15 @@ public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolea epoch ); return false; + } else if (leaderId.isPresent()) { + // If the leader id is known it should behave similar to the follower state + log.debug( + "Rejecting Vote request (preVote=false) from candidate ({}) since we already have a leader {} in epoch {}", + replicaKey, + leaderId, + epoch + ); + return false; } else if (!isLogUpToDate) { log.debug( "Rejecting Vote request (preVote=false) from candidate ({}) since candidate's log is not up to date with us", diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 85c34e2b200a0..877663d922afa 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -426,15 +426,16 @@ public void unattachedAddVotedState( if (localId.isPresent() && candidateKey.id() == localId.getAsInt()) { throw new IllegalStateException( String.format( - "Cannot transition to Voted for %s and epoch %d since it matches the local " + + "Cannot add voted key (%s) to current state (%s) in epoch %d since it matches the local " + "broker.id", candidateKey, + state, epoch ) ); } else if (localId.isEmpty()) { throw new IllegalStateException("Cannot add voted state without a replica id"); - } else if (epoch != currentEpoch || isUnattachedAndVoted()) { + } else if (epoch != currentEpoch || !isUnattachedNotVoted()) { throw new IllegalStateException( String.format( "Cannot add voted key (%s) to current state (%s) in epoch %d", @@ -474,9 +475,10 @@ public void prospectiveAddVotedState( if (localId.isPresent() && candidateKey.id() == localId.getAsInt()) { throw new IllegalStateException( String.format( - "Cannot transition to Voted for %s and epoch %d since it matches the local " + + "Cannot add voted key (%s) to current state (%s) in epoch %d since it matches the local " + "broker.id", candidateKey, + state, epoch ) ); @@ -788,11 +790,11 @@ public ProspectiveState prospectiveStateOrThrow() { } public boolean isProspectiveNotVoted() { - return maybeProspectiveState().filter(unattached -> unattached.votedKey().isEmpty()).isPresent(); + return maybeProspectiveState().filter(prospective -> prospective.votedKey().isEmpty()).isPresent(); } public boolean isProspectiveAndVoted() { - return maybeUnattachedState().flatMap(UnattachedState::votedKey).isPresent(); + return maybeProspectiveState().flatMap(ProspectiveState::votedKey).isPresent(); } public CandidateState candidateStateOrThrow() { diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java index a63a169c8632d..fa93633b48016 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java @@ -71,8 +71,10 @@ public KafkaRaftMetrics(Metrics metrics, String metricGrpPrefix, QuorumState sta Gauge stateProvider = (mConfig, currentTimeMs) -> { if (state.isLeader()) { return "leader"; - } else if (state.isProspective()) { + } else if (state.isProspectiveNotVoted()) { return "prospective"; + } else if (state.isProspectiveAndVoted()) { + return "prospective-voted"; } else if (state.isCandidate()) { return "candidate"; } else if (state.isUnattachedAndVoted()) { @@ -98,9 +100,7 @@ public KafkaRaftMetrics(Metrics metrics, String metricGrpPrefix, QuorumState sta if (state.isLeader() || state.isCandidate()) { return state.localIdOrThrow(); } else { - return (double) state.maybeUnattachedState() - .flatMap(votedState -> votedState.votedKey().map(ReplicaKey::id)) - .orElse(-1); + return state.votedKey().map(ReplicaKey::id).orElse(-1); } }); @@ -113,10 +113,7 @@ public KafkaRaftMetrics(Metrics metrics, String metricGrpPrefix, QuorumState sta if (state.isLeader() || state.isCandidate()) { return state.localDirectoryId().toString(); } else { - return state.maybeUnattachedState() - .flatMap(votedState -> votedState.votedKey().flatMap(ReplicaKey::directoryId)) - .orElse(Uuid.ZERO_UUID) - .toString(); + return state.votedKey().flatMap(ReplicaKey::directoryId).orElse(Uuid.ZERO_UUID).toString(); } }; metrics.addMetric(this.currentVotedDirectoryIdMetricName, null, votedDirectoryIdProvider); diff --git a/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java b/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java index dd5d8314e4140..c528e74610942 100644 --- a/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.raft.internals.EpochElection; + import org.junit.jupiter.api.Test; import java.util.Set; diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index 598b7bc0aea8e..9a1092095ce2f 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; @@ -47,6 +48,7 @@ public class FollowerStateTest { InetSocketAddress.createUnresolved("mock-host-3", 1234) ) ); + private final ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); private FollowerState newFollowerState( Set voters, @@ -57,7 +59,7 @@ private FollowerState newFollowerState( epoch, leaderId, leaderEndpoints, - Optional.empty(), + Optional.of(votedKey), voters, highWatermark, fetchTimeoutMs, @@ -104,6 +106,7 @@ public void testPreVoteIfHasNotFetchedFromLeaderYet(boolean isLogUpToDate) { assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); } @ParameterizedTest @@ -118,10 +121,12 @@ public void testGrantVoteAfterSuccessfulFetchFromLeader(boolean isLogUpToDate) { assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); + assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); } @Test diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateWithVoteTest.java deleted file mode 100644 index 9f40d59c52c9b..0000000000000 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateWithVoteTest.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.raft; - -import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.network.ListenerName; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.MockTime; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; - -import java.net.InetSocketAddress; -import java.util.Collections; -import java.util.Optional; -import java.util.Set; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; - -public class FollowerStateWithVoteTest { - private final MockTime time = new MockTime(); - private final LogContext logContext = new LogContext(); - private final int epoch = 5; - private final int leaderId = 1; - private final Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses( - Collections.singletonMap( - ListenerName.normalised("CONTROLLER"), - InetSocketAddress.createUnresolved("mock-host-1", 1234) - ) - ); - private final ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); - private final Set voters = Set.of(1, 2, 3); - private final int fetchTimeoutMs = 15000; - - private FollowerState newFollowerVotedState() { - return new FollowerState( - time, - epoch, - leaderId, - leaderEndpoints, - Optional.of(votedKey), - voters, - Optional.empty(), - fetchTimeoutMs, - logContext - ); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testPreVoteIfHasNotFetchedFromLeaderYet(boolean isLogUpToDate) { - FollowerState state = newFollowerVotedState(); - assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); - assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); - assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); - assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testGrantVoteAfterSuccessfulFetchFromLeader(boolean isLogUpToDate) { - FollowerState state = newFollowerVotedState(); - state.resetFetchTimeoutForSuccessfulFetch(time.milliseconds()); - - assertFalse(state.canGrantVote(ReplicaKey.of(1, Uuid.randomUuid()), isLogUpToDate, true)); - assertFalse(state.canGrantVote(ReplicaKey.of(2, Uuid.randomUuid()), isLogUpToDate, true)); - assertFalse(state.canGrantVote(ReplicaKey.of(3, Uuid.randomUuid()), isLogUpToDate, true)); - assertFalse(state.canGrantVote(ReplicaKey.of(10, Uuid.randomUuid()), isLogUpToDate, true)); - - assertFalse(state.canGrantVote(ReplicaKey.of(1, Uuid.randomUuid()), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(2, Uuid.randomUuid()), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(3, Uuid.randomUuid()), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(10, Uuid.randomUuid()), isLogUpToDate, false)); - } -} diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 758bb724466bb..3752e79e0dcd8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.raft.RaftClientTestContext.RaftProtocol; import org.apache.kafka.server.common.KRaftVersion; + import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; @@ -124,9 +125,7 @@ public void testHandlePreVoteRequestAsFollowerWithVotedCandidate(KRaftVersion kr // follower can grant pre-votes if it has not fetched successfully from leader yet context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - ReplicaKey expectedVotedKey = kraftVersion == KRaftVersion.KRAFT_VERSION_1 - ? votedCandidateKey : replicaKey(votedCandidateKey.id(), false); - context.assertVotedCandidate(epoch, expectedVotedKey); + context.assertVotedCandidate(epoch, votedCandidateKey); } @ParameterizedTest @@ -153,9 +152,7 @@ public void testHandlePreVoteRequestAsCandidate(KRaftVersion kraftVersion) throw context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, leaderEpoch, OptionalInt.empty(), true); - ReplicaKey expectedVotedKey = kraftVersion == KRaftVersion.KRAFT_VERSION_1 - ? localKey : replicaKey(localId, false); - context.assertVotedCandidate(leaderEpoch, expectedVotedKey); + context.assertVotedCandidate(leaderEpoch, localKey); assertTrue(context.client.quorum().isCandidate()); // if an observer sends a pre-vote request for the same epoch, it should also be granted @@ -163,7 +160,7 @@ public void testHandlePreVoteRequestAsCandidate(KRaftVersion kraftVersion) throw context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, leaderEpoch, OptionalInt.empty(), true); - context.assertVotedCandidate(leaderEpoch, expectedVotedKey); + context.assertVotedCandidate(leaderEpoch, localKey); assertTrue(context.client.quorum().isCandidate()); // candidate will transition to unattached if pre-vote request has a higher epoch @@ -1026,6 +1023,59 @@ public void testProspectiveWithoutLeaderTransitionsToFollower( assertEquals(leader.id(), context.client.quorum().leaderId().orElse(-1)); } + @ParameterizedTest + @MethodSource("kraftVersionRaftProtocolCombinations") + public void testPreVoteRequestTimeout( + KRaftVersion kraftVersion, + RaftProtocol raftProtocol + ) throws Exception { + int localId = randomReplicaId(); + int epoch = 1; + ReplicaKey local = replicaKey(localId, true); + ReplicaKey otherNode = replicaKey(localId + 1, true); + RaftClientTestContext context = new RaftClientTestContext.Builder( + Optional.of(local), + Optional.of(VoterSetTest.voterSet(Stream.of(local, otherNode))), + kraftVersion + ) + .withUnknownLeader(epoch) + .withRaftProtocol(raftProtocol) + .build(); + + context.assertUnknownLeaderAndNoVotedCandidate(epoch); + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + + context.pollUntilRequest(); + + RaftRequest.Outbound request = context.assertSentVoteRequest(epoch, 0, 0L, 1); + + context.time.sleep(context.requestTimeoutMs()); + context.client.poll(); + RaftRequest.Outbound retryRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); + + // We will ignore the timed out response if it arrives late + context.deliverResponse( + request.correlationId(), + request.destination(), + context.voteResponse(true, OptionalInt.empty(), epoch) + ); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + + // Become candidate after receiving the retry response + context.deliverResponse( + retryRequest.correlationId(), + retryRequest.destination(), + context.voteResponse(true, OptionalInt.empty(), epoch) + ); + context.client.poll(); + assertTrue(context.client.quorum().isCandidate()); + context.assertVotedCandidate(epoch + 1, local); + } + + static Stream kraftVersionRaftProtocolCombinations() { return Stream.of(KRaftVersion.values()) .flatMap(enum1 -> Stream.of(RaftProtocol.values()) diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 1e03f591c5fe1..53d53c0d21e20 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -81,6 +81,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +@SuppressWarnings({"ClassDataAbstractionCoupling", "ClassFanOutComplexity" }) public class KafkaRaftClientTest { @Test public void testNodeDirectoryId() { @@ -217,7 +218,7 @@ public void testGrantVotesFromHigherEpochAfterResigningLeadership(boolean withKi ); context.client.poll(); - // We will first transition to unattached and then grant vote and then transition to voted + // We will first transition to unattached and then grant vote and then transition to unattached voted assertTrue(context.client.quorum().isUnattachedAndVoted()); context.assertVotedCandidate(epoch + 1, remoteKey.id()); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); @@ -254,7 +255,7 @@ public void testGrantVotesFromHigherEpochAfterResigningCandidacy(boolean withKip ); context.client.poll(); - // We will first transition to unattached and then grant vote and then transition to voted + // We will first transition to unattached and then grant vote and then transition to unattached voted assertTrue(context.client.quorum().isUnattachedAndVoted()); context.assertVotedCandidate(epoch + 1, remoteKey.id()); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); @@ -290,7 +291,7 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Excepti ); context.client.poll(); - // We will first transition to unattached and then grant vote and then transition to voted + // We will first transition to unattached and then grant vote and then transition to unattached voted assertTrue( context.client.quorum().isUnattachedAndVoted(), "Local Id: " + localId + @@ -304,7 +305,7 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Excepti @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testInitializeAsResignedAndBecomeCandidate(boolean withKip853Rpc) throws Exception { + public void testInitializeAsResignedAndBecomeProspective(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; Set voters = Set.of(localId, remoteId); @@ -327,11 +328,6 @@ public void testInitializeAsResignedAndBecomeCandidate(boolean withKip853Rpc) th // Become prospective assertTrue(context.client.quorum().isProspective()); - - // Receive granted PreVote and become candidate - context.expectAndGrantPreVotes(context.currentEpoch()); - assertTrue(context.client.quorum().isCandidate()); - context.assertVotedCandidate(epoch + 1, localId); } @ParameterizedTest @@ -365,10 +361,8 @@ public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) context.client.poll(); context.time.sleep(context.electionTimeoutMs()); - context.expectAndGrantPreVotes(epoch); - context.pollUntilRequest(); - context.assertVotedCandidate(epoch + 1, localId); - context.assertSentVoteRequest(epoch + 1, 0, 0L, 1); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); } @ParameterizedTest @@ -921,20 +915,18 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKi final int secondNodeId = localId + 2; Set voters = Set.of(localId, firstNodeId, secondNodeId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) .withKip853Rpc(withKip853Rpc) .build(); - - context.assertUnknownLeaderAndNoVotedCandidate(0); - assertTrue(context.client.quorum().isUnattached()); - context.unattachedToCandidate(); + assertTrue(context.client.quorum().isCandidate()); context.pollUntilRequest(); - context.assertVotedCandidate(1, localId); + context.assertVotedCandidate(2, localId); - RaftRequest.Outbound request = context.assertSentVoteRequest(1, 0, 0L, 2); + RaftRequest.Outbound request = context.assertSentVoteRequest(2, 0, 0L, 2); context.deliverResponse( request.correlationId(), request.destination(), - context.voteResponse(true, OptionalInt.empty(), 1) + context.voteResponse(true, OptionalInt.empty(), 2) ); VoteRequestData voteRequest = (VoteRequestData) request.data(); @@ -943,7 +935,7 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKi // Become leader after receiving the vote context.pollUntil(() -> context.log.endOffset().offset() == 1L); - context.assertElectedLeader(1, localId); + context.assertElectedLeader(2, localId); long electionTimestamp = context.time.milliseconds(); // Leader change record appended @@ -952,7 +944,7 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKi // Send BeginQuorumEpoch to voters context.client.poll(); - context.assertSentBeginQuorumEpochRequest(1, Set.of(firstNodeId, secondNodeId)); + context.assertSentBeginQuorumEpochRequest(2, Set.of(firstNodeId, secondNodeId)); Records records = context.log.read(0, Isolation.UNCOMMITTED).records; RecordBatch batch = records.batches().iterator().next(); @@ -1062,12 +1054,7 @@ public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) t .withKip853Rpc(withKip853Rpc) .build(); - // Sleep a little to ensure that we become a prospective - context.time.sleep(context.electionTimeoutMs() + jitterMs); - context.client.poll(); - assertTrue(context.client.quorum().isProspective()); - // Become candidate - context.expectAndGrantPreVotes(epoch - 1); + context.unattachedToCandidate(); context.assertVotedCandidate(epoch, localId); context.deliverRequest( @@ -1509,6 +1496,87 @@ public void testHandleVoteRequestAsFollowerWithVotedCandidate(boolean withKip853 context.assertVotedCandidate(epoch, votedCandidateKey.id()); } + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleVoteRequestAsProspective(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + int epoch = 2; + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + Set voters = Set.of(localId, otherNodeKey.id()); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(epoch) + .withKip853Rpc(withKip853Rpc) + .build(); + + // Sleep a little to ensure that we become a prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 1)); + context.pollUntilResponse(); + + context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); + context.assertVotedCandidate(epoch + 1, otherNodeKey.id()); + assertTrue(context.client.quorum().isUnattachedAndVoted()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleVoteRequestAsProspectiveWithLeader(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + int epoch = 2; + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + int electedLeaderId = localId + 2; + Set voters = Set.of(localId, otherNodeKey.id(), electedLeaderId); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withElectedLeader(epoch, electedLeaderId) + .withKip853Rpc(withKip853Rpc) + .build(); + + // Sleep a little to ensure that we become a prospective + context.time.sleep(context.fetchTimeoutMs); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 1)); + context.pollUntilResponse(); + + context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); + assertTrue(context.client.quorum().isUnattachedAndVoted()); + assertEquals(epoch + 1, context.currentEpoch()); + assertFalse(context.client.quorum().hasLeader()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleVoteRequestAsProspectiveWithVotedCandidate(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + int epoch = 2; + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + ReplicaKey votedCandidateKey = replicaKey(localId + 2, withKip853Rpc); + Set voters = Set.of(localId, otherNodeKey.id(), votedCandidateKey.id()); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withVotedCandidate(epoch, votedCandidateKey) + .withKip853Rpc(withKip853Rpc) + .build(); + + // Sleep a little to ensure that we become a prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertTrue(context.client.quorum().isProspectiveAndVoted()); + context.assertVotedCandidate(epoch, votedCandidateKey.id()); + + context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 1)); + context.pollUntilResponse(); + + context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); + context.assertVotedCandidate(epoch + 1, otherNodeKey.id()); + } + @ParameterizedTest @ValueSource(booleans = { true, false }) public void testProspectiveLosesElectionHasLeaderButMissingEndpoint(boolean withKip853Rpc) throws Exception { @@ -1797,7 +1865,7 @@ public void testInitializeAsFollowerNonEmptyLog(boolean withKip853Rpc) throws Ex @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testVoterBecomeCandidateAfterFetchTimeout(boolean withKip853Rpc) throws Exception { + public void testVoterBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -1815,15 +1883,13 @@ public void testVoterBecomeCandidateAfterFetchTimeout(boolean withKip853Rpc) thr context.assertSentFetchRequest(epoch, 1L, lastEpoch); context.time.sleep(context.fetchTimeoutMs); - context.expectAndGrantPreVotes(epoch); - context.pollUntilRequest(); - context.assertSentVoteRequest(epoch + 1, lastEpoch, 1L, 1); - context.assertVotedCandidate(epoch + 1, localId); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); } @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testFollowerAsObserverDoesNotBecomeCandidateAfterFetchTimeout(boolean withKip853Rpc) throws Exception { + public void testFollowerAsObserverDoesNotBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -1853,7 +1919,7 @@ public void testFollowerAsObserverDoesNotBecomeCandidateAfterFetchTimeout(boolea @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testUnattachedAsObserverDoesNotBecomeCandidateAfterElectionTimeout(boolean withKip853Rpc) throws Exception { + public void testUnattachedAsObserverDoesNotBecomeProspectiveAfterElectionTimeout(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -1882,7 +1948,7 @@ public void testUnattachedAsObserverDoesNotBecomeCandidateAfterElectionTimeout(b context.time.sleep(context.electionTimeoutMs() * 2); context.pollUntilRequest(); - // observer cannot transition to candidate though + // observer cannot transition to prospective though assertTrue(context.client.quorum().isUnattached()); context.assertSentFetchRequest(epoch + 1, 0L, 0); assertEquals(0, context.channel.drainSendQueue().size()); @@ -2647,7 +2713,7 @@ public void testPurgatoryFetchCompletedByFollowerTransition(boolean withKip853Rp @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testFetchResponseIgnoredAfterBecomingCandidate(boolean withKip853Rpc) throws Exception { + public void testFetchResponseIgnoredAfterBecomingProspective(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -2664,12 +2730,10 @@ public void testFetchResponseIgnoredAfterBecomingCandidate(boolean withKip853Rpc context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(epoch, 0L, 0); - // Now await the fetch timeout and become prospective, then candidate + // Now await the fetch timeout and become prospective context.time.sleep(context.fetchTimeoutMs); context.client.poll(); assertTrue(context.client.quorum().isProspective()); - context.expectAndGrantPreVotes(epoch); - context.assertVotedCandidate(epoch + 1, localId); // The fetch response from the old leader returns, but it should be ignored Records records = context.buildBatch(0L, 3, Arrays.asList("a", "b")); @@ -2681,7 +2745,7 @@ public void testFetchResponseIgnoredAfterBecomingCandidate(boolean withKip853Rpc context.client.poll(); assertEquals(0, context.log.endOffset().offset()); - context.assertVotedCandidate(epoch + 1, localId); + context.expectAndGrantPreVotes(epoch); } @ParameterizedTest @@ -4395,7 +4459,7 @@ public void testHandleCommitCallbackFiresInCandidateState(boolean withKip853Rpc) context.assertUnknownLeaderAndNoVotedCandidate(epoch + 1); assertEquals(OptionalLong.of(10L), context.client.highWatermark()); - // Timeout the election and become candidate + // Timeout the election and become prospective then candidate context.unattachedToCandidate(); int candidateEpoch = epoch + 2; context.client.poll(); diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java index 4e47a4f98e55e..49f79db8616f7 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -49,7 +49,7 @@ public class ProspectiveStateTest { ); private final int epoch = 5; private final MockTime time = new MockTime(); - private final int electionTimeoutMs = 5000; + private final int electionTimeoutMs = 10000; private final LogContext logContext = new LogContext(); private final int localId = 0; private final int votedId = 1; @@ -173,19 +173,6 @@ public void testThreeNodeQuorumVoteRejected(boolean withDirectoryId) { assertTrue(state.epochElection().isVoteRejected()); } - @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testCannotRejectVoteFromLocalId(boolean withDirectoryId) { - int otherNodeId = 1; - ProspectiveState state = newProspectiveState( - voterSetWithLocal(IntStream.of(otherNodeId), withDirectoryId) - ); - assertThrows( - IllegalStateException.class, - () -> state.recordRejectedVote(localReplicaKey.id()) - ); - } - @ParameterizedTest @ValueSource(booleans = { true, false }) public void testCanChangePreVote(boolean withDirectoryId) { @@ -296,9 +283,9 @@ public void testGrantVoteWithLeader(boolean isLogUpToDate, boolean withDirectory assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, true)); assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, true)); - assertEquals(isLogUpToDate, state.canGrantVote(node0, isLogUpToDate, false)); - assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, false)); - assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, false)); + assertFalse(state.canGrantVote(node0, isLogUpToDate, false)); + assertFalse(state.canGrantVote(node1, isLogUpToDate, false)); + assertFalse(state.canGrantVote(node2, isLogUpToDate, false)); } @ParameterizedTest diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index c342888265ed6..bbc226d78e98c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.KRaftVersion; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; import org.mockito.Mockito; @@ -42,7 +41,6 @@ import java.util.stream.IntStream; import java.util.stream.Stream; -import static org.apache.kafka.server.common.KRaftVersion.KRAFT_VERSION_1; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -157,6 +155,163 @@ private ReplicaKey replicaKey(int id, boolean withDirectoryId) { return ReplicaKey.of(id, directoryId); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testHasRemoteLeader(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + assertFalse(state.hasRemoteLeader()); + + state.transitionToProspective(); + assertFalse(state.hasRemoteLeader()); + state.transitionToCandidate(); + assertFalse(state.hasRemoteLeader()); + + state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); + state.transitionToLeader(0L, accumulator); + assertFalse(state.hasRemoteLeader()); + + state.transitionToUnattached(state.epoch() + 1); + assertFalse(state.hasRemoteLeader()); + + state.unattachedAddVotedState(state.epoch(), otherNodeKey); + assertFalse(state.hasRemoteLeader()); + + state.transitionToFollower( + state.epoch(), + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) + ); + assertTrue(state.hasRemoteLeader()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testHighWatermarkRetained(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.transitionToFollower( + 5, + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) + ); + + FollowerState followerState = state.followerStateOrThrow(); + followerState.updateHighWatermark(OptionalLong.of(10L)); + + Optional highWatermark = Optional.of(new LogOffsetMetadata(10L)); + assertEquals(highWatermark, state.highWatermark()); + + state.transitionToUnattached(6); + assertEquals(highWatermark, state.highWatermark()); + + state.unattachedAddVotedState(6, otherNodeKey); + assertEquals(highWatermark, state.highWatermark()); + + state.transitionToProspective(); + assertEquals(highWatermark, state.highWatermark()); + + state.transitionToCandidate(); + assertEquals(highWatermark, state.highWatermark()); + + CandidateState candidateState = state.candidateStateOrThrow(); + candidateState.recordGrantedVote(otherNodeKey.id()); + assertTrue(candidateState.epochElection().isVoteGranted()); + + state.transitionToLeader(10L, accumulator); + assertEquals(Optional.empty(), state.highWatermark()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCanBecomeFollowerOfNonVoter(KRaftVersion kraftVersion) { + int otherNodeId = 1; + ReplicaKey nonVoterKey = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, 4)); + + // Add voted state + state.unattachedAddVotedState(4, nonVoterKey); + assertTrue(state.isUnattachedAndVoted()); + + UnattachedState votedState = state.unattachedStateOrThrow(); + assertEquals(4, votedState.epoch()); + assertEquals(nonVoterKey, votedState.votedKey().get()); + + // Transition to follower + state.transitionToFollower( + 4, + nonVoterKey.id(), + Endpoints.fromInetSocketAddresses( + Collections.singletonMap( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("non-voter-host", 1234) + ) + ) + ); + assertEquals( + new LeaderAndEpoch(OptionalInt.of(nonVoterKey.id()), 4), + state.leaderAndEpoch() + ); + assertEquals( + new ElectionState( + 4, + OptionalInt.of(nonVoterKey.id()), + Optional.of(persistedVotedKey(nonVoterKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCannotFollowSelf(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); + QuorumState state = initializeEmptyState(voters, kraftVersion); + + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 0, + localId, + voters.listeners(localId) + ) + ); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(0, localVoterKey)); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCannotTransitionToFollowerWithNoLeaderEndpoint(KRaftVersion kraftVersion) { + int leaderId = 1; + int followerId = 2; + int epoch = 5; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), kraftVersion); + store.writeElectionState(ElectionState.withUnknownLeader(epoch, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + + assertThrows( + NullPointerException.class, + () -> state.transitionToFollower(epoch + 1, leaderId, null) + ); + assertThrows( + IllegalArgumentException.class, + () -> state.transitionToFollower(epoch + 2, leaderId, Endpoints.empty()) + ); + } + + /** + * Initialization tests + */ + @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testInitializePrimordialEpoch(KRaftVersion kraftVersion) { @@ -434,200 +589,228 @@ public void testInitializeAsOnlyVoter(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToProspective(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - assertEquals(Optional.empty(), store.readElectionState()); + public void testInitializeWithCorruptedStore(KRaftVersion kraftVersion) { + QuorumStateStore stateStore = Mockito.mock(QuorumStateStore.class); + Mockito.doThrow(UncheckedIOException.class).when(stateStore).readElectionState(); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.transitionToProspective(); - int jitterMs1 = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs1); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - assertEquals(1, state.epoch()); - CandidateState candidate = state.candidateStateOrThrow(); - assertEquals( - electionTimeoutMs + jitterMs1, - candidate.remainingElectionTimeMs(time.milliseconds()) + QuorumState state = buildQuorumState( + OptionalInt.of(localId), + localStandaloneVoterSet(), + kraftVersion ); - // The election timeout should be reset after we transition to prospective - time.sleep(candidate.remainingElectionTimeMs(time.milliseconds())); - assertEquals(0, candidate.remainingElectionTimeMs(time.milliseconds())); - int jitterMs2 = 3000; - random.mockNextInt(electionTimeoutMs, jitterMs2); - state.transitionToProspective(); - ProspectiveState prospective = state.prospectiveStateOrThrow(); - assertEquals( - electionTimeoutMs + jitterMs2, - prospective.remainingElectionTimeMs(time.milliseconds()) - ); - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 1, - persistedVotedKey(localVoterKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() - ); + int epoch = 2; + state.initialize(new OffsetAndEpoch(0L, epoch)); + assertEquals(epoch, state.epoch()); + assertTrue(state.isUnattached()); + assertFalse(state.hasLeader()); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToCandidate(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - assertEquals(Optional.empty(), store.readElectionState()); - - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.transitionToProspective(); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); - } + public void testInitializeWithEmptyLocalId(KRaftVersion kraftVersion) { + boolean withDirectoryId = kraftVersion.featureLevel() > 0; + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(0, 1), withDirectoryId) + ); + QuorumState state = buildQuorumState(OptionalInt.empty(), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, 0)); - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testCandidateToResigned(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - assertEquals(Optional.empty(), store.readElectionState()); + assertTrue(state.isObserver()); + assertFalse(state.isVoter()); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.transitionToProspective(); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - assertEquals(1, state.epoch()); + assertThrows(IllegalStateException.class, state::transitionToProspective); assertThrows( - IllegalStateException.class, () -> - state.transitionToResigned(Collections.emptyList()) + IllegalStateException.class, + () -> state.unattachedAddVotedState(1, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)) ); - assertTrue(state.isCandidate()); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); + + state.transitionToFollower(1, 1, voters.listeners(1)); + assertTrue(state.isFollower()); + + state.transitionToUnattached(2); + assertTrue(state.isUnattached()); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToLeader(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); + public void testNoLocalIdInitializationFailsIfElectionStateHasVotedCandidate(KRaftVersion kraftVersion) { + boolean withDirectoryId = kraftVersion.featureLevel() > 0; + int epoch = 5; + int votedId = 1; + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(0, votedId), withDirectoryId) + ); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.transitionToProspective(); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - assertEquals(1, state.epoch()); + store.writeElectionState( + ElectionState.withVotedCandidate( + epoch, + ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), + voters.voterIds() + ), + kraftVersion + ); - state.transitionToLeader(0L, accumulator); - LeaderState leaderState = state.leaderStateOrThrow(); - assertTrue(state.isLeader()); - assertEquals(1, leaderState.epoch()); - assertEquals(Optional.empty(), leaderState.highWatermark()); + QuorumState state2 = buildQuorumState(OptionalInt.empty(), voters, kraftVersion); + assertThrows(IllegalStateException.class, () -> state2.initialize(new OffsetAndEpoch(0, 0))); } + /** + * Test transitions from Unattached + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToLeaderWithoutGrantedVote(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + public void testUnattachedToUnattached(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); - assertFalse(state.candidateStateOrThrow().epochElection().isVoteGranted()); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); - state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - assertTrue(state.candidateStateOrThrow().epochElection().isVoteGranted()); - state.transitionToLeader(0L, accumulator); - assertTrue(state.isLeader()); + state.transitionToUnattached(5); + assertTrue(state.isUnattachedNotVoted()); + + long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); + time.sleep(1000); + + // cannot transition to unattached in same epoch + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + + // can transition to unattached in higher epoch + state.transitionToUnattached(6); + assertTrue(state.isUnattachedNotVoted()); + UnattachedState unattachedState = state.unattachedStateOrThrow(); + assertEquals(6, unattachedState.epoch()); + + // Verify that the election timer does not get reset + assertEquals(remainingElectionTimeMs - 1000, + unattachedState.remainingElectionTimeMs(time.milliseconds())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToFollowerSameEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; + public void testUnattachedCannotAddVotedStateForSelf(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); + QuorumState state = initializeEmptyState(voters, kraftVersion); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(0, localVoterKey)); + } - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testUnattachedToUnattachedVotedSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); + state.transitionToUnattached(5); + + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); + state.unattachedAddVotedState(5, otherNodeKey); + + UnattachedState votedState = state.unattachedStateOrThrow(); + assertEquals(5, votedState.epoch()); + assertEquals(otherNodeKey, votedState.votedKey().get()); - state.transitionToFollower(state.epoch(), otherNodeId, voters.listeners(otherNodeId)); - assertEquals(1, state.epoch()); - assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( Optional.of( - ElectionState.withElectedLeader( - 1, - otherNodeId, + ElectionState.withVotedCandidate( + 5, + persistedVotedKey(otherNodeKey, kraftVersion), persistedVoters(voters.voterIds(), kraftVersion) ) ), store.readElectionState() ); + + // Verify election timeout is reset when we vote for a candidate + assertEquals(electionTimeoutMs + jitterMs, + votedState.remainingElectionTimeMs(time.milliseconds())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToFollowerHigherEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; + public void testUnattachedToUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToUnattached(5); + assertTrue(state.isUnattachedNotVoted()); - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, otherNodeKey)); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testUnattachedToFollowerSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); + state.transitionToUnattached(5); - state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId)); - assertEquals(5, state.epoch()); - assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); + state.transitionToFollower( + 5, + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) + ); + assertTrue(state.isFollower()); + FollowerState followerState = state.followerStateOrThrow(); + assertEquals(5, followerState.epoch()); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - otherNodeId, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() + voters.listeners(otherNodeKey.id()), + followerState.leaderEndpoints() ); + assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToUnattachedSameEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + public void testUnattachedToFollowerHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); + state.transitionToUnattached(5); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + state.transitionToFollower( + 8, + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) + ); + assertTrue(state.isFollower()); + FollowerState followerState = state.followerStateOrThrow(); + assertEquals(8, followerState.epoch()); + assertEquals( + voters.listeners(otherNodeKey.id()), + followerState.leaderEndpoints() + ); + assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToUnattachedHigherEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + public void testUnattachedToProspective(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); + assertTrue(state.isUnattached()); + assertEquals(logEndEpoch, state.epoch()); - state.transitionToUnattached(5); - assertEquals(5, state.epoch()); - assertEquals(OptionalInt.empty(), state.leaderId()); + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); + state.transitionToProspective(); + ProspectiveState prospective = state.prospectiveStateOrThrow(); + assertEquals(electionTimeoutMs + jitterMs, + prospective.remainingElectionTimeMs(time.milliseconds())); assertEquals( Optional.of( ElectionState.withUnknownLeader( - 5, + logEndEpoch, persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -637,15 +820,42 @@ public void testCandidateToUnattachedHigherEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToAnyStateLowerEpoch(KRaftVersion kraftVersion) { + public void testUnattachedToCandidate(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToUnattached(5); + assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testUnattachedToLeaderOrResigned(KRaftVersion kraftVersion) { + ReplicaKey leaderKey = ReplicaKey.of(1, Uuid.randomUuid()); + int epoch = 5; + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leaderKey)); + store.writeElectionState( + ElectionState.withVotedCandidate(epoch, leaderKey, voters.voterIds()), + kraftVersion + ); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isUnattachedAndVoted()); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); + assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testUnattachedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToUnattached(5); - state.transitionToProspective(); - state.transitionToCandidate(); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -654,12 +864,11 @@ public void testCandidateToAnyStateLowerEpoch(KRaftVersion kraftVersion) { voters.listeners(otherNodeKey.id()) ) ); - assertEquals(6, state.epoch()); + assertEquals(5, state.epoch()); assertEquals( Optional.of( - ElectionState.withVotedCandidate( - 6, - persistedVotedKey(localVoterKey, kraftVersion), + ElectionState.withUnknownLeader( + 5, persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -667,127 +876,96 @@ public void testCandidateToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ); } + /** + * Tests transitions from Unattached with votedKey + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderToLeader(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); - + public void testUnattachedVotedToUnattachedSameEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); - state.transitionToLeader(0L, accumulator); - assertTrue(state.isLeader()); - assertEquals(1, state.epoch()); - - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); - assertTrue(state.isLeader()); - assertEquals(1, state.epoch()); + state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(logEndEpoch)); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderToResigned(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); - + public void testUnattachedVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); - state.transitionToLeader(0L, accumulator); - assertTrue(state.isLeader()); - assertEquals(1, state.epoch()); - - state.transitionToResigned(Collections.singletonList(localVoterKey)); - assertTrue(state.isResigned()); - ResignedState resignedState = state.resignedStateOrThrow(); - assertEquals( - ElectionState.withElectedLeader(1, localId, voters.voterIds()), - resignedState.election() - ); - assertEquals(1, resignedState.epoch()); - assertEquals(Collections.emptySet(), resignedState.unackedVoters()); - } + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID)); - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testLeaderToProspective(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); - - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); - state.transitionToLeader(0L, accumulator); - assertTrue(state.isLeader()); - assertEquals(1, state.epoch()); - - assertThrows(IllegalStateException.class, state::transitionToProspective); - assertTrue(state.isLeader()); - assertEquals(1, state.epoch()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testLeaderToCandidate(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); + long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); + time.sleep(1000); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); - state.transitionToLeader(0L, accumulator); - assertTrue(state.isLeader()); - assertEquals(1, state.epoch()); + state.transitionToUnattached(6); + UnattachedState unattachedState = state.unattachedStateOrThrow(); + assertEquals(6, unattachedState.epoch()); - assertThrows(IllegalStateException.class, state::transitionToCandidate); - assertTrue(state.isLeader()); - assertEquals(1, state.epoch()); + // Verify that the election timer does not get reset + assertEquals(remainingElectionTimeMs - 1000, + unattachedState.remainingElectionTimeMs(time.milliseconds())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderToFollowerSameEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); - + public void testUnattachedVotedToUnattachedVoted(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - - state.transitionToProspective(); - state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - state.transitionToLeader(0L, accumulator); + state.initialize(new OffsetAndEpoch(0L, 8)); + state.unattachedAddVotedState(8, ReplicaKey.of(node1, Uuid.randomUuid())); + // same epoch assertThrows( IllegalStateException.class, - () -> state.transitionToFollower(state.epoch(), otherNodeId, voters.listeners(otherNodeId)) + () -> state.unattachedAddVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) + ); + // same votedKey + assertThrows( + IllegalStateException.class, + () -> state.unattachedAddVotedState(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) + ); + // higher epoch + assertThrows( + IllegalStateException.class, + () -> state.unattachedAddVotedState(10, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderToFollowerHigherEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); - + public void testUnattachedVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + ReplicaKey node1Key = ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID); + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, node1Key); + state.transitionToFollower( + 5, + node2, + voters.listeners(node2) + ); - state.transitionToProspective(); - state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - state.transitionToLeader(0L, accumulator); - state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId)); - - assertEquals(5, state.epoch()); - assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); + FollowerState followerState = state.followerStateOrThrow(); + assertEquals(5, followerState.epoch()); + assertEquals( + voters.listeners(node2), + followerState.leaderEndpoints() + ); assertEquals( Optional.of( - ElectionState.withElectedLeader( + new ElectionState( 5, - otherNodeId, + OptionalInt.of(node2), + Optional.of(persistedVotedKey(node1Key, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -797,56 +975,90 @@ public void testLeaderToFollowerHigherEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderToUnattachedSameEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + public void testUnattachedVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - state.transitionToLeader(0L, accumulator); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); + + FollowerState followerState = state.followerStateOrThrow(); + assertEquals(8, followerState.epoch()); + assertEquals( + voters.listeners(node2), + followerState.leaderEndpoints() + ); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 8, + node2, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderToUnattachedHigherEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + public void testUnattachedVotedToProspective(KRaftVersion kraftVersion) { + int node1 = 1; + Uuid node1DirectoryId = Uuid.randomUuid(); + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, ReplicaKey.of(node1, node1DirectoryId)); + + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); state.transitionToProspective(); - state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - state.transitionToLeader(0L, accumulator); - state.transitionToUnattached(5); - assertEquals(5, state.epoch()); - assertEquals(OptionalInt.empty(), state.leaderId()); + assertTrue(state.isProspective()); + ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); + assertEquals(5, prospectiveState.epoch()); + assertEquals(electionTimeoutMs + jitterMs, + prospectiveState.remainingElectionTimeMs(time.milliseconds())); assertEquals( Optional.of( - ElectionState.withUnknownLeader( + ElectionState.withVotedCandidate( 5, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), + persistedVotedKey(ReplicaKey.of(node1, node1DirectoryId), kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion))), store.readElectionState() ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { + public void testUnattachedVotedToCandidateOrLeaderOrResigned(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); + assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testUnattachedVotedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - state.transitionToProspective(); - state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); - state.transitionToLeader(0L, accumulator); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, otherNodeKey); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -855,12 +1067,12 @@ public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { voters.listeners(otherNodeKey.id()) ) ); - assertEquals(6, state.epoch()); + assertEquals(5, state.epoch()); assertEquals( Optional.of( - ElectionState.withElectedLeader( - 6, - localId, + ElectionState.withVotedCandidate( + 5, + persistedVotedKey(otherNodeKey, kraftVersion), persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -868,226 +1080,248 @@ public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ); } + /** + * Test transitions from Unattached with leader + */ + @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCannotFollowSelf(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); - QuorumState state = initializeEmptyState(voters, kraftVersion); + public void testUnattachedWithLeaderToProspective(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + int epoch = 5; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + store.writeElectionState(ElectionState.withElectedLeader(epoch, node1, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - assertThrows( - IllegalStateException.class, - () -> state.transitionToFollower( - 0, - localId, - voters.listeners(localId) - ) + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertEquals( + Optional.of(ElectionState.withElectedLeader(epoch, node1, persistedVoters(voters.voterIds(), kraftVersion))), + store.readElectionState() ); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(0, localVoterKey)); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testCannotAddVotedStateForSelfAsUnattached(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); - QuorumState state = initializeEmptyState(voters, kraftVersion); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(0, localVoterKey)); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedToLeaderOrResigned(KRaftVersion kraftVersion) { - ReplicaKey leaderKey = ReplicaKey.of(1, Uuid.randomUuid()); + public void testUnattachedWithLeaderNoEndpointToAndFromProspective(KRaftVersion kraftVersion) { + int leaderId = 1; + ReplicaKey leaderKey = ReplicaKey.of(leaderId, Uuid.randomUuid()); + int followerId = 2; + ReplicaKey followerKey = ReplicaKey.of(followerId, Uuid.randomUuid()); int epoch = 5; - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leaderKey)); - store.writeElectionState( - ElectionState.withVotedCandidate(epoch, leaderKey, voters.voterIds()), - kraftVersion - ); + Map voterMap = new HashMap<>(); + voterMap.put(localId, VoterSetTest.voterNode(localVoterKey)); + voterMap.put(leaderId, VoterSetTest.voterNode(leaderKey, Endpoints.empty())); + voterMap.put(followerId, VoterSetTest.voterNode(followerKey, Endpoints.empty())); + VoterSet voters = VoterSetTest.voterSet(voterMap); + + store.writeElectionState(ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()), kraftVersion); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - assertTrue(state.isUnattachedAndVoted()); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); - assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testUnattachedToUnattachedVotedSameEpoch(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); - state.unattachedAddVotedState(5, otherNodeKey); - - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(5, votedState.epoch()); - assertEquals(otherNodeKey, votedState.votedKey().get()); + assertTrue(state.isUnattached()); + assertTrue(state.hasLeader()); + assertTrue(state.leaderEndpoints().isEmpty()); + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertTrue(state.leaderEndpoints().isEmpty()); assertEquals( Optional.of( - ElectionState.withVotedCandidate( + ElectionState.withElectedLeader( 5, - persistedVotedKey(otherNodeKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), + leaderId, + persistedVoters(voters.voterIds(), kraftVersion))), store.readElectionState() ); - // Verify election timeout is reset when we vote for a candidate - assertEquals(electionTimeoutMs + jitterMs, - votedState.remainingElectionTimeMs(time.milliseconds())); + state.transitionToUnattached(5, OptionalInt.of(leaderId)); + assertTrue(state.isUnattached()); + assertTrue(state.leaderEndpoints().isEmpty()); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 5, + leaderId, + persistedVoters(voters.voterIds(), kraftVersion))), + store.readElectionState() + ); } + /** + * Test transitions from Follower + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedToUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + public void testFollowerToUnattachedSameEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - assertTrue(state.isUnattachedNotVoted()); - - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, otherNodeKey)); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8)); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(8, votedKey)); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedToProspectiveToCandidate(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + public void testFollowerToUnattachedHigherEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToProspective(); - state.transitionToCandidate(); - - assertTrue(state.isCandidate()); - CandidateState candidateState = state.candidateStateOrThrow(); - assertEquals(6, candidateState.epoch()); + state.transitionToUnattached(9); + assertTrue(state.isUnattached()); + UnattachedState unattachedState = state.unattachedStateOrThrow(); + assertEquals(9, unattachedState.epoch()); assertEquals(electionTimeoutMs + jitterMs, - candidateState.remainingElectionTimeMs(time.milliseconds())); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testUnattachedToCandidate(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); + unattachedState.remainingElectionTimeMs(time.milliseconds())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedToUnattached(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + public void testFollowerToUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + ReplicaKey votedKey = ReplicaKey.of(node2, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - assertTrue(state.isUnattachedNotVoted()); - - long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); - time.sleep(1000); - - // cannot transition to unattached in same epoch - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); - - // can transition to unattached in higher epoch - state.transitionToUnattached(6); - assertTrue(state.isUnattachedNotVoted()); - UnattachedState unattachedState = state.unattachedStateOrThrow(); - assertEquals(6, unattachedState.epoch()); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); - // Verify that the election timer does not get reset - assertEquals(remainingElectionTimeMs - 1000, - unattachedState.remainingElectionTimeMs(time.milliseconds())); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(9, votedKey)); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedToFollowerSameEpoch(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + public void testFollowerToFollowerSameEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - state.transitionToFollower( - 5, - otherNodeKey.id(), - voters.listeners(otherNodeKey.id()) + 8, + node2, + voters.listeners(node2) ); - assertTrue(state.isFollower()); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 8, + node1, + voters.listeners(node1) + ) + ); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ) + ); + FollowerState followerState = state.followerStateOrThrow(); - assertEquals(5, followerState.epoch()); + assertEquals(8, followerState.epoch()); assertEquals( - voters.listeners(otherNodeKey.id()), + voters.listeners(node2), followerState.leaderEndpoints() ); - assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 8, + node2, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedToFollowerHigherEpoch(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + public void testFollowerToFollowerSameEpochAndMoreEndpoints(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - state.transitionToFollower( 8, - otherNodeKey.id(), - voters.listeners(otherNodeKey.id()) + node2, + voters.listeners(node2) ); - assertTrue(state.isFollower()); - FollowerState followerState = state.followerStateOrThrow(); - assertEquals(8, followerState.epoch()); - assertEquals( - voters.listeners(otherNodeKey.id()), - followerState.leaderEndpoints() + + HashMap newNode2ListenersMap = new HashMap<>(2); + newNode2ListenersMap.put( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("localhost", 9990 + node2) + ); + newNode2ListenersMap.put( + ListenerName.normalised("ANOTHER_LISTENER"), + InetSocketAddress.createUnresolved("localhost", 8990 + node2) + ); + Endpoints newNode2Endpoints = Endpoints.fromInetSocketAddresses(newNode2ListenersMap); + + state.transitionToFollower( + 8, + node2, + newNode2Endpoints ); - assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + public void testFollowerToFollowerHigherEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); - assertThrows( - IllegalStateException.class, - () -> state.transitionToFollower( - 4, - otherNodeKey.id(), - voters.listeners(otherNodeKey.id()) - ) + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); + state.transitionToFollower( + 9, + node1, + voters.listeners(node1) + ); + + FollowerState followerState = state.followerStateOrThrow(); + assertEquals(9, followerState.epoch()); + assertEquals( + voters.listeners(node1), + followerState.leaderEndpoints() ); - assertEquals(5, state.epoch()); assertEquals( Optional.of( - ElectionState.withUnknownLeader( - 5, + ElectionState.withElectedLeader( + 9, + node1, persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -1097,92 +1331,98 @@ public void testUnattachedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToCandidateOrLeaderOrResigned(KRaftVersion kraftVersion) { + public void testFollowerToProspective(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); + store.writeElectionState(ElectionState.withUnknownLeader(logEndEpoch, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); - assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); - assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); + state.transitionToProspective(); + ProspectiveState prospective = state.prospectiveStateOrThrow(); + assertEquals(8, prospective.epoch()); + assertEquals( + electionTimeoutMs + jitterMs, + prospective.remainingElectionTimeMs(time.milliseconds()) + ); + assertEquals( + ElectionState.withElectedLeader( + 8, + node2, + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToProspectiveToCandidate(KRaftVersion kraftVersion) { + public void testFollowerToCandidate(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); - state.transitionToProspective(); - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - CandidateState candidateState = state.candidateStateOrThrow(); - assertEquals(logEndEpoch + 1, candidateState.epoch()); - assertEquals(electionTimeoutMs + jitterMs, - candidateState.remainingElectionTimeMs(time.milliseconds())); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); + assertThrows(IllegalStateException.class, state::transitionToCandidate); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToUnattachedVoted(KRaftVersion kraftVersion) { + public void testFollowerToLeaderOrResigned(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, 8)); - state.unattachedAddVotedState(8, ReplicaKey.of(node1, Uuid.randomUuid())); - // same epoch - assertThrows( - IllegalStateException.class, - () -> state.unattachedAddVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) - ); - // same votedKey - assertThrows( - IllegalStateException.class, - () -> state.unattachedAddVotedState(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) - ); - // higher epoch - assertThrows( - IllegalStateException.class, - () -> state.unattachedAddVotedState(10, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) ); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); + assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { - int node1 = 1; - ReplicaKey node1Key = ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID); - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, 5)); - state.unattachedAddVotedState(5, node1Key); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower( 5, - node2, - voters.listeners(node2) + otherNodeId, + voters.listeners(otherNodeId) ); - - FollowerState followerState = state.followerStateOrThrow(); - assertEquals(5, followerState.epoch()); - assertEquals( - voters.listeners(node2), - followerState.leaderEndpoints() + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + otherNodeId, + voters.listeners(otherNodeId) + ) ); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(4, accumulator)); + assertEquals(5, state.epoch()); assertEquals( Optional.of( - new ElectionState( + ElectionState.withElectedLeader( 5, - OptionalInt.of(node2), - Optional.of(persistedVotedKey(node1Key, kraftVersion)), + otherNodeId, persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -1190,96 +1430,132 @@ public void testUnattachedVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { ); } + /** + * Test transitions from Follower with votedKey + */ + @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) { + public void testFollowerVotedToUnattachedSameEpoch(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, 5)); - state.unattachedAddVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.prospectiveStateOrThrow().recordGrantedVote(node1); state.transitionToFollower( - 8, + state.epoch(), node2, voters.listeners(node2) ); + assertEquals(0, state.epoch()); - FollowerState followerState = state.followerStateOrThrow(); - assertEquals(8, followerState.epoch()); - assertEquals( - voters.listeners(node2), - followerState.leaderEndpoints() - ); - assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 8, - node2, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() + assertThrows( + IllegalStateException.class, + () -> state.transitionToUnattached(0) ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToUnattachedSameEpoch(KRaftVersion kraftVersion) { + public void testFollowerVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(logEndEpoch)); - } + state.transitionToProspective(); + state.prospectiveStateOrThrow().recordGrantedVote(node1); + state.transitionToFollower( + state.epoch(), + node2, + voters.listeners(node2) + ); + assertEquals(0, state.epoch()); - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, 5)); - state.unattachedAddVotedState(5, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID)); + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); - long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); - time.sleep(1000); + state.transitionToUnattached(10); + assertTrue(state.isUnattachedNotVoted()); - state.transitionToUnattached(6); - UnattachedState unattachedState = state.unattachedStateOrThrow(); - assertEquals(6, unattachedState.epoch()); + UnattachedState unattached = state.unattachedStateOrThrow(); + assertEquals(10, unattached.epoch()); - // Verify that the election timer does not get reset - assertEquals(remainingElectionTimeMs - 1000, - unattachedState.remainingElectionTimeMs(time.milliseconds())); + assertEquals(electionTimeoutMs + jitterMs, + unattached.remainingElectionTimeMs(time.milliseconds())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); + public void testFollowerVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + ReplicaKey votedKey = ReplicaKey.of(node1, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, 5)); - state.unattachedAddVotedState(5, otherNodeKey); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.prospectiveAddVotedState(state.epoch(), votedKey); + state.transitionToFollower( + state.epoch(), + node2, + voters.listeners(node2) + ); + assertEquals( + Optional.of( + new ElectionState( + 0, + OptionalInt.of(node2), + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); + assertThrows( IllegalStateException.class, - () -> state.transitionToFollower( - 4, - otherNodeKey.id(), - voters.listeners(otherNodeKey.id()) - ) + () -> state.transitionToFollower(state.epoch(), node1, voters.listeners(node1)) + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + ReplicaKey votedKey = ReplicaKey.of(node1, Uuid.randomUuid()); + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.prospectiveAddVotedState(state.epoch(), votedKey); + state.transitionToFollower( + state.epoch(), + node2, + voters.listeners(node2) ); - assertEquals(5, state.epoch()); assertEquals( Optional.of( - ElectionState.withVotedCandidate( - 5, - persistedVotedKey(otherNodeKey, kraftVersion), + new ElectionState( + 0, + OptionalInt.of(node2), + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); + + state.transitionToFollower(state.epoch() + 1, node1, voters.listeners(node1)); + assertEquals(1, state.epoch()); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 1, + node1, persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -1287,6 +1563,9 @@ public void testUnattachedVotedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ); } + /** + * Test transitions from Prospective + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testProspectiveToUnattachedInSameEpoch(KRaftVersion kraftVersion) { @@ -1309,37 +1588,27 @@ public void testProspectiveToUnattachedInSameEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testProspectiveWithLeaderToUnattachedInSameEpoch(KRaftVersion kraftVersion) { + public void testProspectiveToUnattachedInHigherEpoch(KRaftVersion kraftVersion) { ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); + + state.transitionToUnattached(20); assertEquals( - ElectionState.withElectedLeader( - logEndEpoch, - voter1.id(), + ElectionState.withUnknownLeader( + 20, persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() ); - - state.transitionToUnattached(state.epoch(), OptionalInt.of(voter1.id())); - assertEquals( - ElectionState.withElectedLeader( - logEndEpoch, - voter1.id(), - persistedVoters(voters.voterIds(), kraftVersion) - ), - store.readElectionState().get() - ); - } + } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testProspectiveToUnattachedInHigherEpoch(KRaftVersion kraftVersion) { + public void testProspectiveToUnattachedVoted(KRaftVersion kraftVersion) { ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); @@ -1347,14 +1616,13 @@ public void testProspectiveToUnattachedInHigherEpoch(KRaftVersion kraftVersion) state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.transitionToUnattached(20); - assertEquals( - ElectionState.withUnknownLeader( - 20, - persistedVoters(voters.voterIds(), kraftVersion) - ), - store.readElectionState().get() - ); + // in same epoch + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(logEndEpoch, voter1)); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(logEndEpoch, localVoterKey)); + + // in higher epoch + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, voter1)); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, localVoterKey)); } @ParameterizedTest @@ -1378,102 +1646,20 @@ public void testProspectiveToUnattachedWithLeaderInHigherEpoch(KRaftVersion kraf ); } - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testProspectiveWithLeaderToUnattachedInHigherEpoch(KRaftVersion kraftVersion) { - ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); - ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - assertEquals( - ElectionState.withElectedLeader( - logEndEpoch, - voter1.id(), - persistedVoters(voters.voterIds(), kraftVersion) - ), - store.readElectionState().get() - ); - - // transition to unattached - state.transitionToUnattached(10); - assertEquals( - ElectionState.withUnknownLeader( - 10, - persistedVoters(voters.voterIds(), kraftVersion) - ), - store.readElectionState().get() - ); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testProspectiveWithLeaderToUnattachedWithLeaderInHigherEpoch(KRaftVersion kraftVersion) { - ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); - ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - assertEquals( - ElectionState.withElectedLeader( - logEndEpoch, - voter1.id(), - persistedVoters(voters.voterIds(), kraftVersion) - ), - store.readElectionState().get() - ); - - // transition to unattached with different leader state - state.transitionToUnattached(10, OptionalInt.of(voter2.id())); - assertEquals( - ElectionState.withElectedLeader( - 10, - voter2.id(), - persistedVoters(voters.voterIds(), kraftVersion) - ), - store.readElectionState().get() - ); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testProspectiveToUnattachedVoted(KRaftVersion kraftVersion) { - ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); - ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - - // in same epoch - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(logEndEpoch, voter1)); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(logEndEpoch, localVoterKey)); - - // in higher epoch - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, voter1)); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, localVoterKey)); - } - @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testProspectiveToFollowerSameEpoch(KRaftVersion kraftVersion) { ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - - state.transitionToFollower(state.epoch(), voter2.id(), voters.listeners(voter2.id())); + state.transitionToFollower(state.epoch(), voter1.id(), voters.listeners(voter1.id())); assertEquals( ElectionState.withElectedLeader( - logEndEpoch, - voter2.id(), + 0, + voter1.id(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1486,16 +1672,14 @@ public void testProspectiveToFollowerHigherEpoch(KRaftVersion kraftVersion) { ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - - state.transitionToFollower(10, voter2.id(), voters.listeners(voter2.id())); + state.transitionToFollower(state.epoch() + 1, voter1.id(), voters.listeners(voter1.id())); assertEquals( ElectionState.withElectedLeader( - 10, - voter2.id(), + 1, + voter1.id(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1553,33 +1737,6 @@ public void testProspectiveToCandidate(KRaftVersion kraftVersion) { ); } - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testProspectiveWithLeaderToCandidate(KRaftVersion kraftVersion) { - int leaderId = 1; - int followerId = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), kraftVersion); - - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, leaderId, voters.voterIds()), kraftVersion); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - assertTrue(state.isProspective()); - assertTrue(state.hasLeader()); - - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - assertEquals(logEndEpoch + 1, state.epoch()); - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - logEndEpoch + 1, - persistedVotedKey(localVoterKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() - ); - } - @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testProspectiveToLeaderOrResigned(KRaftVersion kraftVersion) { @@ -1598,6 +1755,9 @@ public void testProspectiveToLeaderOrResigned(KRaftVersion kraftVersion) { assertThrows(IllegalStateException.class, () -> state.transitionToLeader(10, accumulator)); } + /** + * Transitions from Prospective with votedKey + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testProspectiveVotedToUnattachedInSameEpoch(KRaftVersion kraftVersion) { @@ -1621,35 +1781,6 @@ public void testProspectiveVotedToUnattachedInSameEpoch(KRaftVersion kraftVersio ); } - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testProspectiveVotedWithLeaderToUnattachedInSameEpoch(KRaftVersion kraftVersion) { - ReplicaKey leader = ReplicaKey.of(1, Uuid.randomUuid()); - ReplicaKey candidate = ReplicaKey.of(2, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leader, candidate)); - store.writeElectionState( - ElectionState.withElectedLeader(logEndEpoch, leader.id(), voters.voterIds()), - kraftVersion - ); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToProspective(); - state.prospectiveAddVotedState(logEndEpoch, candidate); - - // transition to unattached with different leader state - state.transitionToUnattached(state.epoch(), OptionalInt.of(candidate.id())); - - assertEquals( - new ElectionState( - logEndEpoch, - OptionalInt.of(candidate.id()), - Optional.of(persistedVotedKey(candidate, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ), - store.readElectionState().get() - ); - } - @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testProspectiveVotedToAndFromFollowerSameEpoch(KRaftVersion kraftVersion) { @@ -1734,294 +1865,282 @@ public void testProspectiveVotedToCandidate(KRaftVersion kraftVersion) { ); } + /** + * Test transitions from Prospective with leader + */ + @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToFollowerSameEpoch(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); + public void testProspectiveWithLeaderToUnattachedInSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) - ); - assertThrows( - IllegalStateException.class, - () -> state.transitionToFollower( - 8, - node1, - voters.listeners(node1) - ) - ); - assertThrows( - IllegalStateException.class, - () -> state.transitionToFollower( - 8, - node2, - voters.listeners(node2) - ) - ); - - FollowerState followerState = state.followerStateOrThrow(); - assertEquals(8, followerState.epoch()); + state.transitionToProspective(); assertEquals( - voters.listeners(node2), - followerState.leaderEndpoints() + ElectionState.withElectedLeader( + logEndEpoch, + voter1.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); + + state.transitionToUnattached(state.epoch(), OptionalInt.of(voter1.id())); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 8, - node2, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + logEndEpoch, + voter1.id(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToFollowerSameEpochAndMoreEndpoints(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); + public void testProspectiveWithLeaderToUnattachedInHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) - ); - - HashMap newNode2ListenersMap = new HashMap<>(2); - newNode2ListenersMap.put( - VoterSetTest.DEFAULT_LISTENER_NAME, - InetSocketAddress.createUnresolved("localhost", 9990 + node2) - ); - newNode2ListenersMap.put( - ListenerName.normalised("ANOTHER_LISTENER"), - InetSocketAddress.createUnresolved("localhost", 8990 + node2) + state.transitionToProspective(); + assertEquals( + ElectionState.withElectedLeader( + logEndEpoch, + voter1.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); - Endpoints newNode2Endpoints = Endpoints.fromInetSocketAddresses(newNode2ListenersMap); - state.transitionToFollower( - 8, - node2, - newNode2Endpoints + // transition to unattached + state.transitionToUnattached(10); + assertEquals( + ElectionState.withUnknownLeader( + 10, + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToFollowerHigherEpoch(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); + public void testProspectiveWithLeaderToUnattachedWithLeaderInHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) - ); - state.transitionToFollower( - 9, - node1, - voters.listeners(node1) - ); - - FollowerState followerState = state.followerStateOrThrow(); - assertEquals(9, followerState.epoch()); + state.transitionToProspective(); assertEquals( - voters.listeners(node1), - followerState.leaderEndpoints() + ElectionState.withElectedLeader( + logEndEpoch, + voter1.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); + + // transition to unattached with different leader state + state.transitionToUnattached(10, OptionalInt.of(voter2.id())); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 9, - node1, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 10, + voter2.id(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToLeaderOrResigned(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); + public void testProspectiveWithLeaderToFollowerSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) + state.transitionToProspective(); + + state.transitionToFollower(state.epoch(), voter2.id(), voters.listeners(voter2.id())); + assertEquals( + ElectionState.withElectedLeader( + logEndEpoch, + voter2.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); - assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToProspective(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); + public void testProspectiveWithLeaderToFollowerHigherEpoch(KRaftVersion kraftVersion) { + ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) - ); - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); state.transitionToProspective(); - ProspectiveState prospective = state.prospectiveStateOrThrow(); - assertEquals(8, prospective.epoch()); - assertEquals(electionTimeoutMs + jitterMs, - prospective.remainingElectionTimeMs(time.milliseconds())); - } - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testFollowerToCandidate(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) + state.transitionToFollower(10, voter2.id(), voters.listeners(voter2.id())); + assertEquals( + ElectionState.withElectedLeader( + 10, + voter2.id(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); - assertThrows(IllegalStateException.class, state::transitionToCandidate); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToUnattachedSameEpoch(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) - ); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8)); - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(8, votedKey)); - } + public void testProspectiveWithLeaderToCandidate(KRaftVersion kraftVersion) { + int leaderId = 1; + int followerId = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), kraftVersion); - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testFollowerToUnattachedHigherEpoch(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); + store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, leaderId, voters.voterIds()), kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) - ); - - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToUnattached(9); - assertTrue(state.isUnattached()); - UnattachedState unattachedState = state.unattachedStateOrThrow(); - assertEquals(9, unattachedState.epoch()); - assertEquals(electionTimeoutMs + jitterMs, - unattachedState.remainingElectionTimeMs(time.milliseconds())); - } + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertTrue(state.hasLeader()); - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testFollowerToUnattachedVotedHigherEpoch(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - ReplicaKey votedKey = ReplicaKey.of(node2, Uuid.randomUuid()); - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertEquals(logEndEpoch + 1, state.epoch()); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion))), + store.readElectionState() ); - - assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(9, votedKey)); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToProspectiveSameEpoch(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - store.writeElectionState(ElectionState.withUnknownLeader(logEndEpoch, voters.voterIds()), kraftVersion); + public void testProspectiveVotedWithLeaderToUnattachedInSameEpoch(KRaftVersion kraftVersion) { + ReplicaKey leader = ReplicaKey.of(1, Uuid.randomUuid()); + ReplicaKey candidate = ReplicaKey.of(2, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leader, candidate)); + store.writeElectionState( + ElectionState.withElectedLeader(logEndEpoch, leader.id(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - node2, - voters.listeners(node2) - ); - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); state.transitionToProspective(); - ProspectiveState prospective = state.prospectiveStateOrThrow(); - assertEquals(8, prospective.epoch()); - assertEquals( - electionTimeoutMs + jitterMs, - prospective.remainingElectionTimeMs(time.milliseconds()) - ); + state.prospectiveAddVotedState(logEndEpoch, candidate); + + // transition to unattached with different leader state + state.transitionToUnattached(state.epoch(), OptionalInt.of(candidate.id())); + assertEquals( - ElectionState.withElectedLeader( - 8, - node2, + new ElectionState( + logEndEpoch, + OptionalInt.of(candidate.id()), + Optional.of(persistedVotedKey(candidate, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() ); } + /** + * Test transitions from Candidate + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { + public void testCandidateToUnattachedSameEpoch(KRaftVersion kraftVersion) { int otherNodeId = 1; VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 5, - otherNodeId, - voters.listeners(otherNodeId) + state.transitionToProspective(); + state.transitionToCandidate(); + + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToUnattachedHigherEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + + state.transitionToUnattached(5); + assertEquals(5, state.epoch()); + assertEquals(OptionalInt.empty(), state.leaderId()); + assertEquals( + Optional.of( + ElectionState.withUnknownLeader( + 5, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() ); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows( - IllegalStateException.class, - () -> state.transitionToFollower( - 4, - otherNodeId, - voters.listeners(otherNodeId) - ) + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToFollowerSameEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + + state.transitionToFollower(state.epoch(), otherNodeId, voters.listeners(otherNodeId)); + assertEquals(1, state.epoch()); + assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); + assertEquals( + Optional.of( + new ElectionState( + 1, + OptionalInt.of(otherNodeId), + Optional.of(persistedVotedKey(localVoterKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() ); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(4, accumulator)); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToFollowerHigherEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + + state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId)); assertEquals(5, state.epoch()); + assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( Optional.of( ElectionState.withElectedLeader( @@ -2036,126 +2155,143 @@ public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerVotedToUnattachedSameEpoch(KRaftVersion kraftVersion) { + public void testCandidateToProspective(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + assertEquals(Optional.empty(), store.readElectionState()); + QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.prospectiveStateOrThrow().recordGrantedVote(node1); - state.transitionToFollower( - state.epoch(), - node2, - voters.listeners(node2) + int jitterMs1 = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs1); + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertEquals(1, state.epoch()); + CandidateState candidate = state.candidateStateOrThrow(); + assertEquals( + electionTimeoutMs + jitterMs1, + candidate.remainingElectionTimeMs(time.milliseconds()) ); - assertEquals(0, state.epoch()); - assertThrows( - IllegalStateException.class, - () -> state.transitionToUnattached(0) + // The election timeout should be reset after we transition to prospective + time.sleep(candidate.remainingElectionTimeMs(time.milliseconds())); + assertEquals(0, candidate.remainingElectionTimeMs(time.milliseconds())); + int jitterMs2 = 3000; + random.mockNextInt(electionTimeoutMs, jitterMs2); + state.transitionToProspective(); + ProspectiveState prospective = state.prospectiveStateOrThrow(); + assertEquals( + electionTimeoutMs + jitterMs2, + prospective.remainingElectionTimeMs(time.milliseconds()) + ); + assertEquals( + Optional.of( + ElectionState.withVotedCandidate( + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion))), + store.readElectionState() ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion) { + public void testCandidateToCandidate(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + assertEquals(Optional.empty(), store.readElectionState()); + QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.prospectiveStateOrThrow().recordGrantedVote(node1); - state.transitionToFollower( - state.epoch(), - node2, - voters.listeners(node2) - ); - assertEquals(0, state.epoch()); + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); + } - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToLeader(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); - state.transitionToUnattached(10); - assertTrue(state.isUnattachedNotVoted()); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.transitionToProspective(); + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertEquals(1, state.epoch()); - UnattachedState unattached = state.unattachedStateOrThrow(); - assertEquals(10, unattached.epoch()); + state.transitionToLeader(0L, accumulator); + LeaderState leaderState = state.leaderStateOrThrow(); + assertTrue(state.isLeader()); + assertEquals(1, leaderState.epoch()); + assertEquals(Optional.empty(), leaderState.highWatermark()); + } - assertEquals(electionTimeoutMs + jitterMs, - unattached.remainingElectionTimeMs(time.milliseconds())); + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToLeaderWithoutGrantedVote(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + assertFalse(state.candidateStateOrThrow().epochElection().isVoteGranted()); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); + state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + assertTrue(state.candidateStateOrThrow().epochElection().isVoteGranted()); + state.transitionToLeader(0L, accumulator); + assertTrue(state.isLeader()); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { + public void testCandidateToResigned(KRaftVersion kraftVersion) { int node1 = 1; int node2 = 2; - ReplicaKey votedKey = ReplicaKey.of(node1, Uuid.randomUuid()); VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + assertEquals(Optional.empty(), store.readElectionState()); + QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.prospectiveAddVotedState(state.epoch(), votedKey); - state.transitionToFollower( - state.epoch(), - node2, - voters.listeners(node2) - ); - assertEquals( - Optional.of( - new ElectionState( - 0, - OptionalInt.of(node2), - Optional.of(persistedVotedKey(votedKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() - ); + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertEquals(1, state.epoch()); assertThrows( - IllegalStateException.class, - () -> state.transitionToFollower(state.epoch(), node1, voters.listeners(node1)) + IllegalStateException.class, () -> + state.transitionToResigned(Collections.emptyList()) ); + assertTrue(state.isCandidate()); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testFollowerVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) { - int node1 = 1; - int node2 = 2; - ReplicaKey votedKey = ReplicaKey.of(node1, Uuid.randomUuid()); - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + public void testCandidateToAnyStateLowerEpoch(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToUnattached(5); state.transitionToProspective(); - state.prospectiveAddVotedState(state.epoch(), votedKey); - state.transitionToFollower( - state.epoch(), - node2, - voters.listeners(node2) - ); - assertEquals( - Optional.of( - new ElectionState( - 0, - OptionalInt.of(node2), - Optional.of(persistedVotedKey(votedKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() + state.transitionToCandidate(); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) + ) ); - - state.transitionToFollower(state.epoch() + 1, node1, voters.listeners(node1)); - assertEquals(1, state.epoch()); + assertEquals(6, state.epoch()); assertEquals( Optional.of( - ElectionState.withElectedLeader( - 1, - node1, + ElectionState.withVotedCandidate( + 6, + persistedVotedKey(localVoterKey, kraftVersion), persistedVoters(voters.voterIds(), kraftVersion) ) ), @@ -2163,108 +2299,230 @@ public void testFollowerVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) { ); } + /** + * Test transitions from Leader + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCanBecomeFollowerOfNonVoter(KRaftVersion kraftVersion) { + public void testLeaderToUnattachedSameEpoch(KRaftVersion kraftVersion) { int otherNodeId = 1; - ReplicaKey nonVoterKey = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, 4)); - - // Add voted state - state.unattachedAddVotedState(4, nonVoterKey); - assertTrue(state.isUnattachedAndVoted()); - - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(4, votedState.epoch()); - assertEquals(nonVoterKey, votedState.votedKey().get()); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + state.transitionToLeader(0L, accumulator); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + } - // Transition to follower - state.transitionToFollower( - 4, - nonVoterKey.id(), - Endpoints.fromInetSocketAddresses( - Collections.singletonMap( - VoterSetTest.DEFAULT_LISTENER_NAME, - InetSocketAddress.createUnresolved("non-voter-host", 1234) + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderToUnattachedHigherEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + state.transitionToLeader(0L, accumulator); + state.transitionToUnattached(5); + assertEquals(5, state.epoch()); + assertEquals(OptionalInt.empty(), state.leaderId()); + assertEquals( + Optional.of( + ElectionState.withUnknownLeader( + 5, + persistedVoters(voters.voterIds(), kraftVersion) ) - ) + ), + store.readElectionState() ); - assertEquals( - new LeaderAndEpoch(OptionalInt.of(nonVoterKey.id()), 4), - state.leaderAndEpoch() + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderToFollowerSameEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + + state.transitionToProspective(); + state.transitionToCandidate(); + state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + state.transitionToLeader(0L, accumulator); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower(state.epoch(), otherNodeId, voters.listeners(otherNodeId)) ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderToFollowerHigherEpoch(KRaftVersion kraftVersion) { + int otherNodeId = 1; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + + state.transitionToProspective(); + state.transitionToCandidate(); + state.candidateStateOrThrow().recordGrantedVote(otherNodeId); + state.transitionToLeader(0L, accumulator); + state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId)); + + assertEquals(5, state.epoch()); + assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( - new ElectionState( - 4, - OptionalInt.of(nonVoterKey.id()), - Optional.of(persistedVotedKey(nonVoterKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) + Optional.of( + ElectionState.withElectedLeader( + 5, + otherNodeId, + persistedVoters(voters.voterIds(), kraftVersion) + ) ), - store.readElectionState().get() + store.readElectionState() ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testObserverCannotBecomeProspectiveOrCandidateOrLeader(KRaftVersion kraftVersion) { - boolean withDirectoryId = kraftVersion.featureLevel() > 0; - int otherNodeId = 1; - VoterSet voters = VoterSetTest.voterSet( - VoterSetTest.voterMap(IntStream.of(otherNodeId), withDirectoryId) - ); + public void testLeaderToProspective(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); + QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - assertTrue(state.isObserver()); + state.transitionToProspective(); + state.transitionToCandidate(); + state.transitionToLeader(0L, accumulator); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); + assertThrows(IllegalStateException.class, state::transitionToProspective); - assertThrows(IllegalStateException.class, state::transitionToCandidate); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testObserverWithIdCanVote(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(otherNodeKey)); + public void testLeaderToCandidate(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, 5)); - assertTrue(state.isObserver()); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + state.transitionToLeader(0L, accumulator); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); - state.unattachedAddVotedState(5, otherNodeKey); - assertTrue(state.isUnattachedAndVoted()); + assertThrows(IllegalStateException.class, state::transitionToCandidate); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); + } - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(5, votedState.epoch()); - assertEquals(otherNodeKey, votedState.votedKey().get()); + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderToLeader(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + state.transitionToLeader(0L, accumulator); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); + + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testObserverFollowerToUnattached(KRaftVersion kraftVersion) { - boolean withDirectoryId = kraftVersion.featureLevel() > 0; - int node1 = 1; - int node2 = 2; - VoterSet voters = VoterSetTest.voterSet( - VoterSetTest.voterMap(IntStream.of(node1, node2), withDirectoryId) + public void testLeaderToResigned(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); + state.transitionToLeader(0L, accumulator); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); + + state.transitionToResigned(Collections.singletonList(localVoterKey)); + assertTrue(state.isResigned()); + ResignedState resignedState = state.resignedStateOrThrow(); + assertEquals( + ElectionState.withElectedLeader(1, localId, voters.voterIds()), + resignedState.election() ); + assertEquals(1, resignedState.epoch()); + assertEquals(Collections.emptySet(), resignedState.unackedVoters()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToUnattached(5); + state.transitionToProspective(); + state.transitionToCandidate(); + state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); + state.transitionToLeader(0L, accumulator); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) + ) + ); + assertEquals(6, state.epoch()); + assertEquals( + Optional.of( + ElectionState.withElectedLeader( + 6, + localId, + persistedVoters(voters.voterIds(), kraftVersion) + ) + ), + store.readElectionState() + ); + } + + /** + * Test transitions from Observer as Unattached + */ + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testObserverUnattachedToUnattachedVoted(KRaftVersion kraftVersion) { + ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); + VoterSet voters = VoterSetTest.voterSet(Stream.of(otherNodeKey)); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, 5)); assertTrue(state.isObserver()); - state.transitionToFollower( - 2, - node1, - voters.listeners(node1) - ); - state.transitionToUnattached(3); - assertTrue(state.isUnattached()); - UnattachedState unattachedState = state.unattachedStateOrThrow(); - assertEquals(3, unattachedState.epoch()); + state.unattachedAddVotedState(5, otherNodeKey); - // Observers can remain in the unattached state indefinitely until a leader is found - assertEquals(Long.MAX_VALUE, unattachedState.electionTimeoutMs()); + UnattachedState votedState = state.unattachedStateOrThrow(); + assertTrue(state.isUnattachedAndVoted()); + assertTrue(state.isObserver()); + assertEquals(5, votedState.epoch()); + assertEquals(otherNodeKey, votedState.votedKey().get()); } @ParameterizedTest @@ -2301,396 +2559,85 @@ public void testObserverUnattachedToProspective(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isObserver()); assertTrue(state.isUnattachedNotVoted()); assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID)); assertTrue(state.isUnattachedAndVoted()); + assertTrue(state.isObserver()); assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); - state.transitionToFollower(logEndEpoch + 1, voter1, voters.listeners(voter1)); + state.transitionToFollower(logEndEpoch, voter2, voters.listeners(voter2)); + assertTrue(state.isFollower()); + assertTrue(state.isObserver()); + assertTrue(state.votedKey().isPresent()); + assertTrue(state.hasLeader()); assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testInitializeWithCorruptedStore(KRaftVersion kraftVersion) { - QuorumStateStore stateStore = Mockito.mock(QuorumStateStore.class); - Mockito.doThrow(UncheckedIOException.class).when(stateStore).readElectionState(); - - QuorumState state = buildQuorumState( - OptionalInt.of(localId), - localStandaloneVoterSet(), - kraftVersion + public void testObserverUnattachedToCandidateOrLeaderOrResigned(KRaftVersion kraftVersion) { + boolean withDirectoryId = kraftVersion.featureLevel() > 0; + int otherNodeId = 1; + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(otherNodeId), withDirectoryId) ); - - int epoch = 2; - state.initialize(new OffsetAndEpoch(0L, epoch)); - assertEquals(epoch, state.epoch()); - assertTrue(state.isUnattached()); - assertFalse(state.hasLeader()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testHasRemoteLeader(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); - - QuorumState state = initializeEmptyState(voters, kraftVersion); - assertFalse(state.hasRemoteLeader()); - - state.transitionToProspective(); - assertFalse(state.hasRemoteLeader()); - state.transitionToCandidate(); - assertFalse(state.hasRemoteLeader()); - - state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); - state.transitionToLeader(0L, accumulator); - assertFalse(state.hasRemoteLeader()); - - state.transitionToUnattached(state.epoch() + 1); - assertFalse(state.hasRemoteLeader()); - - state.unattachedAddVotedState(state.epoch(), otherNodeKey); - assertFalse(state.hasRemoteLeader()); - - state.transitionToFollower( - state.epoch(), - otherNodeKey.id(), - voters.listeners(otherNodeKey.id()) - ); - assertTrue(state.hasRemoteLeader()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testHighWatermarkRetained(KRaftVersion kraftVersion) { - ReplicaKey otherNodeKey = ReplicaKey.of(1, Uuid.randomUuid()); - VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.transitionToFollower( - 5, - otherNodeKey.id(), - voters.listeners(otherNodeKey.id()) - ); - - FollowerState followerState = state.followerStateOrThrow(); - followerState.updateHighWatermark(OptionalLong.of(10L)); - - Optional highWatermark = Optional.of(new LogOffsetMetadata(10L)); - assertEquals(highWatermark, state.highWatermark()); - - state.transitionToUnattached(6); - assertEquals(highWatermark, state.highWatermark()); - - state.unattachedAddVotedState(6, otherNodeKey); - assertEquals(highWatermark, state.highWatermark()); - - state.transitionToProspective(); - assertEquals(highWatermark, state.highWatermark()); - - state.transitionToCandidate(); - assertEquals(highWatermark, state.highWatermark()); - - CandidateState candidateState = state.candidateStateOrThrow(); - candidateState.recordGrantedVote(otherNodeKey.id()); - assertTrue(candidateState.epochElection().isVoteGranted()); - - state.transitionToLeader(10L, accumulator); - assertEquals(Optional.empty(), state.highWatermark()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testInitializeWithEmptyLocalId(KRaftVersion kraftVersion) { - boolean withDirectoryId = kraftVersion.featureLevel() > 0; - VoterSet voters = VoterSetTest.voterSet( - VoterSetTest.voterMap(IntStream.of(0, 1), withDirectoryId) - ); - QuorumState state = buildQuorumState(OptionalInt.empty(), voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, 0)); - + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isObserver()); - assertFalse(state.isVoter()); - - assertThrows(IllegalStateException.class, state::transitionToProspective); - - assertThrows( - IllegalStateException.class, - () -> state.unattachedAddVotedState(1, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)) - ); + assertThrows(IllegalStateException.class, state::transitionToCandidate); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); - - state.transitionToFollower(1, 1, voters.listeners(1)); - assertTrue(state.isFollower()); - - state.transitionToUnattached(2); - assertTrue(state.isUnattached()); + assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } + /** + * Test transitions from Observer as Follower + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testNoLocalIdInitializationFailsIfElectionStateHasVotedCandidate(KRaftVersion kraftVersion) { + public void testObserverFollowerToUnattached(KRaftVersion kraftVersion) { boolean withDirectoryId = kraftVersion.featureLevel() > 0; - int epoch = 5; - int votedId = 1; - VoterSet voters = VoterSetTest.voterSet( - VoterSetTest.voterMap(IntStream.of(0, votedId), withDirectoryId) - ); - - store.writeElectionState( - ElectionState.withVotedCandidate( - epoch, - ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), - voters.voterIds() - ), - kraftVersion - ); - - QuorumState state2 = buildQuorumState(OptionalInt.empty(), voters, kraftVersion); - assertThrows(IllegalStateException.class, () -> state2.initialize(new OffsetAndEpoch(0, 0))); - } - - @Test - public void testUnattachedWithLeaderToProspective() { int node1 = 1; int node2 = 2; - int epoch = 5; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); - store.writeElectionState(ElectionState.withElectedLeader(epoch, node1, voters.voterIds()), KRAFT_VERSION_1); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - - state.transitionToProspective(); - assertTrue(state.isProspective()); - assertEquals( - Optional.of(ElectionState.withElectedLeader(epoch, node1, persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(node1, node2), withDirectoryId) ); - } - - @Test - public void testIllegalTransitionsToCandidate() { - int node1 = 1; - int node2 = 2; - int epoch = 5; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); - QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, epoch)); - - // unattached - assertTrue(state.isUnattached()); - assertThrows(IllegalStateException.class, state::transitionToCandidate); - - // follower - state.transitionToFollower(epoch, node1, voters.listeners(node1)); - assertThrows(IllegalStateException.class, state::transitionToCandidate); - - // candidate - state.transitionToProspective(); - state.transitionToCandidate(); - assertThrows(IllegalStateException.class, state::transitionToCandidate); - - // leader - state.candidateStateOrThrow().recordGrantedVote(1); - state.transitionToLeader(0L, accumulator); - assertThrows(IllegalStateException.class, state::transitionToCandidate); - - // resigned - state.transitionToResigned(Collections.emptyList()); - assertThrows(IllegalStateException.class, state::transitionToCandidate); - } - - @Test - public void testIllegalTransitionsToProspective() { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); - QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - - // prospective - state.transitionToProspective(); - assertThrows(IllegalStateException.class, state::transitionToProspective); - - // leader - state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(1); - state.transitionToLeader(0L, accumulator); - assertThrows(IllegalStateException.class, state::transitionToProspective); - - // observer - voters = withRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); - state = initializeEmptyState(voters, KRAFT_VERSION_1); + QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - assertTrue(state.isUnattached()); assertTrue(state.isObserver()); - assertThrows(IllegalStateException.class, state::transitionToProspective); - } - - @Test - public void testIllegalTransitionsFromProspective() { - int leaderId = 1; - int followerId = 2; - int epoch = 5; - Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses( - Collections.singletonMap( - ListenerName.normalised("CONTROLLER"), - InetSocketAddress.createUnresolved("host-1", 1234))); - VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), KRAFT_VERSION_1); - QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, epoch)); - assertTrue(state.isUnattached()); - state.transitionToProspective(); - - assertThrows(IllegalStateException.class, state::transitionToProspective); - assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(epoch - 1)); - assertThrows(IllegalStateException.class, () -> state.transitionToFollower(epoch - 1, leaderId, leaderEndpoints)); - assertThrows(IllegalArgumentException.class, () -> state.transitionToFollower(epoch, leaderId, Endpoints.empty())); - } - - @Test - public void testUnattachedToAndFromProspective() { - int node1 = 1; - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); - QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - assertTrue(state.isUnattached()); - assertEquals(logEndEpoch, state.epoch()); - state.transitionToProspective(); - assertEquals(logEndEpoch, state.epoch()); - state.transitionToUnattached(5); - assertTrue(state.isUnattached()); - assertEquals(5, state.epoch()); - } - @Test - public void testUnattachedVotedToAndFromProspectiveVoted() { - int node1 = 1; - Uuid node1DirectoryId = Uuid.randomUuid(); - int node2 = 2; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), KRAFT_VERSION_1); - QuorumState state = initializeEmptyState(voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, 5)); - state.unattachedAddVotedState(5, ReplicaKey.of(node1, node1DirectoryId)); - - state.transitionToProspective(); - assertTrue(state.isProspective()); - ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); - assertEquals(5, prospectiveState.epoch()); - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 5, - persistedVotedKey(ReplicaKey.of(node1, node1DirectoryId), KRAFT_VERSION_1), - persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() - ); - - state.transitionToUnattached(prospectiveState.epoch()); - assertTrue(state.isUnattachedAndVoted()); - assertEquals(prospectiveState.epoch(), state.epoch()); - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - prospectiveState.epoch(), - persistedVotedKey(ReplicaKey.of(node1, node1DirectoryId), KRAFT_VERSION_1), - persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() + state.transitionToFollower( + 2, + node1, + voters.listeners(node1) ); - } - - @Test - public void testUnattachedWithLeaderNoEndpointToAndFromProspective() { - int leaderId = 1; - ReplicaKey leaderKey = ReplicaKey.of(leaderId, Uuid.randomUuid()); - int followerId = 2; - ReplicaKey followerKey = ReplicaKey.of(followerId, Uuid.randomUuid()); - int epoch = 5; - Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses( - Collections.singletonMap( - ListenerName.normalised("CONTROLLER"), - InetSocketAddress.createUnresolved("host-1", 1234))); - Map voterMap = new HashMap<>(); - voterMap.put(localId, VoterSetTest.voterNode(localVoterKey)); - voterMap.put(leaderId, VoterSetTest.voterNode(leaderKey, Endpoints.empty())); - voterMap.put(followerId, VoterSetTest.voterNode(followerKey, Endpoints.empty())); - VoterSet voters = VoterSetTest.voterSet(voterMap); - - store.writeElectionState(ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()), KRAFT_VERSION_1); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToUnattached(3); assertTrue(state.isUnattached()); - assertTrue(state.hasLeader()); - - state.transitionToProspective(); - assertTrue(state.isProspective()); - assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - leaderId, - persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() - ); + UnattachedState unattachedState = state.unattachedStateOrThrow(); + assertEquals(3, unattachedState.epoch()); - state.transitionToFollower(epoch, leaderId, leaderEndpoints); - assertTrue(state.isFollower()); - assertEquals(epoch, state.epoch()); - assertEquals(leaderEndpoints, state.leaderEndpoints()); - assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - leaderId, - persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() - ); + // Observers can remain in the unattached state indefinitely until a leader is found + assertEquals(Long.MAX_VALUE, unattachedState.electionTimeoutMs()); } - @Test - public void testFollowerToAndFromProspectiveWithLeader() { - int leaderId = 1; - int followerId = 2; - int epoch = 5; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), KRAFT_VERSION_1); - - store.writeElectionState(ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()), KRAFT_VERSION_1); - QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, KRAFT_VERSION_1); + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testObserverFollowerToProspectiveOrCandidateOrLeaderOrResigned(KRaftVersion kraftVersion) { + int voter1 = 1; + int voter2 = 2; + VoterSet voters = withRemoteVoterSet(IntStream.of(voter1, voter2), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - assertTrue(state.isFollower()); - assertTrue(state.hasLeader()); - state.transitionToProspective(); - assertTrue(state.isProspective()); - ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); - assertFalse(prospectiveState.leaderEndpoints().isEmpty()); - assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - leaderId, - persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() - ); + state.transitionToFollower(logEndEpoch, voter1, voters.listeners(voter1)); + assertTrue(state.isObserver()); + assertEquals(fetchTimeoutMs, state.followerStateOrThrow().remainingFetchTimeMs(time.milliseconds())); - state.transitionToFollower(epoch, leaderId, prospectiveState.leaderEndpoints()); - assertTrue(state.isFollower()); - assertEquals(epoch, state.epoch()); - assertFalse(state.leaderEndpoints().isEmpty()); - assertEquals(prospectiveState.leaderEndpoints(), state.leaderEndpoints()); - assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - leaderId, - persistedVoters(voters.voterIds(), KRAFT_VERSION_1))), - store.readElectionState() - ); + assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); + assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(logEndEpoch + 1, accumulator)); + assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 419a36f7f32bd..314da6b1e723c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -691,10 +691,13 @@ void assertVotedCandidate(int epoch, int candidateId) { } void assertVotedCandidate(int epoch, ReplicaKey candidateKey) { + ReplicaKey expectedKey = kraftVersion == KRaftVersion.KRAFT_VERSION_0 ? + ReplicaKey.of(candidateKey.id(), ReplicaKey.NO_DIRECTORY_ID) : + candidateKey; assertEquals( ElectionState.withVotedCandidate( epoch, - candidateKey, + expectedKey, expectedVoters() ), quorumStateStore.readElectionState().get() diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java index 883bb1f17f9bb..bfc8b03e3538d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java @@ -16,11 +16,12 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import java.util.Optional; @@ -37,16 +38,19 @@ public class UnattachedStateTest { private final LogContext logContext = new LogContext(); private final int epoch = 5; private final int electionTimeoutMs = 10000; + private final Set voters = Set.of(1, 2, 3); + private final ReplicaKey voter1Key = ReplicaKey.of(1, Uuid.randomUuid()); + private final ReplicaKey votedKey = voter1Key; private UnattachedState newUnattachedState( - Set voters, - OptionalInt leaderId + OptionalInt leaderId, + Optional votedKey ) { return new UnattachedState( time, epoch, leaderId, - Optional.empty(), + votedKey, voters, Optional.empty(), electionTimeoutMs, @@ -54,15 +58,17 @@ private UnattachedState newUnattachedState( ); } - @Test - public void testElectionTimeout() { - Set voters = Set.of(1, 2, 3); - - UnattachedState state = newUnattachedState(voters, OptionalInt.empty()); - - assertEquals(epoch, state.epoch()); + @ParameterizedTest + @CsvSource({ "true,false", "false,true", "false,false" }) + public void testElectionTimeout(boolean hasVotedKey, boolean hasLeaderId) { + OptionalInt leader = hasLeaderId ? OptionalInt.of(3) : OptionalInt.empty(); + Optional votedKey = hasVotedKey ? Optional.of(this.votedKey) : Optional.empty(); + UnattachedState state = newUnattachedState(leader, votedKey); - assertEquals(ElectionState.withUnknownLeader(epoch, voters), state.election()); + assertEquals( + new ElectionState(epoch, leader, votedKey, voters), + state.election() + ); assertEquals(electionTimeoutMs, state.remainingElectionTimeMs(time.milliseconds())); assertFalse(state.hasElectionTimeoutExpired(time.milliseconds())); @@ -77,37 +83,39 @@ public void testElectionTimeout() { @ParameterizedTest @ValueSource(booleans = {true, false}) - public void testGrantVote(boolean isLogUpToDate) { - UnattachedState state = newUnattachedState(Set.of(1, 2, 3), OptionalInt.empty()); + public void testGrantVoteWithoutVotedKey(boolean isLogUpToDate) { + UnattachedState state = newUnattachedState(OptionalInt.empty(), Optional.empty()); assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + state.canGrantVote(voter1Key, isLogUpToDate, true) ); assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + state.canGrantVote(voter1Key, isLogUpToDate, false) ); + assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) ); assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) + state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) + state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) ); + assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) + state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); assertEquals( isLogUpToDate, @@ -115,20 +123,11 @@ public void testGrantVote(boolean isLogUpToDate) { ); } - @Test - void testLeaderEndpoints() { - UnattachedState state = newUnattachedState(Set.of(1, 2, 3), OptionalInt.empty()); - - assertEquals(Endpoints.empty(), state.leaderEndpoints()); - } - @ParameterizedTest @ValueSource(booleans = {true, false}) - void testUnattachedWithLeader(boolean isLogUpToDate) { + void testGrantVoteWithLeader(boolean isLogUpToDate) { int leaderId = 3; - Set voters = Set.of(1, 2, leaderId); - - UnattachedState state = newUnattachedState(voters, OptionalInt.of(leaderId)); + UnattachedState state = newUnattachedState(OptionalInt.of(leaderId), Optional.empty()); // Check that the leader is persisted if the leader is known assertEquals(ElectionState.withElectedLeader(epoch, leaderId, voters), state.election()); @@ -137,7 +136,7 @@ void testUnattachedWithLeader(boolean isLogUpToDate) { // This is because nodes in Unattached have not successfully fetched from the leader yet assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + state.canGrantVote(voter1Key, isLogUpToDate, true) ); assertEquals( isLogUpToDate, @@ -145,7 +144,7 @@ void testUnattachedWithLeader(boolean isLogUpToDate) { ); assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + state.canGrantVote(ReplicaKey.of(leaderId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) ); assertEquals( isLogUpToDate, @@ -155,7 +154,66 @@ void testUnattachedWithLeader(boolean isLogUpToDate) { // Check that the replica rejects all standard votes request if the leader is known assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(leaderId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testCanGrantVoteWithVotedKey(boolean isLogUpToDate) { + UnattachedState state = newUnattachedState(OptionalInt.empty(), Optional.of(votedKey)); + + // Same voterKey + // We can reject PreVote for a replica we have already granted a standard vote to if their log is behind + assertEquals( + isLogUpToDate, + state.canGrantVote(votedKey, isLogUpToDate, true) + ); + assertTrue(state.canGrantVote(votedKey, isLogUpToDate, false)); + + // Different directoryId + // We can grant PreVote for a replica we have already granted a standard vote to if their log is up-to-date, + // even if the directoryId is different + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(votedKey.id(), Uuid.randomUuid()), isLogUpToDate, true) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedKey.id(), Uuid.randomUuid()), isLogUpToDate, false)); + + // Missing directoryId + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(votedKey.id(), ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(votedKey.id(), ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + + // Different voterId + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(2, votedKey.directoryId().get()), isLogUpToDate, true) + ); + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); + assertFalse(state.canGrantVote(ReplicaKey.of(2, votedKey.directoryId().get()), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + + // Observer + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); } + + @ParameterizedTest + @CsvSource({ "true,false", "false,true", "false,false" }) + public void testLeaderEndpoints(boolean hasVotedKey, boolean hasLeaderId) { + OptionalInt leaderId = hasLeaderId ? OptionalInt.of(3) : OptionalInt.empty(); + Optional votedKey = hasVotedKey ? Optional.of(this.votedKey) : Optional.empty(); + UnattachedState state = newUnattachedState(leaderId, votedKey); + + assertEquals(Endpoints.empty(), state.leaderEndpoints()); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index f16066b90e908..689249839994b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -116,6 +116,7 @@ private VoterSet localStandaloneVoterSet(KRaftVersion kraftVersion) { public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { boolean withDirectoryId = kraftVersion.featureLevel() > 0; Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2), withDirectoryId); + Uuid voter2DirectoryId = voterMap.get(2).voterKey().directoryId().orElse(Uuid.ZERO_UUID); voterMap.put( localId, VoterSetTest.voterNode( @@ -131,39 +132,82 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { state.initialize(new OffsetAndEpoch(0L, 0)); raftMetrics = new KafkaRaftMetrics(metrics, "raft", state); + // unattached assertEquals("unattached", getMetric(metrics, "current-state").metricValue()); - assertEquals((double) -1L, getMetric(metrics, "current-leader").metricValue()); - assertEquals((double) -1L, getMetric(metrics, "current-vote").metricValue()); + assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); + assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); assertEquals( Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) -1, getMetric(metrics, "high-watermark").metricValue()); + + // prospective + state.transitionToProspective(); + assertEquals("prospective", getMetric(metrics, "current-state").metricValue()); + assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); + assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); + assertEquals( + Uuid.ZERO_UUID.toString(), + getMetric(metrics, "current-vote-directory-id").metricValue() + ); + assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - assertEquals("prospective", getMetric(metrics, "current-state").metricValue()); - assertEquals((double) -1L, getMetric(metrics, "current-leader").metricValue()); - assertEquals((double) -1L, getMetric(metrics, "current-vote").metricValue()); - assertEquals( - Uuid.ZERO_UUID.toString(), - getMetric(metrics, "current-vote-directory-id").metricValue() - ); - assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); - assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); - } + // prospective with votedKey + state.prospectiveAddVotedState(0, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)); + assertEquals("prospective-voted", getMetric(metrics, "current-state").metricValue()); + assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); + assertEquals((double) 1, getMetric(metrics, "current-vote").metricValue()); + assertEquals( + Uuid.ZERO_UUID.toString(), + getMetric(metrics, "current-vote-directory-id").metricValue() + ); + assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); + + // follower with votedKey and leader + state.transitionToFollower(0, 2, voters.listeners(2)); + assertEquals("follower", getMetric(metrics, "current-state").metricValue()); + assertEquals((double) 2, getMetric(metrics, "current-leader").metricValue()); + assertEquals((double) 1, getMetric(metrics, "current-vote").metricValue()); + assertEquals( + Uuid.ZERO_UUID.toString(), + getMetric(metrics, "current-vote-directory-id").metricValue() + ); + assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); + + // follower with updated HW + state.followerStateOrThrow().updateHighWatermark(OptionalLong.of(5L)); + assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); + + // prospective with votedKey and leader + state.transitionToProspective(); + assertEquals("prospective-voted", getMetric(metrics, "current-state").metricValue()); + assertEquals((double) 2, getMetric(metrics, "current-leader").metricValue()); + assertEquals((double) 1, getMetric(metrics, "current-vote").metricValue()); + assertEquals( + Uuid.ZERO_UUID.toString(), + getMetric(metrics, "current-vote-directory-id").metricValue() + ); + assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); + + // candidate state.transitionToCandidate(); assertEquals("candidate", getMetric(metrics, "current-state").metricValue()); - assertEquals((double) -1L, getMetric(metrics, "current-leader").metricValue()); + assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) localId, getMetric(metrics, "current-vote").metricValue()); assertEquals( localDirectoryId.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 1, getMetric(metrics, "current-epoch").metricValue()); - assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); + assertEquals((double) 1L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); + // leader state.candidateStateOrThrow().recordGrantedVote(1); state.transitionToLeader(2L, accumulator); assertEquals("leader", getMetric(metrics, "current-state").metricValue()); @@ -173,17 +217,19 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { localDirectoryId.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 1, getMetric(metrics, "current-epoch").metricValue()); - assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); + assertEquals((double) 1L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); // todo, bug fix - state.leaderStateOrThrow().updateLocalState(new LogOffsetMetadata(5L), voters); + // leader with updated HW + state.leaderStateOrThrow().updateLocalState(new LogOffsetMetadata(10L), voters); state.leaderStateOrThrow().updateReplicaState( voterMap.get(1).voterKey(), 0, - new LogOffsetMetadata(5L) + new LogOffsetMetadata(10L) ); - assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); + assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); + // follower state.transitionToFollower(2, 1, voters.listeners(1)); assertEquals("follower", getMetric(metrics, "current-state").metricValue()); assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); @@ -192,37 +238,23 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 2, getMetric(metrics, "current-epoch").metricValue()); - assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); - - state.followerStateOrThrow().updateHighWatermark(OptionalLong.of(10L)); + assertEquals((double) 2L, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); - if (kraftVersion.isReconfigSupported()) { - state.transitionToProspective(); - assertEquals("prospective", getMetric(metrics, "current-state").metricValue()); - assertEquals((double) 1L, getMetric(metrics, "current-leader").metricValue()); - assertEquals((double) -1L, getMetric(metrics, "current-vote").metricValue()); - assertEquals( - Uuid.ZERO_UUID.toString(), - getMetric(metrics, "current-vote-directory-id").metricValue() - ); - assertEquals((double) 2, getMetric(metrics, "current-epoch").metricValue()); - assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); - } - + // unattached with votedKey state.transitionToUnattached(3); - state.unattachedAddVotedState(3, ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID)); + state.unattachedAddVotedState(3, ReplicaKey.of(2, voter2DirectoryId)); assertEquals("unattached-voted", getMetric(metrics, "current-state").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) 2, getMetric(metrics, "current-vote").metricValue()); assertEquals( - Uuid.ZERO_UUID.toString(), + voter2DirectoryId.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 3, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 3L, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); + // unattached with leader without votedKey state.transitionToUnattached(4, OptionalInt.of(1)); assertEquals("unattached", getMetric(metrics, "current-state").metricValue()); assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); @@ -231,7 +263,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 4, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 4L, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); } From b01aa1100a38835327f2bd7ca9901d1e6011e54b Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Mon, 23 Dec 2024 08:03:57 -0800 Subject: [PATCH 13/23] some cleanup --- raft/src/main/java/org/apache/kafka/raft/CandidateState.java | 2 +- raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java | 4 ++-- .../src/main/java/org/apache/kafka/raft/ProspectiveState.java | 2 +- raft/src/main/java/org/apache/kafka/raft/QuorumState.java | 3 ++- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 62a31011519f9..6034c804b712a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -73,7 +73,7 @@ protected CandidateState( this.log = logContext.logger(CandidateState.class); this.epochElection = new EpochElection(voters.voterKeys()); - epochElection.recordVote(localId, true); //voterStates().get(localId).setState(State.GRANTED); + epochElection.recordVote(localId, true); } public int localId() { diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 2e6f7b0ab5832..38559d00d8afb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -675,7 +675,7 @@ private boolean maybeTransitionToCandidate(ProspectiveState state, long currentT } /** - * Only applies to VotingStates (Prospective or Candidate). If enough votes were granted + * Only applies to NomineeStates (Prospective or Candidate). If enough votes were granted * then this method is called to transition the state forward - either from Prospective to Candidate * or from Candidate to Leader. * @return true if the state transitioned forward, false otherwise @@ -929,7 +929,7 @@ private boolean handleVoteResponse( if (topLevelError != Errors.NONE) { if (topLevelError == Errors.UNSUPPORTED_VERSION && quorum.isProspective()) { logger.info("Prospective received unsupported version error in vote response in epoch {}, " + - "transitioning to Candidate state immediately since entire quorum does not support PreVote.", + "transitioning to Candidate state immediately since entire quorum may not support PreVote.", quorum.epoch()); transitionToCandidate(currentTimeMs); return true; diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index 7f6a0e5d69248..4ecbec627f944 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -70,7 +70,7 @@ public ProspectiveState( this.log = logContext.logger(ProspectiveState.class); this.epochElection = new EpochElection(voters.voterKeys()); - epochElection.recordVote(localId, true); //voterStates().get(localId).setState(State.GRANTED); + epochElection.recordVote(localId, true); } public int localId() { diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 877663d922afa..a34d53cc48aef 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -69,7 +69,7 @@ * Prospective: After expiration of the fetch timeout * Follower: After discovering a leader with a larger epoch * - * Observers follow a simpler state machine. The Voted/Prospective/Candidate/Leader/Resigned + * Observers follow a simpler state machine. The Prospective/Candidate/Leader/Resigned * states are not possible for observers, so the only transitions that are possible * are between Unattached and Follower. * @@ -373,6 +373,7 @@ public void transitionToResigned(List preferredSuccessors) { * Note, if we are transitioning from unattached and there is no epoch change, we take the path of * unattachedAddVotedState instead. */ + // Used in testing public void transitionToUnattached(int epoch) { transitionToUnattached(epoch, OptionalInt.empty()); } From f9d1812e1e525fbf1f5ec1107d3b017308a201c4 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Tue, 24 Dec 2024 09:36:54 -0800 Subject: [PATCH 14/23] from latest discussions --- .../org/apache/kafka/raft/CandidateState.java | 47 +++++++- .../org/apache/kafka/raft/FollowerState.java | 3 +- .../apache/kafka/raft/KafkaRaftClient.java | 40 ++++++- .../org/apache/kafka/raft/NomineeState.java | 5 +- .../apache/kafka/raft/ProspectiveState.java | 78 +++++--------- .../org/apache/kafka/raft/QuorumState.java | 60 ++++++++++- .../apache/kafka/raft/UnattachedState.java | 70 ++++-------- .../apache/kafka/raft/CandidateStateTest.java | 4 +- .../apache/kafka/raft/FollowerStateTest.java | 2 +- .../raft/KafkaRaftClientPreVoteTest.java | 100 +++++++++++------- .../kafka/raft/KafkaRaftClientTest.java | 71 ++++++++++++- .../kafka/raft/ProspectiveStateTest.java | 2 + .../kafka/raft/RaftClientTestContext.java | 10 +- .../raft/internals/KafkaRaftMetricsTest.java | 20 ++-- 14 files changed, 348 insertions(+), 164 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 6034c804b712a..d0bbb696c9ced 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -30,18 +30,22 @@ public class CandidateState implements NomineeState { private final int localId; private final Uuid localDirectoryId; private final int epoch; + private final int retries; private final EpochElection epochElection; private final Optional highWatermark; private final int electionTimeoutMs; private final Timer electionTimer; + private final Timer backoffTimer; private final Logger log; + private boolean isBackingOff; /** * The lifetime of a candidate state is the following. * - * 1. Once started, it will keep record of the received votes. + * 1. Once started, it will send vote requests and keep record of the received vote responses. * 2. If majority votes granted, it will transition to leader state. - * 3. If majority votes rejected or election timed out, it will transition to prospective. + * 3. If majority votes rejected, it will transition to prospective after a backoff phase. + * 4. If election times out, it will transition immediately to prospective. */ protected CandidateState( Time time, @@ -50,6 +54,7 @@ protected CandidateState( int epoch, VoterSet voters, Optional highWatermark, + int retries, int electionTimeoutMs, LogContext logContext ) { @@ -68,8 +73,11 @@ protected CandidateState( this.localDirectoryId = localDirectoryId; this.epoch = epoch; this.highWatermark = highWatermark; + this.retries = retries; + this.isBackingOff = false; this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); + this.backoffTimer = time.timer(0); this.log = logContext.logger(CandidateState.class); this.epochElection = new EpochElection(voters.voterKeys()); @@ -80,6 +88,39 @@ public int localId() { return localId; } + public int retries() { + return retries; + } + + /** + * Check if the candidate is backing off for the next election + */ + public boolean isBackingOff() { + return isBackingOff; + } + + /** + * Record the current election has failed since we've either received sufficient rejecting voters or election timed out + */ + public void startBackingOff(long currentTimeMs, long backoffDurationMs) { + this.backoffTimer.update(currentTimeMs); + this.backoffTimer.reset(backoffDurationMs); + this.isBackingOff = true; + } + + public boolean isBackoffComplete(long currentTimeMs) { + backoffTimer.update(currentTimeMs); + return backoffTimer.isExpired(); + } + + public long remainingBackoffMs(long currentTimeMs) { + if (!isBackingOff) { + throw new IllegalStateException("Candidate is not currently backing off"); + } + backoffTimer.update(currentTimeMs); + return backoffTimer.remainingMs(); + } + @Override public EpochElection epochElection() { return epochElection; @@ -164,7 +205,7 @@ public boolean canGrantVote( @Override public String toString() { return String.format( - "CandidateState(localId=%d, localDirectoryId=%s,epoch=%d, voteStates=%s, " + + "Candidate(localId=%d, localDirectoryId=%s, epoch=%d, voteStates=%s, " + "highWatermark=%s, electionTimeoutMs=%d)", localId, localDirectoryId, diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index 576b8a59a4816..8f84b554f66ef 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -236,12 +236,13 @@ public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolea @Override public String toString() { return String.format( - "FollowerState(fetchTimeoutMs=%d, epoch=%d, leader=%d, leaderEndpoints=%s, " + + "FollowerState(fetchTimeoutMs=%d, epoch=%d, leader=%d, leaderEndpoints=%s, votedKey=%s, " + "voters=%s, highWatermark=%s, fetchingSnapshot=%s)", fetchTimeoutMs, epoch, leaderId, leaderEndpoints, + votedKey, voters, highWatermark, fetchingSnapshot diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 38559d00d8afb..323c092eb15a7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -161,6 +161,7 @@ */ @SuppressWarnings({ "ClassDataAbstractionCoupling", "ClassFanOutComplexity", "ParameterNumber", "NPathComplexity" }) public final class KafkaRaftClient implements RaftClient { + private static final int RETRY_BACKOFF_BASE_MS = 100; private static final int MAX_NUMBER_OF_BATCHES = 10; public static final int MAX_FETCH_WAIT_MS = 500; public static final int MAX_BATCH_SIZE_BYTES = 8 * 1024 * 1024; @@ -984,6 +985,7 @@ private boolean handleVoteResponse( maybeTransitionForward(state, currentTimeMs); } else { state.recordRejectedVote(remoteNodeId); + maybeCandidateStartBackingOff(currentTimeMs); } } else { logger.debug("Ignoring vote response {} since we are no longer a VotingState " + @@ -996,6 +998,35 @@ private boolean handleVoteResponse( } } + private void maybeCandidateStartBackingOff(long currentTimeMs) { + // If in candidate state and vote is rejected, go immediately to a random, exponential backoff. The + // backoff starts low to prevent needing to wait the entire election timeout when the vote + // result has already been determined. The randomness prevents the next election from being + // gridlocked with another nominee due to timing. The exponential aspect limits epoch churn when + // the replica has failed multiple elections in succession. + if (quorum.isCandidate()) { + CandidateState candidate = quorum.candidateStateOrThrow(); + if (candidate.epochElection().isVoteRejected() && !candidate.isBackingOff()) { + logger.info("Insufficient remaining votes to become leader (rejected by {}). " + + "We will backoff before retrying election again", candidate.epochElection().rejectingVoters()); + + candidate.startBackingOff( + currentTimeMs, + binaryExponentialElectionBackoffMs(candidate.retries()) + ); + } + } + } + + private int binaryExponentialElectionBackoffMs(int retries) { + if (retries <= 0) { + throw new IllegalArgumentException("Retries " + retries + " should be larger than zero"); + } + // upper limit exponential co-efficients at 20 to avoid overflow + return Math.min(RETRY_BACKOFF_BASE_MS * random.nextInt(2 << Math.min(20, retries - 1)), + quorumConfig.electionBackoffMaxMs()); + } + private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) { if (positionInSuccessors == 0) { return 0; @@ -3055,7 +3086,14 @@ private long pollCandidate(long currentTimeMs) { // 3) the shutdown timer expires long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); - } else if (state.epochElection().isVoteRejected() || state.hasElectionTimeoutExpired(currentTimeMs)) { + } else if (state.isBackingOff()) { + if (state.isBackoffComplete(currentTimeMs)) { + logger.info("Transition to prospective after election backoff has completed"); + transitionToProspective(currentTimeMs); + return 0L; + } + return state.remainingBackoffMs(currentTimeMs); + } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { logger.info("Election was not granted, transitioning to prospective"); transitionToProspective(currentTimeMs); return 0L; diff --git a/raft/src/main/java/org/apache/kafka/raft/NomineeState.java b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java index 7c1b6e5abeeab..dc8952048c688 100644 --- a/raft/src/main/java/org/apache/kafka/raft/NomineeState.java +++ b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java @@ -34,7 +34,7 @@ interface NomineeState extends EpochState { * Record a rejected vote from one of the voters. * * @param remoteNodeId The id of the voter - * @return true if the rejected vote had not been previously recorded + * @return true if the voter had not been previously recorded * @throws IllegalArgumentException */ boolean recordRejectedVote(int remoteNodeId); @@ -47,8 +47,7 @@ interface NomineeState extends EpochState { /** * Returns the remaining time in milliseconds until the election timeout expires. - * @param currentTimeMs - * @return + * @param currentTimeMs The current time in milliseconds */ long remainingElectionTimeMs(long currentTimeMs); } diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index 4ecbec627f944..7fc03ef71fa2b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -26,6 +26,8 @@ import java.util.Optional; import java.util.OptionalInt; +import static org.apache.kafka.raft.QuorumState.unattachedOrProspectiveCanGrantVote; + public class ProspectiveState implements NomineeState { private final int localId; private final int epoch; @@ -35,17 +37,19 @@ public class ProspectiveState implements NomineeState { private final VoterSet voters; private final EpochElection epochElection; private final Optional highWatermark; + private final int retries; + private final long electionTimeoutMs; private final Timer electionTimer; private final Logger log; /** * The lifetime of a prospective state is the following. * - * 1. Once started, it will keep record of the received votes and continue to fetch from bootstrap voters. - * 2. If it receives a fetch response denoting a leader with a higher epoch, it will transition to follower state. - * 3. If majority votes granted, it will transition to leader state. - * 4. If majority votes rejected or election times out, it will enter a backing off phase; - * after the backoff phase completes, it will send out another round of PreVote requests. + * 1. Once started, it will send prevote requests and keep record of the received vote responses + * 2. If it receives a message denoting a leader with a higher epoch, it will transition to follower state. + * 3. If majority votes granted, it will transition to candidate state. + * 4. If majority votes rejected or election times out, it will transition to unattached or follower state + * depending on if it knows the leader id and endpoints or not */ public ProspectiveState( Time time, @@ -56,6 +60,7 @@ public ProspectiveState( Optional votedKey, VoterSet voters, Optional highWatermark, + int retries, int electionTimeoutMs, LogContext logContext ) { @@ -66,6 +71,8 @@ public ProspectiveState( this.votedKey = votedKey; this.voters = voters; this.highWatermark = highWatermark; + this.retries = retries; + this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); this.log = logContext.logger(ProspectiveState.class); @@ -86,6 +93,10 @@ public EpochElection epochElection() { return epochElection; } + public int retries() { + return retries; + } + @Override public boolean recordGrantedVote(int remoteNodeId) { return epochElection().recordVote(remoteNodeId, true); @@ -101,50 +112,15 @@ public boolean recordRejectedVote(int remoteNodeId) { @Override public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolean isPreVote) { - if (isPreVote) { - return canGrantPreVote(replicaKey, isLogUpToDate); - } - if (votedKey.isPresent()) { - ReplicaKey votedReplicaKey = votedKey.get(); - if (votedReplicaKey.id() == replicaKey.id()) { - return votedReplicaKey.directoryId().isEmpty() || votedReplicaKey.directoryId().equals(replicaKey.directoryId()); - } - log.debug( - "Rejecting Vote request (preVote=false) from candidate ({}), already have voted for another " + - "candidate ({}) in epoch {}", - replicaKey, - votedKey, - epoch - ); - return false; - } else if (leaderId.isPresent()) { - // If the leader id is known it should behave similar to the follower state - log.debug( - "Rejecting Vote request (preVote=false) from candidate ({}) since we already have a leader {} in epoch {}", - replicaKey, - leaderId, - epoch - ); - return false; - } else if (!isLogUpToDate) { - log.debug( - "Rejecting Vote request (preVote=false) from candidate ({}) since candidate's log is not up to date with us", - replicaKey - ); - } - - return isLogUpToDate; - } - - private boolean canGrantPreVote(ReplicaKey replicaKey, boolean isLogUpToDate) { - if (!isLogUpToDate) { - log.debug( - "Rejecting Vote request (preVote=true) from prospective ({}) since prospective's log is not up to date with us", - replicaKey - ); - } - - return isLogUpToDate; + return unattachedOrProspectiveCanGrantVote( + leaderId, + votedKey, + epoch, + replicaKey, + isLogUpToDate, + isPreVote, + log + ); } @Override @@ -182,10 +158,12 @@ public Optional highWatermark() { @Override public String toString() { return String.format( - "ProspectiveState(epoch=%d, votedKey=%s, voters=%s, highWatermark=%s)", + "Prospective(epoch=%d, leaderId=%s, votedKey=%s, voters=%s, electionTimeoutMs=%s, highWatermark=%s)", epoch, + leaderId, votedKey, voters, + electionTimeoutMs, highWatermark ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index a34d53cc48aef..4909b5aac2b81 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -196,6 +196,7 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IllegalStateE election.epoch(), partitionState.lastVoterSet(), Optional.empty(), + 1, randomElectionTimeoutMs(), logContext ); @@ -485,7 +486,7 @@ public void prospectiveAddVotedState( ); } else if (localId.isEmpty()) { throw new IllegalStateException("Cannot add voted state without a replica id"); - } else if (epoch != currentEpoch || isProspectiveAndVoted()) { + } else if (epoch != currentEpoch || !isProspectiveNotVoted()) { throw new IllegalStateException( String.format( "Cannot add voted key (%s) to current state (%s) in epoch %d", @@ -496,6 +497,7 @@ public void prospectiveAddVotedState( ); } + ProspectiveState prospectiveState = prospectiveStateOrThrow(); // Note that we reset the election timeout after voting for a candidate because we // know that the candidate has at least as good of a chance of getting elected as us durableTransitionTo( @@ -508,6 +510,7 @@ public void prospectiveAddVotedState( Optional.of(candidateKey), partitionState.lastVoterSet(), state.highWatermark(), + prospectiveState.retries(), randomElectionTimeoutMs(), logContext ) @@ -602,6 +605,8 @@ public void transitionToProspective() { " is state " + state); } + int retries = isCandidate() ? candidateStateOrThrow().retries() + 1 : 1; + durableTransitionTo(new ProspectiveState( time, localIdOrThrow(), @@ -611,6 +616,7 @@ public void transitionToProspective() { votedKey(), partitionState.lastVoterSet(), state.highWatermark(), + retries, randomElectionTimeoutMs(), logContext )); @@ -622,6 +628,8 @@ public void transitionToCandidate() { int newEpoch = epoch() + 1; int electionTimeoutMs = randomElectionTimeoutMs(); + int retries = isProspective() ? prospectiveStateOrThrow().retries() : 1; + durableTransitionTo(new CandidateState( time, localIdOrThrow(), @@ -629,6 +637,7 @@ public void transitionToCandidate() { newEpoch, partitionState.lastVoterSet(), state.highWatermark(), + retries, electionTimeoutMs, logContext )); @@ -851,4 +860,53 @@ public boolean isCandidate() { public boolean isNomineeState() { return state instanceof NomineeState; } + + public static boolean unattachedOrProspectiveCanGrantVote( + OptionalInt leaderId, + Optional votedKey, + int epoch, + ReplicaKey replicaKey, + boolean isLogUpToDate, + boolean isPreVote, + Logger log + ) { + if (isPreVote) { + if (!isLogUpToDate) { + log.debug( + "Rejecting Vote request (preVote=true) from prospective ({}) since prospective's log is not up to date with us", + replicaKey + ); + } + return isLogUpToDate; + } else if (votedKey.isPresent()) { + ReplicaKey votedReplicaKey = votedKey.get(); + if (votedReplicaKey.id() == replicaKey.id()) { + return votedReplicaKey.directoryId().isEmpty() || votedReplicaKey.directoryId().equals(replicaKey.directoryId()); + } + log.debug( + "Rejecting Vote request (preVote=false) from candidate ({}), already have voted for another " + + "candidate ({}) in epoch {}", + replicaKey, + votedKey, + epoch + ); + return false; + } else if (leaderId.isPresent()) { + // If the leader id is known it should behave similar to the follower state + log.debug( + "Rejecting Vote request (preVote=false) from candidate ({}) since we already have a leader {} in epoch {}", + replicaKey, + leaderId.getAsInt(), + epoch + ); + return false; + } else if (!isLogUpToDate) { + log.debug( + "Rejecting Vote request (preVote=false) from candidate ({}) since candidate's log is not up to date with us", + replicaKey + ); + } + + return isLogUpToDate; + } } diff --git a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java index 9e1aef7d63fe7..794b3b91f5f3d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java @@ -26,6 +26,8 @@ import java.util.OptionalInt; import java.util.Set; +import static org.apache.kafka.raft.QuorumState.unattachedOrProspectiveCanGrantVote; + /** * A replica is "unattached" when it doesn't know the leader or the leader's endpoint. * @@ -114,60 +116,28 @@ public Optional highWatermark() { @Override public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolean isPreVote) { - if (isPreVote) { - return canGrantPreVote(replicaKey, isLogUpToDate); - } else if (votedKey.isPresent()) { - ReplicaKey votedReplicaKey = votedKey.get(); - if (votedReplicaKey.id() == replicaKey.id()) { - return votedReplicaKey.directoryId().isEmpty() || votedReplicaKey.directoryId().equals(replicaKey.directoryId()); - } - log.debug( - "Rejecting Vote request (preVote=false) from candidate ({}), already have voted for another " + - "candidate ({}) in epoch {}", - replicaKey, - votedKey, - epoch - ); - return false; - } else if (leaderId.isPresent()) { - // If the leader id is known it should behave similar to the follower state - log.debug( - "Rejecting Vote request (preVote=false) from candidate ({}) since we already have a leader {} in epoch {}", - replicaKey, - leaderId, - epoch - ); - return false; - } else if (!isLogUpToDate) { - log.debug( - "Rejecting Vote request (preVote=false) from candidate ({}) since candidate's log is not up to date with us", - replicaKey - ); - } - - return isLogUpToDate; - } - - private boolean canGrantPreVote(ReplicaKey replicaKey, boolean isLogUpToDate) { - if (!isLogUpToDate) { - log.debug( - "Rejecting Vote request (preVote=true) from replica ({}) since prospective's log is not up to date with us", - replicaKey - ); - } - - return isLogUpToDate; + return unattachedOrProspectiveCanGrantVote( + leaderId, + votedKey, + epoch, + replicaKey, + isLogUpToDate, + isPreVote, + log + ); } @Override public String toString() { - return "Unattached(" + - "epoch=" + epoch + - ", votedKey=" + votedKey.map(ReplicaKey::toString).orElse("null") + - ", voters=" + voters + - ", electionTimeoutMs=" + electionTimeoutMs + - ", highWatermark=" + highWatermark + - ')'; + return String.format("Unattached(epoch=%d, leaderId=%s, votedKey=%s, voters=%s, " + + "electionTimeoutMs=%d, highWatermark=%s)", + epoch, + leaderId, + votedKey, + voters, + electionTimeoutMs, + highWatermark + ); } @Override diff --git a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java index 235de20f75473..84b6c4fb67cf7 100644 --- a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java @@ -51,7 +51,9 @@ private CandidateState newCandidateState(VoterSet voters) { epoch, voters, Optional.empty(), - electionTimeoutMs, + 1, + + electionTimeoutMs, logContext ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index 9a1092095ce2f..bb1a016c4fef1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -136,4 +136,4 @@ public void testLeaderIdAndEndpoint() { assertEquals(leaderId, state.leaderId()); assertEquals(leaderEndpoints, state.leaderEndpoints()); } -} \ No newline at end of file +} diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 3752e79e0dcd8..c4b4c7fec7cac 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -110,22 +110,42 @@ public void testHandlePreVoteRequestAsFollowerWithVotedCandidate(KRaftVersion kr ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); ReplicaKey votedCandidateKey = replicaKey(localId + 2, true); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey, votedCandidateKey)); RaftClientTestContext context = new RaftClientTestContext.Builder( Optional.of(localKey), - Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey, votedCandidateKey))), + Optional.of(voters), kraftVersion ) .withVotedCandidate(epoch, votedCandidateKey) .withRaftProtocol(KIP_996_PROTOCOL) .build(); + context.pollUntilRequest(); + context.assertSentFetchRequest(); + context.deliverRequest(context.beginEpochRequest(epoch, votedCandidateKey.id(), voters.listeners(votedCandidateKey.id()))); + context.pollUntilResponse(); + context.assertSentBeginQuorumEpochResponse(Errors.NONE); + assertTrue(context.client.quorum().isFollower()); + // follower can grant PreVotes if it has not fetched successfully from leader yet context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, epoch, 1)); context.pollUntilResponse(); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(votedCandidateKey.id()), true); - // follower can grant pre-votes if it has not fetched successfully from leader yet - context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - context.assertVotedCandidate(epoch, votedCandidateKey); + // after fetching from leader, follower should reject PreVote requests + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse(epoch, votedCandidateKey.id(), MemoryRecords.EMPTY, 0L, Errors.NONE) + ); + + context.client.poll(); + assertTrue(context.client.quorum().isFollower()); + context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, epoch, 1)); + context.pollUntilResponse(); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(votedCandidateKey.id()), false); } @ParameterizedTest @@ -156,7 +176,7 @@ public void testHandlePreVoteRequestAsCandidate(KRaftVersion kraftVersion) throw assertTrue(context.client.quorum().isCandidate()); // if an observer sends a pre-vote request for the same epoch, it should also be granted - context.deliverRequest(context.preVoteRequest(leaderEpoch, observer, leaderEpoch, 1)); + context.deliverRequest(context.preVoteRequest(leaderEpoch, observer, leaderEpoch, 2)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, leaderEpoch, OptionalInt.empty(), true); @@ -164,7 +184,7 @@ public void testHandlePreVoteRequestAsCandidate(KRaftVersion kraftVersion) throw assertTrue(context.client.quorum().isCandidate()); // candidate will transition to unattached if pre-vote request has a higher epoch - context.deliverRequest(context.preVoteRequest(leaderEpoch + 1, otherNodeKey, leaderEpoch + 1, 1)); + context.deliverRequest(context.preVoteRequest(leaderEpoch + 1, otherNodeKey, leaderEpoch + 1, 2)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, leaderEpoch + 1, OptionalInt.of(-1), true); @@ -211,7 +231,7 @@ public void testHandlePreVoteRequestAsUnattachedObserver(KRaftVersion kraftVersi context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); // if an observer sends a pre-vote request for the same epoch, it should also be granted - context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 1)); + context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 2)); context.pollUntilResponse(); assertTrue(context.client.quorum().isUnattached()); @@ -258,7 +278,7 @@ public void testHandlePreVoteRequestAsUnattachedVoted(KRaftVersion kraftVersion) context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); // if an observer sends a pre-vote request for the same epoch, it should also be granted - context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 1)); + context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 2)); context.pollUntilResponse(); assertTrue(context.client.quorum().isUnattached()); @@ -306,7 +326,7 @@ public void testHandlePreVoteRequestAsUnattachedWithLeader(KRaftVersion kraftVer context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), true); // if an observer sends a pre-vote request for the same epoch, it should also be granted - context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 1)); + context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 2)); context.pollUntilResponse(); assertTrue(context.client.quorum().isUnattached()); @@ -549,6 +569,7 @@ public void testInvalidPreVoteRequest() throws Exception { assertEquals(epoch, context.currentEpoch()); context.assertElectedLeader(epoch, otherNodeKey.id()); + // invalid offset context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, 0, -5L)); context.pollUntilResponse(); context.assertSentVoteResponse( @@ -560,6 +581,7 @@ public void testInvalidPreVoteRequest() throws Exception { assertEquals(epoch, context.currentEpoch()); context.assertElectedLeader(epoch, otherNodeKey.id()); + // invalid epoch context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, -1, 0L)); context.pollUntilResponse(); context.assertSentVoteResponse( @@ -571,6 +593,7 @@ public void testInvalidPreVoteRequest() throws Exception { assertEquals(epoch, context.currentEpoch()); context.assertElectedLeader(epoch, otherNodeKey.id()); + // lastEpoch > replicaEpoch context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, epoch + 1, 0L)); context.pollUntilResponse(); context.assertSentVoteResponse( @@ -605,14 +628,14 @@ public void testFollowerGrantsPreVoteIfHasNotFetchedYet(KRaftVersion kraftVersio assertTrue(context.client.quorum().isFollower()); - // We will grant PreVotes before fetching successfully from the leader, it will NOT contain the leaderId + // Follower will grant PreVotes before fetching successfully from the leader, it will NOT contain the leaderId context.deliverRequest(context.preVoteRequest(epoch, replica2, epoch, 1)); context.pollUntilResponse(); assertTrue(context.client.quorum().isFollower()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(replica1.id()), true); - // After fetching successfully from the leader once, we will no longer grant PreVotes + // After fetching successfully from the leader once, follower will no longer grant PreVotes context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); @@ -626,6 +649,7 @@ public void testFollowerGrantsPreVoteIfHasNotFetchedYet(KRaftVersion kraftVersio context.deliverRequest(context.preVoteRequest(epoch, replica2, epoch, 1)); context.pollUntilResponse(); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(replica1.id()), false); assertTrue(context.client.quorum().isFollower()); } @@ -685,7 +709,7 @@ public void testPreVoteResponseIgnoredAfterBecomingFollower(KRaftVersion kraftVe context.assertUnknownLeaderAndNoVotedCandidate(epoch); - // Sleep a little to ensure that we become a prospective + // Sleep a little to ensure transition to prospective context.time.sleep(context.electionTimeoutMs() * 2L); // Wait until the vote requests are inflight @@ -694,7 +718,7 @@ public void testPreVoteResponseIgnoredAfterBecomingFollower(KRaftVersion kraftVe List voteRequests = context.collectVoteRequests(epoch, 0, 0); assertEquals(2, voteRequests.size()); - // While the vote requests are still inflight, we receive a BeginEpoch for the same epoch + // While the vote requests are still inflight, replica receives a BeginEpoch for the same epoch context.deliverRequest(context.beginEpochRequest(epoch, voter3.id())); context.client.poll(); context.assertElectedLeader(epoch, voter3.id()); @@ -738,7 +762,7 @@ public void testPreVoteNotSupportedByRemote(KRaftVersion kraftVersion) throws Ex context.assertUnknownLeaderAndNoVotedCandidate(epoch); - // Sleep a little to ensure that we transition to Prospective + // Sleep a little to ensure transition to Prospective context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); assertEquals(epoch, context.currentEpoch()); @@ -790,7 +814,7 @@ public void testPreVoteNotSupportedByRemote(KRaftVersion kraftVersion) throws Ex assertEquals(epoch + 2, context.currentEpoch()); context.client.quorum().isCandidate(); - // Any further PreVote requests should be ignored + // Any further PreVote responses should be ignored context.deliverResponse( voteRequests.get(1).correlationId(), voteRequests.get(1).destination(), @@ -823,7 +847,7 @@ public void testProspectiveReceivesBeginQuorumRequest( context.assertUnknownLeaderAndNoVotedCandidate(epoch); - // Sleep a little to ensure that we transition to Prospective + // Sleep a little to ensure transition to prospective context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); @@ -857,39 +881,39 @@ public void testProspectiveTransitionsToUnattachedOnElectionFailure( .build(); context.assertUnknownLeaderAndNoVotedCandidate(epoch); - // Sleep a little to ensure that we transition to Prospective + // Sleep a little to ensure that transition to prospective context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); assertTrue(context.client.quorum().isProspective()); context.assertSentVoteRequest(epoch, 0, 0L, 1); - // If election timeout expires, we should transition to Unattached to attempt re-discovering leader + // If election timeout expires, replica should transition to unattached to attempt re-discovering leader context.time.sleep(context.electionTimeoutMs() * 2L); context.client.poll(); assertTrue(context.client.quorum().isUnattached()); - // After election times out again, we will transition back to Prospective and continue sending PreVote requests + // After election times out again, replica will transition back to prospective and send PreVote requests context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); RaftRequest.Outbound voteRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); - // If we receive enough rejected votes, we also transition to Unattached immediately + // If prospective receives enough rejected votes, it also transitions to unattached immediately context.deliverResponse( voteRequest.correlationId(), voteRequest.destination(), context.voteResponse(false, OptionalInt.empty(), epoch)); - // handle vote response and mark we should transition out of prospective + // handle vote response and mark rejected vote context.client.poll(); - // transition + // transition to unattached after seeing election has failed context.client.poll(); assertTrue(context.client.quorum().isUnattached()); - // After election times out again, we will transition back to Prospective and continue sending PreVote requests + // After election times out again, replica will transition back to prospective and send PreVote requests context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); voteRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); - // If we receive vote response with different leaderId but empty leader endpoint, we will transition to + // If prospective receive vote response with leaderId but empty leader endpoint, it will transition to // unattached with leader immediately context.deliverResponse( voteRequest.correlationId(), @@ -924,20 +948,20 @@ public void testProspectiveWithLeaderTransitionsToFollower( context.assertElectedLeader(epoch, replica1.id()); assertTrue(context.client.quorum().isFollower()); - // Sleep a little to ensure that we transition to Prospective + // Sleep a little to ensure transition to prospective context.time.sleep(context.fetchTimeoutMs); context.pollUntilRequest(); assertTrue(context.client.quorum().isProspective()); context.assertSentVoteRequest(epoch, 0, 0L, 2); - // If election timeout expires, we should transition back to Follower if we haven't found new leader yet + // If election timeout expires, replica should transition back to follower if it hasn't found new leader yet context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); context.assertSentFetchRequest(); assertTrue(context.client.quorum().isFollower()); context.assertElectedLeader(epoch, replica1.id()); - // After election times out again, we will transition back to Prospective and continue sending PreVote requests + // After election times out again, replica will transition back to prospective and send PreVote requests context.time.sleep(context.fetchTimeoutMs); context.pollUntilRequest(); List voteRequests = context.collectVoteRequests(epoch, 0, 0); @@ -945,7 +969,7 @@ public void testProspectiveWithLeaderTransitionsToFollower( assertTrue(context.client.quorum().isProspective()); context.assertElectedLeader(epoch, replica1.id()); - // If we receive enough rejected votes without leaderId, we also transition to Follower immediately + // If prospective receives enough rejected votes without leaderId, it also transitions to follower immediately context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), @@ -954,14 +978,14 @@ public void testProspectiveWithLeaderTransitionsToFollower( voteRequests.get(1).correlationId(), voteRequests.get(1).destination(), context.voteResponse(false, OptionalInt.empty(), epoch)); - // handle vote response and mark we should transition out of prospective + // handle vote response and mark rejected vote context.client.poll(); - // transition + // transition to follower after seeing election has failed context.pollUntilRequest(); assertTrue(context.client.quorum().isFollower()); context.assertSentFetchRequest(); - // After election times out again, we will transition back to Prospective and continue sending PreVote requests + // After election times out again, transition back to prospective and send PreVote requests context.time.sleep(context.fetchTimeoutMs); context.pollUntilRequest(); voteRequests = context.collectVoteRequests(epoch, 0, 0); @@ -969,11 +993,11 @@ public void testProspectiveWithLeaderTransitionsToFollower( assertTrue(context.client.quorum().isProspective()); context.assertElectedLeader(epoch, replica1.id()); - // If we receive vote response with different leaderId, we will transition to follower immediately + // If prospective receives vote response with different leaderId, it will transition to follower immediately context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), - context.voteResponse(false, OptionalInt.of(replica2.id()), epoch + 1)); + context.voteResponse(Errors.FENCED_LEADER_EPOCH, OptionalInt.of(replica2.id()), epoch + 1)); context.client.poll(); assertTrue(context.client.quorum().isFollower()); context.assertElectedLeader(epoch + 1, replica2.id()); @@ -1003,9 +1027,7 @@ public void testProspectiveWithoutLeaderTransitionsToFollower( // Sleep a little to ensure that we transition to Prospective context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); - assertTrue(context.client.quorum().isProspective()); - List voteRequests = context.collectVoteRequests(epoch, 0, 0); assertEquals(2, voteRequests.size()); @@ -1041,21 +1063,21 @@ public void testPreVoteRequestTimeout( .withUnknownLeader(epoch) .withRaftProtocol(raftProtocol) .build(); - context.assertUnknownLeaderAndNoVotedCandidate(epoch); context.time.sleep(context.electionTimeoutMs() * 2L); context.client.poll(); assertTrue(context.client.quorum().isProspective()); + // Simulate a request timeout context.pollUntilRequest(); - RaftRequest.Outbound request = context.assertSentVoteRequest(epoch, 0, 0L, 1); - context.time.sleep(context.requestTimeoutMs()); + + // Prospective should retry the request context.client.poll(); RaftRequest.Outbound retryRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); - // We will ignore the timed out response if it arrives late + // Ignore the timed out response if it arrives late context.deliverResponse( request.correlationId(), request.destination(), diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 53d53c0d21e20..b42974f92e512 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -1777,7 +1777,7 @@ public void testCandidateIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) thr @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testRetryElection(boolean withKip853Rpc) throws Exception { + public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 1; @@ -1794,6 +1794,12 @@ public void testRetryElection(boolean withKip853Rpc) throws Exception { context.unattachedToCandidate(); context.pollUntilRequest(); context.assertVotedCandidate(epoch, localId); + CandidateState candidate = context.client.quorum().candidateStateOrThrow(); + assertEquals(1, candidate.retries()); + assertEquals( + context.electionTimeoutMs() + exponentialFactor, + candidate.remainingElectionTimeMs(context.time.milliseconds())); + assertFalse(candidate.isBackingOff()); // Quorum size is two. If the other member rejects, then we need to schedule a revote. RaftRequest.Outbound request = context.assertSentVoteRequest(epoch, 0, 0L, 1); @@ -1804,6 +1810,10 @@ public void testRetryElection(boolean withKip853Rpc) throws Exception { ); context.client.poll(); + assertTrue(candidate.isBackingOff()); + assertEquals( + context.electionBackoffMaxMs, + candidate.remainingBackoffMs(context.time.milliseconds())); // All nodes have rejected our candidacy, but we should still remember that we had voted context.assertVotedCandidate(epoch, localId); @@ -1818,8 +1828,67 @@ public void testRetryElection(boolean withKip853Rpc) throws Exception { context.time.sleep(1); context.client.poll(); assertTrue(context.client.quorum().isProspective()); + ProspectiveState prospective = context.client.quorum().prospectiveStateOrThrow(); + assertEquals(2, prospective.retries()); + context.pollUntilRequest(); + request = context.assertSentVoteRequest(epoch, 0, 0L, 1); + assertEquals( + context.electionTimeoutMs() + exponentialFactor, + prospective.remainingElectionTimeMs(context.time.milliseconds()) + ); + + // If we become candidate again, retries should be 2 + context.deliverResponse( + request.correlationId(), + request.destination(), + context.voteResponse(true, OptionalInt.empty(), 1) + ); + context.client.poll(); + context.assertVotedCandidate(epoch + 1, localId); + candidate = context.client.quorum().candidateStateOrThrow(); + assertEquals(2, candidate.retries()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testCandidateElectionTimeout(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + int otherNodeId = localId + 1; + int epoch = 1; + int jitter = 100; + Set voters = Set.of(localId, otherNodeId); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .updateRandom(r -> r.mockNextInt(jitter)) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.assertUnknownLeaderAndNoVotedCandidate(0); + + context.unattachedToCandidate(); + context.pollUntilRequest(); + context.assertVotedCandidate(epoch, localId); + context.assertSentVoteRequest(epoch, 0, 0L, 1); + CandidateState candidate = context.client.quorum().candidateStateOrThrow(); + assertEquals(1, candidate.retries()); + assertEquals( + context.electionTimeoutMs() + jitter, + candidate.remainingElectionTimeMs(context.time.milliseconds())); + assertFalse(candidate.isBackingOff()); + + // If election times out, we transition to prospective without any additional backoff + context.time.sleep(candidate.remainingElectionTimeMs(context.time.milliseconds())); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + + ProspectiveState prospective = context.client.quorum().prospectiveStateOrThrow(); + assertEquals(2, prospective.retries()); context.pollUntilRequest(); context.assertSentVoteRequest(epoch, 0, 0L, 1); + assertEquals( + context.electionTimeoutMs() + jitter, + prospective.remainingElectionTimeMs(context.time.milliseconds()) + ); } @ParameterizedTest diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java index 49f79db8616f7..98dbe08fefc76 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -71,6 +71,7 @@ private ProspectiveState newProspectiveState( votedKey, voters, Optional.empty(), + 1, electionTimeoutMs, logContext ); @@ -86,6 +87,7 @@ private ProspectiveState newProspectiveState(VoterSet voters) { Optional.empty(), voters, Optional.empty(), + 1, electionTimeoutMs, logContext ); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 314da6b1e723c..c41a2f7be0680 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -1595,16 +1595,20 @@ VoteRequestData voteRequest( } VoteResponseData voteResponse(boolean voteGranted, OptionalInt leaderId, int epoch) { - return voteResponse(voteGranted, leaderId, epoch, voteRpcVersion()); + return voteResponse(Errors.NONE, voteGranted, leaderId, epoch, voteRpcVersion()); } - VoteResponseData voteResponse(boolean voteGranted, OptionalInt leaderId, int epoch, short version) { + VoteResponseData voteResponse(Errors error, OptionalInt leaderId, int epoch) { + return voteResponse(error, false, leaderId, epoch, voteRpcVersion()); + } + + VoteResponseData voteResponse(Errors error, boolean voteGranted, OptionalInt leaderId, int epoch, short version) { return RaftUtil.singletonVoteResponse( channel.listenerName(), version, Errors.NONE, metadataPartition, - Errors.NONE, + error, epoch, leaderId.orElse(-1), voteGranted, diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index 689249839994b..8ed124f795e54 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -140,7 +140,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1, getMetric(metrics, "high-watermark").metricValue()); // prospective @@ -152,7 +152,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); // prospective with votedKey @@ -164,7 +164,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); // follower with votedKey and leader @@ -176,7 +176,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); // follower with updated HW @@ -192,7 +192,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 0L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 0, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); // candidate @@ -204,7 +204,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { localDirectoryId.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 1L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 1, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) 5L, getMetric(metrics, "high-watermark").metricValue()); // leader @@ -217,7 +217,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { localDirectoryId.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 1L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 1, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); // todo, bug fix // leader with updated HW @@ -238,7 +238,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 2L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 2, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); // unattached with votedKey @@ -251,7 +251,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { voter2DirectoryId.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 3L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 3, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); // unattached with leader without votedKey @@ -263,7 +263,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { Uuid.ZERO_UUID.toString(), getMetric(metrics, "current-vote-directory-id").metricValue() ); - assertEquals((double) 4L, getMetric(metrics, "current-epoch").metricValue()); + assertEquals((double) 4, getMetric(metrics, "current-epoch").metricValue()); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); } From 3268784333412268980150ae5ada012492d15d49 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Sat, 28 Dec 2024 23:48:46 -0800 Subject: [PATCH 15/23] addressing comments and non-simulation test failures --- .../org/apache/kafka/raft/CandidateState.java | 53 ++++---- .../org/apache/kafka/raft/FollowerState.java | 4 - .../apache/kafka/raft/KafkaRaftClient.java | 121 +++++++++++------- .../apache/kafka/raft/ProspectiveState.java | 6 +- .../org/apache/kafka/raft/QuorumState.java | 77 ++++++----- .../apache/kafka/raft/UnattachedState.java | 3 +- .../kafka/raft/internals/EpochElection.java | 30 ++++- .../raft/KafkaRaftClientPreVoteTest.java | 16 +-- .../raft/KafkaRaftClientSnapshotTest.java | 16 +-- .../kafka/raft/KafkaRaftClientTest.java | 14 +- .../apache/kafka/raft/QuorumStateTest.java | 70 +++++----- .../kafka/raft/RaftClientTestContext.java | 25 +++- .../raft/internals/KafkaRaftMetricsTest.java | 4 +- 13 files changed, 257 insertions(+), 182 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index d0bbb696c9ced..8541be42fcf24 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -88,10 +88,6 @@ public int localId() { return localId; } - public int retries() { - return retries; - } - /** * Check if the candidate is backing off for the next election */ @@ -99,26 +95,8 @@ public boolean isBackingOff() { return isBackingOff; } - /** - * Record the current election has failed since we've either received sufficient rejecting voters or election timed out - */ - public void startBackingOff(long currentTimeMs, long backoffDurationMs) { - this.backoffTimer.update(currentTimeMs); - this.backoffTimer.reset(backoffDurationMs); - this.isBackingOff = true; - } - - public boolean isBackoffComplete(long currentTimeMs) { - backoffTimer.update(currentTimeMs); - return backoffTimer.isExpired(); - } - - public long remainingBackoffMs(long currentTimeMs) { - if (!isBackingOff) { - throw new IllegalStateException("Candidate is not currently backing off"); - } - backoffTimer.update(currentTimeMs); - return backoffTimer.remainingMs(); + public int retries() { + return retries; } @Override @@ -144,12 +122,34 @@ public boolean recordRejectedVote(int remoteNodeId) { return epochElection().recordVote(remoteNodeId, false); } + /** + * Record the current election has failed since we've either received sufficient rejecting voters or election timed out + */ + public void startBackingOff(long currentTimeMs, long backoffDurationMs) { + this.backoffTimer.update(currentTimeMs); + this.backoffTimer.reset(backoffDurationMs); + this.isBackingOff = true; + } + @Override public boolean hasElectionTimeoutExpired(long currentTimeMs) { electionTimer.update(currentTimeMs); return electionTimer.isExpired(); } + public boolean isBackoffComplete(long currentTimeMs) { + backoffTimer.update(currentTimeMs); + return backoffTimer.isExpired(); + } + + public long remainingBackoffMs(long currentTimeMs) { + if (!isBackingOff) { + throw new IllegalStateException("Candidate is not currently backing off"); + } + backoffTimer.update(currentTimeMs); + return backoffTimer.remainingMs(); + } + @Override public long remainingElectionTimeMs(long currentTimeMs) { electionTimer.update(currentTimeMs); @@ -205,12 +205,13 @@ public boolean canGrantVote( @Override public String toString() { return String.format( - "Candidate(localId=%d, localDirectoryId=%s, epoch=%d, voteStates=%s, " + + "CandidateState(localId=%d, localDirectoryId=%s, epoch=%d, retries=%d, epochElection=%s, " + "highWatermark=%s, electionTimeoutMs=%d)", localId, localDirectoryId, epoch, - epochElection().voterStates(), + retries, + epochElection(), highWatermark, electionTimeoutMs ); diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index 8f84b554f66ef..0a6a48b46d63a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -122,10 +122,6 @@ public Node leaderNode(ListenerName listener) { ); } - public Optional votedKey() { - return votedKey; - } - public boolean hasFetchTimeoutExpired(long currentTimeMs) { fetchTimer.update(currentTimeMs); return fetchTimer.isExpired(); diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 323c092eb15a7..9b6bf4ef07d3a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -720,10 +720,6 @@ private void transitionToProspective(long currentTimeMs) { onBecomeProspective(currentTimeMs); } - private void transitionToUnattached(int epoch) { - transitionToUnattached(epoch, OptionalInt.empty()); - } - private void transitionToUnattached(int epoch, OptionalInt leaderId) { quorum.transitionToUnattached(epoch, leaderId); maybeFireLeaderChange(); @@ -865,7 +861,7 @@ private VoteResponseData handleVoteRequest( } if (replicaEpoch > quorum.epoch()) { - transitionToUnattached(replicaEpoch); + transitionToUnattached(replicaEpoch, OptionalInt.empty()); } // Check that the request was intended for this replica @@ -927,14 +923,15 @@ private boolean handleVoteResponse( int remoteNodeId = responseMetadata.source().id(); VoteResponseData response = (VoteResponseData) responseMetadata.data(); Errors topLevelError = Errors.forCode(response.errorCode()); - if (topLevelError != Errors.NONE) { - if (topLevelError == Errors.UNSUPPORTED_VERSION && quorum.isProspective()) { - logger.info("Prospective received unsupported version error in vote response in epoch {}, " + - "transitioning to Candidate state immediately since entire quorum may not support PreVote.", - quorum.epoch()); - transitionToCandidate(currentTimeMs); - return true; - } + if (topLevelError == Errors.UNSUPPORTED_VERSION && quorum.isProspective()) { + logger.info( + "Prospective received unsupported version error in vote response in epoch {}, " + + "transitioning to Candidate state immediately since entire quorum may not support PreVote.", + quorum.epoch() + ); + transitionToCandidate(currentTimeMs); + return true; + } else if (topLevelError != Errors.NONE) { return handleTopLevelError(topLevelError, responseMetadata); } @@ -985,12 +982,15 @@ private boolean handleVoteResponse( maybeTransitionForward(state, currentTimeMs); } else { state.recordRejectedVote(remoteNodeId); - maybeCandidateStartBackingOff(currentTimeMs); + maybeHandleElectionLoss(currentTimeMs); } } else { - logger.debug("Ignoring vote response {} since we are no longer a VotingState " + - "(Prospective or Candidate) in epoch {}", - partitionResponse, quorum.epoch()); + logger.debug( + "Ignoring vote response {} since we are no longer a NomineeState " + + "(Prospective or Candidate) in epoch {}", + partitionResponse, + quorum.epoch() + ); } return true; } else { @@ -998,23 +998,38 @@ private boolean handleVoteResponse( } } - private void maybeCandidateStartBackingOff(long currentTimeMs) { - // If in candidate state and vote is rejected, go immediately to a random, exponential backoff. The - // backoff starts low to prevent needing to wait the entire election timeout when the vote - // result has already been determined. The randomness prevents the next election from being - // gridlocked with another nominee due to timing. The exponential aspect limits epoch churn when - // the replica has failed multiple elections in succession. + /** + * On election loss, if replica is prospective it will transition to unattached or follower state. + * If replica is candidate, it will start backing off. + */ + private void maybeHandleElectionLoss(long currentTimeMs) { if (quorum.isCandidate()) { CandidateState candidate = quorum.candidateStateOrThrow(); if (candidate.epochElection().isVoteRejected() && !candidate.isBackingOff()) { - logger.info("Insufficient remaining votes to become leader (rejected by {}). " + - "We will backoff before retrying election again", candidate.epochElection().rejectingVoters()); - + logger.info( + "Insufficient remaining votes to become leader (rejected by {}). " + + "We will backoff before retrying election again", + candidate.epochElection().rejectingVoters() + ); + // Go immediately to a random, exponential backoff. The backoff starts low to prevent + // needing to wait the entire election timeout when the vote result has already been + // determined. The randomness prevents the next election from being gridlocked with + // another nominee due to timing. The exponential aspect limits epoch churn when the + // replica has failed multiple elections in succession. candidate.startBackingOff( currentTimeMs, binaryExponentialElectionBackoffMs(candidate.retries()) ); } + } else if (quorum.isProspective()) { + ProspectiveState prospective = quorum.prospectiveStateOrThrow(); + if (prospective.epochElection().isVoteRejected()) { + logger.info( + "Insufficient remaining votes to become candidate (rejected by {}). ", + prospective.epochElection().rejectingVoters() + ); + prospectiveTransitionAfterElectionLoss(prospective, currentTimeMs); + } } } @@ -1023,8 +1038,10 @@ private int binaryExponentialElectionBackoffMs(int retries) { throw new IllegalArgumentException("Retries " + retries + " should be larger than zero"); } // upper limit exponential co-efficients at 20 to avoid overflow - return Math.min(RETRY_BACKOFF_BASE_MS * random.nextInt(2 << Math.min(20, retries - 1)), - quorumConfig.electionBackoffMaxMs()); + return Math.min( + RETRY_BACKOFF_BASE_MS * random.nextInt(2 << Math.min(20, retries - 1)), + quorumConfig.electionBackoffMaxMs() + ); } private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) { @@ -2503,10 +2520,10 @@ private void maybeTransition( " and epoch " + epoch + " which is inconsistent with current leader " + quorum.leaderId() + " and epoch " + quorum.epoch()); } else if (epoch > quorum.epoch()) { - if (leaderId.isPresent() && !leaderEndpoints.isEmpty()) { + if (leaderId.isPresent()) { transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs); } else { - transitionToUnattached(epoch, leaderId); + transitionToUnattached(epoch, OptionalInt.empty()); } } else if ( leaderId.isPresent() && @@ -2811,9 +2828,9 @@ private BeginQuorumEpochRequestData buildBeginQuorumEpochRequest(ReplicaKey remo ); } - private VoteRequestData buildVoteRequest(ReplicaKey remoteVoter) { + private VoteRequestData buildVoteRequest(ReplicaKey remoteVoter, boolean preVote) { OffsetAndEpoch endOffset = endOffset(); - boolean isPreVote = quorum.isProspective(); +// boolean isPreVote = quorum.isProspective(); return RaftUtil.singletonVoteRequest( log.topicPartition(), clusterId, @@ -2822,7 +2839,7 @@ private VoteRequestData buildVoteRequest(ReplicaKey remoteVoter) { remoteVoter, endOffset.epoch(), endOffset.offset(), - isPreVote + preVote ); } @@ -3003,7 +3020,7 @@ private long pollResigned(long currentTimeMs) { // It is possible that the old leader is not a voter in the new voter set. // In that case increase the epoch and transition to unattached. The epoch needs // to be increased to avoid FETCH responses with the leader being this replica. - transitionToUnattached(quorum.epoch() + 1); + transitionToUnattached(quorum.epoch() + 1, OptionalInt.empty()); } stateTimeoutMs = 0L; } else { @@ -3054,6 +3071,7 @@ private long maybeSendVoteRequests( // Continue sending Vote requests as long as we still have a chance to win the election if (!state.epochElection().isVoteRejected()) { VoterSet voters = partitionState.lastVoterSet(); + boolean preVote = quorum.isProspective(); return maybeSendRequest( currentTimeMs, state.epochElection().unrecordedVoters(), @@ -3068,7 +3086,7 @@ private long maybeSendVoteRequests( ) ) ), - this::buildVoteRequest + voterId -> buildVoteRequest(voterId, preVote) ); } return Long.MAX_VALUE; @@ -3110,20 +3128,9 @@ private long pollProspective(long currentTimeMs) { if (shutdown != null) { long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); - } else if (state.epochElection().isVoteRejected() || state.hasElectionTimeoutExpired(currentTimeMs)) { - if (state.election().hasLeader() && !state.leaderEndpoints().isEmpty()) { - logger.info( - "Election was not granted, transitioning to Follower of leader {}", - state.election().leaderId()); - transitionToFollower( - quorum().epoch(), - state.election().leaderId(), - state.leaderEndpoints(), - currentTimeMs); - } else { - logger.info("Election was not granted, transitioning to Unattached to attempt rediscovering leader"); - transitionToUnattached(quorum().epoch(), state.election().optionalLeaderId()); - } + } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { + logger.info("Election timed out before receiving sufficient vote responses to become candidate"); + prospectiveTransitionAfterElectionLoss(state, currentTimeMs); return 0L; } else { long minVoteRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); @@ -3131,6 +3138,22 @@ private long pollProspective(long currentTimeMs) { } } + private void prospectiveTransitionAfterElectionLoss(ProspectiveState prospective, long currentTimeMs) { + if (prospective.election().hasLeader() && !prospective.leaderEndpoints().isEmpty()) { + logger.info( + "Transitioning to Follower of leader {}", + prospective.election().leaderId()); + transitionToFollower( + quorum().epoch(), + prospective.election().leaderId(), + prospective.leaderEndpoints(), + currentTimeMs); + } else { + logger.info("Transitioning to Unattached to attempt rediscovering leader"); + transitionToUnattached(quorum().epoch(), prospective.election().optionalLeaderId()); + } + } + private long pollFollower(long currentTimeMs) { FollowerState state = quorum.followerStateOrThrow(); if (quorum.isVoter()) { diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index 7fc03ef71fa2b..59343be013053 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -158,11 +158,13 @@ public Optional highWatermark() { @Override public String toString() { return String.format( - "Prospective(epoch=%d, leaderId=%s, votedKey=%s, voters=%s, electionTimeoutMs=%s, highWatermark=%s)", + "ProspectiveState(epoch=%d, leaderId=%s, retries=%d, votedKey=%s, epochElection=%s, " + + "electionTimeoutMs=%s, highWatermark=%s)", epoch, leaderId, + retries, votedKey, - voters, + epochElection, electionTimeoutMs, highWatermark ); diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 4909b5aac2b81..d2cefd399648f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -369,16 +369,11 @@ public void transitionToResigned(List preferredSuccessors) { } /** - * Transition to the "unattached" state. This means we have found an epoch greater than the current epoch - * and do not yet know of the elected leader, or we have transitioned from Prospective with the same epoch. - * Note, if we are transitioning from unattached and there is no epoch change, we take the path of - * unattachedAddVotedState instead. + * Transition to the "unattached" state. This means the replica has found an epoch greater than the current epoch, + * or the replica has transitioned from Prospective with the same epoch. + * Note, if the replica is transitioning from unattached to add voted state and there is no epoch change, + * it takes the route of unattachedAddVotedState instead. */ - // Used in testing - public void transitionToUnattached(int epoch) { - transitionToUnattached(epoch, OptionalInt.empty()); - } - public void transitionToUnattached(int epoch, OptionalInt leaderId) { int currentEpoch = state.epoch(); if (epoch < currentEpoch || (epoch == currentEpoch && !isProspective())) { @@ -404,11 +399,14 @@ public void transitionToUnattached(int epoch, OptionalInt leaderId) { electionTimeoutMs = randomElectionTimeoutMs(); } + // If the local replica is transitioning to Unattached in the same epoch (i.e. from Prospective), it + // should retain its voted key if it exists, so that it will not vote again in the same epoch. + Optional votedKey = epoch == currentEpoch ? votedKey() : Optional.empty(); durableTransitionTo(new UnattachedState( time, epoch, leaderId, - epoch == currentEpoch ? votedKey() : Optional.empty(), + votedKey, partitionState.lastVoterSet().voterIds(), state.highWatermark(), electionTimeoutMs, @@ -417,8 +415,10 @@ public void transitionToUnattached(int epoch, OptionalInt leaderId) { } /** - * Grant a vote to a candidate as Unattached. We will transition to Unattached with votedKey - * state and remain there until either the election timeout expires or we discover the leader. + * Grant a vote to a candidate as Unattached. The replica will transition to Unattached with votedKey + * state in the same epoch and remain there until either the election timeout expires or it discovers the leader. + * Note, if the replica discovers a higher epoch or is transitioning from Prospective, it takes + * the route of transitionToUnattached instead. */ public void unattachedAddVotedState( int epoch, @@ -466,8 +466,9 @@ public void unattachedAddVotedState( } /** - * Grant a vote to a candidate as Prospective. We will transition to Prospective with votedKey - * state and remain there until either the election timeout expires or we discover the leader. + * Grant a vote to a candidate as Prospective. The replica will transition to Prospective with votedKey + * state in the same epoch. Note, if the replica is transitioning to Prospective due to a fetch/election timeout + * or loss of election as candidate, it takes the route of transitionToProspective instead. */ public void prospectiveAddVotedState( int epoch, @@ -478,7 +479,7 @@ public void prospectiveAddVotedState( throw new IllegalStateException( String.format( "Cannot add voted key (%s) to current state (%s) in epoch %d since it matches the local " + - "broker.id", + "broker.id", candidateKey, state, epoch @@ -589,12 +590,17 @@ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { ); } + /** + * Transition to the "prospective" state. This means the replica experienced a fetch/election timeout or + * loss of election as candidate. Note, if the replica is transitioning from prospective to add voted state + * and there is no epoch change, it takes the route of prospectiveAddVotedState instead. + */ public void transitionToProspective() { if (isObserver()) { throw new IllegalStateException( String.format( "Cannot transition to Prospective since the local id (%s) and directory id (%s) " + - "is not one of the voters %s", + "is not one of the voters %s", localId, localDirectoryId, partitionState.lastVoterSet() @@ -607,19 +613,21 @@ public void transitionToProspective() { int retries = isCandidate() ? candidateStateOrThrow().retries() + 1 : 1; - durableTransitionTo(new ProspectiveState( - time, - localIdOrThrow(), - epoch(), - leaderId(), - Optional.of(state.leaderEndpoints()), - votedKey(), - partitionState.lastVoterSet(), - state.highWatermark(), - retries, - randomElectionTimeoutMs(), - logContext - )); + durableTransitionTo( + new ProspectiveState( + time, + localIdOrThrow(), + epoch(), + leaderId(), + Optional.of(state.leaderEndpoints()), + votedKey(), + partitionState.lastVoterSet(), + state.highWatermark(), + retries, + randomElectionTimeoutMs(), + logContext + ) + ); } public void transitionToCandidate() { @@ -861,6 +869,17 @@ public boolean isNomineeState() { return state instanceof NomineeState; } + /** + * Determines if replica in unattached or prospective state can grant a vote request. + * @param leaderId local replica's optional leader id. + * @param votedKey local replica's optional voted key. + * @param epoch local replica's epoch + * @param replicaKey replicaKey of nominee which sent the vote request + * @param isLogUpToDate whether the log of the nominee is up-to-date with the local replica's log + * @param isPreVote whether the vote request is a PreVote request + * @param log logger + * @return true if the local replica can grant the vote request, false otherwise + */ public static boolean unattachedOrProspectiveCanGrantVote( OptionalInt leaderId, Optional votedKey, diff --git a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java index 794b3b91f5f3d..9d9ac30213cc3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java @@ -129,7 +129,8 @@ public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolea @Override public String toString() { - return String.format("Unattached(epoch=%d, leaderId=%s, votedKey=%s, voters=%s, " + + return String.format( + "UnattachedState(epoch=%d, leaderId=%s, votedKey=%s, voters=%s, " + "electionTimeoutMs=%d, highWatermark=%s)", epoch, leaderId, diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java index 19cf7715b4920..a837bfe90da7d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java @@ -33,10 +33,12 @@ public class EpochElection { public EpochElection(Set voters) { this.voterStates = voters.stream() - .collect(Collectors.toMap( - ReplicaKey::id, - VoterState::new - )); + .collect( + Collectors.toMap( + ReplicaKey::id, + VoterState::new + ) + ); } /** @@ -165,6 +167,17 @@ private int majoritySize() { return voterStates.size() / 2 + 1; } + @Override + public String toString() { + return String.format( + "EpochElection(%s)", + voterStates.values().stream() + .map(VoterState::toString) + .collect( + Collectors.joining(", ")) + ); + } + private static final class VoterState { private final ReplicaKey replicaKey; private State state = State.UNRECORDED; @@ -190,5 +203,14 @@ enum State { GRANTED, REJECTED } + + @Override + public String toString() { + return String.format( + "VoterState(%s, state=%s)", + replicaKey, + state + ); + } } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index c4b4c7fec7cac..6669b2b032c8d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -902,27 +902,13 @@ public void testProspectiveTransitionsToUnattachedOnElectionFailure( voteRequest.correlationId(), voteRequest.destination(), context.voteResponse(false, OptionalInt.empty(), epoch)); - // handle vote response and mark rejected vote - context.client.poll(); - // transition to unattached after seeing election has failed context.client.poll(); assertTrue(context.client.quorum().isUnattached()); // After election times out again, replica will transition back to prospective and send PreVote requests context.time.sleep(context.electionTimeoutMs() * 2L); context.pollUntilRequest(); - voteRequest = context.assertSentVoteRequest(epoch, 0, 0L, 1); - - // If prospective receive vote response with leaderId but empty leader endpoint, it will transition to - // unattached with leader immediately - context.deliverResponse( - voteRequest.correlationId(), - voteRequest.destination(), - context.voteResponse(false, OptionalInt.of(localId + 2), epoch + 1)); - context.client.poll(); - assertTrue(context.client.quorum().isUnattached()); - assertEquals(epoch + 1, context.currentEpoch()); - context.assertElectedLeader(epoch + 1, localId + 2); + context.assertSentVoteRequest(epoch, 0, 0L, 1); } @ParameterizedTest diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 66b83b0035f89..ae2ab225b1ffa 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -1182,13 +1182,13 @@ public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); - // Fetch timer is not reset; sleeping for remainder should transition to candidate + // Fetch timer is not reset; sleeping for remainder should transition to prospective context.time.sleep(context.fetchTimeoutMs - slept); context.pollUntilRequest(); - context.assertSentVoteRequest(epoch + 1, 0, 0L, 1); - context.assertVotedCandidate(epoch + 1, localId); + context.assertSentVoteRequest(epoch, 0, 0L, 1); + assertTrue(context.client.quorum().isProspective()); } @ParameterizedTest @@ -1777,13 +1777,13 @@ public void testFetchSnapshotResponseToNotFollower(boolean withKip853Rpc) throws assertEquals(snapshotId.epoch(), request.snapshotId().epoch()); assertEquals(0, request.position()); - // Sleeping for fetch timeout should transition to candidate + // Sleeping for fetch timeout should transition to prospective context.time.sleep(context.fetchTimeoutMs); context.pollUntilRequest(); - context.assertSentVoteRequest(epoch + 1, 0, 0L, 1); - context.assertVotedCandidate(epoch + 1, localId); + context.assertSentVoteRequest(epoch, 0, 0L, 1); + assertTrue(context.client.quorum().isProspective()); // Send the response late context.deliverResponse( @@ -1809,9 +1809,9 @@ public void testFetchSnapshotResponseToNotFollower(boolean withKip853Rpc) throws ) ); - // Assert that the response is ignored and the replicas stays as a candidate + // Assert that the response is ignored and the replicas stays as a prospective context.client.poll(); - context.assertVotedCandidate(epoch + 1, localId); + assertTrue(context.client.quorum().isProspective()); } @ParameterizedTest diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index b42974f92e512..2838286999dab 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -972,7 +972,7 @@ public void testHandleBeginQuorumRequest(boolean withKip853Rpc) throws Exception context.deliverRequest(context.beginEpochRequest(votedCandidateEpoch, otherNodeKey.id())); context.pollUntilResponse(); - context.assertElectedLeader(votedCandidateEpoch, otherNodeKey.id()); + context.assertElectedLeaderAndVotedCandidate(votedCandidateEpoch, otherNodeKey.id(), otherNodeKey); context.assertSentBeginQuorumEpochResponse( Errors.NONE, @@ -2910,7 +2910,11 @@ public void testVoteResponseIgnoredAfterBecomingFollower(RaftProtocol raftProtoc // While the vote requests are still inflight, we receive a BeginEpoch for the same epoch context.deliverRequest(context.beginEpochRequest(epoch + 1, voter3)); context.client.poll(); - context.assertElectedLeader(epoch + 1, voter3); + context.assertElectedLeaderAndVotedCandidate( + epoch + 1, + voter3, + ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID) + ); // The vote requests now return and should be ignored voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch + 1); @@ -2928,7 +2932,11 @@ public void testVoteResponseIgnoredAfterBecomingFollower(RaftProtocol raftProtoc ); context.client.poll(); - context.assertElectedLeader(epoch + 1, voter3); + context.assertElectedLeaderAndVotedCandidate( + epoch + 1, + voter3, + ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID) + ); } @ParameterizedTest diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index bbc226d78e98c..af5e4011ad005 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -173,7 +173,7 @@ public void testHasRemoteLeader(KRaftVersion kraftVersion) { state.transitionToLeader(0L, accumulator); assertFalse(state.hasRemoteLeader()); - state.transitionToUnattached(state.epoch() + 1); + state.transitionToUnattached(state.epoch() + 1, OptionalInt.empty()); assertFalse(state.hasRemoteLeader()); state.unattachedAddVotedState(state.epoch(), otherNodeKey); @@ -206,7 +206,7 @@ public void testHighWatermarkRetained(KRaftVersion kraftVersion) { Optional highWatermark = Optional.of(new LogOffsetMetadata(10L)); assertEquals(highWatermark, state.highWatermark()); - state.transitionToUnattached(6); + state.transitionToUnattached(6, OptionalInt.empty()); assertEquals(highWatermark, state.highWatermark()); state.unattachedAddVotedState(6, otherNodeKey); @@ -630,7 +630,7 @@ public void testInitializeWithEmptyLocalId(KRaftVersion kraftVersion) { state.transitionToFollower(1, 1, voters.listeners(1)); assertTrue(state.isFollower()); - state.transitionToUnattached(2); + state.transitionToUnattached(2, OptionalInt.empty()); assertTrue(state.isUnattached()); } @@ -667,17 +667,17 @@ public void testUnattachedToUnattached(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); assertTrue(state.isUnattachedNotVoted()); long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); time.sleep(1000); // cannot transition to unattached in same epoch - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), OptionalInt.empty())); // can transition to unattached in higher epoch - state.transitionToUnattached(6); + state.transitionToUnattached(6, OptionalInt.empty()); assertTrue(state.isUnattachedNotVoted()); UnattachedState unattachedState = state.unattachedStateOrThrow(); assertEquals(6, unattachedState.epoch()); @@ -703,7 +703,7 @@ public void testUnattachedToUnattachedVotedSameEpoch(KRaftVersion kraftVersion) VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); @@ -736,7 +736,7 @@ public void testUnattachedToUnattachedVotedHigherEpoch(KRaftVersion kraftVersion VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); assertTrue(state.isUnattachedNotVoted()); assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, otherNodeKey)); @@ -749,7 +749,7 @@ public void testUnattachedToFollowerSameEpoch(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); state.transitionToFollower( 5, @@ -773,7 +773,7 @@ public void testUnattachedToFollowerHigherEpoch(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); state.transitionToFollower( 8, @@ -825,7 +825,7 @@ public void testUnattachedToCandidate(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); } @@ -853,8 +853,8 @@ public void testUnattachedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + state.transitionToUnattached(5, OptionalInt.empty()); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); assertThrows( IllegalStateException.class, @@ -888,7 +888,7 @@ public void testUnattachedVotedToUnattachedSameEpoch(KRaftVersion kraftVersion) QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(logEndEpoch)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(logEndEpoch, OptionalInt.empty())); } @ParameterizedTest @@ -903,7 +903,7 @@ public void testUnattachedVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); time.sleep(1000); - state.transitionToUnattached(6); + state.transitionToUnattached(6, OptionalInt.empty()); UnattachedState unattachedState = state.unattachedStateOrThrow(); assertEquals(6, unattachedState.epoch()); @@ -1057,7 +1057,7 @@ public void testUnattachedVotedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, 5)); state.unattachedAddVotedState(5, otherNodeKey); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); assertThrows( IllegalStateException.class, @@ -1166,7 +1166,7 @@ public void testFollowerToUnattachedSameEpoch(KRaftVersion kraftVersion) { node2, voters.listeners(node2) ); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(8, OptionalInt.empty())); assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(8, votedKey)); } @@ -1186,7 +1186,7 @@ public void testFollowerToUnattachedHigherEpoch(KRaftVersion kraftVersion) { int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToUnattached(9); + state.transitionToUnattached(9, OptionalInt.empty()); assertTrue(state.isUnattached()); UnattachedState unattachedState = state.unattachedStateOrThrow(); assertEquals(9, unattachedState.epoch()); @@ -1407,7 +1407,7 @@ public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { otherNodeId, voters.listeners(otherNodeId) ); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -1453,7 +1453,7 @@ public void testFollowerVotedToUnattachedSameEpoch(KRaftVersion kraftVersion) { assertThrows( IllegalStateException.class, - () -> state.transitionToUnattached(0) + () -> state.transitionToUnattached(0, OptionalInt.empty()) ); } @@ -1477,7 +1477,7 @@ public void testFollowerVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion) int jitterMs = 2500; random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToUnattached(10); + state.transitionToUnattached(10, OptionalInt.empty()); assertTrue(state.isUnattachedNotVoted()); UnattachedState unattached = state.unattachedStateOrThrow(); @@ -1576,7 +1576,7 @@ public void testProspectiveToUnattachedInSameEpoch(KRaftVersion kraftVersion) { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.transitionToUnattached(state.epoch()); + state.transitionToUnattached(state.epoch(), OptionalInt.empty()); assertEquals( ElectionState.withUnknownLeader( logEndEpoch, @@ -1596,7 +1596,7 @@ public void testProspectiveToUnattachedInHigherEpoch(KRaftVersion kraftVersion) state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); - state.transitionToUnattached(20); + state.transitionToUnattached(20, OptionalInt.empty()); assertEquals( ElectionState.withUnknownLeader( 20, @@ -1770,7 +1770,7 @@ public void testProspectiveVotedToUnattachedInSameEpoch(KRaftVersion kraftVersio state.transitionToProspective(); state.prospectiveAddVotedState(logEndEpoch, voter1); - state.transitionToUnattached(logEndEpoch); + state.transitionToUnattached(logEndEpoch, OptionalInt.empty()); assertEquals( ElectionState.withVotedCandidate( logEndEpoch, @@ -1919,7 +1919,7 @@ public void testProspectiveWithLeaderToUnattachedInHigherEpoch(KRaftVersion kraf ); // transition to unattached - state.transitionToUnattached(10); + state.transitionToUnattached(10, OptionalInt.empty()); assertEquals( ElectionState.withUnknownLeader( 10, @@ -2073,7 +2073,7 @@ public void testCandidateToUnattachedSameEpoch(KRaftVersion kraftVersion) { state.transitionToProspective(); state.transitionToCandidate(); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), OptionalInt.empty())); } @ParameterizedTest @@ -2086,7 +2086,7 @@ public void testCandidateToUnattachedHigherEpoch(KRaftVersion kraftVersion) { state.transitionToProspective(); state.transitionToCandidate(); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); assertEquals(5, state.epoch()); assertEquals(OptionalInt.empty(), state.leaderId()); assertEquals( @@ -2274,10 +2274,10 @@ public void testCandidateToAnyStateLowerEpoch(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); state.transitionToProspective(); state.transitionToCandidate(); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -2313,7 +2313,7 @@ public void testLeaderToUnattachedSameEpoch(KRaftVersion kraftVersion) { state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); state.transitionToLeader(0L, accumulator); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch())); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), OptionalInt.empty())); } @ParameterizedTest @@ -2327,7 +2327,7 @@ public void testLeaderToUnattachedHigherEpoch(KRaftVersion kraftVersion) { state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); state.transitionToLeader(0L, accumulator); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); assertEquals(5, state.epoch()); assertEquals(OptionalInt.empty(), state.leaderId()); assertEquals( @@ -2476,12 +2476,12 @@ public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, otherNodeKey)); QuorumState state = initializeEmptyState(voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToUnattached(5); + state.transitionToUnattached(5, OptionalInt.empty()); state.transitionToProspective(); state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); state.transitionToLeader(0L, accumulator); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -2538,7 +2538,7 @@ public void testObserverUnattachedToFollower(KRaftVersion kraftVersion) { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isObserver()); - state.transitionToUnattached(2); + state.transitionToUnattached(2, OptionalInt.empty()); state.transitionToFollower(3, node1, voters.listeners(node1)); assertTrue(state.isFollower()); FollowerState followerState = state.followerStateOrThrow(); @@ -2613,7 +2613,7 @@ public void testObserverFollowerToUnattached(KRaftVersion kraftVersion) { node1, voters.listeners(node1) ); - state.transitionToUnattached(3); + state.transitionToUnattached(3, OptionalInt.empty()); assertTrue(state.isUnattached()); UnattachedState unattachedState = state.unattachedStateOrThrow(); assertEquals(3, unattachedState.epoch()); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index c41a2f7be0680..1b19a49d5f857 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -691,13 +691,10 @@ void assertVotedCandidate(int epoch, int candidateId) { } void assertVotedCandidate(int epoch, ReplicaKey candidateKey) { - ReplicaKey expectedKey = kraftVersion == KRaftVersion.KRAFT_VERSION_0 ? - ReplicaKey.of(candidateKey.id(), ReplicaKey.NO_DIRECTORY_ID) : - candidateKey; assertEquals( ElectionState.withVotedCandidate( epoch, - expectedKey, + persistedVotedKey(candidateKey, kraftVersion), expectedVoters() ), quorumStateStore.readElectionState().get() @@ -711,6 +708,26 @@ public void assertElectedLeader(int epoch, int leaderId) { ); } + public void assertElectedLeaderAndVotedCandidate(int epoch, int leaderId, ReplicaKey candidateKey) { + assertEquals( + new ElectionState( + epoch, + OptionalInt.of(leaderId), + Optional.of(persistedVotedKey(candidateKey, kraftVersion)), + expectedVoters() + ), + quorumStateStore.readElectionState().get() + ); + } + + private ReplicaKey persistedVotedKey(ReplicaKey replicaKey, KRaftVersion kraftVersion) { + if (kraftVersion.featureLevel() == 1) { + return replicaKey; + } + + return ReplicaKey.of(replicaKey.id(), ReplicaKey.NO_DIRECTORY_ID); + } + void assertUnknownLeaderAndNoVotedCandidate(int epoch) { assertEquals( ElectionState.withUnknownLeader(epoch, expectedVoters()), diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index 8ed124f795e54..f0c6710cd604a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -242,7 +242,7 @@ public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) { assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); // unattached with votedKey - state.transitionToUnattached(3); + state.transitionToUnattached(3, OptionalInt.empty()); state.unattachedAddVotedState(3, ReplicaKey.of(2, voter2DirectoryId)); assertEquals("unattached-voted", getMetric(metrics, "current-state").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); @@ -302,7 +302,7 @@ public void shouldRecordNonVoterQuorumState(KRaftVersion kraftVersion) { state.followerStateOrThrow().updateHighWatermark(OptionalLong.of(10L)); assertEquals((double) 10L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToUnattached(4); + state.transitionToUnattached(4, OptionalInt.empty()); assertEquals("unattached", getMetric(metrics, "current-state").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); From 1972b58725ad4c29399410f7c48d76f7678143b9 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Sun, 29 Dec 2024 23:09:51 -0800 Subject: [PATCH 16/23] resigned edge case --- .../org/apache/kafka/raft/KafkaRaftClient.java | 14 +++++++++----- .../apache/kafka/raft/RaftEventSimulationTest.java | 6 +++--- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 9b6bf4ef07d3a..fe656770451f7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -3014,14 +3014,18 @@ private long pollResigned(long currentTimeMs) { // until either the shutdown expires or an election bumps the epoch stateTimeoutMs = shutdown.remainingTimeMs(); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { - if (quorum.isVoter()) { - transitionToProspective(currentTimeMs); - } else { +// if (quorum.isVoter()) { + // canElectNewLeaderAfterOldLeaderPartitioned fails if we do not bump epoch since it is possible + // that the replica ends up as follower in the same epoch. + // resigned(leaderId=local) -> prospective(leaderId=local) -> follower(leaderId=local) which is illegal +// transitionToProspective(quorum.epoch() + 1, currentTimeMs); +// transitionToCandidate(currentTimeMs); +// } else { // It is possible that the old leader is not a voter in the new voter set. // In that case increase the epoch and transition to unattached. The epoch needs // to be increased to avoid FETCH responses with the leader being this replica. - transitionToUnattached(quorum.epoch() + 1, OptionalInt.empty()); - } + transitionToUnattached(quorum.epoch() + 1, OptionalInt.empty()); +// } stateTimeoutMs = 0L; } else { stateTimeoutMs = state.remainingElectionTimeMs(currentTimeMs); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index a011ddc438c6e..086a5ebad3b18 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -651,13 +651,13 @@ boolean hasConsistentLeader() { return false; RaftNode first = iter.next(); - ElectionState election = first.store.readElectionState().get(); - if (!election.hasLeader()) + OptionalInt leaderId = first.store.readElectionState().get().optionalLeaderId(); + if (leaderId.isEmpty()) return false; while (iter.hasNext()) { RaftNode next = iter.next(); - if (!election.equals(next.store.readElectionState().get())) + if (!leaderId.equals(next.store.readElectionState().get().optionalLeaderId())) return false; } From 8e851ed8129da2015310a9c4437701738e845162 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Mon, 30 Dec 2024 11:34:08 -0800 Subject: [PATCH 17/23] comments and test fixes --- .../apache/kafka/raft/KafkaRaftClient.java | 17 +----- .../org/apache/kafka/raft/QuorumState.java | 2 + .../kafka/raft/KafkaRaftClientTest.java | 56 +++++++++++++++++-- .../apache/kafka/raft/QuorumStateTest.java | 36 ++++++++++++ 4 files changed, 91 insertions(+), 20 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index fe656770451f7..9524d45909bba 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -681,11 +681,11 @@ private boolean maybeTransitionToCandidate(ProspectiveState state, long currentT * or from Candidate to Leader. * @return true if the state transitioned forward, false otherwise */ - private boolean maybeTransitionForward(NomineeState state, long currentTimeMs) { + private void maybeTransitionForward(NomineeState state, long currentTimeMs) { if (quorum.isProspective()) { - return maybeTransitionToCandidate(quorum.prospectiveStateOrThrow(), currentTimeMs); + maybeTransitionToCandidate(quorum.prospectiveStateOrThrow(), currentTimeMs); } else if (quorum.isCandidate()) { - return maybeTransitionToLeader(quorum.candidateStateOrThrow(), currentTimeMs); + maybeTransitionToLeader(quorum.candidateStateOrThrow(), currentTimeMs); } else { throw new IllegalStateException("Expected to be a VotingState (Prospective or Candidate), " + "but current state is " + state); @@ -3014,18 +3014,7 @@ private long pollResigned(long currentTimeMs) { // until either the shutdown expires or an election bumps the epoch stateTimeoutMs = shutdown.remainingTimeMs(); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { -// if (quorum.isVoter()) { - // canElectNewLeaderAfterOldLeaderPartitioned fails if we do not bump epoch since it is possible - // that the replica ends up as follower in the same epoch. - // resigned(leaderId=local) -> prospective(leaderId=local) -> follower(leaderId=local) which is illegal -// transitionToProspective(quorum.epoch() + 1, currentTimeMs); -// transitionToCandidate(currentTimeMs); -// } else { - // It is possible that the old leader is not a voter in the new voter set. - // In that case increase the epoch and transition to unattached. The epoch needs - // to be increased to avoid FETCH responses with the leader being this replica. transitionToUnattached(quorum.epoch() + 1, OptionalInt.empty()); -// } stateTimeoutMs = 0L; } else { stateTimeoutMs = state.remainingElectionTimeMs(currentTimeMs); diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index d2cefd399648f..8272c889736e1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -395,6 +395,8 @@ public void transitionToUnattached(int epoch, OptionalInt leaderId) { electionTimeoutMs = unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); } else if (isProspective() && !prospectiveStateOrThrow().epochElection().isVoteRejected()) { electionTimeoutMs = prospectiveStateOrThrow().remainingElectionTimeMs(time.milliseconds()); + } else if (isResigned()) { + electionTimeoutMs = resignedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); } else { electionTimeoutMs = randomElectionTimeoutMs(); } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 2838286999dab..88c6bf27040bf 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -305,7 +305,7 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Excepti @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testInitializeAsResignedAndBecomeProspective(boolean withKip853Rpc) throws Exception { + public void testInitializeAsResignedAndBecomeUnattached(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; Set voters = Set.of(localId, remoteId); @@ -326,8 +326,46 @@ public void testInitializeAsResignedAndBecomeProspective(boolean withKip853Rpc) context.time.sleep(context.electionTimeoutMs()); context.client.poll(); + // Become unattached with expired election timeout + assertTrue(context.client.quorum().isUnattached()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + int remoteId = localId + 1; + Set voters = Set.of(localId, remoteId); + int epoch = 2; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) + .withElectedLeader(epoch, localId) + .withKip853Rpc(withKip853Rpc) + .build(); + + // Resign from leader, will restart in resigned state + assertTrue(context.client.quorum().isResigned()); + assertEquals(0L, context.log.endOffset().offset()); + context.assertElectedLeader(epoch, localId); + + // Election timeout + context.time.sleep(context.electionTimeoutMs()); + context.client.poll(); + + // Become unattached with expired election timeout + assertTrue(context.client.quorum().isUnattached()); + // Become prospective + context.time.sleep(1); + context.client.poll(); assertTrue(context.client.quorum().isProspective()); + + // Become unattached again after election timeout + ProspectiveState prospectiveState = context.client.quorum().prospectiveStateOrThrow(); + context.time.sleep(prospectiveState.remainingElectionTimeMs(context.time.milliseconds())); + context.client.poll(); + assertTrue(context.client.quorum().isUnattached()); } @ParameterizedTest @@ -734,7 +772,7 @@ public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) t context.deliverResponse(request.correlationId(), request.destination(), response); context.client.poll(); - // We do not resend `EndQuorumRequest` once the other voter has acknowledged it. + // Local does not resend `EndQuorumRequest` once the other voter has acknowledged it. context.time.sleep(context.retryBackoffMs); context.client.poll(); assertFalse(context.channel.hasSentRequests()); @@ -749,12 +787,18 @@ public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) t OptionalInt.of(localId) ); - // After the election timer, we should become a prospective. + // After the election timer, local should become unattached. context.time.sleep(2L * context.electionTimeoutMs()); - context.pollUntil(context.client.quorum()::isProspective); - assertEquals(resignedEpoch, context.currentEpoch()); - assertEquals(new LeaderAndEpoch(OptionalInt.of(localId), resignedEpoch), + context.pollUntil(context.client.quorum()::isUnattached); + assertEquals(resignedEpoch + 1, context.currentEpoch()); + assertEquals(new LeaderAndEpoch(OptionalInt.empty(), resignedEpoch + 1), context.listener.currentLeaderAndEpoch()); + + // Local will become prospective right away + assertEquals(0, context.client.quorum().unattachedStateOrThrow().electionTimeoutMs()); + context.time.sleep(1); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); } @ParameterizedTest diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index af5e4011ad005..91911d974ce90 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -2503,6 +2503,42 @@ public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ); } + /** + * Test transitions from Resigned + */ + // KAFKA-18379 to fill in the rest of the cases + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testResignedToFollowerInSameEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + int epoch = 5; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + ElectionState election = ElectionState.withElectedLeader(epoch, localId, voters.voterIds()); + store.writeElectionState(election, kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isResigned()); + assertThrows(IllegalStateException.class, () -> state.transitionToFollower(epoch, localId, voters.listeners(localId))); + // KAFKA-18379 will fix this + state.transitionToFollower(epoch, node1, voters.listeners(node1)); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testResignedToUnattachedInHigherEpoch(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + int epoch = 5; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + ElectionState election = ElectionState.withElectedLeader(epoch, localId, voters.voterIds()); + store.writeElectionState(election, kraftVersion); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isResigned()); + state.transitionToUnattached(epoch + 1, OptionalInt.empty()); + } + /** * Test transitions from Observer as Unattached */ From 8c68a9f4ac251c6257cf2bd583f8c5ec52ff025d Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Thu, 2 Jan 2025 00:38:54 -0800 Subject: [PATCH 18/23] more src main change requests --- .../org/apache/kafka/raft/CandidateState.java | 4 - .../org/apache/kafka/raft/ElectionState.java | 9 +- .../org/apache/kafka/raft/FollowerState.java | 3 +- .../apache/kafka/raft/KafkaRaftClient.java | 47 +- .../org/apache/kafka/raft/LeaderState.java | 2 +- .../apache/kafka/raft/ProspectiveState.java | 12 +- .../org/apache/kafka/raft/QuorumState.java | 45 +- .../org/apache/kafka/raft/ResignedState.java | 3 +- .../apache/kafka/raft/UnattachedState.java | 8 +- .../kafka/raft/internals/EpochElection.java | 20 +- .../apache/kafka/raft/ElectionStateTest.java | 7 +- .../kafka/raft/FileQuorumStateStoreTest.java | 6 +- .../kafka/raft/KafkaRaftClientTest.java | 42 +- .../kafka/raft/ProspectiveStateTest.java | 2 +- .../apache/kafka/raft/QuorumStateTest.java | 520 +++++++++--------- .../kafka/raft/RaftClientTestContext.java | 6 +- .../apache/kafka/raft/ResignedStateTest.java | 5 +- .../kafka/raft/UnattachedStateTest.java | 2 +- 18 files changed, 395 insertions(+), 348 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 8541be42fcf24..d66de84e28445 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -84,10 +84,6 @@ protected CandidateState( epochElection.recordVote(localId, true); } - public int localId() { - return localId; - } - /** * Check if the candidate is backing off for the next election */ diff --git a/raft/src/main/java/org/apache/kafka/raft/ElectionState.java b/raft/src/main/java/org/apache/kafka/raft/ElectionState.java index 675436cc52c2a..6b4f775caeebc 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ElectionState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ElectionState.java @@ -185,12 +185,17 @@ public static ElectionState withVotedCandidate(int epoch, ReplicaKey votedKey, S return new ElectionState(epoch, OptionalInt.empty(), Optional.of(votedKey), voters); } - public static ElectionState withElectedLeader(int epoch, int leaderId, Set voters) { + public static ElectionState withElectedLeader( + int epoch, + int leaderId, + Optional votedKey, + Set voters + ) { if (leaderId < 0) { throw new IllegalArgumentException("Illegal leader Id " + leaderId + ": must be non-negative"); } - return new ElectionState(epoch, OptionalInt.of(leaderId), Optional.empty(), voters); + return new ElectionState(epoch, OptionalInt.of(leaderId), votedKey, voters); } public static ElectionState withUnknownLeader(int epoch, Set voters) { diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index 0a6a48b46d63a..4d2357fdef58e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -26,7 +26,6 @@ import org.slf4j.Logger; import java.util.Optional; -import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Set; @@ -79,7 +78,7 @@ public FollowerState( @Override public ElectionState election() { - return new ElectionState(epoch, OptionalInt.of(leaderId), votedKey, voters); + return ElectionState.withElectedLeader(epoch, leaderId, votedKey, voters); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 9524d45909bba..683a43b9f7e6d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -550,9 +550,10 @@ public void initialize( onBecomeFollower(currentTimeMs); } - // When there is only a single voter, become candidate immediately - if (quorum.isOnlyVoter() && !quorum.isCandidate()) { - transitionToCandidate(currentTimeMs); + // When there is only a single voter, become prospective immediately. + // transitionToProspective will handle short-circuiting voter to candidate state + if (quorum.isOnlyVoter() && !quorum.isNomineeState() && !quorum.isLeader()) { + transitionToProspective(currentTimeMs); } // Specialized add voter handler @@ -667,7 +668,8 @@ private boolean maybeTransitionToLeader(CandidateState state, long currentTimeMs } private boolean maybeTransitionToCandidate(ProspectiveState state, long currentTimeMs) { - if (state.epochElection().isVoteGranted()) { + // If replica is the only voter, it should transition to candidate immediately + if (state.epochElection().isVoteGranted() || quorum.isOnlyVoter()) { transitionToCandidate(currentTimeMs); return true; } else { @@ -716,7 +718,6 @@ private void onBecomeProspective(long currentTimeMs) { private void transitionToProspective(long currentTimeMs) { quorum.transitionToProspective(); - maybeFireLeaderChange(); onBecomeProspective(currentTimeMs); } @@ -1007,9 +1008,9 @@ private void maybeHandleElectionLoss(long currentTimeMs) { CandidateState candidate = quorum.candidateStateOrThrow(); if (candidate.epochElection().isVoteRejected() && !candidate.isBackingOff()) { logger.info( - "Insufficient remaining votes to become leader (rejected by {}). " + - "We will backoff before retrying election again", - candidate.epochElection().rejectingVoters() + "Insufficient remaining votes to become leader. We will backoff before retrying election again. " + + "Current epoch election state is {}.", + candidate.epochElection() ); // Go immediately to a random, exponential backoff. The backoff starts low to prevent // needing to wait the entire election timeout when the vote result has already been @@ -1025,11 +1026,15 @@ private void maybeHandleElectionLoss(long currentTimeMs) { ProspectiveState prospective = quorum.prospectiveStateOrThrow(); if (prospective.epochElection().isVoteRejected()) { logger.info( - "Insufficient remaining votes to become candidate (rejected by {}). ", - prospective.epochElection().rejectingVoters() + "Insufficient remaining votes to become candidate. Current epoch election state is {}. ", + prospective.epochElection() ); prospectiveTransitionAfterElectionLoss(prospective, currentTimeMs); } + } else { + throw new IllegalStateException( + "Expected to be a NomineeState (Prospective or Candidate), but quorum state is " + quorum + ); } } @@ -2830,7 +2835,6 @@ private BeginQuorumEpochRequestData buildBeginQuorumEpochRequest(ReplicaKey remo private VoteRequestData buildVoteRequest(ReplicaKey remoteVoter, boolean preVote) { OffsetAndEpoch endOffset = endOffset(); -// boolean isPreVote = quorum.isProspective(); return RaftUtil.singletonVoteRequest( log.topicPartition(), clusterId, @@ -3010,10 +3014,15 @@ private long pollResigned(long currentTimeMs) { GracefulShutdown shutdown = this.shutdown.get(); final long stateTimeoutMs; if (shutdown != null) { - // If we are shutting down, then we will remain in the resigned state + // If the replica is shutting down, it will remain in the resigned state // until either the shutdown expires or an election bumps the epoch stateTimeoutMs = shutdown.remainingTimeMs(); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { + // The replica stays in resigned state for an election timeout period to allow end quorum requests + // to be processed, and to give other replicas a chance to become leader. When transitioning out + // of resigned state, the epoch must be increased to avoid FETCH responses with the leader + // being this replica, and to avoid this replica attempting to transition into follower state with + // itself as the leader. transitionToUnattached(quorum.epoch() + 1, OptionalInt.empty()); stateTimeoutMs = 0L; } else { @@ -3122,7 +3131,11 @@ private long pollProspective(long currentTimeMs) { long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); return Math.min(shutdown.remainingTimeMs(), minRequestBackoffMs); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { - logger.info("Election timed out before receiving sufficient vote responses to become candidate"); + logger.info( + "Election timed out before receiving sufficient vote responses to become candidate. " + + "Current epoch election state: {}", + state.epochElection() + ); prospectiveTransitionAfterElectionLoss(state, currentTimeMs); return 0L; } else { @@ -3132,17 +3145,15 @@ private long pollProspective(long currentTimeMs) { } private void prospectiveTransitionAfterElectionLoss(ProspectiveState prospective, long currentTimeMs) { + // If the replica knows of a leader, it transitions to follower. Otherwise, it transitions to unattached. if (prospective.election().hasLeader() && !prospective.leaderEndpoints().isEmpty()) { - logger.info( - "Transitioning to Follower of leader {}", - prospective.election().leaderId()); transitionToFollower( quorum().epoch(), prospective.election().leaderId(), prospective.leaderEndpoints(), - currentTimeMs); + currentTimeMs + ); } else { - logger.info("Transitioning to Unattached to attempt rediscovering leader"); transitionToUnattached(quorum().epoch(), prospective.election().optionalLeaderId()); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index eb5f8cb3d87ba..dffee814d72ae 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -406,7 +406,7 @@ public Optional highWatermark() { @Override public ElectionState election() { - return ElectionState.withElectedLeader(epoch, localReplicaKey.id(), voterStates.keySet()); + return ElectionState.withElectedLeader(epoch, localReplicaKey.id(), Optional.empty(), voterStates.keySet()); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java index 59343be013053..a4981dc5d2549 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -80,10 +80,6 @@ public ProspectiveState( epochElection.recordVote(localId, true); } - public int localId() { - return localId; - } - public Optional votedKey() { return votedKey; } @@ -137,7 +133,13 @@ public long remainingElectionTimeMs(long currentTimeMs) { @Override public ElectionState election() { - return new ElectionState(epoch, leaderId, votedKey, voters.voterIds()); + if (leaderId.isPresent()) { + return ElectionState.withElectedLeader(epoch, leaderId.getAsInt(), votedKey, voters.voterIds()); + } else if (votedKey.isPresent()) { + return ElectionState.withVotedCandidate(epoch, votedKey.get(), voters.voterIds()); + } else { + return ElectionState.withUnknownLeader(epoch, voters.voterIds()); + } } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 8272c889736e1..4c79d2601f0a4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -39,9 +39,8 @@ * how they are triggered: * * Resigned transitions to: - * Unattached: After learning of a new election with a higher epoch - * Prospective: After expiration of the election timeout - * Follower: After discovering a leader with an equal or larger epoch + * Unattached: After learning of a new election with a higher epoch, or expiration of the election timeout + * Follower: After discovering a leader with a larger epoch * * Unattached transitions to: * Unattached: After learning of a new election with a higher epoch or after giving a binding vote @@ -369,8 +368,10 @@ public void transitionToResigned(List preferredSuccessors) { } /** - * Transition to the "unattached" state. This means the replica has found an epoch greater than the current epoch, - * or the replica has transitioned from Prospective with the same epoch. + * Transition to the "unattached" state. This means one of the following + * 1. the replica has found an epoch greater than the current epoch. + * 2. the replica has transitioned from Prospective with the same epoch. + * 3. the replica has transitioned from Resigned with current epoch + 1. * Note, if the replica is transitioning from unattached to add voted state and there is no epoch change, * it takes the route of unattachedAddVotedState instead. */ @@ -379,7 +380,7 @@ public void transitionToUnattached(int epoch, OptionalInt leaderId) { if (epoch < currentEpoch || (epoch == currentEpoch && !isProspective())) { throw new IllegalStateException( String.format( - "Cannot transition to Unattached with epoch= %d from current state %s", + "Cannot transition to Unattached with epoch %d from current state %s", epoch, state ) @@ -464,7 +465,6 @@ public void unattachedAddVotedState( logContext ) ); - log.debug("Voted for candidate {} in epoch {}", candidateKey, epoch); } /** @@ -518,7 +518,6 @@ public void prospectiveAddVotedState( logContext ) ); - log.debug("Voted for candidate {} in epoch {}", candidateKey, epoch); } /** @@ -526,7 +525,6 @@ public void prospectiveAddVotedState( */ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { int currentEpoch = state.epoch(); - boolean retainVotedKey = false; if (endpoints.isEmpty()) { throw new IllegalArgumentException( String.format( @@ -554,7 +552,6 @@ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { ) ); } else if (epoch == currentEpoch) { - retainVotedKey = true; if (isFollower() && state.leaderEndpoints().size() >= endpoints.size()) { throw new IllegalStateException( String.format( @@ -577,13 +574,18 @@ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { } } + // State transitions within the same epoch should preserve voted key if it exists. This prevents + // replicas from voting multiple times in the same epoch, which could violate the Raft invariant of + // at most one leader elected in an epoch. + Optional votedKey = epoch == currentEpoch ? votedKey() : Optional.empty(); + durableTransitionTo( new FollowerState( time, epoch, leaderId, endpoints, - retainVotedKey ? votedKey() : Optional.empty(), + votedKey, partitionState.lastVoterSet().voterIds(), state.highWatermark(), fetchTimeoutMs, @@ -615,7 +617,8 @@ public void transitionToProspective() { int retries = isCandidate() ? candidateStateOrThrow().retries() + 1 : 1; - durableTransitionTo( + // Durable transition is not necessary since there is no change to the persisted electionState + memoryTransitionTo( new ProspectiveState( time, localIdOrThrow(), @@ -665,9 +668,8 @@ private void checkValidTransitionToCandidate() { ) ); } - // Leader state can never transition to Candidate state - // Only Prospective is allowed to transition to Candidate, unless the local replica is the only voter - if (isLeader() || (!isProspective() && !isOnlyVoter())) { + // Only Prospective is allowed to transition to Candidate + if (!isProspective()) { throw new IllegalStateException("Cannot transition to Candidate since the local broker.id=" + localId + " is state " + state); } @@ -768,9 +770,9 @@ public Optional maybeUnattachedState() { } public UnattachedState unattachedStateOrThrow() { - if (isUnattached()) - return (UnattachedState) state; - throw new IllegalStateException("Expected to be Unattached, but current state is " + state); + return maybeUnattachedState().orElseThrow( + () -> new IllegalStateException("Expected to be Unattached, but current state is " + state) + ); } public LeaderState leaderStateOrThrow() { @@ -804,9 +806,9 @@ public Optional maybeProspectiveState() { } public ProspectiveState prospectiveStateOrThrow() { - if (isProspective()) - return (ProspectiveState) state; - throw new IllegalStateException("Expected to be Prospective, but current state is " + state); + return maybeProspectiveState().orElseThrow( + () -> new IllegalStateException("Expected to be Prospective, but current state is " + state) + ); } public boolean isProspectiveNotVoted() { @@ -873,6 +875,7 @@ public boolean isNomineeState() { /** * Determines if replica in unattached or prospective state can grant a vote request. + * * @param leaderId local replica's optional leader id. * @param votedKey local replica's optional voted key. * @param epoch local replica's epoch diff --git a/raft/src/main/java/org/apache/kafka/raft/ResignedState.java b/raft/src/main/java/org/apache/kafka/raft/ResignedState.java index d79ee44c84631..2d5fd27919f8c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ResignedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ResignedState.java @@ -24,6 +24,7 @@ import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; /** @@ -75,7 +76,7 @@ public ResignedState( @Override public ElectionState election() { - return ElectionState.withElectedLeader(epoch, localId, voters); + return ElectionState.withElectedLeader(epoch, localId, Optional.empty(), voters); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java index 9d9ac30213cc3..6b7e4b700f241 100644 --- a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java @@ -73,7 +73,13 @@ public UnattachedState( @Override public ElectionState election() { - return new ElectionState(epoch, leaderId, votedKey, voters); + if (leaderId.isPresent()) { + return ElectionState.withElectedLeader(epoch, leaderId.getAsInt(), votedKey, voters); + } else if (votedKey.isPresent()) { + return ElectionState.withVotedCandidate(epoch, votedKey.get(), voters); + } else { + return ElectionState.withUnknownLeader(epoch, voters); + } } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java index a837bfe90da7d..538553a99cdc3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java @@ -18,7 +18,6 @@ import org.apache.kafka.raft.ReplicaKey; -import java.util.Collection; import java.util.Collections; import java.util.Map; import java.util.Set; @@ -48,11 +47,8 @@ public EpochElection(Set voters) { * @return true if the voter had not been previously recorded */ public boolean recordVote(int voterId, boolean isGranted) { - boolean wasUnrecorded = false; VoterState voterState = getVoterStateOrThrow(voterId); - if (voterState.state == VoterState.State.UNRECORDED) { - wasUnrecorded = true; - } + boolean wasUnrecorded = voterState.state == VoterState.State.UNRECORDED; if (isGranted) { voterState.setState(VoterState.State.GRANTED); } else { @@ -86,13 +82,6 @@ public Set voterIds() { return Collections.unmodifiableSet(voterStates.keySet()); } - /** - * Get the collection of voter states. - */ - public Collection voterStates() { - return Collections.unmodifiableCollection(voterStates.values()); - } - /** * Check whether we have received enough votes to conclude the election and become leader. * @@ -170,11 +159,8 @@ private int majoritySize() { @Override public String toString() { return String.format( - "EpochElection(%s)", - voterStates.values().stream() - .map(VoterState::toString) - .collect( - Collectors.joining(", ")) + "EpochElection(voterStates=%s)", + voterStates ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java index 43ac53d11ac90..af443bd1dd18d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -70,7 +71,7 @@ void testQuorumStateDataRoundTrip(short version) { ReplicaKey votedKey = ReplicaKey.of(1, Uuid.randomUuid()); List electionStates = Arrays.asList( ElectionState.withUnknownLeader(5, Set.of(1, 2, 3)), - ElectionState.withElectedLeader(5, 1, Set.of(1, 2, 3)), + ElectionState.withElectedLeader(5, 1, Optional.empty(), Set.of(1, 2, 3)), ElectionState.withVotedCandidate(5, votedKey, Set.of(1, 2, 3)) ); @@ -78,7 +79,7 @@ void testQuorumStateDataRoundTrip(short version) { if (version == 0) { expected = Arrays.asList( ElectionState.withUnknownLeader(5, Set.of(1, 2, 3)), - ElectionState.withElectedLeader(5, 1, Set.of(1, 2, 3)), + ElectionState.withElectedLeader(5, 1, Optional.empty(), Set.of(1, 2, 3)), ElectionState.withVotedCandidate( 5, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), @@ -88,7 +89,7 @@ void testQuorumStateDataRoundTrip(short version) { } else { expected = Arrays.asList( ElectionState.withUnknownLeader(5, Collections.emptySet()), - ElectionState.withElectedLeader(5, 1, Collections.emptySet()), + ElectionState.withElectedLeader(5, 1, Optional.empty(), Collections.emptySet()), ElectionState.withVotedCandidate(5, votedKey, Collections.emptySet()) ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java b/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java index 9782bc05a2283..a265803d72de8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java @@ -59,17 +59,17 @@ void testWriteReadElectedLeader(KRaftVersion kraftVersion) throws IOException { Set voters = Set.of(voter1, voter2, voter3); stateStore.writeElectionState( - ElectionState.withElectedLeader(epoch, voter1, voters), + ElectionState.withElectedLeader(epoch, voter1, Optional.empty(), voters), kraftVersion ); final Optional expected; if (kraftVersion.isReconfigSupported()) { expected = Optional.of( - ElectionState.withElectedLeader(epoch, voter1, Collections.emptySet()) + ElectionState.withElectedLeader(epoch, voter1, Optional.empty(), Collections.emptySet()) ); } else { - expected = Optional.of(ElectionState.withElectedLeader(epoch, voter1, voters)); + expected = Optional.of(ElectionState.withElectedLeader(epoch, voter1, Optional.empty(), voters)); } assertEquals(expected, stateStore.readElectionState()); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 88c6bf27040bf..86d3cdf129d20 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -398,9 +398,13 @@ public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) ); context.client.poll(); + // The node will transition to unattached with epoch + 1 after election timeout passes context.time.sleep(context.electionTimeoutMs()); context.client.poll(); - assertTrue(context.client.quorum().isProspective()); + assertTrue(context.client.quorum().isUnattached()); + assertEquals(epoch + 1, context.currentEpoch()); + UnattachedState unattached = context.client.quorum().unattachedStateOrThrow(); + assertEquals(0, unattached.remainingElectionTimeMs(context.time.milliseconds())); } @ParameterizedTest @@ -1000,6 +1004,42 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree(boolean withKi Arrays.asList(voterId, localId), record.key(), record.value()); } + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testInitializeAsOnlyVoterWithEmptyElectionState(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + .withKip853Rpc(withKip853Rpc) + .build(); + context.assertElectedLeader(1, localId); + assertEquals(0L, context.log.endOffset().offset()); + assertTrue(context.client.quorum().isLeader()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testInitializeAsCandidateAndOnlyVoter(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + .withKip853Rpc(withKip853Rpc) + .withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) + .build(); + context.assertElectedLeader(2, localId); + assertTrue(context.client.quorum().isLeader()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testInitializeAsResignedAndOnlyVoter(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + .withKip853Rpc(withKip853Rpc) + .withElectedLeader(2, localId) + .build(); + context.assertElectedLeader(3, localId); + assertTrue(context.client.quorum().isLeader()); + } + @ParameterizedTest @ValueSource(booleans = { true, false }) public void testHandleBeginQuorumRequest(boolean withKip853Rpc) throws Exception { diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java index 98dbe08fefc76..2a39fc90fd544 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -309,7 +309,7 @@ public void testElectionState(boolean withDirectoryId) { ElectionState.withElectedLeader( epoch, 1, - voters.voterIds() + Optional.empty(), voters.voterIds() ), state.election() ); diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index 91911d974ce90..04fb1f5473889 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -259,9 +259,9 @@ public void testCanBecomeFollowerOfNonVoter(KRaftVersion kraftVersion) { state.leaderAndEpoch() ); assertEquals( - new ElectionState( + ElectionState.withElectedLeader( 4, - OptionalInt.of(nonVoterKey.id()), + nonVoterKey.id(), Optional.of(persistedVotedKey(nonVoterKey, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion) ), @@ -359,7 +359,10 @@ public void testInitializeAsFollower(KRaftVersion kraftVersion) { int node2 = 2; int epoch = 5; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - store.writeElectionState(ElectionState.withElectedLeader(epoch, node1, voters.voterIds()), kraftVersion); + store.writeElectionState( + ElectionState.withElectedLeader(epoch, node1, Optional.empty(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -379,7 +382,7 @@ public void testInitializeVotedAndLeaderAreSame(KRaftVersion kraftVersion) { ReplicaKey leaderKey = ReplicaKey.of(1, Uuid.randomUuid()); VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderKey.id(), 2), kraftVersion); store.writeElectionState( - new ElectionState(epoch, OptionalInt.of(leaderKey.id()), Optional.of(leaderKey), voters.voterIds()), + ElectionState.withElectedLeader(epoch, leaderKey.id(), Optional.of(leaderKey), voters.voterIds()), kraftVersion ); @@ -393,15 +396,13 @@ public void testInitializeVotedAndLeaderAreSame(KRaftVersion kraftVersion) { assertEquals(leaderKey.id(), followerState.leaderId()); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); assertEquals( - Optional.of( - new ElectionState( - epoch, - OptionalInt.of(leaderKey.id()), - Optional.of(persistedVotedKey(leaderKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + epoch, + leaderKey.id(), + Optional.of(persistedVotedKey(leaderKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -413,7 +414,7 @@ public void testInitializeVotedAndLeaderAreDifferent(KRaftVersion kraftVersion) ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = localWithRemoteVoterSet(IntStream.of(leader, votedKey.id()), kraftVersion); store.writeElectionState( - new ElectionState(epoch, OptionalInt.of(leader), Optional.of(votedKey), voters.voterIds()), + ElectionState.withElectedLeader(epoch, leader, Optional.of(votedKey), voters.voterIds()), kraftVersion ); @@ -427,15 +428,13 @@ public void testInitializeVotedAndLeaderAreDifferent(KRaftVersion kraftVersion) assertEquals(leader, followerState.leaderId()); assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); assertEquals( - Optional.of( - new ElectionState( - epoch, - OptionalInt.of(leader), - Optional.of(persistedVotedKey(votedKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + epoch, + leader, + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -449,7 +448,7 @@ public void testInitializeAsUnattachedWhenMissingEndpoints(KRaftVersion kraftVer ReplicaKey votedKey = ReplicaKey.of(leader, Uuid.randomUuid()); VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); store.writeElectionState( - new ElectionState(epoch, OptionalInt.of(leader), Optional.of(votedKey), voters.voterIds()), + ElectionState.withElectedLeader(epoch, leader, Optional.of(votedKey), voters.voterIds()), kraftVersion ); @@ -461,15 +460,13 @@ public void testInitializeAsUnattachedWhenMissingEndpoints(KRaftVersion kraftVer UnattachedState unattachedState = state.unattachedStateOrThrow(); assertEquals(epoch, unattachedState.epoch()); assertEquals( - Optional.of( - new ElectionState( - epoch, - OptionalInt.of(leader), - Optional.of(persistedVotedKey(votedKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + epoch, + leader, + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -549,7 +546,7 @@ public void testInitializeAsResignedLeader(KRaftVersion kraftVersion) { int node2 = 2; int epoch = 5; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - ElectionState election = ElectionState.withElectedLeader(epoch, localId, voters.voterIds()); + ElectionState election = ElectionState.withElectedLeader(epoch, localId, Optional.empty(), voters.voterIds()); store.writeElectionState(election, kraftVersion); // If we were previously a leader, we will start as resigned in order to ensure @@ -569,8 +566,10 @@ public void testInitializeAsResignedLeader(KRaftVersion kraftVersion) { assertEquals(epoch, resignedState.epoch()); assertEquals(election, resignedState.election()); assertEquals(Set.of(node1, node2), resignedState.unackedVoters()); - assertEquals(electionTimeoutMs + jitterMs, - resignedState.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + electionTimeoutMs + jitterMs, + resignedState.remainingElectionTimeMs(time.milliseconds()) + ); } @ParameterizedTest @@ -683,8 +682,10 @@ public void testUnattachedToUnattached(KRaftVersion kraftVersion) { assertEquals(6, unattachedState.epoch()); // Verify that the election timer does not get reset - assertEquals(remainingElectionTimeMs - 1000, - unattachedState.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + remainingElectionTimeMs - 1000, + unattachedState.remainingElectionTimeMs(time.milliseconds()) + ); } @ParameterizedTest @@ -714,19 +715,19 @@ public void testUnattachedToUnattachedVotedSameEpoch(KRaftVersion kraftVersion) assertEquals(otherNodeKey, votedState.votedKey().get()); assertEquals( - Optional.of( ElectionState.withVotedCandidate( 5, persistedVotedKey(otherNodeKey, kraftVersion), persistedVoters(voters.voterIds(), kraftVersion) - ) ), - store.readElectionState() + store.readElectionState().get() ); // Verify election timeout is reset when we vote for a candidate - assertEquals(electionTimeoutMs + jitterMs, - votedState.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + electionTimeoutMs + jitterMs, + votedState.remainingElectionTimeMs(time.milliseconds()) + ); } @ParameterizedTest @@ -808,13 +809,8 @@ public void testUnattachedToProspective(KRaftVersion kraftVersion) { assertEquals(electionTimeoutMs + jitterMs, prospective.remainingElectionTimeMs(time.milliseconds())); assertEquals( - Optional.of( - ElectionState.withUnknownLeader( - logEndEpoch, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() + ElectionState.withUnknownLeader(logEndEpoch, persistedVoters(voters.voterIds(), kraftVersion)), + store.readElectionState().get() ); } @@ -866,13 +862,8 @@ public void testUnattachedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ); assertEquals(5, state.epoch()); assertEquals( - Optional.of( - ElectionState.withUnknownLeader( - 5, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() + ElectionState.withUnknownLeader(5, persistedVoters(voters.voterIds(), kraftVersion)), + store.readElectionState().get() ); } @@ -908,8 +899,10 @@ public void testUnattachedVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion assertEquals(6, unattachedState.epoch()); // Verify that the election timer does not get reset - assertEquals(remainingElectionTimeMs - 1000, - unattachedState.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + remainingElectionTimeMs - 1000, + unattachedState.remainingElectionTimeMs(time.milliseconds()) + ); } @ParameterizedTest @@ -961,15 +954,13 @@ public void testUnattachedVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { followerState.leaderEndpoints() ); assertEquals( - Optional.of( - new ElectionState( - 5, - OptionalInt.of(node2), - Optional.of(persistedVotedKey(node1Key, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 5, + node2, + Optional.of(persistedVotedKey(node1Key, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -995,14 +986,13 @@ public void testUnattachedVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) followerState.leaderEndpoints() ); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 8, - node2, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 8, + node2, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -1026,12 +1016,12 @@ public void testUnattachedVotedToProspective(KRaftVersion kraftVersion) { assertEquals(electionTimeoutMs + jitterMs, prospectiveState.remainingElectionTimeMs(time.milliseconds())); assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 5, - persistedVotedKey(ReplicaKey.of(node1, node1DirectoryId), kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() + ElectionState.withVotedCandidate( + 5, + persistedVotedKey(ReplicaKey.of(node1, node1DirectoryId), kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); } @@ -1069,14 +1059,12 @@ public void testUnattachedVotedToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ); assertEquals(5, state.epoch()); assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 5, - persistedVotedKey(otherNodeKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withVotedCandidate( + 5, + persistedVotedKey(otherNodeKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -1091,15 +1079,23 @@ public void testUnattachedWithLeaderToProspective(KRaftVersion kraftVersion) { int node2 = 2; int epoch = 5; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - store.writeElectionState(ElectionState.withElectedLeader(epoch, node1, voters.voterIds()), kraftVersion); + store.writeElectionState( + ElectionState.withElectedLeader(epoch, node1, Optional.empty(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); assertTrue(state.isProspective()); assertEquals( - Optional.of(ElectionState.withElectedLeader(epoch, node1, persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() + ElectionState.withElectedLeader( + epoch, + node1, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); } @@ -1117,7 +1113,10 @@ public void testUnattachedWithLeaderNoEndpointToAndFromProspective(KRaftVersion voterMap.put(followerId, VoterSetTest.voterNode(followerKey, Endpoints.empty())); VoterSet voters = VoterSetTest.voterSet(voterMap); - store.writeElectionState(ElectionState.withElectedLeader(epoch, leaderId, voters.voterIds()), kraftVersion); + store.writeElectionState( + ElectionState.withElectedLeader(epoch, leaderId, Optional.empty(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isUnattached()); @@ -1128,24 +1127,26 @@ public void testUnattachedWithLeaderNoEndpointToAndFromProspective(KRaftVersion assertTrue(state.isProspective()); assertTrue(state.leaderEndpoints().isEmpty()); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - leaderId, - persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() + ElectionState.withElectedLeader( + 5, + leaderId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); state.transitionToUnattached(5, OptionalInt.of(leaderId)); assertTrue(state.isUnattached()); assertTrue(state.leaderEndpoints().isEmpty()); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - leaderId, - persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() + ElectionState.withElectedLeader( + 5, + leaderId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); } @@ -1190,8 +1191,10 @@ public void testFollowerToUnattachedHigherEpoch(KRaftVersion kraftVersion) { assertTrue(state.isUnattached()); UnattachedState unattachedState = state.unattachedStateOrThrow(); assertEquals(9, unattachedState.epoch()); - assertEquals(electionTimeoutMs + jitterMs, - unattachedState.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + electionTimeoutMs + jitterMs, + unattachedState.remainingElectionTimeMs(time.milliseconds()) + ); } @ParameterizedTest @@ -1249,14 +1252,13 @@ public void testFollowerToFollowerSameEpoch(KRaftVersion kraftVersion) { followerState.leaderEndpoints() ); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 8, - node2, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 8, + node2, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -1318,14 +1320,13 @@ public void testFollowerToFollowerHigherEpoch(KRaftVersion kraftVersion) { followerState.leaderEndpoints() ); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 9, - node1, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 9, + node1, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -1356,6 +1357,7 @@ public void testFollowerToProspective(KRaftVersion kraftVersion) { ElectionState.withElectedLeader( 8, node2, + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1419,14 +1421,13 @@ public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { assertThrows(IllegalStateException.class, () -> state.transitionToLeader(4, accumulator)); assertEquals(5, state.epoch()); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - otherNodeId, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 5, + otherNodeId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -1483,8 +1484,10 @@ public void testFollowerVotedToUnattachedHigherEpoch(KRaftVersion kraftVersion) UnattachedState unattached = state.unattachedStateOrThrow(); assertEquals(10, unattached.epoch()); - assertEquals(electionTimeoutMs + jitterMs, - unattached.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + electionTimeoutMs + jitterMs, + unattached.remainingElectionTimeMs(time.milliseconds()) + ); } @ParameterizedTest @@ -1504,15 +1507,13 @@ public void testFollowerVotedToFollowerSameEpoch(KRaftVersion kraftVersion) { voters.listeners(node2) ); assertEquals( - Optional.of( - new ElectionState( - 0, - OptionalInt.of(node2), - Optional.of(persistedVotedKey(votedKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 0, + node2, + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); assertThrows( @@ -1538,28 +1539,25 @@ public void testFollowerVotedToFollowerHigherEpoch(KRaftVersion kraftVersion) { voters.listeners(node2) ); assertEquals( - Optional.of( - new ElectionState( - 0, - OptionalInt.of(node2), - Optional.of(persistedVotedKey(votedKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 0, + node2, + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); state.transitionToFollower(state.epoch() + 1, node1, voters.listeners(node1)); assertEquals(1, state.epoch()); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 1, - node1, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 1, + node1, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -1578,10 +1576,7 @@ public void testProspectiveToUnattachedInSameEpoch(KRaftVersion kraftVersion) { state.transitionToUnattached(state.epoch(), OptionalInt.empty()); assertEquals( - ElectionState.withUnknownLeader( - logEndEpoch, - persistedVoters(voters.voterIds(), kraftVersion) - ), + ElectionState.withUnknownLeader(logEndEpoch, persistedVoters(voters.voterIds(), kraftVersion)), store.readElectionState().get() ); } @@ -1598,10 +1593,7 @@ public void testProspectiveToUnattachedInHigherEpoch(KRaftVersion kraftVersion) state.transitionToUnattached(20, OptionalInt.empty()); assertEquals( - ElectionState.withUnknownLeader( - 20, - persistedVoters(voters.voterIds(), kraftVersion) - ), + ElectionState.withUnknownLeader(20, persistedVoters(voters.voterIds(), kraftVersion)), store.readElectionState().get() ); } @@ -1640,6 +1632,7 @@ public void testProspectiveToUnattachedWithLeaderInHigherEpoch(KRaftVersion kraf ElectionState.withElectedLeader( 10, voter1.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1660,6 +1653,7 @@ public void testProspectiveToFollowerSameEpoch(KRaftVersion kraftVersion) { ElectionState.withElectedLeader( 0, voter1.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1680,6 +1674,7 @@ public void testProspectiveToFollowerHigherEpoch(KRaftVersion kraftVersion) { ElectionState.withElectedLeader( 1, voter1.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1728,12 +1723,12 @@ public void testProspectiveToCandidate(KRaftVersion kraftVersion) { state.candidateStateOrThrow().remainingElectionTimeMs(time.milliseconds()) ); assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - logEndEpoch + 1, - persistedVotedKey(localVoterKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); } @@ -1796,9 +1791,9 @@ public void testProspectiveVotedToAndFromFollowerSameEpoch(KRaftVersion kraftVer // transition to follower of voter2 with votedKey voter1 state.transitionToFollower(state.epoch(), voter2.id(), voters.listeners(voter2.id())); assertEquals( - new ElectionState( + ElectionState.withElectedLeader( logEndEpoch, - OptionalInt.of(voter2.id()), + voter2.id(), Optional.of(persistedVotedKey(voter1, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion) ), @@ -1808,13 +1803,13 @@ public void testProspectiveVotedToAndFromFollowerSameEpoch(KRaftVersion kraftVer // transition back to prospective state.transitionToProspective(); assertEquals( - Optional.of(new ElectionState( + ElectionState.withElectedLeader( logEndEpoch, - OptionalInt.of(voter2.id()), + voter2.id(), Optional.of(persistedVotedKey(voter1, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion)) + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -1856,12 +1851,12 @@ public void testProspectiveVotedToCandidate(KRaftVersion kraftVersion) { ); assertEquals(logEndEpoch + 1, state.epoch()); assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - logEndEpoch + 1, - persistedVotedKey(localVoterKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); } @@ -1875,7 +1870,10 @@ public void testProspectiveWithLeaderToUnattachedInSameEpoch(KRaftVersion kraftV ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + store.writeElectionState( + ElectionState.withElectedLeader(logEndEpoch, voter1.id(), Optional.empty(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); @@ -1883,6 +1881,7 @@ public void testProspectiveWithLeaderToUnattachedInSameEpoch(KRaftVersion kraftV ElectionState.withElectedLeader( logEndEpoch, voter1.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1893,6 +1892,7 @@ public void testProspectiveWithLeaderToUnattachedInSameEpoch(KRaftVersion kraftV ElectionState.withElectedLeader( logEndEpoch, voter1.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1905,7 +1905,10 @@ public void testProspectiveWithLeaderToUnattachedInHigherEpoch(KRaftVersion kraf ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + store.writeElectionState( + ElectionState.withElectedLeader(logEndEpoch, voter1.id(), Optional.empty(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); @@ -1913,6 +1916,7 @@ public void testProspectiveWithLeaderToUnattachedInHigherEpoch(KRaftVersion kraf ElectionState.withElectedLeader( logEndEpoch, voter1.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1921,10 +1925,7 @@ public void testProspectiveWithLeaderToUnattachedInHigherEpoch(KRaftVersion kraf // transition to unattached state.transitionToUnattached(10, OptionalInt.empty()); assertEquals( - ElectionState.withUnknownLeader( - 10, - persistedVoters(voters.voterIds(), kraftVersion) - ), + ElectionState.withUnknownLeader(10, persistedVoters(voters.voterIds(), kraftVersion)), store.readElectionState().get() ); } @@ -1935,7 +1936,10 @@ public void testProspectiveWithLeaderToUnattachedWithLeaderInHigherEpoch(KRaftVe ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + store.writeElectionState( + ElectionState.withElectedLeader(logEndEpoch, voter1.id(), Optional.empty(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); @@ -1943,6 +1947,7 @@ public void testProspectiveWithLeaderToUnattachedWithLeaderInHigherEpoch(KRaftVe ElectionState.withElectedLeader( logEndEpoch, voter1.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1954,6 +1959,7 @@ public void testProspectiveWithLeaderToUnattachedWithLeaderInHigherEpoch(KRaftVe ElectionState.withElectedLeader( 10, voter2.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1966,7 +1972,10 @@ public void testProspectiveWithLeaderToFollowerSameEpoch(KRaftVersion kraftVersi ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + store.writeElectionState( + ElectionState.withElectedLeader(logEndEpoch, voter1.id(), Optional.empty(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); @@ -1976,6 +1985,7 @@ public void testProspectiveWithLeaderToFollowerSameEpoch(KRaftVersion kraftVersi ElectionState.withElectedLeader( logEndEpoch, voter2.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -1988,7 +1998,10 @@ public void testProspectiveWithLeaderToFollowerHigherEpoch(KRaftVersion kraftVer ReplicaKey voter1 = ReplicaKey.of(1, Uuid.randomUuid()); ReplicaKey voter2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, voter1, voter2)); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, voter1.id(), voters.voterIds()), kraftVersion); + store.writeElectionState( + ElectionState.withElectedLeader(logEndEpoch, voter1.id(), Optional.empty(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); @@ -1998,6 +2011,7 @@ public void testProspectiveWithLeaderToFollowerHigherEpoch(KRaftVersion kraftVer ElectionState.withElectedLeader( 10, voter2.id(), + Optional.empty(), persistedVoters(voters.voterIds(), kraftVersion) ), store.readElectionState().get() @@ -2011,7 +2025,10 @@ public void testProspectiveWithLeaderToCandidate(KRaftVersion kraftVersion) { int followerId = 2; VoterSet voters = localWithRemoteVoterSet(IntStream.of(leaderId, followerId), kraftVersion); - store.writeElectionState(ElectionState.withElectedLeader(logEndEpoch, leaderId, voters.voterIds()), kraftVersion); + store.writeElectionState( + ElectionState.withElectedLeader(logEndEpoch, leaderId, Optional.empty(), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToProspective(); @@ -2022,12 +2039,12 @@ public void testProspectiveWithLeaderToCandidate(KRaftVersion kraftVersion) { assertTrue(state.isCandidate()); assertEquals(logEndEpoch + 1, state.epoch()); assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - logEndEpoch + 1, - persistedVotedKey(localVoterKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); } @@ -2038,7 +2055,7 @@ public void testProspectiveVotedWithLeaderToUnattachedInSameEpoch(KRaftVersion k ReplicaKey candidate = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterKey, leader, candidate)); store.writeElectionState( - ElectionState.withElectedLeader(logEndEpoch, leader.id(), voters.voterIds()), + ElectionState.withElectedLeader(logEndEpoch, leader.id(), Optional.empty(), voters.voterIds()), kraftVersion ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); @@ -2050,9 +2067,9 @@ public void testProspectiveVotedWithLeaderToUnattachedInSameEpoch(KRaftVersion k state.transitionToUnattached(state.epoch(), OptionalInt.of(candidate.id())); assertEquals( - new ElectionState( + ElectionState.withElectedLeader( logEndEpoch, - OptionalInt.of(candidate.id()), + candidate.id(), Optional.of(persistedVotedKey(candidate, kraftVersion)), persistedVoters(voters.voterIds(), kraftVersion) ), @@ -2090,13 +2107,8 @@ public void testCandidateToUnattachedHigherEpoch(KRaftVersion kraftVersion) { assertEquals(5, state.epoch()); assertEquals(OptionalInt.empty(), state.leaderId()); assertEquals( - Optional.of( - ElectionState.withUnknownLeader( - 5, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() + ElectionState.withUnknownLeader(5, persistedVoters(voters.voterIds(), kraftVersion)), + store.readElectionState().get() ); } @@ -2115,15 +2127,13 @@ public void testCandidateToFollowerSameEpoch(KRaftVersion kraftVersion) { assertEquals(1, state.epoch()); assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( - Optional.of( - new ElectionState( - 1, - OptionalInt.of(otherNodeId), - Optional.of(persistedVotedKey(localVoterKey, kraftVersion)), - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 1, + otherNodeId, + Optional.of(persistedVotedKey(localVoterKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -2142,14 +2152,13 @@ public void testCandidateToFollowerHigherEpoch(KRaftVersion kraftVersion) { assertEquals(5, state.epoch()); assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - otherNodeId, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 5, + otherNodeId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -2186,12 +2195,12 @@ public void testCandidateToProspective(KRaftVersion kraftVersion) { prospective.remainingElectionTimeMs(time.milliseconds()) ); assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 1, - persistedVotedKey(localVoterKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion))), - store.readElectionState() + ElectionState.withVotedCandidate( + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); } @@ -2280,22 +2289,16 @@ public void testCandidateToAnyStateLowerEpoch(KRaftVersion kraftVersion) { assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); assertThrows( IllegalStateException.class, - () -> state.transitionToFollower( - 4, - otherNodeKey.id(), - voters.listeners(otherNodeKey.id()) - ) + () -> state.transitionToFollower(4, otherNodeKey.id(), voters.listeners(otherNodeKey.id())) ); assertEquals(6, state.epoch()); assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 6, - persistedVotedKey(localVoterKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withVotedCandidate( + 6, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -2331,13 +2334,8 @@ public void testLeaderToUnattachedHigherEpoch(KRaftVersion kraftVersion) { assertEquals(5, state.epoch()); assertEquals(OptionalInt.empty(), state.leaderId()); assertEquals( - Optional.of( - ElectionState.withUnknownLeader( - 5, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() + ElectionState.withUnknownLeader(5, persistedVoters(voters.voterIds(), kraftVersion)), + store.readElectionState().get() ); } @@ -2376,14 +2374,13 @@ public void testLeaderToFollowerHigherEpoch(KRaftVersion kraftVersion) { assertEquals(5, state.epoch()); assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - otherNodeId, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 5, + otherNodeId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -2462,7 +2459,7 @@ public void testLeaderToResigned(KRaftVersion kraftVersion) { assertTrue(state.isResigned()); ResignedState resignedState = state.resignedStateOrThrow(); assertEquals( - ElectionState.withElectedLeader(1, localId, voters.voterIds()), + ElectionState.withElectedLeader(1, localId, Optional.empty(), voters.voterIds()), resignedState.election() ); assertEquals(1, resignedState.epoch()); @@ -2492,14 +2489,13 @@ public void testLeaderToAnyStateLowerEpoch(KRaftVersion kraftVersion) { ); assertEquals(6, state.epoch()); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 6, - localId, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 6, + localId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -2514,7 +2510,7 @@ public void testResignedToFollowerInSameEpoch(KRaftVersion kraftVersion) { int node2 = 2; int epoch = 5; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - ElectionState election = ElectionState.withElectedLeader(epoch, localId, voters.voterIds()); + ElectionState election = ElectionState.withElectedLeader(epoch, localId, Optional.empty(), voters.voterIds()); store.writeElectionState(election, kraftVersion); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -2531,7 +2527,7 @@ public void testResignedToUnattachedInHigherEpoch(KRaftVersion kraftVersion) { int node2 = 2; int epoch = 5; VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); - ElectionState election = ElectionState.withElectedLeader(epoch, localId, voters.voterIds()); + ElectionState election = ElectionState.withElectedLeader(epoch, localId, Optional.empty(), voters.voterIds()); store.writeElectionState(election, kraftVersion); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -2597,19 +2593,19 @@ public void testObserverUnattachedToProspective(KRaftVersion kraftVersion) { assertTrue(state.isObserver()); assertTrue(state.isUnattachedNotVoted()); - assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); + assertThrows(IllegalStateException.class, state::transitionToProspective); state.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID)); assertTrue(state.isUnattachedAndVoted()); assertTrue(state.isObserver()); - assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); + assertThrows(IllegalStateException.class, state::transitionToProspective); state.transitionToFollower(logEndEpoch, voter2, voters.listeners(voter2)); assertTrue(state.isFollower()); assertTrue(state.isObserver()); assertTrue(state.votedKey().isPresent()); assertTrue(state.hasLeader()); - assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); + assertThrows(IllegalStateException.class, state::transitionToProspective); } @ParameterizedTest @@ -2671,8 +2667,8 @@ public void testObserverFollowerToProspectiveOrCandidateOrLeaderOrResigned(KRaft assertTrue(state.isObserver()); assertEquals(fetchTimeoutMs, state.followerStateOrThrow().remainingFetchTimeMs(time.milliseconds())); - assertThrows(IllegalStateException.class, () -> state.transitionToProspective()); - assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); + assertThrows(IllegalStateException.class, state::transitionToProspective); + assertThrows(IllegalStateException.class, state::transitionToCandidate); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(logEndEpoch + 1, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 1b19a49d5f857..5af5a98a1723d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -220,7 +220,7 @@ public Builder( Builder withElectedLeader(int epoch, int leaderId) { quorumStateStore.writeElectionState( - ElectionState.withElectedLeader(epoch, leaderId, startingVoters.voterIds()), + ElectionState.withElectedLeader(epoch, leaderId, Optional.empty(), startingVoters.voterIds()), kraftVersion ); return this; @@ -703,7 +703,7 @@ void assertVotedCandidate(int epoch, ReplicaKey candidateKey) { public void assertElectedLeader(int epoch, int leaderId) { assertEquals( - ElectionState.withElectedLeader(epoch, leaderId, expectedVoters()), + ElectionState.withElectedLeader(epoch, leaderId, Optional.empty(), expectedVoters()), quorumStateStore.readElectionState().get() ); } @@ -737,7 +737,7 @@ void assertUnknownLeaderAndNoVotedCandidate(int epoch) { void assertResignedLeader(int epoch, int leaderId) { assertTrue(client.quorum().isResigned()); assertEquals( - ElectionState.withElectedLeader(epoch, leaderId, expectedVoters()), + ElectionState.withElectedLeader(epoch, leaderId, Optional.empty(), expectedVoters()), quorumStateStore.readElectionState().get() ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java index ba867a01e0d6c..f4fc143e2eb2e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java @@ -25,6 +25,7 @@ import java.net.InetSocketAddress; import java.util.Collections; +import java.util.Optional; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -67,7 +68,7 @@ public void testResignedState() { ResignedState state = newResignedState(voters); - assertEquals(ElectionState.withElectedLeader(epoch, localId, voters), state.election()); + assertEquals(ElectionState.withElectedLeader(epoch, localId, Optional.empty(), voters), state.election()); assertEquals(epoch, state.epoch()); assertEquals(Collections.singleton(remoteId), state.unackedVoters()); @@ -113,7 +114,7 @@ void testNegativeScenarioAcknowledgeResignation() { ResignedState state = newResignedState(voters); - assertEquals(ElectionState.withElectedLeader(epoch, 0, voters), state.election()); + assertEquals(ElectionState.withElectedLeader(epoch, 0, Optional.empty(), voters), state.election()); assertEquals(epoch, state.epoch()); // try non-existed voter must throw an exception diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java index bfc8b03e3538d..4411e7a17cc8a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java @@ -130,7 +130,7 @@ void testGrantVoteWithLeader(boolean isLogUpToDate) { UnattachedState state = newUnattachedState(OptionalInt.of(leaderId), Optional.empty()); // Check that the leader is persisted if the leader is known - assertEquals(ElectionState.withElectedLeader(epoch, leaderId, voters), state.election()); + assertEquals(ElectionState.withElectedLeader(epoch, leaderId, Optional.empty(), voters), state.election()); // Check that the replica can grant PreVotes if the log is up-to-date, even if the last leader is known // This is because nodes in Unattached have not successfully fetched from the leader yet From 2de09bcc30ad45ac418ef34a53d37da9740ec0bc Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Thu, 2 Jan 2025 12:31:38 -0800 Subject: [PATCH 19/23] only voter logic --- .../apache/kafka/raft/KafkaRaftClient.java | 37 ++++++++----------- .../kafka/raft/internals/EpochElection.java | 2 +- .../kafka/raft/KafkaRaftClientTest.java | 13 +++++++ 3 files changed, 30 insertions(+), 22 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 683a43b9f7e6d..2049638be586d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -544,18 +544,17 @@ public void initialize( long currentTimeMs = time.milliseconds(); if (quorum.isLeader()) { throw new IllegalStateException("Voter cannot initialize as a Leader"); + } else if (quorum.isOnlyVoter() && (quorum.isUnattached() || quorum.isFollower())) { + // When there is only a single voter, become leader immediately. + // transitionToProspective will handle short-circuiting voter to candidate state + // and transitionToCandidate will handle short-circuiting voter to leader state + transitionToProspective(currentTimeMs); } else if (quorum.isCandidate()) { onBecomeCandidate(currentTimeMs); } else if (quorum.isFollower()) { onBecomeFollower(currentTimeMs); } - // When there is only a single voter, become prospective immediately. - // transitionToProspective will handle short-circuiting voter to candidate state - if (quorum.isOnlyVoter() && !quorum.isNomineeState() && !quorum.isLeader()) { - transitionToProspective(currentTimeMs); - } - // Specialized add voter handler this.addVoterHandler = new AddVoterHandler( partitionState, @@ -668,8 +667,7 @@ private boolean maybeTransitionToLeader(CandidateState state, long currentTimeMs } private boolean maybeTransitionToCandidate(ProspectiveState state, long currentTimeMs) { - // If replica is the only voter, it should transition to candidate immediately - if (state.epochElection().isVoteGranted() || quorum.isOnlyVoter()) { + if (state.epochElection().isVoteGranted()) { transitionToCandidate(currentTimeMs); return true; } else { @@ -681,15 +679,14 @@ private boolean maybeTransitionToCandidate(ProspectiveState state, long currentT * Only applies to NomineeStates (Prospective or Candidate). If enough votes were granted * then this method is called to transition the state forward - either from Prospective to Candidate * or from Candidate to Leader. - * @return true if the state transitioned forward, false otherwise */ private void maybeTransitionForward(NomineeState state, long currentTimeMs) { - if (quorum.isProspective()) { - maybeTransitionToCandidate(quorum.prospectiveStateOrThrow(), currentTimeMs); - } else if (quorum.isCandidate()) { - maybeTransitionToLeader(quorum.candidateStateOrThrow(), currentTimeMs); + if (state instanceof ProspectiveState prospective) { + maybeTransitionToCandidate(prospective, currentTimeMs); + } else if (state instanceof CandidateState candidate) { + maybeTransitionToLeader(candidate, currentTimeMs); } else { - throw new IllegalStateException("Expected to be a VotingState (Prospective or Candidate), " + + throw new IllegalStateException("Expected to be a NomineeState (Prospective or Candidate), " + "but current state is " + state); } } @@ -983,7 +980,7 @@ private boolean handleVoteResponse( maybeTransitionForward(state, currentTimeMs); } else { state.recordRejectedVote(remoteNodeId); - maybeHandleElectionLoss(currentTimeMs); + maybeHandleElectionLoss(state, currentTimeMs); } } else { logger.debug( @@ -1003,9 +1000,8 @@ private boolean handleVoteResponse( * On election loss, if replica is prospective it will transition to unattached or follower state. * If replica is candidate, it will start backing off. */ - private void maybeHandleElectionLoss(long currentTimeMs) { - if (quorum.isCandidate()) { - CandidateState candidate = quorum.candidateStateOrThrow(); + private void maybeHandleElectionLoss(NomineeState state, long currentTimeMs) { + if (state instanceof CandidateState candidate) { if (candidate.epochElection().isVoteRejected() && !candidate.isBackingOff()) { logger.info( "Insufficient remaining votes to become leader. We will backoff before retrying election again. " + @@ -1022,8 +1018,7 @@ private void maybeHandleElectionLoss(long currentTimeMs) { binaryExponentialElectionBackoffMs(candidate.retries()) ); } - } else if (quorum.isProspective()) { - ProspectiveState prospective = quorum.prospectiveStateOrThrow(); + } else if (state instanceof ProspectiveState prospective) { if (prospective.epochElection().isVoteRejected()) { logger.info( "Insufficient remaining votes to become candidate. Current epoch election state is {}. ", @@ -1033,7 +1028,7 @@ private void maybeHandleElectionLoss(long currentTimeMs) { } } else { throw new IllegalStateException( - "Expected to be a NomineeState (Prospective or Candidate), but quorum state is " + quorum + "Expected to be a NomineeState (Prospective or Candidate), but quorum state is " + state ); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java index 538553a99cdc3..dfb77848087e0 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java @@ -193,7 +193,7 @@ enum State { @Override public String toString() { return String.format( - "VoterState(%s, state=%s)", + "VoterState(replicaKey=%s, state=%s)", replicaKey, state ); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 86d3cdf129d20..06e657915b8d9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -1016,6 +1016,19 @@ public void testInitializeAsOnlyVoterWithEmptyElectionState(boolean withKip853Rp assertTrue(context.client.quorum().isLeader()); } + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testInitializeAsFollowerAndOnlyVoter(boolean withKip853Rpc) throws Exception { + int localId = randomReplicaId(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + .withKip853Rpc(withKip853Rpc) + .withElectedLeader(2, localId + 1) + .build(); + context.assertElectedLeader(3, localId); + assertEquals(0L, context.log.endOffset().offset()); + assertTrue(context.client.quorum().isLeader()); + } + @ParameterizedTest @ValueSource(booleans = { true, false }) public void testInitializeAsCandidateAndOnlyVoter(boolean withKip853Rpc) throws Exception { From d034c8385ba6aedff5382e21a15e591aee382fe7 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Thu, 2 Jan 2025 12:39:03 -0800 Subject: [PATCH 20/23] nits --- raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java | 2 +- .../java/org/apache/kafka/raft/internals/EpochElection.java | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 2049638be586d..751ea04fdb6eb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -1028,7 +1028,7 @@ private void maybeHandleElectionLoss(NomineeState state, long currentTimeMs) { } } else { throw new IllegalStateException( - "Expected to be a NomineeState (Prospective or Candidate), but quorum state is " + state + "Expected to be a NomineeState (Prospective or Candidate), but current state is " + state ); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java index dfb77848087e0..da3e29b71c22f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java @@ -42,6 +42,7 @@ public EpochElection(Set voters) { /** * Record a vote from a voter. + * * @param voterId The id of the voter * @param isGranted true if the vote is granted, false if it is rejected * @return true if the voter had not been previously recorded @@ -59,6 +60,7 @@ public boolean recordVote(int voterId, boolean isGranted) { /** * Returns if a voter has granted the vote. + * * @param voterId The id of the voter * @throws IllegalArgumentException if the voter is not in the set of voters */ @@ -68,6 +70,7 @@ public boolean isGrantedVoter(int voterId) { /** * Returns if a voter has rejected the vote. + * * @param voterId The id of the voter * @throws IllegalArgumentException if the voter is not in the set of voters */ From 578e4c357d56eaa70302e970186319ae4d2a70b1 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Fri, 3 Jan 2025 00:20:12 -0800 Subject: [PATCH 21/23] some test improvements, onlyvoter fix --- .../apache/kafka/raft/KafkaRaftClient.java | 2 +- .../kafka/raft/internals/EpochElection.java | 2 +- .../apache/kafka/raft/CandidateStateTest.java | 17 +- .../apache/kafka/raft/ElectionStateTest.java | 12 +- .../apache/kafka/raft/EpochElectionTest.java | 82 +++--- .../kafka/raft/FileQuorumStateStoreTest.java | 14 +- .../apache/kafka/raft/FollowerStateTest.java | 34 +-- .../raft/KafkaRaftClientPreVoteTest.java | 236 +++++++++--------- .../kafka/raft/RaftClientTestContext.java | 17 +- 9 files changed, 221 insertions(+), 195 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 751ea04fdb6eb..253d25cd84739 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -544,7 +544,7 @@ public void initialize( long currentTimeMs = time.milliseconds(); if (quorum.isLeader()) { throw new IllegalStateException("Voter cannot initialize as a Leader"); - } else if (quorum.isOnlyVoter() && (quorum.isUnattached() || quorum.isFollower())) { + } else if (quorum.isOnlyVoter() && (quorum.isUnattached() || quorum.isFollower() || quorum.isResigned())) { // When there is only a single voter, become leader immediately. // transitionToProspective will handle short-circuiting voter to candidate state // and transitionToCandidate will handle short-circuiting voter to leader state diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java index da3e29b71c22f..e15092ddc2bb0 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java @@ -70,7 +70,7 @@ public boolean isGrantedVoter(int voterId) { /** * Returns if a voter has rejected the vote. - * + * * @param voterId The id of the voter * @throws IllegalArgumentException if the voter is not in the set of voters */ diff --git a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java index 84b6c4fb67cf7..217efad22f5f8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java @@ -45,16 +45,15 @@ public class CandidateStateTest { private CandidateState newCandidateState(VoterSet voters) { return new CandidateState( - time, - localReplicaKey.id(), - localReplicaKey.directoryId().get(), - epoch, - voters, - Optional.empty(), - 1, - + time, + localReplicaKey.id(), + localReplicaKey.directoryId().get(), + epoch, + voters, + Optional.empty(), + 1, electionTimeoutMs, - logContext + logContext ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java index af443bd1dd18d..85a3ca3951a07 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java @@ -72,7 +72,8 @@ void testQuorumStateDataRoundTrip(short version) { List electionStates = Arrays.asList( ElectionState.withUnknownLeader(5, Set.of(1, 2, 3)), ElectionState.withElectedLeader(5, 1, Optional.empty(), Set.of(1, 2, 3)), - ElectionState.withVotedCandidate(5, votedKey, Set.of(1, 2, 3)) + ElectionState.withVotedCandidate(5, votedKey, Set.of(1, 2, 3)), + ElectionState.withElectedLeader(5, 1, Optional.of(votedKey), Set.of(1, 2, 3)) ); final List expected; @@ -84,13 +85,20 @@ void testQuorumStateDataRoundTrip(short version) { 5, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), Set.of(1, 2, 3) + ), + ElectionState.withElectedLeader( + 5, + 1, + Optional.of(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)), + Set.of(1, 2, 3) ) ); } else { expected = Arrays.asList( ElectionState.withUnknownLeader(5, Collections.emptySet()), ElectionState.withElectedLeader(5, 1, Optional.empty(), Collections.emptySet()), - ElectionState.withVotedCandidate(5, votedKey, Collections.emptySet()) + ElectionState.withVotedCandidate(5, votedKey, Collections.emptySet()), + ElectionState.withElectedLeader(5, 1, Optional.of(votedKey), Collections.emptySet()) ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java b/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java index c528e74610942..e14e4cb17ffe8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java @@ -29,111 +29,111 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class EpochElectionTest { - static final int VOTER_1_ID = randomReplicaId(); - static final Set VOTERS = Set.of( - ReplicaKey.of(VOTER_1_ID, Uuid.randomUuid()), - ReplicaKey.of(VOTER_1_ID + 1, Uuid.randomUuid()), - ReplicaKey.of(VOTER_1_ID + 2, Uuid.randomUuid()) + private final int voter1 = randomReplicaId(); + private final Set voters = Set.of( + ReplicaKey.of(voter1, Uuid.randomUuid()), + ReplicaKey.of(voter1 + 1, Uuid.randomUuid()), + ReplicaKey.of(voter1 + 2, Uuid.randomUuid()) ); @Test public void testStateOnInitialization() { - EpochElection epochElection = new EpochElection(VOTERS); + EpochElection epochElection = new EpochElection(voters); - assertEquals(VOTERS, epochElection.unrecordedVoters()); + assertEquals(voters, epochElection.unrecordedVoters()); assertTrue(epochElection.grantingVoters().isEmpty()); assertTrue(epochElection.rejectingVoters().isEmpty()); assertFalse(epochElection.isVoteGranted()); assertFalse(epochElection.isVoteRejected()); - assertFalse(epochElection.isGrantedVoter(VOTER_1_ID)); - assertFalse(epochElection.isRejectedVoter(VOTER_1_ID)); + assertFalse(epochElection.isGrantedVoter(voter1)); + assertFalse(epochElection.isRejectedVoter(voter1)); } @Test public void testRecordGrantedVote() { - EpochElection epochElection = new EpochElection(VOTERS); + EpochElection epochElection = new EpochElection(voters); - assertTrue(epochElection.recordVote(VOTER_1_ID, true)); + assertTrue(epochElection.recordVote(voter1, true)); assertEquals(1, epochElection.grantingVoters().size()); - assertTrue(epochElection.grantingVoters().contains(VOTER_1_ID)); + assertTrue(epochElection.grantingVoters().contains(voter1)); assertEquals(0, epochElection.rejectingVoters().size()); assertEquals(2, epochElection.unrecordedVoters().size()); - assertTrue(epochElection.isGrantedVoter(VOTER_1_ID)); - assertFalse(epochElection.isRejectedVoter(VOTER_1_ID)); + assertTrue(epochElection.isGrantedVoter(voter1)); + assertFalse(epochElection.isRejectedVoter(voter1)); assertFalse(epochElection.isVoteGranted()); assertFalse(epochElection.isVoteRejected()); // recording same id as granted - assertFalse(epochElection.recordVote(VOTER_1_ID, true)); - assertTrue(epochElection.isGrantedVoter(VOTER_1_ID)); + assertFalse(epochElection.recordVote(voter1, true)); + assertTrue(epochElection.isGrantedVoter(voter1)); assertFalse(epochElection.isVoteGranted()); // recording majority as granted - assertTrue(epochElection.recordVote(VOTER_1_ID + 1, true)); + assertTrue(epochElection.recordVote(voter1 + 1, true)); assertEquals(2, epochElection.grantingVoters().size()); assertEquals(0, epochElection.rejectingVoters().size()); assertEquals(1, epochElection.unrecordedVoters().size()); - assertTrue(epochElection.isGrantedVoter(VOTER_1_ID + 1)); - assertFalse(epochElection.isRejectedVoter(VOTER_1_ID + 1)); + assertTrue(epochElection.isGrantedVoter(voter1 + 1)); + assertFalse(epochElection.isRejectedVoter(voter1 + 1)); assertTrue(epochElection.isVoteGranted()); assertFalse(epochElection.isVoteRejected()); } @Test public void testRecordRejectedVote() { - EpochElection epochElection = new EpochElection(VOTERS); + EpochElection epochElection = new EpochElection(voters); - assertTrue(epochElection.recordVote(VOTER_1_ID, false)); + assertTrue(epochElection.recordVote(voter1, false)); assertEquals(0, epochElection.grantingVoters().size()); assertEquals(1, epochElection.rejectingVoters().size()); - assertTrue(epochElection.rejectingVoters().contains(VOTER_1_ID)); + assertTrue(epochElection.rejectingVoters().contains(voter1)); assertEquals(2, epochElection.unrecordedVoters().size()); - assertFalse(epochElection.isGrantedVoter(VOTER_1_ID)); - assertTrue(epochElection.isRejectedVoter(VOTER_1_ID)); + assertFalse(epochElection.isGrantedVoter(voter1)); + assertTrue(epochElection.isRejectedVoter(voter1)); assertFalse(epochElection.isVoteGranted()); assertFalse(epochElection.isVoteRejected()); // recording same id as rejected - assertFalse(epochElection.recordVote(VOTER_1_ID, false)); - assertFalse(epochElection.isGrantedVoter(VOTER_1_ID)); + assertFalse(epochElection.recordVote(voter1, false)); + assertFalse(epochElection.isGrantedVoter(voter1)); assertFalse(epochElection.isVoteRejected()); // recording majority as rejected - assertTrue(epochElection.recordVote(VOTER_1_ID + 1, false)); + assertTrue(epochElection.recordVote(voter1 + 1, false)); assertEquals(0, epochElection.grantingVoters().size()); assertEquals(2, epochElection.rejectingVoters().size()); assertEquals(1, epochElection.unrecordedVoters().size()); - assertFalse(epochElection.isGrantedVoter(VOTER_1_ID + 1)); - assertTrue(epochElection.isRejectedVoter(VOTER_1_ID + 1)); + assertFalse(epochElection.isGrantedVoter(voter1 + 1)); + assertTrue(epochElection.isRejectedVoter(voter1 + 1)); assertFalse(epochElection.isVoteGranted()); assertTrue(epochElection.isVoteRejected()); } @Test public void testOverWritingVote() { - EpochElection epochElection = new EpochElection(VOTERS); + EpochElection epochElection = new EpochElection(voters); - assertTrue(epochElection.recordVote(VOTER_1_ID, true)); - assertFalse(epochElection.recordVote(VOTER_1_ID, false)); + assertTrue(epochElection.recordVote(voter1, true)); + assertFalse(epochElection.recordVote(voter1, false)); assertEquals(0, epochElection.grantingVoters().size()); assertEquals(1, epochElection.rejectingVoters().size()); - assertTrue(epochElection.rejectingVoters().contains(VOTER_1_ID)); - assertFalse(epochElection.isGrantedVoter(VOTER_1_ID)); - assertTrue(epochElection.isRejectedVoter(VOTER_1_ID)); + assertTrue(epochElection.rejectingVoters().contains(voter1)); + assertFalse(epochElection.isGrantedVoter(voter1)); + assertTrue(epochElection.isRejectedVoter(voter1)); assertFalse(epochElection.isVoteGranted()); assertFalse(epochElection.isVoteRejected()); - assertTrue(epochElection.recordVote(VOTER_1_ID + 2, false)); - assertFalse(epochElection.recordVote(VOTER_1_ID + 2, true)); + assertTrue(epochElection.recordVote(voter1 + 2, false)); + assertFalse(epochElection.recordVote(voter1 + 2, true)); assertEquals(1, epochElection.grantingVoters().size()); assertEquals(1, epochElection.rejectingVoters().size()); - assertTrue(epochElection.grantingVoters().contains(VOTER_1_ID + 2)); - assertTrue(epochElection.isGrantedVoter(VOTER_1_ID + 2)); - assertFalse(epochElection.isRejectedVoter(VOTER_1_ID + 2)); + assertTrue(epochElection.grantingVoters().contains(voter1 + 2)); + assertTrue(epochElection.isGrantedVoter(voter1 + 2)); + assertFalse(epochElection.isRejectedVoter(voter1 + 2)); assertFalse(epochElection.isVoteGranted()); assertFalse(epochElection.isVoteRejected()); } - static int randomReplicaId() { + private static int randomReplicaId() { return ThreadLocalRandom.current().nextInt(1025); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java b/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java index a265803d72de8..a0893ae2aa0f9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java @@ -56,20 +56,28 @@ void testWriteReadElectedLeader(KRaftVersion kraftVersion) throws IOException { final int voter1 = 1; final int voter2 = 2; final int voter3 = 3; + ReplicaKey votedKey = ReplicaKey.of(voter1, Uuid.randomUuid()); Set voters = Set.of(voter1, voter2, voter3); stateStore.writeElectionState( - ElectionState.withElectedLeader(epoch, voter1, Optional.empty(), voters), + ElectionState.withElectedLeader(epoch, voter1, Optional.of(votedKey), voters), kraftVersion ); final Optional expected; if (kraftVersion.isReconfigSupported()) { expected = Optional.of( - ElectionState.withElectedLeader(epoch, voter1, Optional.empty(), Collections.emptySet()) + ElectionState.withElectedLeader(epoch, voter1, Optional.of(votedKey), Collections.emptySet()) ); } else { - expected = Optional.of(ElectionState.withElectedLeader(epoch, voter1, Optional.empty(), voters)); + expected = Optional.of( + ElectionState.withElectedLeader( + epoch, + voter1, + Optional.of(ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID)), + voters + ) + ); } assertEquals(expected, stateStore.readElectionState()); diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index bb1a016c4fef1..c7e86c3fe4946 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -51,8 +51,7 @@ public class FollowerStateTest { private final ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); private FollowerState newFollowerState( - Set voters, - Optional highWatermark + Set voters ) { return new FollowerState( time, @@ -61,7 +60,7 @@ private FollowerState newFollowerState( leaderEndpoints, Optional.of(votedKey), voters, - highWatermark, + Optional.empty(), fetchTimeoutMs, logContext ); @@ -69,7 +68,7 @@ private FollowerState newFollowerState( @Test public void testFetchTimeoutExpiration() { - FollowerState state = newFollowerState(Set.of(1, 2, 3), Optional.empty()); + FollowerState state = newFollowerState(Set.of(1, 2, 3)); assertFalse(state.hasFetchTimeoutExpired(time.milliseconds())); assertEquals(fetchTimeoutMs, state.remainingFetchTimeMs(time.milliseconds())); @@ -85,7 +84,7 @@ public void testFetchTimeoutExpiration() { @Test public void testMonotonicHighWatermark() { - FollowerState state = newFollowerState(Set.of(1, 2, 3), Optional.empty()); + FollowerState state = newFollowerState(Set.of(1, 2, 3)); OptionalLong highWatermark = OptionalLong.of(15L); state.updateHighWatermark(highWatermark); @@ -98,10 +97,7 @@ public void testMonotonicHighWatermark() { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testPreVoteIfHasNotFetchedFromLeaderYet(boolean isLogUpToDate) { - FollowerState state = newFollowerState( - Set.of(1, 2, 3), - Optional.empty() - ); + FollowerState state = newFollowerState(Set.of(1, 2, 3)); assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); assertEquals(isLogUpToDate, state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); @@ -111,11 +107,8 @@ public void testPreVoteIfHasNotFetchedFromLeaderYet(boolean isLogUpToDate) { @ParameterizedTest @ValueSource(booleans = {true, false}) - public void testGrantVoteAfterSuccessfulFetchFromLeader(boolean isLogUpToDate) { - FollowerState state = newFollowerState( - Set.of(1, 2, 3), - Optional.empty() - ); + public void testPreVoteAfterSuccessfulFetchFromLeader(boolean isLogUpToDate) { + FollowerState state = newFollowerState(Set.of(1, 2, 3)); state.resetFetchTimeoutForSuccessfulFetch(time.milliseconds()); assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); @@ -129,9 +122,20 @@ public void testGrantVoteAfterSuccessfulFetchFromLeader(boolean isLogUpToDate) { assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGrantStandardVote(boolean isLogUpToDate) { + FollowerState state = newFollowerState(Set.of(1, 2, 3)); + + assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + } + @Test public void testLeaderIdAndEndpoint() { - FollowerState state = newFollowerState(Set.of(0, 1, 2), Optional.empty()); + FollowerState state = newFollowerState(Set.of(0, 1, 2)); assertEquals(leaderId, state.leaderId()); assertEquals(leaderEndpoints, state.leaderEndpoints()); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 6669b2b032c8d..6083f3e7b5ee9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -32,9 +32,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.OptionalInt; -import java.util.Set; import java.util.stream.Stream; import static org.apache.kafka.raft.KafkaRaftClientTest.randomReplicaId; @@ -46,7 +44,7 @@ public class KafkaRaftClientPreVoteTest { @ParameterizedTest @MethodSource("kraftVersionHasFetchedCombinations") - public void testHandlePreVoteRequestAsFollowerWithElectedLeader( + public void testHandlePreVoteRequestAsFollower( KRaftVersion kraftVersion, boolean hasFetchedFromLeader ) throws Exception { @@ -58,8 +56,8 @@ public void testHandlePreVoteRequestAsFollowerWithElectedLeader( int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(local, otherNodeKey, electedLeader))), + local, + VoterSetTest.voterSet(Stream.of(local, otherNodeKey, electedLeader)), kraftVersion ) .withElectedLeader(epoch, electedLeader.id()) @@ -78,19 +76,18 @@ public void testHandlePreVoteRequestAsFollowerWithElectedLeader( ); } - // follower should reject pre-vote requests with the same epoch if it has successfully fetched from the leader context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, epoch, 1)); context.pollUntilResponse(); - boolean voteGranted = !hasFetchedFromLeader; - context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(electedLeader.id()), voteGranted); + // follower should reject pre-vote requests if it has successfully fetched from the leader + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(electedLeader.id()), !hasFetchedFromLeader); context.assertElectedLeader(epoch, electedLeader.id()); // same with observers context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 1)); context.pollUntilResponse(); - context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(electedLeader.id()), voteGranted); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(electedLeader.id()), !hasFetchedFromLeader); context.assertElectedLeader(epoch, electedLeader.id()); // follower will transition to unattached if pre-vote request has a higher epoch @@ -112,11 +109,7 @@ public void testHandlePreVoteRequestAsFollowerWithVotedCandidate(KRaftVersion kr ReplicaKey votedCandidateKey = replicaKey(localId + 2, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey, votedCandidateKey)); - RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(localKey), - Optional.of(voters), - kraftVersion - ) + RaftClientTestContext context = new RaftClientTestContext.Builder(localKey, voters, kraftVersion) .withVotedCandidate(epoch, votedCandidateKey) .withRaftProtocol(KIP_996_PROTOCOL) .build(); @@ -155,39 +148,39 @@ public void testHandlePreVoteRequestAsCandidate(KRaftVersion kraftVersion) throw ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); ReplicaKey observer = replicaKey(localId + 2, true); - int leaderEpoch = 2; + int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(localKey), - Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey))), + localKey, + VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), kraftVersion ) - .withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, localKey.directoryId().get())) + .withVotedCandidate(epoch, ReplicaKey.of(localId, localKey.directoryId().get())) .withRaftProtocol(KIP_996_PROTOCOL) .build(); assertTrue(context.client.quorum().isCandidate()); // candidate should grant pre-vote requests with the same epoch if log is up-to-date - context.deliverRequest(context.preVoteRequest(leaderEpoch, otherNodeKey, leaderEpoch, 1)); + context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, epoch, 1)); context.pollUntilResponse(); - context.assertSentVoteResponse(Errors.NONE, leaderEpoch, OptionalInt.empty(), true); - context.assertVotedCandidate(leaderEpoch, localKey); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); + context.assertVotedCandidate(epoch, localKey); assertTrue(context.client.quorum().isCandidate()); - // if an observer sends a pre-vote request for the same epoch, it should also be granted - context.deliverRequest(context.preVoteRequest(leaderEpoch, observer, leaderEpoch, 2)); + // if an observer with up-to-date log sends a pre-vote request for the same epoch, it should also be granted + context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 2)); context.pollUntilResponse(); - context.assertSentVoteResponse(Errors.NONE, leaderEpoch, OptionalInt.empty(), true); - context.assertVotedCandidate(leaderEpoch, localKey); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); + context.assertVotedCandidate(epoch, localKey); assertTrue(context.client.quorum().isCandidate()); // candidate will transition to unattached if pre-vote request has a higher epoch - context.deliverRequest(context.preVoteRequest(leaderEpoch + 1, otherNodeKey, leaderEpoch + 1, 2)); + context.deliverRequest(context.preVoteRequest(epoch + 1, otherNodeKey, epoch + 1, 2)); context.pollUntilResponse(); - context.assertSentVoteResponse(Errors.NONE, leaderEpoch + 1, OptionalInt.of(-1), true); + context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.of(-1), true); assertTrue(context.client.quorum().isUnattached()); } @@ -202,40 +195,39 @@ public void testHandlePreVoteRequestAsUnattachedObserver(KRaftVersion kraftVersi ReplicaKey observer = replicaKey(localId + 3, true); RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(localKey), - Optional.of(VoterSetTest.voterSet(Stream.of(replica1, replica2))), + localKey, + VoterSetTest.voterSet(Stream.of(replica1, replica2)), kraftVersion ) .withUnknownLeader(epoch) .withRaftProtocol(KIP_996_PROTOCOL) .build(); + assertTrue(context.client.quorum().isUnattached()); + assertTrue(context.client.quorum().isObserver()); + // if a voter with up-to-date log sends a pre-vote request, it should be granted context.deliverRequest(context.preVoteRequest(epoch, replica1, epoch, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - // if same replica sends another pre-vote request for the same epoch, it should be granted + // if same voter sends another pre-vote request, it can be granted if the sender's log is still up-to-date context.deliverRequest(context.preVoteRequest(epoch, replica1, epoch, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - // if different replica sends a pre-vote request for the same epoch, it should be granted + // if different voter with up-to-date log sends a pre-vote request for the same epoch, it will be granted context.deliverRequest(context.preVoteRequest(epoch, replica2, epoch, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - // if an observer sends a pre-vote request for the same epoch, it should also be granted + // if an observer with up-to-date log sends a pre-vote request for the same epoch, it will be granted context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 2)); context.pollUntilResponse(); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); + assertEquals(epoch, context.currentEpoch()); assertTrue(context.client.quorum().isUnattached()); - context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); + assertTrue(context.client.quorum().isObserver()); } @ParameterizedTest @@ -249,40 +241,37 @@ public void testHandlePreVoteRequestAsUnattachedVoted(KRaftVersion kraftVersion) int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(localKey), - Optional.of(VoterSetTest.voterSet(Stream.of(replica1, replica2))), + localKey, + VoterSetTest.voterSet(Stream.of(replica1, replica2)), kraftVersion ) .withVotedCandidate(epoch, replica2) .withRaftProtocol(KIP_996_PROTOCOL) .build(); + assertTrue(context.client.quorum().isUnattachedAndVoted()); + // if a voter with up-to-date log sends a pre-vote request, it should be granted context.deliverRequest(context.preVoteRequest(epoch, replica1, epoch, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattachedAndVoted()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - // if same replica sends another pre-vote request for the same epoch, it should be granted + // if same voter sends another pre-vote request, it can be granted if the sender's log is still up-to-date context.deliverRequest(context.preVoteRequest(epoch, replica1, epoch, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - // if different replica sends a pre-vote request for the same epoch, it should be granted + // if different voter with up-to-date log sends a pre-vote request for the same epoch, it will be granted context.deliverRequest(context.preVoteRequest(epoch, replica2, epoch, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); - // if an observer sends a pre-vote request for the same epoch, it should also be granted + // if an observer with up-to-date log sends a pre-vote request for the same epoch, it will be granted context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 2)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); + + assertEquals(epoch, context.currentEpoch()); + assertTrue(context.client.quorum().isUnattachedAndVoted()); } @ParameterizedTest @@ -297,45 +286,42 @@ public void testHandlePreVoteRequestAsUnattachedWithLeader(KRaftVersion kraftVer int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(localKey), - Optional.of(VoterSetTest.voterSet(Stream.of(replica1, replica2))), + localKey, + VoterSetTest.voterSet(Stream.of(replica1, replica2)), kraftVersion ) .withElectedLeader(epoch, leader.id()) .withRaftProtocol(KIP_996_PROTOCOL) .build(); + assertTrue(context.client.quorum().isUnattachedNotVoted()); + // if a voter with up-to-date log sends a pre-vote request, it should be granted context.deliverRequest(context.preVoteRequest(epoch, replica1, epoch, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattachedNotVoted()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), true); - // if same replica sends another pre-vote request for the same epoch, it should be granted + // if same voter sends another pre-vote request, it can be granted if the sender's log is still up-to-date context.deliverRequest(context.preVoteRequest(epoch, replica1, epoch, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), true); - // if different replica sends a pre-vote request for the same epoch, it should be granted + // if different voter with up-to-date log sends a pre-vote request for the same epoch, it will be granted context.deliverRequest(context.preVoteRequest(epoch, replica2, epoch, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), true); - // if an observer sends a pre-vote request for the same epoch, it should also be granted + // if an observer with up-to-date log sends a pre-vote request for the same epoch, it will be granted context.deliverRequest(context.preVoteRequest(epoch, observer, epoch, 2)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), true); + + assertEquals(epoch, context.currentEpoch()); + assertTrue(context.client.quorum().isUnattachedNotVoted()); } @ParameterizedTest @MethodSource("kraftVersionHasFetchedCombinations") - public void testHandlePreVoteRequestAsFollowerObserverThatHasNotFetched( + public void testHandlePreVoteRequestAsFollowerObserver( KRaftVersion kraftVersion, boolean hasFetchedFromLeader ) throws Exception { @@ -346,14 +332,16 @@ public void testHandlePreVoteRequestAsFollowerObserverThatHasNotFetched( int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(localKey), - Optional.of(VoterSetTest.voterSet(Stream.of(localKey, leader, follower))), + localKey, + VoterSetTest.voterSet(Stream.of(localKey, leader, follower)), kraftVersion ) .withElectedLeader(epoch, leader.id()) .withRaftProtocol(KIP_996_PROTOCOL) .build(); context.assertElectedLeader(epoch, leader.id()); + assertTrue(context.client.quorum().isFollower()); + assertTrue(context.client.quorum().isObserver()); if (hasFetchedFromLeader) { context.pollUntilRequest(); @@ -370,20 +358,24 @@ public void testHandlePreVoteRequestAsFollowerObserverThatHasNotFetched( context.deliverRequest(context.preVoteRequest(epoch, follower, epoch, 1)); context.pollUntilResponse(); - boolean voteGranted = !hasFetchedFromLeader; - context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), voteGranted); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), !hasFetchedFromLeader); assertTrue(context.client.quorum().isFollower()); + assertTrue(context.client.quorum().isObserver()); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testHandleInvalidPreVoteRequestWithOlderEpoch() throws Exception { + public void testHandleInvalidPreVoteRequestWithOlderEpoch(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); - int epoch = 2; + ReplicaKey local = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - Set voters = Set.of(localId, otherNodeKey.id()); + int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + local, + VoterSetTest.voterSet(Stream.of(local, otherNodeKey)), + kraftVersion + ) .withUnknownLeader(epoch) .withRaftProtocol(KIP_996_PROTOCOL) .build(); @@ -397,15 +389,18 @@ public void testHandleInvalidPreVoteRequestWithOlderEpoch() throws Exception { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderRejectPreVoteRequestOnSameEpoch() throws Exception { + public void testLeaderRejectPreVoteRequestOnSameEpoch(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey, + VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), + kraftVersion + ) .withUnknownLeader(2) .withRaftProtocol(KIP_996_PROTOCOL) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); context.unattachedToLeader(); @@ -421,14 +416,17 @@ public void testLeaderRejectPreVoteRequestOnSameEpoch() throws Exception { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testPreVoteRequestClusterIdValidation() throws Exception { + public void testPreVoteRequestClusterIdValidation(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey, + VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), + kraftVersion + ) .withRaftProtocol(KIP_996_PROTOCOL) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); context.unattachedToLeader(); @@ -457,14 +455,17 @@ public void testPreVoteRequestClusterIdValidation() throws Exception { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testInvalidVoterReplicaPreVoteRequest() throws Exception { + public void testInvalidVoterReplicaPreVoteRequest(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey, + VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), + kraftVersion + ) .withRaftProtocol(KIP_996_PROTOCOL) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); context.unattachedToLeader(); @@ -503,15 +504,18 @@ public void testInvalidVoterReplicaPreVoteRequest() throws Exception { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testLeaderAcceptPreVoteFromObserver() throws Exception { + public void testLeaderAcceptPreVoteFromObserver(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, localKey.directoryId().get()) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey, + VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), + kraftVersion + ) .withUnknownLeader(4) .withRaftProtocol(KIP_996_PROTOCOL) - .withBootstrapSnapshot(Optional.of(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)))) .build(); context.unattachedToLeader(); @@ -529,12 +533,16 @@ public void testLeaderAcceptPreVoteFromObserver() throws Exception { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testHandlePreVoteRequestAsResigned() throws Exception { + public void testHandlePreVoteRequestAsResigned(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); - Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey, + VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), + kraftVersion + ) .withRaftProtocol(KIP_996_PROTOCOL) .build(); context.unattachedToLeader(); @@ -543,12 +551,12 @@ public void testHandlePreVoteRequestAsResigned() throws Exception { // resigned should grant pre-vote requests with the same epoch if log is up-to-date int epoch = context.currentEpoch(); - context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, epoch, 1)); + context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, epoch, 3)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(localId), true); // resigned will transition to unattached if pre-vote request has a higher epoch - context.deliverRequest(context.preVoteRequest(epoch + 1, otherNodeKey, epoch + 1, 1)); + context.deliverRequest(context.preVoteRequest(epoch + 1, otherNodeKey, epoch + 1, 3)); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.of(-1), true); assertTrue(context.client.quorum().isUnattached()); @@ -556,13 +564,17 @@ public void testHandlePreVoteRequestAsResigned() throws Exception { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testInvalidPreVoteRequest() throws Exception { + public void testInvalidPreVoteRequest(KRaftVersion kraftVersion) throws Exception { int localId = randomReplicaId(); + ReplicaKey localKey = replicaKey(localId, true); ReplicaKey otherNodeKey = replicaKey(localId + 1, true); int epoch = 5; - Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey, + VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), + kraftVersion + ) .withElectedLeader(epoch, otherNodeKey.id()) .withRaftProtocol(KIP_996_PROTOCOL) .build(); @@ -616,8 +628,8 @@ public void testFollowerGrantsPreVoteIfHasNotFetchedYet(KRaftVersion kraftVersio int epoch = 2; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(replica1, replica2))), + local, + VoterSetTest.voterSet(Stream.of(replica1, replica2)), kraftVersion ) .withElectedLeader(epoch, replica1.id()) @@ -663,8 +675,8 @@ public void testRejectPreVoteIfRemoteLogIsNotUpToDate(KRaftVersion kraftVersion) ReplicaKey replica1 = replicaKey(localId + 1, true); ReplicaKey replica2 = replicaKey(localId + 2, true); RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(local, replica1, replica2))), + local, + VoterSetTest.voterSet(Stream.of(local, replica1, replica2)), kraftVersion ) .withUnknownLeader(epoch) @@ -699,8 +711,8 @@ public void testPreVoteResponseIgnoredAfterBecomingFollower(KRaftVersion kraftVe int epoch = 5; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(local, voter2, voter3))), + local, + VoterSetTest.voterSet(Stream.of(local, voter2, voter3)), kraftVersion ) .withUnknownLeader(epoch) @@ -752,8 +764,8 @@ public void testPreVoteNotSupportedByRemote(KRaftVersion kraftVersion) throws Ex int epoch = 5; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(local, voter2Key, voter3Key))), + local, + VoterSetTest.voterSet(Stream.of(local, voter2Key, voter3Key)), kraftVersion ) .withUnknownLeader(epoch) @@ -837,8 +849,8 @@ public void testProspectiveReceivesBeginQuorumRequest( int epoch = 5; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(local, leader))), + local, + VoterSetTest.voterSet(Stream.of(local, leader)), kraftVersion ) .withUnknownLeader(epoch) @@ -872,8 +884,8 @@ public void testProspectiveTransitionsToUnattachedOnElectionFailure( int epoch = 5; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(local, otherNode))), + local, + VoterSetTest.voterSet(Stream.of(local, otherNode)), kraftVersion ) .withUnknownLeader(epoch) @@ -924,8 +936,8 @@ public void testProspectiveWithLeaderTransitionsToFollower( int epoch = 5; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(local, replica1, replica2))), + local, + VoterSetTest.voterSet(Stream.of(local, replica1, replica2)), kraftVersion ) .withElectedLeader(epoch, replica1.id()) @@ -1001,8 +1013,8 @@ public void testProspectiveWithoutLeaderTransitionsToFollower( int epoch = 5; RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(local, leader, follower))), + local, + VoterSetTest.voterSet(Stream.of(local, leader, follower)), kraftVersion ) .withUnknownLeader(epoch) @@ -1042,8 +1054,8 @@ public void testPreVoteRequestTimeout( ReplicaKey local = replicaKey(localId, true); ReplicaKey otherNode = replicaKey(localId + 1, true); RaftClientTestContext context = new RaftClientTestContext.Builder( - Optional.of(local), - Optional.of(VoterSetTest.voterSet(Stream.of(local, otherNode))), + local, + VoterSetTest.voterSet(Stream.of(local, otherNode)), kraftVersion ) .withUnknownLeader(epoch) diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 5af5a98a1723d..f1c0cbea4b104 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -199,22 +199,17 @@ public Builder(OptionalInt localId, Uuid localDirectoryId) { } public Builder( - Optional localKey, - Optional voters, + ReplicaKey localKey, + VoterSet voters, KRaftVersion kraftVersion ) { - this.localId = localKey.map(replicaKey -> - OptionalInt.of(replicaKey.id())).orElseGet(OptionalInt::empty); - this.localDirectoryId = localKey.map(replicaKey -> - replicaKey.directoryId().orElse(Uuid.randomUuid())).orElse(Uuid.randomUuid()); + this.localId = OptionalInt.of(localKey.id()); + this.localDirectoryId = localKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID); this.kraftVersion = kraftVersion; if (kraftVersion == KRaftVersion.KRAFT_VERSION_0) { - withStaticVoters( - voters.orElseThrow(() -> - new IllegalArgumentException("Static voters must be provided for KRaft version 0")) - .voterIds()); + withStaticVoters(voters.voterIds()); } else { - withBootstrapSnapshot(voters); + withBootstrapSnapshot(Optional.of(voters)); } } From 2c3286684a0987e988080e73b7df24a6da1c5fbd Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Fri, 3 Jan 2025 11:44:59 -0800 Subject: [PATCH 22/23] more test revisions --- .../raft/KafkaRaftClientPreVoteTest.java | 41 +++- .../kafka/raft/KafkaRaftClientTest.java | 194 ++++-------------- .../kafka/raft/ProspectiveStateTest.java | 12 ++ .../kafka/raft/UnattachedStateTest.java | 84 ++++---- .../raft/UnattachedStateWithVoteTest.java | 164 --------------- 5 files changed, 134 insertions(+), 361 deletions(-) delete mode 100644 raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java index 6083f3e7b5ee9..fced3bda6f16c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -333,7 +333,7 @@ public void testHandlePreVoteRequestAsFollowerObserver( RaftClientTestContext context = new RaftClientTestContext.Builder( localKey, - VoterSetTest.voterSet(Stream.of(localKey, leader, follower)), + VoterSetTest.voterSet(Stream.of(leader, follower)), kraftVersion ) .withElectedLeader(epoch, leader.id()) @@ -1001,6 +1001,45 @@ public void testProspectiveWithLeaderTransitionsToFollower( context.assertElectedLeader(epoch + 1, replica2.id()); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testProspectiveLosesElectionHasLeaderButMissingEndpoint(KRaftVersion kraftVersion) throws Exception { + int localId = randomReplicaId(); + ReplicaKey local = replicaKey(localId, true); + ReplicaKey voter1 = replicaKey(localId + 1, true); + int electedLeaderId = localId + 3; + int epoch = 2; + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local, + VoterSetTest.voterSet(Stream.of(local, voter1)), + kraftVersion + ) + .withElectedLeader(epoch, electedLeaderId) + .withRaftProtocol(KIP_996_PROTOCOL) + .build(); + context.assertElectedLeader(epoch, electedLeaderId); + assertTrue(context.client.quorum().isUnattached()); + // Sleep a little to ensure that we become a prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + + // Sleep past election timeout + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + + // Prospective should transition to unattached + assertTrue(context.client.quorum().isUnattached()); + assertTrue(context.client.quorum().hasLeader()); + + // If election timeout expires again, it should transition back to prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + assertTrue(context.client.quorum().hasLeader()); + } + @ParameterizedTest @MethodSource("kraftVersionRaftProtocolCombinations") public void testProspectiveWithoutLeaderTransitionsToFollower( diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 06e657915b8d9..ad90dd8007b18 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -46,7 +46,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; -import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; @@ -71,7 +70,6 @@ import static java.util.Collections.singletonList; import static org.apache.kafka.raft.RaftClientTestContext.Builder.DEFAULT_ELECTION_TIMEOUT_MS; -import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol; import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -218,7 +216,7 @@ public void testGrantVotesFromHigherEpochAfterResigningLeadership(boolean withKi ); context.client.poll(); - // We will first transition to unattached and then grant vote and then transition to unattached voted + // Replica will first transition to unattached, then grant vote, then transition to unattached voted assertTrue(context.client.quorum().isUnattachedAndVoted()); context.assertVotedCandidate(epoch + 1, remoteKey.id()); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); @@ -255,7 +253,7 @@ public void testGrantVotesFromHigherEpochAfterResigningCandidacy(boolean withKip ); context.client.poll(); - // We will first transition to unattached and then grant vote and then transition to unattached voted + // Replica will first transition to unattached, then grant vote, then transition to unattached voted assertTrue(context.client.quorum().isUnattachedAndVoted()); context.assertVotedCandidate(epoch + 1, remoteKey.id()); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); @@ -291,7 +289,7 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Excepti ); context.client.poll(); - // We will first transition to unattached and then grant vote and then transition to unattached voted + // Replica will first transition to unattached, then grant vote, then transition to unattached voted assertTrue( context.client.quorum().isUnattachedAndVoted(), "Local Id: " + localId + @@ -303,33 +301,6 @@ public void testGrantVotesWhenShuttingDown(boolean withKip853Rpc) throws Excepti context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); } - @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testInitializeAsResignedAndBecomeUnattached(boolean withKip853Rpc) throws Exception { - int localId = randomReplicaId(); - int remoteId = localId + 1; - Set voters = Set.of(localId, remoteId); - int epoch = 2; - - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0)) - .withElectedLeader(epoch, localId) - .withKip853Rpc(withKip853Rpc) - .build(); - - // Resign from leader, will restart in resigned state - assertTrue(context.client.quorum().isResigned()); - assertEquals(0L, context.log.endOffset().offset()); - context.assertElectedLeader(epoch, localId); - - // Election timeout - context.time.sleep(context.electionTimeoutMs()); - context.client.poll(); - - // Become unattached with expired election timeout - assertTrue(context.client.quorum().isUnattached()); - } - @ParameterizedTest @ValueSource(booleans = { true, false }) public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853Rpc) throws Exception { @@ -355,6 +326,7 @@ public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853R // Become unattached with expired election timeout assertTrue(context.client.quorum().isUnattached()); + assertEquals(epoch + 1, context.currentEpoch()); // Become prospective context.time.sleep(1); @@ -366,6 +338,7 @@ public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853R context.time.sleep(prospectiveState.remainingElectionTimeMs(context.time.milliseconds())); context.client.poll(); assertTrue(context.client.quorum().isUnattached()); + assertEquals(epoch + 1, context.currentEpoch()); } @ParameterizedTest @@ -1165,7 +1138,7 @@ public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) t context.client.poll(); context.assertSentEndQuorumEpochResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.empty()); - // We should still be candidate until expiration of election timeout + // Replica should still be candidate until expiration of election timeout context.time.sleep(context.electionTimeoutMs() + jitterMs - 1); context.client.poll(); context.assertVotedCandidate(epoch, localId); @@ -1175,7 +1148,7 @@ public void testEndQuorumIgnoredAsCandidateIfOlderEpoch(boolean withKip853Rpc) t context.client.poll(); context.assertVotedCandidate(epoch, localId); - // After backoff, we will become prospective again + // After backoff, replica will become prospective again context.time.sleep(context.electionBackoffMaxMs); context.client.poll(); assertTrue(context.client.quorum().isProspective()); @@ -1204,7 +1177,7 @@ public void testEndQuorumIgnoredAsLeaderIfOlderEpoch(boolean withKip853Rpc) thro context.pollUntilResponse(); context.assertSentEndQuorumEpochResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.of(localId)); - // We should still be leader as long as fetch timeout has not expired + // Replica should still be leader as long as fetch timeout has not expired context.time.sleep(context.fetchTimeoutMs - 1); context.client.poll(); context.assertElectedLeader(epoch, localId); @@ -1472,14 +1445,14 @@ public void testHandleEndQuorumRequestWithLowerPriorityToBecomeLeader(boolean wi // The election won't trigger by one round retry backoff context.time.sleep(1); - context.pollUntilRequest(); - + context.client.poll(); context.assertSentFetchRequest(leaderEpoch, 0, 0); - context.time.sleep(context.retryBackoffMs); - - context.pollUntilRequest(); + context.time.sleep(context.electionBackoffMaxMs); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + context.client.poll(); List voteRequests = context.collectVoteRequests(leaderEpoch, 0, 0); assertEquals(2, voteRequests.size()); @@ -1596,32 +1569,6 @@ public void testHandleVoteRequestAsFollowerWithVotedCandidate(boolean withKip853 @ParameterizedTest @ValueSource(booleans = { true, false }) public void testHandleVoteRequestAsProspective(boolean withKip853Rpc) throws Exception { - int localId = randomReplicaId(); - int epoch = 2; - ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); - Set voters = Set.of(localId, otherNodeKey.id()); - - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withUnknownLeader(epoch) - .withKip853Rpc(withKip853Rpc) - .build(); - - // Sleep a little to ensure that we become a prospective - context.time.sleep(context.electionTimeoutMs() * 2L); - context.client.poll(); - assertTrue(context.client.quorum().isProspective()); - - context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 1)); - context.pollUntilResponse(); - - context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); - context.assertVotedCandidate(epoch + 1, otherNodeKey.id()); - assertTrue(context.client.quorum().isUnattachedAndVoted()); - } - - @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testHandleVoteRequestAsProspectiveWithLeader(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int epoch = 2; ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); @@ -1674,41 +1621,6 @@ public void testHandleVoteRequestAsProspectiveWithVotedCandidate(boolean withKip context.assertVotedCandidate(epoch + 1, otherNodeKey.id()); } - @ParameterizedTest - @ValueSource(booleans = { true, false }) - public void testProspectiveLosesElectionHasLeaderButMissingEndpoint(boolean withKip853Rpc) throws Exception { - int localId = randomReplicaId(); - int epoch = 2; - ReplicaKey voter1 = replicaKey(localId + 1, withKip853Rpc); - int electedLeaderId = localId + 3; - Set voters = Set.of(localId, voter1.id()); - - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withElectedLeader(epoch, electedLeaderId) - .withKip853Rpc(withKip853Rpc) - .build(); - context.assertElectedLeader(epoch, electedLeaderId); - assertTrue(context.client.quorum().isUnattached()); - // Sleep a little to ensure that we become a prospective - context.time.sleep(context.electionTimeoutMs() * 2L); - context.client.poll(); - assertTrue(context.client.quorum().isProspective()); - - // Sleep past election timeout - context.time.sleep(context.electionTimeoutMs() * 2L); - context.client.poll(); - - // Prospective should transition to unattached - assertTrue(context.client.quorum().isUnattached()); - assertTrue(context.client.quorum().hasLeader()); - - // If election timeout expires again, it should transition back to prospective - context.time.sleep(context.electionTimeoutMs() * 2L); - context.client.poll(); - assertTrue(context.client.quorum().isProspective()); - assertTrue(context.client.quorum().hasLeader()); - } - @ParameterizedTest @ValueSource(booleans = { true, false }) public void testHandleInvalidVoteRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception { @@ -1878,7 +1790,7 @@ public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 1; - int exponentialFactor = 85; // set it large enough so that we will bound on jitter + int exponentialFactor = 85; // set it large enough so that replica will bound on jitter Set voters = Set.of(localId, otherNodeId); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) @@ -1895,10 +1807,11 @@ public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception assertEquals(1, candidate.retries()); assertEquals( context.electionTimeoutMs() + exponentialFactor, - candidate.remainingElectionTimeMs(context.time.milliseconds())); + candidate.remainingElectionTimeMs(context.time.milliseconds()) + ); assertFalse(candidate.isBackingOff()); - // Quorum size is two. If the other member rejects, then we need to schedule a revote. + // Quorum size is two. If the other member rejects, then the local replica will lose the election. RaftRequest.Outbound request = context.assertSentVoteRequest(epoch, 0, 0L, 1); context.deliverResponse( request.correlationId(), @@ -1910,18 +1823,19 @@ public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception assertTrue(candidate.isBackingOff()); assertEquals( context.electionBackoffMaxMs, - candidate.remainingBackoffMs(context.time.milliseconds())); + candidate.remainingBackoffMs(context.time.milliseconds()) + ); - // All nodes have rejected our candidacy, but we should still remember that we had voted + // Election is lost, but local replica should still remember that it has voted context.assertVotedCandidate(epoch, localId); - // Even though our candidacy was rejected, we will backoff for jitter period - // before we transition to prospective and start a new election. + // Even though candidacy was rejected, local replica will backoff for jitter period + // before transitioning to prospective and starting a new election. context.time.sleep(context.electionBackoffMaxMs - 1); context.client.poll(); context.assertVotedCandidate(epoch, localId); - // After jitter expires, we become a prospective again + // After jitter expires, become a prospective again context.time.sleep(1); context.client.poll(); assertTrue(context.client.quorum().isProspective()); @@ -1934,7 +1848,7 @@ public void testCandidateBackoffElection(boolean withKip853Rpc) throws Exception prospective.remainingElectionTimeMs(context.time.milliseconds()) ); - // If we become candidate again, retries should be 2 + // After becoming candidate again, retries should be 2 context.deliverResponse( request.correlationId(), request.destination(), @@ -1970,10 +1884,11 @@ public void testCandidateElectionTimeout(boolean withKip853Rpc) throws Exception assertEquals(1, candidate.retries()); assertEquals( context.electionTimeoutMs() + jitter, - candidate.remainingElectionTimeMs(context.time.milliseconds())); + candidate.remainingElectionTimeMs(context.time.milliseconds()) + ); assertFalse(candidate.isBackingOff()); - // If election times out, we transition to prospective without any additional backoff + // If election times out, replica transition to prospective without any additional backoff context.time.sleep(candidate.remainingElectionTimeMs(context.time.milliseconds())); context.client.poll(); assertTrue(context.client.quorum().isProspective()); @@ -2051,6 +1966,8 @@ public void testVoterBecomeProspectiveAfterFetchTimeout(boolean withKip853Rpc) t context.time.sleep(context.fetchTimeoutMs); context.client.poll(); assertTrue(context.client.quorum().isProspective()); + context.client.poll(); + context.assertSentVoteRequest(epoch, lastEpoch, 1L, 1); } @ParameterizedTest @@ -2400,17 +2317,15 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E } @ParameterizedTest - @EnumSource(RaftProtocol.class) - public void testInvalidFetchRequest(RaftProtocol raftProtocol) throws Exception { - boolean withDynamicReconfig = raftProtocol.isReconfigSupported(); + @ValueSource(booleans = { true, false }) + public void testInvalidFetchRequest(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); - int otherNodeId = localId + 1; - ReplicaKey otherNodeKey = replicaKey(otherNodeId, withDynamicReconfig); - Set voters = Set.of(localId, otherNodeId); + ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc); + Set voters = Set.of(localId, otherNodeKey.id()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(4) - .withKip853Rpc(withDynamicReconfig) + .withKip853Rpc(withKip853Rpc) .build(); context.unattachedToLeader(); @@ -2956,52 +2871,25 @@ public void testFetchResponseIgnoredAfterBecomingFollowerOfDifferentLeader( } @ParameterizedTest - @EnumSource(RaftProtocol.class) - public void testVoteResponseIgnoredAfterBecomingFollower(RaftProtocol raftProtocol) throws Exception { - boolean withDynamicReconfig = raftProtocol.isReconfigSupported(); + @ValueSource(booleans = { true, false }) + public void testVoteResponseIgnoredAfterBecomingFollower(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int voter2 = localId + 1; int voter3 = localId + 2; int epoch = 5; - Set voters = Set.of(localId, voter2, voter3); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withUnknownLeader(epoch) - .withKip853Rpc(withDynamicReconfig) + .withKip853Rpc(withKip853Rpc) .build(); context.assertUnknownLeaderAndNoVotedCandidate(epoch); - - // Sleep a little to ensure that we become a prospective - context.time.sleep(context.electionTimeoutMs() * 2L); - - // Wait until the PreVote requests are inflight - context.pollUntilRequest(); - assertTrue(context.client.quorum().isProspective()); - List voteRequests = context.collectVoteRequests(epoch, 0, 0); - assertEquals(2, voteRequests.size()); - - // Become candidate after PreVote requests are granted - VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch); - context.deliverResponse( - voteRequests.get(0).correlationId(), - voteRequests.get(0).destination(), - voteResponse1 - ); - VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch); - context.deliverResponse( - voteRequests.get(1).correlationId(), - voteRequests.get(1).destination(), - voteResponse2 - ); - - context.client.poll(); - assertTrue(context.client.quorum().isCandidate()); + context.unattachedToCandidate(); // Wait until the vote requests are inflight context.pollUntilRequest(); context.assertVotedCandidate(epoch + 1, localId); - voteRequests = context.collectVoteRequests(epoch + 1, 0, 0); + List voteRequests = context.collectVoteRequests(epoch + 1, 0, 0); assertEquals(2, voteRequests.size()); // While the vote requests are still inflight, we receive a BeginEpoch for the same epoch @@ -3014,14 +2902,14 @@ public void testVoteResponseIgnoredAfterBecomingFollower(RaftProtocol raftProtoc ); // The vote requests now return and should be ignored - voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch + 1); + VoteResponseData voteResponse1 = context.voteResponse(true, OptionalInt.empty(), epoch + 1); context.deliverResponse( voteRequests.get(0).correlationId(), voteRequests.get(0).destination(), voteResponse1 ); - voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch + 1); + VoteResponseData voteResponse2 = context.voteResponse(true, OptionalInt.of(voter3), epoch + 1); context.deliverResponse( voteRequests.get(1).correlationId(), voteRequests.get(1).destination(), diff --git a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java index 2a39fc90fd544..5e06cfccf25c1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -325,6 +325,18 @@ public void testElectionState(boolean withDirectoryId) { ), state.election() ); + + // with both + state = newProspectiveState(voters, OptionalInt.of(1), Optional.of(votedKey)); + assertEquals( + ElectionState.withElectedLeader( + epoch, + 1, + Optional.of(votedKey), + voters.voterIds() + ), + state.election() + ); } @Test diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java index 4411e7a17cc8a..5e9c68d2d053c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; @@ -60,7 +61,7 @@ private UnattachedState newUnattachedState( @ParameterizedTest @CsvSource({ "true,false", "false,true", "false,false" }) - public void testElectionTimeout(boolean hasVotedKey, boolean hasLeaderId) { + public void testElectionStateAndElectionTimeout(boolean hasVotedKey, boolean hasLeaderId) { OptionalInt leader = hasLeaderId ? OptionalInt.of(3) : OptionalInt.empty(); Optional votedKey = hasVotedKey ? Optional.of(this.votedKey) : Optional.empty(); UnattachedState state = newUnattachedState(leader, votedKey); @@ -123,48 +124,13 @@ public void testGrantVoteWithoutVotedKey(boolean isLogUpToDate) { ); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - void testGrantVoteWithLeader(boolean isLogUpToDate) { - int leaderId = 3; - UnattachedState state = newUnattachedState(OptionalInt.of(leaderId), Optional.empty()); - - // Check that the leader is persisted if the leader is known - assertEquals(ElectionState.withElectedLeader(epoch, leaderId, Optional.empty(), voters), state.election()); - - // Check that the replica can grant PreVotes if the log is up-to-date, even if the last leader is known - // This is because nodes in Unattached have not successfully fetched from the leader yet - assertEquals( - isLogUpToDate, - state.canGrantVote(voter1Key, isLogUpToDate, true) - ); - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) - ); - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(leaderId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) - ); - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) - ); - - // Check that the replica rejects all standard votes request if the leader is known - assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(leaderId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - } - @ParameterizedTest @ValueSource(booleans = {true, false}) public void testCanGrantVoteWithVotedKey(boolean isLogUpToDate) { UnattachedState state = newUnattachedState(OptionalInt.empty(), Optional.of(votedKey)); // Same voterKey - // We can reject PreVote for a replica we have already granted a standard vote to if their log is behind + // Local can reject PreVote for a replica that local has already granted a standard vote to if their log is behind assertEquals( isLogUpToDate, state.canGrantVote(votedKey, isLogUpToDate, true) @@ -172,7 +138,7 @@ public void testCanGrantVoteWithVotedKey(boolean isLogUpToDate) { assertTrue(state.canGrantVote(votedKey, isLogUpToDate, false)); // Different directoryId - // We can grant PreVote for a replica we have already granted a standard vote to if their log is up-to-date, + // Local can grant PreVote for a replica that local has already granted a standard vote to if their log is up-to-date, // even if the directoryId is different assertEquals( isLogUpToDate, @@ -208,11 +174,43 @@ public void testCanGrantVoteWithVotedKey(boolean isLogUpToDate) { } @ParameterizedTest - @CsvSource({ "true,false", "false,true", "false,false" }) - public void testLeaderEndpoints(boolean hasVotedKey, boolean hasLeaderId) { - OptionalInt leaderId = hasLeaderId ? OptionalInt.of(3) : OptionalInt.empty(); - Optional votedKey = hasVotedKey ? Optional.of(this.votedKey) : Optional.empty(); - UnattachedState state = newUnattachedState(leaderId, votedKey); + @ValueSource(booleans = {true, false}) + void testGrantVoteWithLeader(boolean isLogUpToDate) { + int leaderId = 3; + UnattachedState state = newUnattachedState(OptionalInt.of(leaderId), Optional.empty()); + + // Check that the leader is persisted if the leader is known + assertEquals(ElectionState.withElectedLeader(epoch, leaderId, Optional.empty(), voters), state.election()); + + // Check that the replica can grant PreVotes if the log is up-to-date, even if the last leader is known + // This is because nodes in Unattached have not successfully fetched from the leader yet + assertEquals( + isLogUpToDate, + state.canGrantVote(voter1Key, isLogUpToDate, true) + ); + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(leaderId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); + + // Check that the replica rejects all standard votes request if the leader is known + assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(leaderId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + } + + @Test + public void testLeaderEndpoints() { + UnattachedState state = newUnattachedState(OptionalInt.of(3), Optional.of(this.votedKey)); assertEquals(Endpoints.empty(), state.leaderEndpoints()); } diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java deleted file mode 100644 index 008706abf3064..0000000000000 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java +++ /dev/null @@ -1,164 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.raft; - -import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.common.utils.MockTime; - -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; - -import java.util.Optional; -import java.util.OptionalInt; -import java.util.Set; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - -class UnattachedStateWithVoteTest { - - private final MockTime time = new MockTime(); - private final LogContext logContext = new LogContext(); - private final Set voters = Set.of(1, 2, 3); - private final int epoch = 5; - private final int votedId = 1; - private final int electionTimeoutMs = 10000; - - private UnattachedState newUnattachedVotedState( - Uuid votedDirectoryId - ) { - return new UnattachedState( - time, - epoch, - OptionalInt.empty(), - Optional.of(ReplicaKey.of(votedId, votedDirectoryId)), - voters, - Optional.empty(), - electionTimeoutMs, - logContext - ); - } - - @Test - public void testElectionTimeout() { - UnattachedState state = newUnattachedVotedState(ReplicaKey.NO_DIRECTORY_ID); - ReplicaKey votedKey = ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID); - - assertEquals(epoch, state.epoch()); - assertEquals(votedKey, state.votedKey().get()); - assertEquals( - ElectionState.withVotedCandidate(epoch, votedKey, voters), - state.election() - ); - assertEquals(electionTimeoutMs, state.remainingElectionTimeMs(time.milliseconds())); - assertFalse(state.hasElectionTimeoutExpired(time.milliseconds())); - - time.sleep(5000); - assertEquals(electionTimeoutMs - 5000, state.remainingElectionTimeMs(time.milliseconds())); - assertFalse(state.hasElectionTimeoutExpired(time.milliseconds())); - - time.sleep(5000); - assertEquals(0, state.remainingElectionTimeMs(time.milliseconds())); - assertTrue(state.hasElectionTimeoutExpired(time.milliseconds())); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { - UnattachedState state = newUnattachedVotedState(ReplicaKey.NO_DIRECTORY_ID); - - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) - ); - assertTrue(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate, true) - ); - assertTrue(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate, false)); - - // Can grant PreVote to other replicas even if we have granted a standard vote to another replica - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) - ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testCanGrantVoteWithDirectoryId(boolean isLogUpToDate) { - Uuid votedDirectoryId = Uuid.randomUuid(); - UnattachedState state = newUnattachedVotedState(votedDirectoryId); - - // Same voterKey - // We can reject PreVote for a replica we have already granted a standard vote to if their log is behind - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(votedId, votedDirectoryId), isLogUpToDate, true) - ); - assertTrue(state.canGrantVote(ReplicaKey.of(votedId, votedDirectoryId), isLogUpToDate, false)); - - // Different directoryId - // We can grant PreVote for a replica we have already granted a standard vote to if their log is up-to-date, - // even if the directoryId is different - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate, true) - ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId, Uuid.randomUuid()), isLogUpToDate, false)); - - // Missing directoryId - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) - ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - - // Different voterId - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), isLogUpToDate, true) - ); - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) - ); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, votedDirectoryId), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - - // Observer - assertEquals( - isLogUpToDate, - state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) - ); - assertFalse(state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - } - - @Test - public void testLeaderEndpoints() { - Uuid votedDirectoryId = Uuid.randomUuid(); - UnattachedState state = newUnattachedVotedState(votedDirectoryId); - - assertEquals(Endpoints.empty(), state.leaderEndpoints()); - } -} From 99c7817b92747e467504fa971ad9cc2e971d33b8 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Tue, 7 Jan 2025 15:13:50 -0500 Subject: [PATCH 23/23] new raft simulation test --- .../kafka/raft/RaftEventSimulationTest.java | 103 ++++++++++++++++++ 1 file changed, 103 insertions(+) diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 086a5ebad3b18..469bc885b7507 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -295,6 +295,67 @@ void canMakeProgressIfMajorityIsReachable( scheduler.runUntil(() -> cluster.allReachedHighWatermark(2 * restoredLogEndOffset)); } + @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY) + void leadershipWillNotChangeIfMajorityIsReachable( + @ForAll int seed, + @ForAll @IntRange(min = 0, max = 3) int numObservers + ) { + int numVoters = 5; + Random random = new Random(seed); + Cluster cluster = new Cluster(numVoters, numObservers, random); + MessageRouter router = new MessageRouter(cluster); + EventScheduler scheduler = schedulerWithDefaultInvariants(cluster); + scheduler.addInvariant(new StableLeadershipWhenMajorityReachable(cluster)); + + // Seed the cluster with some data + cluster.startAll(); + schedulePolling(scheduler, cluster, 3, 5); + scheduler.schedule(router::deliverAll, 0, 2, 1); + scheduler.schedule(new SequentialAppendAction(cluster), 0, 2, 1); + scheduler.runUntil(cluster::hasConsistentLeader); + scheduler.runUntil(() -> cluster.allReachedHighWatermark(5)); + + int leaderId = cluster.latestLeader().orElseThrow(() -> + new AssertionError("Failed to find current leader during setup") + ); + + int leaderEpoch = cluster.epoch(leaderId); + + // Create network partition which would result in ping-pong of leadership between nodes C and D without PreVote + // A B + // | | + // C - D (have leader start in position C) + // \ / + // E + int nodeA = (leaderId + 1) % numVoters; + int nodeB = (leaderId + 2) % numVoters; + int nodeD = (leaderId + 3) % numVoters; + int nodeE = (leaderId + 4) % numVoters; + router.filter( + nodeA, + new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(nodeB, nodeD, nodeE))) + ); + router.filter( + nodeB, + new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(nodeA, leaderId, nodeE))) + ); + router.filter(leaderId, new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(nodeB)))); + router.filter(nodeD, new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(nodeA)))); + router.filter(nodeE, new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(nodeA, nodeB)))); + + // Check that leadership is stable + scheduler.runUntil(() -> cluster.allReachedHighWatermark(20, Set.of(nodeA, leaderId, nodeD, nodeE))); + // Leader and leader epoch should be the same + assertEquals(leaderId, cluster.latestLeader().orElseThrow( + () -> new AssertionError("Failed to find current leader after partition")) + ); + assertEquals(leaderEpoch, cluster.epoch(leaderId)); + // All nodes should be on the same epoch + for (int nodeId : cluster.nodeIds()) { + assertEquals(leaderEpoch, cluster.epoch(nodeId)); + } + } + @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY) void canMakeProgressAfterBackToBackLeaderFailures( @ForAll int seed, @@ -645,6 +706,14 @@ OptionalInt latestLeader() { return latestLeader; } + int epoch(int nodeId) { + return running.values().stream() + .filter(node -> node.nodeId == nodeId) + .findFirst() + .orElseThrow(() -> new IllegalStateException("Node " + nodeId + " is not running")) + .client.quorum().epoch(); + } + boolean hasConsistentLeader() { Iterator iter = running.values().iterator(); if (!iter.hasNext()) @@ -1057,6 +1126,40 @@ public void verify() { } } + private static class StableLeadershipWhenMajorityReachable implements Invariant { + final Cluster cluster; + OptionalInt epochWithFirstLeader = OptionalInt.empty(); + OptionalInt firstLeaderId = OptionalInt.empty(); + + private StableLeadershipWhenMajorityReachable(Cluster cluster) { + this.cluster = cluster; + } + + @Override + public void verify() { + // Todo: currently this just checks the leader is never changed after the first successful election. + // KAFKA-18439 will generalize the invariant so it holds for all tests even if routing filters are changed. + // i.e. if the current leader is reachable by majority, we do not expect leadership to change + for (Map.Entry nodeEntry : cluster.nodes.entrySet()) { + PersistentState state = nodeEntry.getValue(); + Optional electionState = state.store.readElectionState(); + + electionState.ifPresent(election -> { + if (election.hasLeader()) { + // verify there were no leaders prior to this one + if (epochWithFirstLeader.isEmpty()) { + epochWithFirstLeader = OptionalInt.of(election.epoch()); + firstLeaderId = OptionalInt.of(election.leaderId()); + } else { + assertEquals(epochWithFirstLeader.getAsInt(), election.epoch()); + assertEquals(firstLeaderId.getAsInt(), election.leaderId()); + } + } + }); + } + } + } + private static class MonotonicHighWatermark implements Invariant { final Cluster cluster; long highWatermark = 0;