diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 19cc0975a09..51d2b3c186a 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="(KafkaConsumer|ConsumerCoordinator|AbstractFetch|KafkaProducer|AbstractRequest|AbstractResponse|TransactionManager|Admin|KafkaAdminClient|MockAdminClient|KafkaNetworkChannelTest).java"/> @@ -102,10 +102,10 @@ 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"/> + 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"/> @@ -187,6 +187,9 @@ + + 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 4acf8d31ca5..619b5bd78e5 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 class VoteRequest extends AbstractRequest { 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 class VoteRequest extends AbstractRequest { .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 122ee1f6d89..85c4f8ba2a2 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 902d334ad79..d66de84e284 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -20,38 +20,33 @@ import org.apache.kafka.common.Uuid; 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.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 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; private final Timer backoffTimer; private final Logger log; + private boolean isBackingOff; /** * 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 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, it will transition to prospective after a backoff phase. + * 4. If election times out, it will transition immediately to prospective. */ - private boolean isBackingOff; - protected CandidateState( Time time, int localId, @@ -85,26 +80,8 @@ public class CandidateState implements EpochState { 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); - } - - 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(); + this.epochElection = new EpochElection(voters.voterKeys()); + epochElection.recordVote(localId, true); } /** @@ -118,69 +95,27 @@ public class CandidateState implements EpochState { 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(); + @Override + public EpochElection epochElection() { + return epochElection; } - /** - * 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 - */ + @Override 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)) { + if (epochElection().isRejectedVoter(remoteNodeId)) { 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; + return epochElection().recordVote(remoteNodeId, true); } - /** - * 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 - */ + @Override 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)) { + if (epochElection().isGrantedVoter(remoteNodeId)) { 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; + return epochElection().recordVote(remoteNodeId, false); } /** @@ -192,41 +127,7 @@ public class CandidateState implements EpochState { 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) { - return voteStates - .values() - .stream() - .filter(voterState -> voterState.state().equals(state)) - .map(VoterState::replicaKey); - } - + @Override public boolean hasElectionTimeoutExpired(long currentTimeMs) { electionTimer.update(currentTimeMs); return electionTimer.isExpired(); @@ -245,6 +146,7 @@ public class CandidateState implements EpochState { return backoffTimer.remainingMs(); } + @Override public long remainingElectionTimeMs(long currentTimeMs) { electionTimer.update(currentTimeMs); return electionTimer.remainingMs(); @@ -255,7 +157,7 @@ public class CandidateState implements EpochState { return ElectionState.withVotedCandidate( epoch, ReplicaKey.of(localId, localDirectoryId), - voteStates.keySet() + epochElection.voterIds() ); } @@ -299,13 +201,13 @@ public class CandidateState implements EpochState { @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, retries=%d, epochElection=%s, " + "highWatermark=%s, electionTimeoutMs=%d)", localId, localDirectoryId, epoch, retries, - voteStates, + epochElection(), highWatermark, electionTimeoutMs ); @@ -318,31 +220,4 @@ public class CandidateState implements EpochState { @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/ElectionState.java b/raft/src/main/java/org/apache/kafka/raft/ElectionState.java index 675436cc52c..6b4f775caee 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 final class ElectionState { 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/EpochState.java b/raft/src/main/java/org/apache/kafka/raft/EpochState.java index c47fc087f0d..338f6603181 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 @@ public interface EpochState extends Closeable { * 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/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index e3db67893bf..4d2357fdef5 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 class FollowerState implements EpochState { int epoch, int leaderId, Endpoints leaderEndpoints, + Optional votedKey, Set voters, Optional highWatermark, int fetchTimeoutMs, @@ -65,6 +67,7 @@ public class FollowerState implements EpochState { 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()); @@ -75,7 +78,7 @@ public class FollowerState implements EpochState { @Override public ElectionState election() { - return ElectionState.withElectedLeader(epoch, leaderId, voters); + return ElectionState.withElectedLeader(epoch, leaderId, votedKey, voters); } @Override @@ -228,12 +231,13 @@ public class FollowerState implements EpochState { @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 ccd5873d036..f1e0e4e41b4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -159,6 +159,7 @@ import static org.apache.kafka.snapshot.Snapshots.BOOTSTRAP_SNAPSHOT_ID; * as FileRecords, but we use {@link UnalignedRecords} in FetchSnapshotResponse because the records * are not necessarily offset-aligned. */ +@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; @@ -543,17 +544,17 @@ public final class KafkaRaftClient implements RaftClient { long currentTimeMs = time.milliseconds(); if (quorum.isLeader()) { throw new IllegalStateException("Voter cannot initialize as a Leader"); + } 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 + transitionToProspective(currentTimeMs); } else if (quorum.isCandidate()) { onBecomeCandidate(currentTimeMs); } else if (quorum.isFollower()) { onBecomeFollower(currentTimeMs); } - // When there is only a single voter, become candidate immediately - if (quorum.isOnlyVoter() && !quorum.isCandidate()) { - transitionToCandidate(currentTimeMs); - } - // Specialized add voter handler this.addVoterHandler = new AddVoterHandler( partitionState, @@ -657,7 +658,7 @@ public final class KafkaRaftClient implements RaftClient { } private boolean maybeTransitionToLeader(CandidateState state, long currentTimeMs) { - if (state.isVoteGranted()) { + if (state.epochElection().isVoteGranted()) { onBecomeLeader(currentTimeMs); return true; } else { @@ -665,6 +666,32 @@ public final class KafkaRaftClient implements RaftClient { } } + private boolean maybeTransitionToCandidate(ProspectiveState state, long currentTimeMs) { + if (state.epochElection().isVoteGranted()) { + transitionToCandidate(currentTimeMs); + return true; + } else { + return false; + } + } + + /** + * 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. + */ + private void maybeTransitionForward(NomineeState state, long 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 NomineeState (Prospective or Candidate), but current state is " + state + ); + } + } + private void onBecomeCandidate(long currentTimeMs) { CandidateState state = quorum.candidateStateOrThrow(); if (!maybeTransitionToLeader(state, currentTimeMs)) { @@ -679,8 +706,21 @@ public final class KafkaRaftClient implements RaftClient { onBecomeCandidate(currentTimeMs); } - private void transitionToUnattached(int epoch) { - quorum.transitionToUnattached(epoch); + private void onBecomeProspective(long currentTimeMs) { + ProspectiveState state = quorum.prospectiveStateOrThrow(); + if (!maybeTransitionToCandidate(state, currentTimeMs)) { + resetConnections(); + kafkaRaftMetrics.updateElectionStartMs(currentTimeMs); + } + } + + private void transitionToProspective(long currentTimeMs) { + quorum.transitionToProspective(); + onBecomeProspective(currentTimeMs); + } + + private void transitionToUnattached(int epoch, OptionalInt leaderId) { + quorum.transitionToUnattached(epoch, leaderId); maybeFireLeaderChange(); resetConnections(); } @@ -692,10 +732,6 @@ public final class KafkaRaftClient implements RaftClient { resetConnections(); } - private void transitionToUnattachedVoted(ReplicaKey candidateKey, int epoch) { - quorum.transitionToUnattachedVotedState(epoch, candidateKey); - } - private void onBecomeFollower(long currentTimeMs) { kafkaRaftMetrics.maybeUpdateElectionLatency(currentTimeMs); @@ -824,7 +860,7 @@ public final class KafkaRaftClient implements RaftClient { } if (replicaEpoch > quorum.epoch()) { - transitionToUnattached(replicaEpoch); + transitionToUnattached(replicaEpoch, OptionalInt.empty()); } // Check that the request was intended for this replica @@ -857,8 +893,12 @@ public final class KafkaRaftClient implements RaftClient { preVote ); - if (!preVote && voteGranted && quorum.isUnattachedNotVoted()) { - transitionToUnattachedVoted(replicaKey, replicaEpoch); + if (!preVote && voteGranted) { + if (quorum.isUnattachedNotVoted()) { + quorum.unattachedAddVotedState(replicaEpoch, replicaKey); + } else if (quorum.isProspectiveNotVoted()) { + quorum.prospectiveAddVotedState(replicaEpoch, replicaKey); + } } logger.info( @@ -882,7 +922,15 @@ public final class KafkaRaftClient implements RaftClient { 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 at least one voter doesn't support PreVote.", + quorum.epoch() + ); + transitionToCandidate(currentTimeMs); + return true; + } else if (topLevelError != Errors.NONE) { return handleTopLevelError(topLevelError, responseMetadata); } @@ -926,30 +974,22 @@ public final class KafkaRaftClient implements RaftClient { 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(); + } else if (quorum.isNomineeState()) { + NomineeState state = quorum.nomineeStateOrThrow(); if (partitionResponse.voteGranted()) { state.recordGrantedVote(remoteNodeId); - maybeTransitionToLeader(state, currentTimeMs); + 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 become leader (rejected by {}). " + - "We will backoff before retrying election again", state.rejectingVoters()); - - state.startBackingOff( - currentTimeMs, - binaryExponentialElectionBackoffMs(state.retries()) - ); - } + maybeHandleElectionLoss(state, currentTimeMs); } } else { - logger.debug("Ignoring vote response {} since we are no longer a 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 { @@ -957,13 +997,52 @@ public final class KafkaRaftClient implements RaftClient { } } + /** + * 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(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. " + + "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 + // 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 (state instanceof ProspectiveState prospective) { + if (prospective.epochElection().isVoteRejected()) { + logger.info( + "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 current state is " + state + ); + } + } + 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()); + return Math.min( + RETRY_BACKOFF_BASE_MS * random.nextInt(2 << Math.min(20, retries - 1)), + quorumConfig.electionBackoffMaxMs() + ); } private int strictExponentialElectionBackoffMs(int positionInSuccessors, int totalNumSuccessors) { @@ -2368,7 +2447,7 @@ public final class KafkaRaftClient implements RaftClient { * - 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, @@ -2445,7 +2524,7 @@ public final class KafkaRaftClient implements RaftClient { if (leaderId.isPresent()) { transitionToFollower(epoch, leaderId.getAsInt(), leaderEndpoints, currentTimeMs); } else { - transitionToUnattached(epoch); + transitionToUnattached(epoch, OptionalInt.empty()); } } else if ( leaderId.isPresent() && @@ -2748,7 +2827,7 @@ public final class KafkaRaftClient implements RaftClient { ); } - private VoteRequestData buildVoteRequest(ReplicaKey remoteVoter) { + private VoteRequestData buildVoteRequest(ReplicaKey remoteVoter, boolean preVote) { OffsetAndEpoch endOffset = endOffset(); return RaftUtil.singletonVoteRequest( log.topicPartition(), @@ -2758,7 +2837,7 @@ public final class KafkaRaftClient implements RaftClient { remoteVoter, endOffset.epoch(), endOffset.offset(), - false + preVote ); } @@ -2929,18 +3008,16 @@ public final class KafkaRaftClient implements RaftClient { 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)) { - if (quorum.isVoter()) { - 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); - } + // 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 { stateTimeoutMs = state.remainingElectionTimeMs(currentTimeMs); @@ -2984,15 +3061,16 @@ public final class KafkaRaftClient implements RaftClient { } private long maybeSendVoteRequests( - CandidateState state, + 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(); + boolean preVote = quorum.isProspective(); return maybeSendRequest( currentTimeMs, - state.unrecordedVoters(), + state.epochElection().unrecordedVoters(), voterId -> voters .voterNode(voterId, channel.listenerName()) .orElseThrow(() -> @@ -3004,7 +3082,7 @@ public final class KafkaRaftClient implements RaftClient { ) ) ), - this::buildVoteRequest + voterId -> buildVoteRequest(voterId, preVote) ); } return Long.MAX_VALUE; @@ -3015,7 +3093,7 @@ public final class KafkaRaftClient implements RaftClient { 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 @@ -3024,20 +3102,53 @@ public final class KafkaRaftClient implements RaftClient { 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); } 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); - state.startBackingOff(currentTimeMs, backoffDurationMs); - return backoffDurationMs; + 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)); + } + } + + private long pollProspective(long currentTimeMs) { + ProspectiveState state = quorum.prospectiveStateOrThrow(); + GracefulShutdown shutdown = this.shutdown.get(); + + if (shutdown != null) { long minRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); - return Math.min(minRequestBackoffMs, state.remainingElectionTimeMs(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. " + + "Current epoch election state: {}", + state.epochElection() + ); + prospectiveTransitionAfterElectionLoss(state, currentTimeMs); + return 0L; + } else { + long minVoteRequestBackoffMs = maybeSendVoteRequests(state, currentTimeMs); + return Math.min(minVoteRequestBackoffMs, state.remainingElectionTimeMs(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()) { + transitionToFollower( + quorum().epoch(), + prospective.election().leaderId(), + prospective.leaderEndpoints(), + currentTimeMs + ); + } else { + transitionToUnattached(quorum().epoch(), prospective.election().optionalLeaderId()); } } @@ -3058,8 +3169,8 @@ public final class KafkaRaftClient implements RaftClient { // 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 Prospective state due to fetch timeout"); + transitionToProspective(currentTimeMs); backoffMs = 0; } else if (state.hasUpdateVoterPeriodExpired(currentTimeMs)) { if (partitionState.lastKraftVersion().isReconfigSupported() && @@ -3163,7 +3274,7 @@ public final class KafkaRaftClient implements RaftClient { // shutdown completes or an epoch bump forces another state transition return shutdown.remainingTimeMs(); } else if (state.hasElectionTimeoutExpired(currentTimeMs)) { - transitionToCandidate(currentTimeMs); + transitionToProspective(currentTimeMs); return 0L; } else { return pollUnattachedCommon(state, currentTimeMs); @@ -3180,6 +3291,8 @@ public final class KafkaRaftClient implements RaftClient { return pollLeader(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/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index ff74602217b..36579499e62 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -409,7 +409,7 @@ public class LeaderState implements EpochState { @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/NomineeState.java b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java new file mode 100644 index 00000000000..dc8952048c6 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/NomineeState.java @@ -0,0 +1,53 @@ +/* + * 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.raft.internals.EpochElection; + +interface NomineeState extends EpochState { + EpochElection epochElection(); + + /** + * 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 voter had not been previously recorded + * @throws IllegalArgumentException + */ + 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 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 new file mode 100644 index 00000000000..d66fe2c3e5f --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java @@ -0,0 +1,182 @@ +/* + * 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.apache.kafka.raft.internals.EpochElection; + +import org.slf4j.Logger; + +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; + private final OptionalInt leaderId; + private final Endpoints leaderEndpoints; + private final Optional votedKey; + 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 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, + int localId, + int epoch, + OptionalInt leaderId, + Endpoints 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.electionTimeoutMs = electionTimeoutMs; + this.electionTimer = time.timer(electionTimeoutMs); + this.log = logContext.logger(ProspectiveState.class); + + this.epochElection = new EpochElection(voters.voterKeys()); + epochElection.recordVote(localId, true); + } + + public Optional votedKey() { + return votedKey; + } + + @Override + public EpochElection epochElection() { + return epochElection; + } + + public int retries() { + return retries; + } + + @Override + public boolean recordGrantedVote(int remoteNodeId) { + return epochElection().recordVote(remoteNodeId, true); + } + + @Override + public boolean recordRejectedVote(int remoteNodeId) { + if (remoteNodeId == localId) { + throw new IllegalArgumentException("Attempted to reject vote from ourselves"); + } + return epochElection().recordVote(remoteNodeId, false); + } + + @Override + public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolean isPreVote) { + return unattachedOrProspectiveCanGrantVote( + leaderId, + votedKey, + epoch, + replicaKey, + isLogUpToDate, + isPreVote, + log + ); + } + + @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 ElectionState election() { + 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 + public int epoch() { + return epoch; + } + + @Override + public Endpoints leaderEndpoints() { + return leaderEndpoints; + } + + @Override + public Optional highWatermark() { + return highWatermark; + } + + @Override + public String toString() { + return String.format( + "ProspectiveState(epoch=%d, leaderId=%s, retries=%d, votedKey=%s, epochElection=%s, " + + "electionTimeoutMs=%s, highWatermark=%s)", + epoch, + leaderId, + retries, + votedKey, + epochElection, + electionTimeoutMs, + 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 4e1ba679e35..9233cc19512 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -39,40 +39,46 @@ import java.util.Random; * how they are triggered: * * Resigned transitions to: - * Unattached: After learning of a new election with a higher epoch - * Candidate: 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 - * Candidate: 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 after giving a binding vote + * 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 - * 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 - * Candidate: After expiration of the fetch timeout - * Follower: After discovering a leader with a larger 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 * - * Observers follow a simpler state machine. The Voted/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. * * 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 { @@ -193,17 +199,6 @@ public class QuorumState { 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()); @@ -226,7 +221,7 @@ public class QuorumState { time, election.epoch(), OptionalInt.of(election.leaderId()), - Optional.empty(), + election.optionalVotedKey(), partitionState.lastVoterSet().voterIds(), Optional.empty(), randomElectionTimeoutMs(), @@ -238,6 +233,7 @@ public class QuorumState { election.epoch(), election.leaderId(), leaderEndpoints, + election.optionalVotedKey(), voters.voterIds(), Optional.empty(), fetchTimeoutMs, @@ -249,7 +245,7 @@ public class QuorumState { time, election.epoch(), OptionalInt.empty(), - Optional.empty(), + election.optionalVotedKey(), partitionState.lastVoterSet().voterIds(), Optional.empty(), randomElectionTimeoutMs(), @@ -315,6 +311,10 @@ public class QuorumState { return OptionalInt.empty(); } + public Optional votedKey() { + return state.election().optionalVotedKey(); + } + public boolean hasLeader() { return leaderId().isPresent(); } @@ -368,14 +368,23 @@ public class QuorumState { } /** - * 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 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. */ - public void transitionToUnattached(int epoch) { + public void transitionToUnattached(int epoch, OptionalInt leaderId) { int currentEpoch = state.epoch(); - if (epoch <= currentEpoch) { - throw new IllegalStateException("Cannot transition to Unattached with epoch= " + epoch + - " from current state " + state); + 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; @@ -385,15 +394,22 @@ public class QuorumState { electionTimeoutMs = candidateStateOrThrow().remainingElectionTimeMs(time.milliseconds()); } else if (isUnattached()) { 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(); } + // 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, - OptionalInt.empty(), - Optional.empty(), + leaderId, + votedKey, partitionState.lastVoterSet().voterIds(), state.highWatermark(), electionTimeoutMs, @@ -402,12 +418,12 @@ public class QuorumState { } /** - * 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. 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 transitionToUnattachedVotedState( + public void unattachedAddVotedState( int epoch, ReplicaKey candidateKey ) { @@ -415,31 +431,22 @@ public class QuorumState { 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 transition to voted without a replica id"); - } else if (epoch < currentEpoch) { + 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 since the current epoch " + - "(%d) is larger", + "Cannot add voted key (%s) to current state (%s) in epoch %d", candidateKey, - epoch, - currentEpoch - ) - ); - } else if (epoch == currentEpoch && !isUnattachedNotVoted()) { - throw new IllegalStateException( - String.format( - "Cannot transition to Voted for %s and epoch %d from the current state (%s)", - candidateKey, - epoch, - state + state, + epoch ) ); } @@ -450,7 +457,7 @@ public class QuorumState { new UnattachedState( time, epoch, - OptionalInt.empty(), + state.election().optionalLeaderId(), Optional.of(candidateKey), partitionState.lastVoterSet().voterIds(), state.highWatermark(), @@ -458,7 +465,59 @@ public class QuorumState { logContext ) ); - log.debug("Voted for candidate {} in epoch {}", candidateKey, epoch); + } + + /** + * 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, + ReplicaKey candidateKey + ) { + int currentEpoch = state.epoch(); + if (localId.isPresent() && candidateKey.id() == localId.getAsInt()) { + throw new IllegalStateException( + String.format( + "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 || !isProspectiveNotVoted()) { + throw new IllegalStateException( + String.format( + "Cannot add voted key (%s) to current state (%s) in epoch %d", + candidateKey, + state, + epoch + ) + ); + } + + 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( + new ProspectiveState( + time, + localIdOrThrow(), + epoch, + state.election().optionalLeaderId(), + state.leaderEndpoints(), + Optional.of(candidateKey), + partitionState.lastVoterSet(), + state.highWatermark(), + prospectiveState.retries(), + randomElectionTimeoutMs(), + logContext + ) + ); } /** @@ -515,12 +574,18 @@ public class QuorumState { } } + // 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, + votedKey, partitionState.lastVoterSet().voterIds(), state.highWatermark(), fetchTimeoutMs, @@ -529,26 +594,55 @@ public class QuorumState { ); } - public void transitionToCandidate() { + /** + * 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 Candidate since the local id (%s) and directory id (%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() || isProspective()) { + throw new IllegalStateException("Cannot transition to Prospective since the local broker.id=" + localId + + " is state " + state); } int retries = isCandidate() ? candidateStateOrThrow().retries() + 1 : 1; + + // Durable transition is not necessary since there is no change to the persisted electionState + memoryTransitionTo( + new ProspectiveState( + time, + localIdOrThrow(), + epoch(), + leaderId(), + state.leaderEndpoints(), + votedKey(), + partitionState.lastVoterSet(), + state.highWatermark(), + retries, + randomElectionTimeoutMs(), + logContext + ) + ); + } + + public void transitionToCandidate() { + checkValidTransitionToCandidate(); + int newEpoch = epoch() + 1; int electionTimeoutMs = randomElectionTimeoutMs(); + int retries = isProspective() ? prospectiveStateOrThrow().retries() : 1; + durableTransitionTo(new CandidateState( time, localIdOrThrow(), @@ -562,6 +656,30 @@ public class QuorumState { )); } + 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() + ) + ); + } + // Only Prospective is allowed to transition to Candidate + if (!isProspective()) { + throw new IllegalStateException( + String.format( + "Cannot transition to Candidate since the local broker.id=%s is state %s", + localId, + state + ) + ); + } + } + public LeaderState transitionToLeader(long epochStartOffset, BatchAccumulator accumulator) { if (isObserver()) { throw new IllegalStateException( @@ -578,7 +696,7 @@ public class QuorumState { } 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 @@ -600,7 +718,7 @@ public class QuorumState { partitionState.lastVoterSet(), partitionState.lastVoterSetOffset(), partitionState.lastKraftVersion(), - candidateState.grantingVoters(), + candidateState.epochElection().grantingVoters(), accumulator, localListeners, fetchTimeoutMs, @@ -657,9 +775,9 @@ public class QuorumState { } 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() { @@ -683,12 +801,42 @@ public class QuorumState { 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() { + return maybeProspectiveState().orElseThrow( + () -> new IllegalStateException("Expected to be Prospective, but current state is " + state) + ); + } + + public boolean isProspectiveNotVoted() { + return maybeProspectiveState().filter(prospective -> prospective.votedKey().isEmpty()).isPresent(); + } + + public boolean isProspectiveAndVoted() { + return maybeProspectiveState().flatMap(ProspectiveState::votedKey).isPresent(); + } + public CandidateState candidateStateOrThrow() { if (isCandidate()) return (CandidateState) state; throw new IllegalStateException("Expected to be Candidate, but current state is " + state); } + public NomineeState nomineeStateOrThrow() { + if (isNomineeState()) + return (NomineeState) state; + throw new IllegalStateException("Expected to be a NomineeState (Prospective or Candidate), " + + "but current state is " + state); + } + public LeaderAndEpoch leaderAndEpoch() { ElectionState election = state.election(); return new LeaderAndEpoch(election.optionalLeaderId(), election.epoch()); @@ -718,7 +866,76 @@ public class QuorumState { return state instanceof ResignedState; } + public boolean isProspective() { + return state instanceof ProspectiveState; + } + public boolean isCandidate() { return state instanceof CandidateState; } + + 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, + 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/ResignedState.java b/raft/src/main/java/org/apache/kafka/raft/ResignedState.java index d79ee44c846..2d5fd27919f 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 org.slf4j.Logger; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; /** @@ -75,7 +76,7 @@ public class ResignedState implements EpochState { @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 94a596d4115..6b7e4b700f2 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.Optional; 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. * @@ -71,10 +73,10 @@ public class UnattachedState implements EpochState { @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); + 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); } @@ -120,60 +122,29 @@ public class UnattachedState implements EpochState { @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 epoch/offset 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 replica'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( + "UnattachedState(epoch=%d, leaderId=%s, votedKey=%s, voters=%s, " + + "electionTimeoutMs=%d, highWatermark=%s)", + epoch, + leaderId, + votedKey, + voters, + electionTimeoutMs, + highWatermark + ); } @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 new file mode 100644 index 00000000000..8cebe1becd9 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/EpochElection.java @@ -0,0 +1,203 @@ +/* + * 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; + +import org.apache.kafka.raft.ReplicaKey; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; +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; + + public EpochElection(Set voters) { + this.voterStates = voters.stream() + .collect( + Collectors.toMap( + ReplicaKey::id, + VoterState::new + ) + ); + } + + /** + * 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) { + VoterState voterState = getVoterStateOrThrow(voterId); + boolean wasUnrecorded = voterState.state == VoterState.State.UNRECORDED; + voterState.setState( + isGranted ? VoterState.State.GRANTED : VoterState.State.REJECTED + ); + return wasUnrecorded; + } + + /** + * 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; + } + + /** + * 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; + } + + /** + * The set of voter ids. + */ + public Set voterIds() { + return Collections.unmodifiableSet(voterStates.keySet()); + } + + /** + * 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(); + } + + /** + * 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 votersOfState(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 + */ + public Set grantingVoters() { + return votersOfState(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 + */ + 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() + .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; + } + + @Override + public String toString() { + return String.format( + "EpochElection(voterStates=%s)", + voterStates + ); + } + + private 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 + } + + @Override + public String toString() { + return String.format( + "VoterState(replicaKey=%s, state=%s)", + replicaKey, + state + ); + } + } +} 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 64230b45f72..fa93633b480 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,14 @@ public class KafkaRaftMetrics implements AutoCloseable { Gauge stateProvider = (mConfig, currentTimeMs) -> { if (state.isLeader()) { return "leader"; + } else if (state.isProspectiveNotVoted()) { + return "prospective"; + } else if (state.isProspectiveAndVoted()) { + return "prospective-voted"; } 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()) { @@ -96,9 +100,7 @@ public class KafkaRaftMetrics implements AutoCloseable { 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); } }); @@ -111,10 +113,7 @@ public class KafkaRaftMetrics implements AutoCloseable { 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/CandidateStateTest.java b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java index e494a306cad..217efad22f5 100644 --- a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java @@ -45,15 +45,15 @@ public class CandidateStateTest { private CandidateState newCandidateState(VoterSet voters) { return new CandidateState( - time, - localReplicaKey.id(), - localReplicaKey.directoryId().get(), - epoch, - voters, - Optional.empty(), - 0, - electionTimeoutMs, - logContext + time, + localReplicaKey.id(), + localReplicaKey.directoryId().get(), + epoch, + voters, + Optional.empty(), + 1, + electionTimeoutMs, + logContext ); } @@ -61,9 +61,9 @@ public class CandidateStateTest { @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 class CandidateStateTest { 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 class CandidateStateTest { 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 class CandidateStateTest { 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 class CandidateStateTest { 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 class CandidateStateTest { ); assertTrue(state.recordGrantedVote(otherNodeId)); assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(otherNodeId)); - assertTrue(state.isVoteGranted()); + assertTrue(state.epochElection().isVoteGranted()); } @ParameterizedTest @@ -173,7 +173,7 @@ public class CandidateStateTest { ); 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/ElectionStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java index 43ac53d11ac..85a3ca3951a 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 org.junit.jupiter.params.provider.ValueSource; 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,26 +71,34 @@ final class ElectionStateTest { 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.withVotedCandidate(5, votedKey, 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.withElectedLeader(5, 1, Optional.of(votedKey), Set.of(1, 2, 3)) ); final List expected; 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), 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, Collections.emptySet()), - ElectionState.withVotedCandidate(5, votedKey, Collections.emptySet()) + ElectionState.withElectedLeader(5, 1, Optional.empty(), 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 new file mode 100644 index 00000000000..e14e4cb17ff --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/EpochElectionTest.java @@ -0,0 +1,139 @@ +/* + * 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.raft.internals.EpochElection; + +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 { + 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); + + assertEquals(voters, epochElection.unrecordedVoters()); + assertTrue(epochElection.grantingVoters().isEmpty()); + assertTrue(epochElection.rejectingVoters().isEmpty()); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + assertFalse(epochElection.isGrantedVoter(voter1)); + assertFalse(epochElection.isRejectedVoter(voter1)); + } + + @Test + public void testRecordGrantedVote() { + EpochElection epochElection = new EpochElection(voters); + + assertTrue(epochElection.recordVote(voter1, true)); + assertEquals(1, epochElection.grantingVoters().size()); + assertTrue(epochElection.grantingVoters().contains(voter1)); + assertEquals(0, epochElection.rejectingVoters().size()); + assertEquals(2, epochElection.unrecordedVoters().size()); + assertTrue(epochElection.isGrantedVoter(voter1)); + assertFalse(epochElection.isRejectedVoter(voter1)); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + + // recording same id as granted + assertFalse(epochElection.recordVote(voter1, true)); + assertTrue(epochElection.isGrantedVoter(voter1)); + assertFalse(epochElection.isVoteGranted()); + + // recording majority as granted + assertTrue(epochElection.recordVote(voter1 + 1, true)); + assertEquals(2, epochElection.grantingVoters().size()); + assertEquals(0, epochElection.rejectingVoters().size()); + assertEquals(1, epochElection.unrecordedVoters().size()); + 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); + + assertTrue(epochElection.recordVote(voter1, false)); + assertEquals(0, epochElection.grantingVoters().size()); + assertEquals(1, epochElection.rejectingVoters().size()); + assertTrue(epochElection.rejectingVoters().contains(voter1)); + assertEquals(2, epochElection.unrecordedVoters().size()); + assertFalse(epochElection.isGrantedVoter(voter1)); + assertTrue(epochElection.isRejectedVoter(voter1)); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + + // recording same id as rejected + assertFalse(epochElection.recordVote(voter1, false)); + assertFalse(epochElection.isGrantedVoter(voter1)); + assertFalse(epochElection.isVoteRejected()); + + // recording majority as rejected + assertTrue(epochElection.recordVote(voter1 + 1, false)); + assertEquals(0, epochElection.grantingVoters().size()); + assertEquals(2, epochElection.rejectingVoters().size()); + assertEquals(1, epochElection.unrecordedVoters().size()); + 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); + + assertTrue(epochElection.recordVote(voter1, true)); + assertFalse(epochElection.recordVote(voter1, false)); + assertEquals(0, epochElection.grantingVoters().size()); + assertEquals(1, epochElection.rejectingVoters().size()); + assertTrue(epochElection.rejectingVoters().contains(voter1)); + assertFalse(epochElection.isGrantedVoter(voter1)); + assertTrue(epochElection.isRejectedVoter(voter1)); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + + 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(voter1 + 2)); + assertTrue(epochElection.isGrantedVoter(voter1 + 2)); + assertFalse(epochElection.isRejectedVoter(voter1 + 2)); + assertFalse(epochElection.isVoteGranted()); + assertFalse(epochElection.isVoteRejected()); + } + + 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 9782bc05a22..a0893ae2aa0 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 @@ public class FileQuorumStateStoreTest { 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, voters), + ElectionState.withElectedLeader(epoch, voter1, Optional.of(votedKey), voters), kraftVersion ); final Optional expected; if (kraftVersion.isReconfigSupported()) { expected = Optional.of( - ElectionState.withElectedLeader(epoch, voter1, Collections.emptySet()) + ElectionState.withElectedLeader(epoch, voter1, Optional.of(votedKey), Collections.emptySet()) ); } else { - expected = Optional.of(ElectionState.withElectedLeader(epoch, voter1, 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 4c93cad065a..c7e86c3fe49 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,18 +48,19 @@ public class FollowerStateTest { InetSocketAddress.createUnresolved("mock-host-3", 1234) ) ); + private final ReplicaKey votedKey = ReplicaKey.of(2, Uuid.randomUuid()); private FollowerState newFollowerState( - Set voters, - Optional highWatermark + Set voters ) { return new FollowerState( time, epoch, leaderId, leaderEndpoints, + Optional.of(votedKey), voters, - highWatermark, + Optional.empty(), fetchTimeoutMs, logContext ); @@ -66,7 +68,7 @@ public class FollowerStateTest { @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())); @@ -82,7 +84,7 @@ public class FollowerStateTest { @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); @@ -95,37 +97,45 @@ public class FollowerStateTest { @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)); 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 testGrantVote(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)); 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)); + } + + @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/KafkaNetworkChannelTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java index af44137d061..ba49a9c1431 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 @@ public class KafkaNetworkChannelTest { 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 a27853ce64a..43cfeb29fe1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientPreVoteTest.java @@ -17,37 +17,51 @@ package org.apache.kafka.raft; 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; +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.ValueSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; import java.util.Collections; +import java.util.List; 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; +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; public class KafkaRaftClientPreVoteTest { @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testHandlePreVoteRequestAsFollowerWithElectedLeader(boolean hasFetchedFromLeader) throws Exception { + @MethodSource("kraftVersionHasFetchedCombinations") + public void testHandlePreVoteRequestAsFollower( + 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) - .withKip853Rpc(true) + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(local, otherNodeKey, electedLeader)), kraftVersion) + .withElectedLeader(epoch, electedLeader.id()) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); if (hasFetchedFromLeader) { @@ -58,24 +72,23 @@ public class KafkaRaftClientPreVoteTest { 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) ); } - // 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(electedLeaderId), voteGranted); - context.assertElectedLeader(epoch, electedLeaderId); + // 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(electedLeaderId), voteGranted); - context.assertElectedLeader(epoch, electedLeaderId); + 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 context.deliverRequest(context.preVoteRequest(epoch + 1, otherNodeKey, epoch + 1, 1)); @@ -86,186 +99,255 @@ public class KafkaRaftClientPreVoteTest { assertTrue(context.client.quorum().isUnattachedNotVoted()); } - @Test - public void testHandlePreVoteRequestAsCandidate() 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); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey, votedCandidateKey)); + + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(voters, kraftVersion) + .withVotedCandidate(epoch, votedCandidateKey) + .withRaftProtocol(KIP_996_PROTOCOL) + .build(); + // unattached will send fetch request before transitioning to follower, proactively clear the mock sent queue + context.client.poll(); + 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); + + // 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 + @EnumSource(value = KRaftVersion.class) + 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()); + int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - .withKip853Rpc(true) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), kraftVersion) + .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, localId); + 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, 1)); + // 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, localId); + 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, 1)); + 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()); } - @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( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(replica1, replica2)), kraftVersion) .withUnknownLeader(epoch) - .withKip853Rpc(true) + .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, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); + + assertEquals(epoch, context.currentEpoch()); + assertTrue(context.client.quorum().isUnattached()); + assertTrue(context.client.quorum().isObserver()); } - @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( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(replica1, replica2)), kraftVersion) .withVotedCandidate(epoch, replica2) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - - 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 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 different replica sends a 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(); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); + + // 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, 1)); context.pollUntilResponse(); - - assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), true); + + assertEquals(epoch, context.currentEpoch()); + assertTrue(context.client.quorum().isUnattachedAndVoted()); } - @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( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(replica1, replica2)), kraftVersion) .withElectedLeader(epoch, leader.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - - 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 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.of(leader.id()), true); - // if different replica sends a 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(); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), true); + + // 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, 1)); 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 - @ValueSource(booleans = {true, false}) - public void testHandlePreVoteRequestAsFollowerObserver(boolean hasFetchedFromLeader) throws Exception { + @MethodSource("kraftVersionHasFetchedCombinations") + public void testHandlePreVoteRequestAsFollowerObserver( + KRaftVersion kraftVersion, + 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()); + int epoch = 2; - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(leader, follower)), kraftVersion) .withElectedLeader(epoch, leader.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); context.assertElectedLeader(epoch, leader.id()); + assertTrue(context.client.quorum().isFollower()); + assertTrue(context.client.quorum().isObserver()); if (hasFetchedFromLeader) { context.pollUntilRequest(); @@ -275,49 +357,59 @@ public class KafkaRaftClientPreVoteTest { 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) ); } context.deliverRequest(context.preVoteRequest(epoch, follower, epoch, 1)); context.pollUntilResponse(); - boolean voteGranted = !hasFetchedFromLeader; - context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leaderId), voteGranted); + context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(leader.id()), !hasFetchedFromLeader); assertTrue(context.client.quorum().isFollower()); + assertTrue(context.client.quorum().isObserver()); } - @Test - public void testHandleInvalidPreVoteRequestWithOlderEpoch() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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.id(), + local.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(local, otherNodeKey)), kraftVersion) .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 - public void testLeaderRejectPreVoteRequestOnSameEpoch() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderRejectPreVoteRequestOnSameEpoch(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.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), kraftVersion) .withUnknownLeader(2) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int leaderEpoch = context.currentEpoch(); context.deliverRequest(context.preVoteRequest(leaderEpoch, otherNodeKey, leaderEpoch, 1)); @@ -328,17 +420,22 @@ public class KafkaRaftClientPreVoteTest { context.assertElectedLeader(leaderEpoch, localId); } - @Test - public void testPreVoteRequestClusterIdValidation() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testPreVoteRequestClusterIdValidation(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) - .withKip853Rpc(true) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), kraftVersion) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -362,17 +459,22 @@ public class KafkaRaftClientPreVoteTest { context.assertSentVoteResponse(Errors.INCONSISTENT_CLUSTER_ID); } - @Test - public void testInvalidVoterReplicaPreVoteRequest() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testInvalidVoterReplicaPreVoteRequest(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) - .withKip853Rpc(true) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), kraftVersion) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // invalid voter id is rejected @@ -406,18 +508,23 @@ public class KafkaRaftClientPreVoteTest { context.assertSentVoteResponse(Errors.INVALID_VOTER_KEY, epoch, OptionalInt.of(localId), false); } - @Test - public void testLeaderAcceptPreVoteFromObserver() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderAcceptPreVoteFromObserver(KRaftVersion kraftVersion) 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( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), kraftVersion) .withUnknownLeader(4) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey observerKey = replicaKey(localId + 2, true); @@ -430,46 +537,57 @@ public class KafkaRaftClientPreVoteTest { context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.of(localId), false); } - @Test - public void testHandlePreVoteRequestAsResigned() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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) - .withKip853Rpc(true) + RaftClientTestContext context = new RaftClientTestContext.Builder( + localKey.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), kraftVersion) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.quorum().transitionToResigned(Collections.emptyList()); assertTrue(context.client.quorum().isResigned()); // 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, context.log.endOffset().offset())); 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, context.log.endOffset().offset())); context.pollUntilResponse(); context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.of(-1), true); assertTrue(context.client.quorum().isUnattached()); } - @Test - public void testInvalidVoteRequest() throws Exception { + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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.id(), + localKey.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(localKey, otherNodeKey)), kraftVersion) .withElectedLeader(epoch, otherNodeKey.id()) - .withKip853Rpc(true) + .withRaftProtocol(KIP_996_PROTOCOL) .build(); assertEquals(epoch, context.currentEpoch()); context.assertElectedLeader(epoch, otherNodeKey.id()); + // invalid offset context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, 0, -5L)); context.pollUntilResponse(); context.assertSentVoteResponse( @@ -481,6 +599,7 @@ public class KafkaRaftClientPreVoteTest { assertEquals(epoch, context.currentEpoch()); context.assertElectedLeader(epoch, otherNodeKey.id()); + // invalid epoch context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, -1, 0L)); context.pollUntilResponse(); context.assertSentVoteResponse( @@ -492,6 +611,7 @@ public class KafkaRaftClientPreVoteTest { assertEquals(epoch, context.currentEpoch()); context.assertElectedLeader(epoch, otherNodeKey.id()); + // lastEpoch > replicaEpoch context.deliverRequest(context.preVoteRequest(epoch, otherNodeKey, epoch + 1, 0L)); context.pollUntilResponse(); context.assertSentVoteResponse( @@ -504,28 +624,36 @@ public class KafkaRaftClientPreVoteTest { 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( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(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 + // 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); @@ -539,21 +667,27 @@ public class KafkaRaftClientPreVoteTest { 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()); } - @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( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(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()); @@ -573,4 +707,454 @@ public class KafkaRaftClientPreVoteTest { assertTrue(context.client.quorum().isUnattached()); context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), false); } + + @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( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(local, voter2, voter3)), kraftVersion) + .withUnknownLeader(epoch) + .withRaftProtocol(KIP_996_PROTOCOL) + .build(); + + context.assertUnknownLeaderAndNoVotedCandidate(epoch); + + // Sleep a little to ensure transition to prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + + // Wait until the vote requests are inflight + context.pollUntilRequest(); + assertTrue(context.client.quorum().isProspective()); + List voteRequests = context.collectPreVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + + // 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()); + + // 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()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testPreVoteNotSupportedByRemote(KRaftVersion kraftVersion) throws Exception { + int localId = randomReplicaId(); + ReplicaKey local = replicaKey(localId, true); + ReplicaKey voter2Key = replicaKey(localId + 1, true); + ReplicaKey voter3Key = replicaKey(localId + 2, true); + int epoch = 5; + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(local, voter2Key, voter3Key)), kraftVersion) + .withUnknownLeader(epoch) + .withRaftProtocol(KIP_996_PROTOCOL) + .build(); + + context.assertUnknownLeaderAndNoVotedCandidate(epoch); + + // Sleep a little to ensure transition to Prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + assertEquals(epoch, context.currentEpoch()); + assertTrue(context.client.quorum().isProspective()); + + // 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) + ); + + // 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(); + + // 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(); + context.collectVoteRequests(epoch + 1, 0, 0); + + // Sleep to transition back to Prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertEquals(epoch + 1, context.currentEpoch()); + assertTrue(context.client.quorum().isProspective()); + + // 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(), + context.voteResponse(true, OptionalInt.empty(), epoch + 1) + ); + context.client.poll(); + assertEquals(epoch + 2, context.currentEpoch()); + context.client.quorum().isCandidate(); + + // Any further PreVote responses 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(); + } + + @ParameterizedTest + @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( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(local, leader)), kraftVersion) + .withUnknownLeader(epoch) + .withRaftProtocol(raftProtocol) + .build(); + + context.assertUnknownLeaderAndNoVotedCandidate(epoch); + + // Sleep a little to ensure 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()); + } + + @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( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(local, otherNode)), kraftVersion) + .withUnknownLeader(epoch) + .withRaftProtocol(raftProtocol) + .build(); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); + + // Sleep a little to ensure that transition to prospective + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + assertTrue(context.client.quorum().isProspective()); + context.assertSentPreVoteRequest(epoch, 0, 0L, 1); + + // 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, replica will transition back to prospective and send PreVote requests + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + RaftRequest.Outbound voteRequest = context.assertSentPreVoteRequest(epoch, 0, 0L, 1); + + // If prospective receives enough rejected votes, it also transitions to unattached immediately + context.deliverResponse( + voteRequest.correlationId(), + voteRequest.destination(), + context.voteResponse(false, OptionalInt.empty(), epoch) + ); + 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(); + context.assertSentPreVoteRequest(epoch, 0, 0L, 1); + } + + @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( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(VoterSetTest.voterSet(Stream.of(local, replica1, replica2)), kraftVersion) + .withElectedLeader(epoch, replica1.id()) + .withRaftProtocol(raftProtocol) + .build(); + context.assertElectedLeader(epoch, replica1.id()); + assertTrue(context.client.quorum().isFollower()); + + // Sleep a little to ensure transition to prospective + context.time.sleep(context.fetchTimeoutMs); + context.pollUntilRequest(); + assertTrue(context.client.quorum().isProspective()); + context.assertSentPreVoteRequest(epoch, 0, 0L, 2); + + // 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, replica will transition back to prospective and send PreVote requests + context.time.sleep(context.fetchTimeoutMs); + context.pollUntilRequest(); + List voteRequests = context.collectPreVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + assertTrue(context.client.quorum().isProspective()); + context.assertElectedLeader(epoch, replica1.id()); + + // If prospective receives enough rejected votes without leaderId, it also transitions to follower immediately + context.deliverResponse( + voteRequests.get(0).correlationId(), + voteRequests.get(0).destination(), + context.voteResponse(false, OptionalInt.empty(), epoch) + ); + context.client.poll(); + + context.deliverResponse( + voteRequests.get(1).correlationId(), + voteRequests.get(1).destination(), + context.voteResponse(false, OptionalInt.empty(), epoch) + ); + context.client.poll(); + assertTrue(context.client.quorum().isFollower()); + + context.client.poll(); + context.assertSentFetchRequest(); + + // After election times out again, transition back to prospective and send PreVote requests + context.time.sleep(context.fetchTimeoutMs); + context.pollUntilRequest(); + voteRequests = context.collectPreVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + assertTrue(context.client.quorum().isProspective()); + context.assertElectedLeader(epoch, replica1.id()); + + // 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(Errors.FENCED_LEADER_EPOCH, OptionalInt.of(replica2.id()), epoch + 1)); + context.client.poll(); + assertTrue(context.client.quorum().isFollower()); + 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.id(), + local.directoryId().get() + ) + .withStartingVoters(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( + 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; + + RaftClientTestContext context = new RaftClientTestContext.Builder( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(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 + context.time.sleep(context.electionTimeoutMs() * 2L); + context.pollUntilRequest(); + assertTrue(context.client.quorum().isProspective()); + List voteRequests = context.collectPreVoteRequests(epoch, 0, 0); + assertEquals(2, voteRequests.size()); + + // 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(), + voteRequests.get(0).destination(), + voteResponse1 + ); + + // Prospective should transition to Follower + context.client.poll(); + assertTrue(context.client.quorum().isFollower()); + assertEquals(OptionalInt.of(leader.id()), context.client.quorum().leaderId()); + } + + @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( + local.id(), + local.directoryId().get() + ) + .withStartingVoters(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()); + + // Simulate a request timeout + context.pollUntilRequest(); + RaftRequest.Outbound request = context.assertSentPreVoteRequest(epoch, 0, 0L, 1); + context.time.sleep(context.requestTimeoutMs()); + + // Prospective should retry the request + context.client.poll(); + RaftRequest.Outbound retryRequest = context.assertSentPreVoteRequest(epoch, 0, 0L, 1); + + // 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()) + .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/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index 8dddba6a10d..493083831d1 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 class KafkaRaftClientReconfigTest { 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 class KafkaRaftClientReconfigTest { .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 class KafkaRaftClientReconfigTest { ); // 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 class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -412,7 +412,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); InetSocketAddress newAddress = InetSocketAddress.createUnresolved( @@ -477,7 +477,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); InetSocketAddress newAddress = InetSocketAddress.createUnresolved( @@ -507,7 +507,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -563,7 +563,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -601,7 +601,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -646,7 +646,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(follower.id(), true); @@ -691,7 +691,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -763,7 +763,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -823,7 +823,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -883,7 +883,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -936,7 +936,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -985,7 +985,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, false); @@ -1024,7 +1024,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertTrue(context.client.quorum().isVoter(follower2)); @@ -1073,7 +1073,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1145,7 +1145,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); // empty cluster id is rejected context.deliverRequest(context.removeVoterRequest("", follower1)); @@ -1194,7 +1194,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1232,7 +1232,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); // Attempt to remove follower2 context.deliverRequest(context.removeVoterRequest(follower2)); @@ -1254,7 +1254,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1284,7 +1284,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1314,7 +1314,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1366,7 +1366,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1410,7 +1410,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1452,7 +1452,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1497,7 +1497,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey newVoter = replicaKey(local.id() + 2, true); @@ -1545,7 +1545,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertTrue(context.client.quorum().isVoter(follower)); @@ -1615,7 +1615,7 @@ public class KafkaRaftClientReconfigTest { .withLocalListeners(localListeners) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertTrue(context.client.quorum().isVoter(follower)); @@ -1650,7 +1650,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // empty cluster id is rejected @@ -1701,7 +1701,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -1734,7 +1734,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -1796,7 +1796,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Attempt to update the follower @@ -1841,7 +1841,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1892,7 +1892,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1943,7 +1943,7 @@ public class KafkaRaftClientReconfigTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Establish a HWM and fence previous leaders @@ -1994,7 +1994,7 @@ public class KafkaRaftClientReconfigTest { .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 6ef692229af..1f3307f9ada 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 final class KafkaRaftClientSnapshotTest { RaftClientTestContext context = contextBuilder.build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Advance the highWatermark @@ -236,7 +236,7 @@ public final class KafkaRaftClientSnapshotTest { .deleteBeforeSnapshot(snapshotId) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Stop the listener from reading commit batches @@ -293,7 +293,7 @@ public final class KafkaRaftClientSnapshotTest { .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 final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); List appendRecords = Arrays.asList("a", "b", "c"); @@ -371,7 +371,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); List appendRecords = Arrays.asList("a", "b", "c"); @@ -423,7 +423,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 1, epoch); @@ -465,7 +465,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch); @@ -516,7 +516,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch); @@ -562,7 +562,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch); @@ -616,7 +616,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch); @@ -660,7 +660,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -693,7 +693,7 @@ public final class KafkaRaftClientSnapshotTest { .withUnknownLeader(3) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -724,7 +724,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -756,7 +756,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.advanceLocalLeaderHighWatermarkToLogEndOffset(); @@ -813,7 +813,7 @@ public final class KafkaRaftClientSnapshotTest { .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); FetchSnapshotRequestData voter1FetchSnapshotRequest = fetchSnapshotRequest( @@ -902,7 +902,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.advanceLocalLeaderHighWatermarkToLogEndOffset(); @@ -1015,7 +1015,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.advanceLocalLeaderHighWatermarkToLogEndOffset(); @@ -1074,7 +1074,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -1107,7 +1107,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest( @@ -1182,13 +1182,13 @@ public final class KafkaRaftClientSnapshotTest { 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.assertSentPreVoteRequest(epoch, 0, 0L, 1); + assertTrue(context.client.quorum().isProspective()); } @ParameterizedTest @@ -1777,13 +1777,13 @@ public final class KafkaRaftClientSnapshotTest { 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.assertSentPreVoteRequest(epoch, 0, 0L, 1); + assertTrue(context.client.quorum().isProspective()); // Send the response late context.deliverResponse( @@ -1809,9 +1809,9 @@ public final class KafkaRaftClientSnapshotTest { ) ); - // 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 @@ -1828,7 +1828,7 @@ public final class KafkaRaftClientSnapshotTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -1909,7 +1909,7 @@ public final class KafkaRaftClientSnapshotTest { .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 ab6b698cc38..48b8c8b72c1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -70,6 +70,7 @@ import java.util.stream.Stream; 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.KIP_853_PROTOCOL; import static org.apache.kafka.test.TestUtils.assertFutureThrows; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -79,6 +80,7 @@ import static org.junit.jupiter.api.Assertions.assertNull; 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() { @@ -215,7 +217,7 @@ public class KafkaRaftClientTest { ); context.client.poll(); - // We will first transition to unattached and then grant vote and then transition to 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); @@ -252,7 +254,7 @@ public class KafkaRaftClientTest { ); context.client.poll(); - // We will first transition to unattached and then grant vote and then transition to 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); @@ -270,7 +272,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Beginning shutdown @@ -288,7 +290,7 @@ public class KafkaRaftClientTest { ); context.client.poll(); - // We will first transition to unattached and then grant vote and then transition to voted + // Replica will first transition to unattached, then grant vote, then transition to unattached voted assertTrue( context.client.quorum().isUnattachedAndVoted(), "Local Id: " + localId + @@ -302,7 +304,7 @@ public class KafkaRaftClientTest { @ParameterizedTest @ValueSource(booleans = { true, false }) - public void testInitializeAsResignedAndBecomeCandidate(boolean withKip853Rpc) throws Exception { + public void testInitializeAsResignedAndUnableToContactQuorum(boolean withKip853Rpc) throws Exception { int localId = randomReplicaId(); int remoteId = localId + 1; Set voters = Set.of(localId, remoteId); @@ -323,9 +325,19 @@ public class KafkaRaftClientTest { context.time.sleep(context.electionTimeoutMs()); context.client.poll(); - // Become candidate in a new epoch - assertTrue(context.client.quorum().isCandidate()); - context.assertVotedCandidate(epoch + 1, localId); + // Become unattached with expired election timeout + assertTrue(context.client.quorum().isUnattached()); + assertEquals(epoch + 1, context.currentEpoch()); + + // Become prospective immediately + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + + // Become unattached again after election timeout + context.time.sleep(context.electionTimeoutMs() * 2L); + context.client.poll(); + assertTrue(context.client.quorum().isUnattached()); + assertEquals(epoch + 1, context.currentEpoch()); } @ParameterizedTest @@ -358,10 +370,13 @@ public class KafkaRaftClientTest { ); context.client.poll(); + // The node will transition to unattached with epoch + 1 after election timeout passes context.time.sleep(context.electionTimeoutMs()); - context.pollUntilRequest(); - context.assertVotedCandidate(epoch + 1, localId); - context.assertSentVoteRequest(epoch + 1, 0, 0L, 1); + context.client.poll(); + assertTrue(context.client.quorum().isUnattached()); + assertEquals(epoch + 1, context.currentEpoch()); + UnattachedState unattached = context.client.quorum().unattachedStateOrThrow(); + assertEquals(0, unattached.remainingElectionTimeMs(context.time.milliseconds())); } @ParameterizedTest @@ -399,7 +414,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -418,7 +433,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -439,7 +454,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -512,7 +527,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); // send fetch request when become leader @@ -550,7 +565,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int currentEpoch = context.currentEpoch(); @@ -577,7 +592,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int resignedEpoch = context.currentEpoch(); @@ -605,7 +620,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(OptionalInt.of(localId), context.currentLeader()); @@ -641,7 +656,7 @@ public class KafkaRaftClientTest { .build(); int resignLeadershipTimeout = context.checkQuorumTimeoutMs; - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); assertEquals(OptionalInt.of(localId), context.currentLeader()); @@ -714,7 +729,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int resignedEpoch = context.currentEpoch(); @@ -733,7 +748,7 @@ public class KafkaRaftClientTest { 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()); @@ -748,12 +763,17 @@ public class KafkaRaftClientTest { OptionalInt.of(localId) ); - // After the election timer, we should become a candidate. + // After the election timer, local should become unattached. context.time.sleep(2L * context.electionTimeoutMs()); - context.pollUntil(context.client.quorum()::isCandidate); + 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.client.poll(); + assertTrue(context.client.quorum().isProspective()); } @ParameterizedTest @@ -766,7 +786,7 @@ public class KafkaRaftClientTest { 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 +859,7 @@ public class KafkaRaftClientTest { @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 +867,7 @@ public class KafkaRaftClientTest { .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 +886,13 @@ public class KafkaRaftClientTest { 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); @@ -912,20 +934,18 @@ public class KafkaRaftClientTest { 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.assertUnknownLeader(0); - context.time.sleep(2L * context.electionTimeoutMs()); - + 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(); @@ -934,7 +954,7 @@ public class KafkaRaftClientTest { // 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 @@ -943,7 +963,7 @@ public class KafkaRaftClientTest { // 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(); @@ -955,6 +975,52 @@ public class KafkaRaftClientTest { 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()); + } + + @Test + public void testInitializeAsFollowerAndOnlyVoter() throws Exception { + int localId = randomReplicaId(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + .withRaftProtocol(KIP_853_PROTOCOL) + .withElectedLeader(2, localId + 1) + .build(); + context.assertElectedLeader(3, localId); + assertEquals(0L, context.log.endOffset().offset()); + assertTrue(context.client.quorum().isLeader()); + } + + @Test + public void testInitializeAsCandidateAndOnlyVoter() throws Exception { + int localId = randomReplicaId(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + .withRaftProtocol(KIP_853_PROTOCOL) + .withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) + .build(); + context.assertElectedLeader(2, localId); + assertTrue(context.client.quorum().isLeader()); + } + + @Test + public void testInitializeAsResignedAndOnlyVoter() throws Exception { + int localId = randomReplicaId(); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Set.of(localId)) + .withRaftProtocol(KIP_853_PROTOCOL) + .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 { @@ -971,7 +1037,7 @@ public class KafkaRaftClientTest { context.deliverRequest(context.beginEpochRequest(votedCandidateEpoch, otherNodeKey.id())); context.pollUntilResponse(); - context.assertElectedLeader(votedCandidateEpoch, otherNodeKey.id()); + context.assertElectedLeaderAndVotedKey(votedCandidateEpoch, otherNodeKey.id(), otherNodeKey); context.assertSentBeginQuorumEpochResponse( Errors.NONE, @@ -1053,9 +1119,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - // Sleep a little to ensure that we become a candidate - context.time.sleep(context.electionTimeoutMs() + jitterMs); - context.client.poll(); + context.unattachedToCandidate(); context.assertVotedCandidate(epoch, localId); context.deliverRequest( @@ -1069,7 +1133,7 @@ public class KafkaRaftClientTest { 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); @@ -1079,10 +1143,10 @@ public class KafkaRaftClientTest { context.client.poll(); context.assertVotedCandidate(epoch, localId); - // After backoff, we will become a candidate again + // After backoff, replica 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 +1161,7 @@ public class KafkaRaftClientTest { .withUnknownLeader(6) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // One of the voters may have sent EndQuorumEpoch from an earlier epoch @@ -1108,7 +1172,7 @@ public class KafkaRaftClientTest { 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); @@ -1141,9 +1205,9 @@ public class KafkaRaftClientTest { 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 @@ -1165,7 +1229,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -1198,7 +1262,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -1232,7 +1296,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); @@ -1241,7 +1305,7 @@ public class KafkaRaftClientTest { 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 +1325,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); assertEquals(1L, context.log.endOffset().offset()); @@ -1297,7 +1361,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); assertEquals(OptionalInt.of(localId), context.currentLeader()); assertEquals(1L, context.log.endOffset().offset()); @@ -1344,7 +1408,8 @@ public class KafkaRaftClientTest { 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 @@ -1375,19 +1440,19 @@ public class KafkaRaftClientTest { // 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.time.sleep(context.electionBackoffMaxMs); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); - context.pollUntilRequest(); - - List voteRequests = context.collectVoteRequests(leaderEpoch + 1, 0, 0); + context.client.poll(); + List voteRequests = context.collectPreVoteRequests(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 +1466,9 @@ public class KafkaRaftClientTest { 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); @@ -1496,6 +1561,61 @@ public class KafkaRaftClientTest { 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); + 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 testHandleInvalidVoteRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception { @@ -1513,7 +1633,7 @@ public class KafkaRaftClientTest { context.pollUntilResponse(); context.assertSentVoteResponse(Errors.FENCED_LEADER_EPOCH, epoch, OptionalInt.empty(), false); - context.assertUnknownLeader(epoch); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); } @ParameterizedTest @@ -1549,7 +1669,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int leaderEpoch = context.currentEpoch(); context.deliverRequest(context.voteRequest(leaderEpoch, otherNodeKey, leaderEpoch - 1, 1)); @@ -1572,7 +1692,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // First poll has no high watermark advance @@ -1622,7 +1742,7 @@ public class KafkaRaftClientTest { .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 @@ -1661,11 +1781,11 @@ public class KafkaRaftClientTest { @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; - 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) @@ -1673,13 +1793,20 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.assertUnknownLeader(0); + context.assertUnknownLeaderAndNoVotedCandidate(0); - context.time.sleep(2L * context.electionTimeoutMs()); + 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. + // 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(), @@ -1688,22 +1815,87 @@ public class KafkaRaftClientTest { ); 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 + // 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 bump the epoch 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 candidate again + // After jitter expires, become a prospective again 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.assertSentPreVoteRequest(epoch, 0, 0L, 1); + assertEquals( + context.electionTimeoutMs() + exponentialFactor, + prospective.remainingElectionTimeMs(context.time.milliseconds()) + ); + + // After becoming 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); - context.assertSentVoteRequest(epoch + 1, 0, 0L, 1); + 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, replica 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.assertSentPreVoteRequest(epoch, 0, 0L, 1); + assertEquals( + context.electionTimeoutMs() + jitter, + prospective.remainingElectionTimeMs(context.time.milliseconds()) + ); } @ParameterizedTest @@ -1749,7 +1941,7 @@ public class KafkaRaftClientTest { @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; @@ -1767,14 +1959,15 @@ public class KafkaRaftClientTest { context.assertSentFetchRequest(epoch, 1L, lastEpoch); context.time.sleep(context.fetchTimeoutMs); - context.pollUntilRequest(); - context.assertSentVoteRequest(epoch + 1, lastEpoch, 1L, 1); - context.assertVotedCandidate(epoch + 1, localId); + context.client.poll(); + assertTrue(context.client.quorum().isProspective()); + context.client.poll(); + context.assertSentPreVoteRequest(epoch, lastEpoch, 1L, 1); } @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; @@ -1804,7 +1997,7 @@ public class KafkaRaftClientTest { @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; @@ -1833,7 +2026,7 @@ public class KafkaRaftClientTest { 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()); @@ -1987,7 +2180,7 @@ public class KafkaRaftClientTest { @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(); @@ -2130,7 +2323,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, -5L, 0, 0)); @@ -2172,7 +2365,11 @@ public class KafkaRaftClientTest { int epoch = 5; Set voters = Set.of(localId, otherNodeKey.id()); - RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withUnknownLeader(epoch - 1) + .build(); + context.assertUnknownLeaderAndNoVotedCandidate(epoch - 1); + context.unattachedToLeader(); // First poll has no high watermark advance. context.client.poll(); @@ -2202,7 +2399,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -2237,7 +2434,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -2271,7 +2468,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(true) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // invalid voter id is rejected @@ -2317,7 +2514,7 @@ public class KafkaRaftClientTest { .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( @@ -2370,7 +2567,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -2406,7 +2603,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // valid cluster id is accepted @@ -2442,7 +2639,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); ReplicaKey observerKey = replicaKey(localId + 2, withKip853Rpc); @@ -2512,7 +2709,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Follower sends a fetch which cannot be satisfied immediately @@ -2540,7 +2737,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Follower sends a fetch which cannot be satisfied immediately @@ -2571,7 +2768,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Follower sends a fetch which cannot be satisfied immediately @@ -2596,7 +2793,7 @@ public class KafkaRaftClientTest { @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; @@ -2613,10 +2810,10 @@ public class KafkaRaftClientTest { 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 context.time.sleep(context.fetchTimeoutMs); context.client.poll(); - context.assertVotedCandidate(epoch + 1, localId); + assertTrue(context.client.quorum().isProspective()); // The fetch response from the old leader returns, but it should be ignored Records records = context.buildBatch(0L, 3, Arrays.asList("a", "b")); @@ -2628,7 +2825,7 @@ public class KafkaRaftClientTest { context.client.poll(); assertEquals(0, context.log.endOffset().offset()); - context.assertVotedCandidate(epoch + 1, localId); + context.expectAndGrantPreVotes(epoch); } @ParameterizedTest @@ -2682,34 +2879,36 @@ public class KafkaRaftClientTest { Set voters = Set.of(localId, voter2, voter3); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withUnknownLeader(epoch - 1) + .withUnknownLeader(epoch) .withKip853Rpc(withKip853Rpc) .build(); - context.assertUnknownLeader(epoch - 1); - - // Sleep a little to ensure that we become a candidate - context.time.sleep(context.electionTimeoutMs() * 2L); + context.assertUnknownLeaderAndNoVotedCandidate(epoch); + context.unattachedToCandidate(); // Wait until the vote requests are inflight context.pollUntilRequest(); - context.assertVotedCandidate(epoch, localId); - List voteRequests = context.collectVoteRequests(epoch, 0, 0); + context.assertVotedCandidate(epoch + 1, localId); + 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.assertElectedLeaderAndVotedKey( + epoch + 1, + voter3, + ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID) + ); // 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); 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); context.deliverResponse( voteRequests.get(1).correlationId(), voteRequests.get(1).destination(), @@ -2717,7 +2916,11 @@ public class KafkaRaftClientTest { ); context.client.poll(); - context.assertElectedLeader(epoch, voter3); + context.assertElectedLeaderAndVotedKey( + epoch + 1, + voter3, + ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID) + ); } @ParameterizedTest @@ -2899,7 +3102,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Now shutdown @@ -2947,7 +3150,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // The lagging follower fetches first @@ -3030,7 +3233,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Describe quorum response will not include directory ids @@ -3081,7 +3284,7 @@ public class KafkaRaftClientTest { } RaftClientTestContext context = builder.build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Describe quorum response before any fetches made @@ -3187,7 +3390,7 @@ public class KafkaRaftClientTest { } RaftClientTestContext context = builder.build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Update HW to non-initial value @@ -3329,7 +3532,7 @@ public class KafkaRaftClientTest { } RaftClientTestContext context = builder.build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Update HW to non-initial value @@ -3400,7 +3603,7 @@ public class KafkaRaftClientTest { .withBootstrapSnapshot(Optional.of(voterSet)) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // check describe quorum response has both followers context.deliverRequest(context.describeQuorumRequest()); @@ -3440,7 +3643,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); // Now shutdown @@ -3506,7 +3709,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); context.client.poll(); - context.assertUnknownLeader(5); + context.assertUnknownLeaderAndNoVotedCandidate(5); // Observer shutdown should complete immediately even if the // current leader is unknown @@ -3681,7 +3884,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.time.sleep(context.electionTimeoutMs()); + context.unattachedToCandidate(); context.expectAndGrantVotes(epoch); context.pollUntilRequest(); @@ -3908,6 +4111,7 @@ public class KafkaRaftClientTest { .build(); context.time.sleep(context.electionTimeoutMs()); + context.expectAndGrantPreVotes(epoch - 1); context.expectAndGrantVotes(epoch); context.pollUntilRequest(); @@ -3935,8 +4139,8 @@ public class KafkaRaftClientTest { .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); @@ -3960,7 +4164,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); int epoch = context.currentEpoch(); context.client.shutdown(5000); @@ -3987,7 +4191,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.poll(); int epoch = context.currentEpoch(); @@ -4044,7 +4248,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.poll(); // After becoming leader, we expect the `LeaderChange` record to be appended @@ -4110,7 +4314,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.poll(); assertEquals(10L, context.log.endOffset().offset()); @@ -4153,7 +4357,7 @@ public class KafkaRaftClientTest { .withKip853Rpc(withKip853Rpc) .build(); - context.becomeLeader(); + context.unattachedToLeader(); context.client.poll(); assertEquals(10L, context.log.endOffset().offset()); @@ -4255,7 +4459,7 @@ public class KafkaRaftClientTest { .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()); @@ -4302,7 +4506,7 @@ public class KafkaRaftClientTest { .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)); @@ -4313,12 +4517,12 @@ public class KafkaRaftClientTest { // 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 + // Timeout the election and become prospective then 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 new file mode 100644 index 00000000000..d74eba4ab22 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/ProspectiveStateTest.java @@ -0,0 +1,480 @@ +/* + * 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.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +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; + +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 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 = 10000; + private final LogContext logContext = new LogContext(); + 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, + leaderId, + leaderId.isPresent() ? leaderEndpoints : Endpoints.empty(), + votedKey, + voters, + Optional.empty(), + 1, + electionTimeoutMs, + logContext + ); + } + + private ProspectiveState newProspectiveState(VoterSet voters) { + return new ProspectiveState( + time, + localReplicaKey.id(), + epoch, + OptionalInt.empty(), + Endpoints.empty(), + Optional.empty(), + voters, + Optional.empty(), + 1, + electionTimeoutMs, + logContext + ); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testSingleNodeQuorum(boolean withDirectoryId) { + ProspectiveState state = newProspectiveState(voterSetWithLocal(IntStream.empty(), withDirectoryId)); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Collections.emptySet(), state.epochElection().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.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Collections.singleton(otherNode), state.epochElection().unrecordedVoters()); + assertTrue(state.recordRejectedVote(otherNode.id())); + assertFalse(state.epochElection().isVoteGranted()); + assertTrue(state.epochElection().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.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Collections.singleton(otherNode), state.epochElection().unrecordedVoters()); + assertTrue(state.recordGrantedVote(otherNode.id())); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); + assertFalse(state.epochElection().isVoteRejected()); + assertTrue(state.epochElection().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.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Set.of(node1, node2), state.epochElection().unrecordedVoters()); + assertTrue(state.recordGrantedVote(node1.id())); + assertEquals(Collections.singleton(node2), state.epochElection().unrecordedVoters()); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertTrue(state.recordRejectedVote(node2.id())); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().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.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertEquals(Set.of(node1, node2), state.epochElection().unrecordedVoters()); + assertTrue(state.recordRejectedVote(node1.id())); + assertEquals(Collections.singleton(node2), state.epochElection().unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertFalse(state.epochElection().isVoteRejected()); + assertTrue(state.recordRejectedVote(node2.id())); + assertEquals(Collections.emptySet(), state.epochElection().unrecordedVoters()); + assertFalse(state.epochElection().isVoteGranted()); + assertTrue(state.epochElection().isVoteRejected()); + } + + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testCanChangePreVote(boolean withDirectoryId) { + int voter1 = 1; + int voter2 = 2; + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.of(voter1, voter2), withDirectoryId) + ); + assertTrue(state.recordGrantedVote(voter1)); + assertTrue(state.epochElection().isVoteGranted()); + assertFalse(state.recordRejectedVote(voter1)); + assertFalse(state.epochElection().isVoteGranted()); + + assertTrue(state.recordRejectedVote(voter2)); + assertTrue(state.epochElection().isVoteRejected()); + assertFalse(state.recordGrantedVote(voter2)); + assertFalse(state.epochElection().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 testConsecutiveGrant(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 testConsecutiveReject(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,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); + + ProspectiveState state = newProspectiveState( + voterSetWithLocal(Stream.of(node1, node2), withDirectoryId) + ); + + 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, false)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, false)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, false)); + } + + @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); + + ProspectiveState state = newProspectiveState( + voterSetWithLocal(Stream.of(node1, node2), withDirectoryId), + OptionalInt.empty(), + Optional.of(node1) + ); + + 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, false)); + assertTrue(state.canGrantVote(node1, isLogUpToDate, false)); + assertFalse(state.canGrantVote(node2, isLogUpToDate, false)); + } + + @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.canGrantVote(node0, isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(node1, isLogUpToDate, true)); + assertEquals(isLogUpToDate, state.canGrantVote(node2, isLogUpToDate, true)); + + assertFalse(state.canGrantVote(node0, isLogUpToDate, false)); + assertFalse(state.canGrantVote(node1, isLogUpToDate, false)); + assertFalse(state.canGrantVote(node2, isLogUpToDate, false)); + } + + @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.withUnknownLeader( + epoch, + voters.voterIds() + ), + state.election() + ); + + // with leader + state = newProspectiveState(voters, OptionalInt.of(1), Optional.empty()); + assertEquals( + ElectionState.withElectedLeader( + epoch, + 1, + Optional.empty(), voters.voterIds() + ), + state.election() + ); + + // with voted key + ReplicaKey votedKey = replicaKey(1, withDirectoryId); + state = newProspectiveState(voters, OptionalInt.empty(), Optional.of(votedKey)); + assertEquals( + ElectionState.withVotedCandidate( + epoch, + votedKey, + voters.voterIds() + ), + 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 + 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 testCanGrantVoteWithoutDirectoryId(boolean isLogUpToDate) { + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.empty(), true), + OptionalInt.empty(), + Optional.of(votedKeyWithoutDirectoryId)); + + 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}) + 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.canGrantVote(votedKeyWithDirectoryId, isLogUpToDate, true) + ); + 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.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), true, false)); + assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), true, false)); + } + + @Test + public void testLeaderEndpoints() { + ProspectiveState state = newProspectiveState( + voterSetWithLocal(IntStream.of(1, 2, 3), true), + OptionalInt.empty(), + Optional.of(ReplicaKey.of(1, Uuid.randomUuid())) + ); + assertEquals(Endpoints.empty(), state.leaderEndpoints()); + + state = newProspectiveState( + voterSetWithLocal(IntStream.of(1, 2, 3), true), + OptionalInt.of(3), + Optional.of(ReplicaKey.of(1, Uuid.randomUuid())) + ); + assertEquals(leaderEndpoints, 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 360c3bba62a..ffe47f66d9c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -33,6 +33,7 @@ import java.io.UncheckedIOException; 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; @@ -97,7 +98,7 @@ public class QuorumStateTest { } private Set persistedVoters(Set voters, KRaftVersion kraftVersion) { - if (kraftVersion.featureLevel() == 1) { + if (kraftVersion.isReconfigSupported()) { return Collections.emptySet(); } @@ -105,7 +106,7 @@ public class QuorumStateTest { } private ReplicaKey persistedVotedKey(ReplicaKey replicaKey, KRaftVersion kraftVersion) { - if (kraftVersion.featureLevel() == 1) { + if (kraftVersion.isReconfigSupported()) { return replicaKey; } @@ -154,6 +155,163 @@ public class QuorumStateTest { 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, OptionalInt.empty()); + 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, OptionalInt.empty()); + 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( + ElectionState.withElectedLeader( + 4, + 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) { @@ -163,9 +321,10 @@ public class QuorumStateTest { QuorumState state = initializeEmptyState(voters, kraftVersion); assertTrue(state.isUnattached()); assertEquals(0, state.epoch()); + state.transitionToProspective(); state.transitionToCandidate(); CandidateState candidateState = state.candidateStateOrThrow(); - assertTrue(candidateState.isVoteGranted()); + assertTrue(candidateState.epochElection().isVoteGranted()); assertEquals(1, candidateState.epoch()); } @@ -200,7 +359,10 @@ public class QuorumStateTest { 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)); @@ -213,6 +375,69 @@ public class QuorumStateTest { 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( + ElectionState.withElectedLeader(epoch, 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( + ElectionState.withElectedLeader( + epoch, + leaderKey.id(), + Optional.of(persistedVotedKey(leaderKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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( + ElectionState.withElectedLeader(epoch, 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( + ElectionState.withElectedLeader( + epoch, + leader, + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testInitializeAsUnattachedWhenMissingEndpoints(KRaftVersion kraftVersion) { @@ -220,8 +445,12 @@ public class QuorumStateTest { 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( + ElectionState.withElectedLeader(epoch, leader, Optional.of(votedKey), voters.voterIds()), + kraftVersion + ); QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); @@ -230,11 +459,20 @@ public class QuorumStateTest { UnattachedState unattachedState = state.unattachedStateOrThrow(); assertEquals(epoch, unattachedState.epoch()); + assertEquals( + ElectionState.withElectedLeader( + epoch, + leader, + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); } @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()); @@ -292,9 +530,9 @@ public class QuorumStateTest { 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()) @@ -308,7 +546,7 @@ public class QuorumStateTest { 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 @@ -328,507 +566,99 @@ public class QuorumStateTest { assertEquals(epoch, resignedState.epoch()); assertEquals(election, resignedState.election()); assertEquals(Set.of(node1, node2), resignedState.unackedVoters()); - assertEquals(electionTimeoutMs + jitterMs, - resignedState.remainingElectionTimeMs(time.milliseconds())); - } - - @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.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); - - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - CandidateState candidate2 = state.candidateStateOrThrow(); - assertEquals(2, state.epoch()); - assertEquals(Collections.singleton(localId), candidate2.grantingVoters()); - assertEquals(Collections.emptySet(), candidate2.rejectingVoters()); - assertEquals(electionTimeoutMs + jitterMs, - candidate2.remainingElectionTimeMs(time.milliseconds())); - } - - @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()); - - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - assertEquals(1, state.epoch()); - - assertThrows( - IllegalStateException.class, () -> - state.transitionToResigned(Collections.emptyList()) - ); - assertTrue(state.isCandidate()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testCandidateToLeader(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); - - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - assertEquals(1, state.epoch()); - - state.transitionToLeader(0L, accumulator); - LeaderState leaderState = state.leaderStateOrThrow(); - assertTrue(state.isLeader()); - assertEquals(1, leaderState.epoch()); - assertEquals(Optional.empty(), leaderState.highWatermark()); - } - - @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.transitionToCandidate(); - assertFalse(state.candidateStateOrThrow().isVoteGranted()); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); - state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - assertTrue(state.candidateStateOrThrow().isVoteGranted()); - state.transitionToLeader(0L, accumulator); - assertTrue(state.isLeader()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testCandidateToFollower(KRaftVersion kraftVersion) { - int otherNodeId = 1; - - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToCandidate(); - - state.transitionToFollower(5, otherNodeId, voters.listeners(otherNodeId)); - assertEquals(5, state.epoch()); - assertEquals(OptionalInt.of(otherNodeId), state.leaderId()); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - otherNodeId, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() + electionTimeoutMs + jitterMs, + resignedState.remainingElectionTimeMs(time.milliseconds()) ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCandidateToUnattached(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToCandidate(); + public void testInitializeAsOnlyVoter(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + store.writeElectionState(ElectionState.withUnknownLeader(0, voters.voterIds()), kraftVersion); - state.transitionToUnattached(5); - assertEquals(5, state.epoch()); + 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()); - assertEquals( - Optional.of( - ElectionState.withUnknownLeader( - 5, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() - ); } @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)); - state.transitionToCandidate(); + public void testInitializeWithCorruptedStore(KRaftVersion kraftVersion) { + QuorumStateStore stateStore = Mockito.mock(QuorumStateStore.class); + Mockito.doThrow(UncheckedIOException.class).when(stateStore).readElectionState(); - state.transitionToUnattachedVotedState(5, 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) { - 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.transitionToCandidate(); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(4, otherNodeKey)); - assertThrows( - IllegalStateException.class, - () -> 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) - ) - ), - store.readElectionState() - ); - } - - @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.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 testLeaderToResigned(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); - - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - 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 testLeaderToCandidate(KRaftVersion kraftVersion) { - VoterSet voters = localStandaloneVoterSet(); - assertEquals(Optional.empty(), store.readElectionState()); - - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToCandidate(); - state.transitionToLeader(0L, accumulator); - assertTrue(state.isLeader()); - assertEquals(1, state.epoch()); - - assertThrows(IllegalStateException.class, state::transitionToCandidate); - assertTrue(state.isLeader()); - assertEquals(1, state.epoch()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testLeaderToFollower(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); - - QuorumState state = initializeEmptyState(voters, kraftVersion); - - 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( - Optional.of( - ElectionState.withElectedLeader( - 5, - otherNodeId, - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - store.readElectionState() - ); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testLeaderToUnattached(KRaftVersion kraftVersion) { - int otherNodeId = 1; - VoterSet voters = localWithRemoteVoterSet(IntStream.of(otherNodeId), kraftVersion); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - 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() - ); - } - - @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)); - state.transitionToCandidate(); - state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); - state.transitionToLeader(0L, accumulator); - state.transitionToUnattachedVotedState(5, 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) { - 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.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.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() - ); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testCannotFollowOrVoteForSelf(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.transitionToUnattachedVotedState(0, localVoterKey)); - } - - @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()), + QuorumState state = buildQuorumState( + OptionalInt.of(localId), + localStandaloneVoterSet(), kraftVersion ); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + + 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 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)); + + 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.transitionToLeader(0L, accumulator)); - assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); + + state.transitionToFollower(1, 1, voters.listeners(1)); + assertTrue(state.isFollower()); + + state.transitionToUnattached(2, OptionalInt.empty()); + assertTrue(state.isUnattached()); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedSameEpoch(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.transitionToUnattachedVotedState(5, otherNodeKey); - - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(5, votedState.epoch()); - assertEquals(otherNodeKey, votedState.votedKey().get()); - - assertEquals( - Optional.of( - ElectionState.withVotedCandidate( - 5, - persistedVotedKey(otherNodeKey, kraftVersion), - persistedVoters(voters.voterIds(), kraftVersion) - ) - ), - 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) ); - // 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 testUnattachedVotedHigherEpoch(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()); - - state.transitionToUnattachedVotedState(8, 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.writeElectionState( + ElectionState.withVotedCandidate( + epoch, + ReplicaKey.of(votedId, ReplicaKey.NO_DIRECTORY_ID), + voters.voterIds() ), - store.readElectionState() + kraftVersion ); + + QuorumState state2 = buildQuorumState(OptionalInt.empty(), voters, kraftVersion); + assertThrows(IllegalStateException.class, () -> state2.initialize(new OffsetAndEpoch(0, 0))); } - @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); - - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToCandidate(); - - assertTrue(state.isCandidate()); - CandidateState candidateState = state.candidateStateOrThrow(); - assertEquals(6, candidateState.epoch()); - assertEquals(electionTimeoutMs + jitterMs, - candidateState.remainingElectionTimeMs(time.milliseconds())); - } - + /** + * Test transitions from Unattached + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testUnattachedToUnattached(KRaftVersion kraftVersion) { @@ -836,20 +666,81 @@ public class QuorumStateTest { 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); - state.transitionToUnattached(6); + // cannot transition to unattached in same epoch + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), OptionalInt.empty())); + + // can transition to unattached in higher epoch + state.transitionToUnattached(6, OptionalInt.empty()); 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())); + assertEquals( + remainingElectionTimeMs - 1000, + unattachedState.remainingElectionTimeMs(time.milliseconds()) + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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)); + } + + @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, OptionalInt.empty()); + + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); + state.unattachedAddVotedState(5, otherNodeKey); + + UnattachedState votedState = state.unattachedStateOrThrow(); + assertEquals(5, votedState.epoch()); + assertEquals(otherNodeKey, votedState.votedKey().get()); + + assertEquals( + ElectionState.withVotedCandidate( + 5, + persistedVotedKey(otherNodeKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + + // 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 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, OptionalInt.empty()); + assertTrue(state.isUnattachedNotVoted()); + + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(10, otherNodeKey)); } @ParameterizedTest @@ -859,7 +750,7 @@ public class QuorumStateTest { 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, @@ -883,7 +774,7 @@ public class QuorumStateTest { 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, @@ -900,6 +791,57 @@ public class QuorumStateTest { assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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)); + assertTrue(state.isUnattached()); + assertEquals(logEndEpoch, state.epoch()); + + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); + state.transitionToProspective(); + ProspectiveState prospective = state.prospectiveStateOrThrow(); + assertEquals(electionTimeoutMs + jitterMs, + prospective.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + ElectionState.withUnknownLeader(logEndEpoch, persistedVoters(voters.voterIds(), kraftVersion)), + store.readElectionState().get() + ); + } + + @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, OptionalInt.empty()); + 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) { @@ -907,9 +849,9 @@ public class QuorumStateTest { 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)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(4, otherNodeKey)); + state.transitionToUnattached(5, OptionalInt.empty()); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -920,80 +862,72 @@ public class QuorumStateTest { ); 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() + ); + } + + /** + * Tests transitions from Unattached with votedKey + */ + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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.unattachedAddVotedState(logEndEpoch, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(logEndEpoch, OptionalInt.empty())); + } + + @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)); + + long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); + time.sleep(1000); + + state.transitionToUnattached(6, OptionalInt.empty()); + 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 testUnattachedVotedToInvalidLeaderOrResigned(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, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); - assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToCandidate(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.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); - - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - CandidateState candidateState = state.candidateStateOrThrow(); - assertEquals(6, candidateState.epoch()); - assertEquals(electionTimeoutMs + jitterMs, - candidateState.remainingElectionTimeMs(time.milliseconds())); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testObserverFromUnattachedVotedToCandidate(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.transitionToUnattachedVotedState(5, ReplicaKey.of(voter1, ReplicaKey.NO_DIRECTORY_ID)); - - assertThrows(IllegalStateException.class, () -> state.transitionToCandidate()); - assertTrue(state.isUnattached()); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToUnattachedVotedSameEpoch(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.transitionToUnattachedVotedState(8, ReplicaKey.of(node1, Uuid.randomUuid())); + state.initialize(new OffsetAndEpoch(0L, 8)); + state.unattachedAddVotedState(8, ReplicaKey.of(node1, Uuid.randomUuid())); + // same epoch assertThrows( IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) + () -> state.unattachedAddVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) ); + // same votedKey assertThrows( IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) + () -> 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)) ); } @@ -1001,11 +935,12 @@ public class QuorumStateTest { @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.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, node1Key); state.transitionToFollower( 5, node2, @@ -1019,14 +954,13 @@ public class QuorumStateTest { followerState.leaderEndpoints() ); assertEquals( - Optional.of( - ElectionState.withElectedLeader( - 5, - node2, - persistedVoters(voters.voterIds(), kraftVersion) - ) + ElectionState.withElectedLeader( + 5, + node2, + Optional.of(persistedVotedKey(node1Key, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() ); } @@ -1037,8 +971,8 @@ public class QuorumStateTest { int node2 = 2; 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.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); state.transitionToFollower( 8, node2, @@ -1052,48 +986,57 @@ public class QuorumStateTest { 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() ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testUnattachedVotedToUnattachedSameEpoch(KRaftVersion 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, 5)); + state.unattachedAddVotedState(5, ReplicaKey.of(node1, node1DirectoryId)); + + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); + state.transitionToProspective(); + assertTrue(state.isProspective()); + ProspectiveState prospectiveState = state.prospectiveStateOrThrow(); + assertEquals(5, prospectiveState.epoch()); + assertEquals(electionTimeoutMs + jitterMs, + prospectiveState.remainingElectionTimeMs(time.milliseconds())); + assertEquals( + ElectionState.withVotedCandidate( + 5, + persistedVotedKey(ReplicaKey.of(node1, node1DirectoryId), kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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.transitionToUnattachedVotedState(5, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(5)); - } - - @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, logEndEpoch)); - state.transitionToUnattachedVotedState(5, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID)); - - long remainingElectionTimeMs = state.unattachedStateOrThrow().remainingElectionTimeMs(time.milliseconds()); - time.sleep(1000); - - state.transitionToUnattached(6); - UnattachedState unattachedState = state.unattachedStateOrThrow(); - assertEquals(6, unattachedState.epoch()); - - // Verify that the election timer does not get reset - assertEquals(remainingElectionTimeMs - 1000, - unattachedState.remainingElectionTimeMs(time.milliseconds())); + 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 @@ -1102,10 +1045,10 @@ public class QuorumStateTest { 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.transitionToUnattachedVotedState(5, otherNodeKey); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(4, otherNodeKey)); + state.initialize(new OffsetAndEpoch(0L, 5)); + state.unattachedAddVotedState(5, otherNodeKey); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(4, otherNodeKey)); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -1116,52 +1059,160 @@ public class QuorumStateTest { ); 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() + ); + } + + /** + * Test transitions from Unattached with leader + */ + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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, 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( + ElectionState.withElectedLeader( + epoch, + node1, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testAllStatesToUnattachedFailInSameEpoch(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)); + 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; + 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, Optional.empty(), voters.voterIds()), + kraftVersion + ); + QuorumState state = buildQuorumState(OptionalInt.of(localId), voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isUnattached()); + assertTrue(state.hasLeader()); + assertTrue(state.leaderEndpoints().isEmpty()); + + state.transitionToProspective(); + assertTrue(state.isProspective()); + assertTrue(state.leaderEndpoints().isEmpty()); + assertEquals( + 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( + ElectionState.withElectedLeader( + 5, + leaderId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + /** + * Test transitions from Follower + */ + @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, OptionalInt.empty())); + assertThrows(IllegalStateException.class, () -> state.unattachedAddVotedState(8, votedKey)); + } - // unattached to unattached - state.unattachedStateOrThrow(); - state.transitionToUnattachedVotedState(5, 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); + @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); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); - // 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); + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); + state.transitionToUnattached(9, OptionalInt.empty()); + assertTrue(state.isUnattached()); + UnattachedState unattachedState = state.unattachedStateOrThrow(); + assertEquals(9, unattachedState.epoch()); + assertEquals( + electionTimeoutMs + jitterMs, + unattachedState.remainingElectionTimeMs(time.milliseconds()) + ); + } - // candidate - state.transitionToCandidate(); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattachedVotedState(state.epoch(), voter1)); - state.transitionToUnattachedVotedState(state.epoch() + 1, voter1); + @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) + ); - // 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.unattachedAddVotedState(9, votedKey)); } @ParameterizedTest @@ -1201,14 +1252,13 @@ public class QuorumStateTest { 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() ); } @@ -1270,17 +1320,66 @@ public class QuorumStateTest { 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() ); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerToProspective(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, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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 @EnumSource(value = KRaftVersion.class) public void testFollowerToLeaderOrResigned(KRaftVersion kraftVersion) { @@ -1298,127 +1397,6 @@ public class QuorumStateTest { assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } - @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) - ); - - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); - state.transitionToCandidate(); - assertTrue(state.isCandidate()); - CandidateState candidateState = state.candidateStateOrThrow(); - assertEquals(9, candidateState.epoch()); - assertEquals(electionTimeoutMs + jitterMs, - candidateState.remainingElectionTimeMs(time.milliseconds())); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testFollowerToUnattachedSameEpoch(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.transitionToUnattached(8)); - } - - @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); - 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())); - } - - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testFollowerToUnattachedVotedSameEpoch(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.transitionToUnattachedVotedState(8, ReplicaKey.of(node1, ReplicaKey.NO_DIRECTORY_ID)) - ); - assertThrows( - IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(8, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID)) - ); - assertThrows( - IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(8, ReplicaKey.of(node2, ReplicaKey.NO_DIRECTORY_ID)) - ); - } - - @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)); - QuorumState state = initializeEmptyState(voters, kraftVersion); - state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); - state.transitionToFollower( - 8, - nodeKey2.id(), - voters.listeners(nodeKey2.id()) - ); - - int jitterMs = 2500; - random.mockNextInt(electionTimeoutMs, jitterMs); - - state.transitionToUnattachedVotedState(9, nodeKey1); - assertTrue(state.isUnattachedAndVoted()); - - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(9, votedState.epoch()); - assertEquals(nodeKey1, votedState.votedKey().get()); - - assertEquals(electionTimeoutMs + jitterMs, - votedState.remainingElectionTimeMs(time.milliseconds())); - } - @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testFollowerToAnyStateLowerEpoch(KRaftVersion kraftVersion) { @@ -1431,11 +1409,7 @@ public class QuorumStateTest { otherNodeId, voters.listeners(otherNodeId) ); - assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); - assertThrows( - IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(4, ReplicaKey.of(otherNodeId, ReplicaKey.NO_DIRECTORY_ID)) - ); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); assertThrows( IllegalStateException.class, () -> state.transitionToFollower( @@ -1444,56 +1418,1199 @@ public class QuorumStateTest { 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) - ) + ElectionState.withElectedLeader( + 5, + otherNodeId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) ), - store.readElectionState() + store.readElectionState().get() + ); + } + + /** + * Test transitions from Follower with votedKey + */ + + @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( + state.epoch(), + node2, + voters.listeners(node2) + ); + assertEquals(0, state.epoch()); + + assertThrows( + IllegalStateException.class, + () -> state.transitionToUnattached(0, OptionalInt.empty()) ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testCanBecomeFollowerOfNonVoter(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.transitionToProspective(); + state.prospectiveStateOrThrow().recordGrantedVote(node1); + state.transitionToFollower( + state.epoch(), + node2, + voters.listeners(node2) + ); + assertEquals(0, state.epoch()); + + int jitterMs = 2500; + random.mockNextInt(electionTimeoutMs, jitterMs); + + state.transitionToUnattached(10, OptionalInt.empty()); + assertTrue(state.isUnattachedNotVoted()); + + UnattachedState unattached = state.unattachedStateOrThrow(); + assertEquals(10, unattached.epoch()); + + assertEquals( + electionTimeoutMs + jitterMs, + unattached.remainingElectionTimeMs(time.milliseconds()) + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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( + state.epoch(), + node2, + voters.listeners(node2) + ); + assertEquals( + ElectionState.withElectedLeader( + 0, + node2, + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + + assertThrows( + IllegalStateException.class, + () -> 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( + ElectionState.withElectedLeader( + 0, + node2, + Optional.of(persistedVotedKey(votedKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + + state.transitionToFollower(state.epoch() + 1, node1, voters.listeners(node1)); + assertEquals(1, state.epoch()); + assertEquals( + ElectionState.withElectedLeader( + 1, + node1, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + /** + * Test transitions from Prospective + */ + @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, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + + state.transitionToUnattached(state.epoch(), OptionalInt.empty()); + assertEquals( + ElectionState.withUnknownLeader(logEndEpoch, 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, OptionalInt.empty()); + assertEquals( + ElectionState.withUnknownLeader(20, 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 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(), + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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)); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToFollower(state.epoch(), voter1.id(), voters.listeners(voter1.id())); + assertEquals( + ElectionState.withElectedLeader( + 0, + voter1.id(), + Optional.empty(), + 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)); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToProspective(); + state.transitionToFollower(state.epoch() + 1, voter1.id(), voters.listeners(voter1.id())); + assertEquals( + ElectionState.withElectedLeader( + 1, + voter1.id(), + Optional.empty(), + 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( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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)); + } + + /** + * Transitions from Prospective with votedKey + */ + @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, OptionalInt.empty()); + assertEquals( + ElectionState.withVotedCandidate( + logEndEpoch, + persistedVotedKey(voter1, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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.prospectiveAddVotedState(logEndEpoch, voter1); + + // transition to follower of voter2 with votedKey voter1 + state.transitionToFollower(state.epoch(), voter2.id(), voters.listeners(voter2.id())); + assertEquals( + ElectionState.withElectedLeader( + logEndEpoch, + voter2.id(), + Optional.of(persistedVotedKey(voter1, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + + // transition back to prospective + state.transitionToProspective(); + assertEquals( + ElectionState.withElectedLeader( + logEndEpoch, + voter2.id(), + Optional.of(persistedVotedKey(voter1, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + /** + * Test transitions from Prospective with leader + */ + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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(), Optional.empty(), 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(), + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + + state.transitionToUnattached(state.epoch(), OptionalInt.of(voter1.id())); + assertEquals( + ElectionState.withElectedLeader( + logEndEpoch, + voter1.id(), + Optional.empty(), + 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(), Optional.empty(), 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(), + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + + // transition to unattached + state.transitionToUnattached(10, OptionalInt.empty()); + 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(), Optional.empty(), 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(), + Optional.empty(), + 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(), + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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(), Optional.empty(), 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(), + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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(), Optional.empty(), 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(), + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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, Optional.empty(), 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( + ElectionState.withVotedCandidate( + logEndEpoch + 1, + persistedVotedKey(localVoterKey, 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(), Optional.empty(), 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( + ElectionState.withElectedLeader( + logEndEpoch, + 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 testCandidateToUnattachedSameEpoch(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, logEndEpoch)); + state.transitionToProspective(); + state.transitionToCandidate(); - // Add voted state - state.transitionToUnattachedVotedState(4, nonVoterKey); - assertTrue(state.isUnattachedAndVoted()); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(state.epoch(), OptionalInt.empty())); + } - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(4, votedState.epoch()); - assertEquals(nonVoterKey, votedState.votedKey().get()); + @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(); - // Transition to follower - state.transitionToFollower( - 4, - nonVoterKey.id(), - Endpoints.fromInetSocketAddresses( - Collections.singletonMap( - VoterSetTest.DEFAULT_LISTENER_NAME, - InetSocketAddress.createUnresolved("non-voter-host", 1234) - ) - ) - ); + state.transitionToUnattached(5, OptionalInt.empty()); + assertEquals(5, state.epoch()); + assertEquals(OptionalInt.empty(), state.leaderId()); assertEquals( - new LeaderAndEpoch(OptionalInt.of(nonVoterKey.id()), 4), - state.leaderAndEpoch() + ElectionState.withUnknownLeader(5, persistedVoters(voters.voterIds(), kraftVersion)), + store.readElectionState().get() ); } @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testObserverCannotBecomeCandidateOrLeader(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( + ElectionState.withElectedLeader( + 1, + otherNodeId, + Optional.of(persistedVotedKey(localVoterKey, kraftVersion)), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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( + ElectionState.withElectedLeader( + 5, + otherNodeId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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()); + + 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()) + ); + + // 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( + ElectionState.withVotedCandidate( + 1, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testCandidateToLeader(KRaftVersion kraftVersion) { + VoterSet voters = localStandaloneVoterSet(); + assertEquals(Optional.empty(), store.readElectionState()); + + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.transitionToProspective(); + state.transitionToCandidate(); + assertTrue(state.isCandidate()); + assertEquals(1, state.epoch()); + + state.transitionToLeader(0L, accumulator); + LeaderState leaderState = state.leaderStateOrThrow(); + assertTrue(state.isLeader()); + assertEquals(1, leaderState.epoch()); + assertEquals(Optional.empty(), leaderState.highWatermark()); + } + + @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 testCandidateToResigned(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()); + assertEquals(1, state.epoch()); + + assertThrows( + IllegalStateException.class, () -> + state.transitionToResigned(Collections.emptyList()) + ); + assertTrue(state.isCandidate()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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, OptionalInt.empty()); + state.transitionToProspective(); + state.transitionToCandidate(); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower(4, otherNodeKey.id(), voters.listeners(otherNodeKey.id())) + ); + assertEquals(6, state.epoch()); + assertEquals( + ElectionState.withVotedCandidate( + 6, + persistedVotedKey(localVoterKey, kraftVersion), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + /** + * Test transitions from Leader + */ + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + 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(), OptionalInt.empty())); + } + + @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, OptionalInt.empty()); + assertEquals(5, state.epoch()); + assertEquals(OptionalInt.empty(), state.leaderId()); + assertEquals( + ElectionState.withUnknownLeader(5, persistedVoters(voters.voterIds(), kraftVersion)), + store.readElectionState().get() + ); + } + + @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( + ElectionState.withElectedLeader( + 5, + otherNodeId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + @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()); + + 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::transitionToCandidate); + assertTrue(state.isLeader()); + assertEquals(1, state.epoch()); + } + + @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 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, Optional.empty(), 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, OptionalInt.empty()); + state.transitionToProspective(); + state.transitionToCandidate(); + state.candidateStateOrThrow().recordGrantedVote(otherNodeKey.id()); + state.transitionToLeader(0L, accumulator); + assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4, OptionalInt.empty())); + assertThrows( + IllegalStateException.class, + () -> state.transitionToFollower( + 4, + otherNodeKey.id(), + voters.listeners(otherNodeKey.id()) + ) + ); + assertEquals(6, state.epoch()); + assertEquals( + ElectionState.withElectedLeader( + 6, + localId, + Optional.empty(), + persistedVoters(voters.voterIds(), kraftVersion) + ), + store.readElectionState().get() + ); + } + + /** + * 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, Optional.empty(), 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, Optional.empty(), 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 + */ + @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.unattachedAddVotedState(5, otherNodeKey); + + UnattachedState votedState = state.unattachedStateOrThrow(); + assertTrue(state.isUnattachedAndVoted()); + assertTrue(state.isObserver()); + assertEquals(5, votedState.epoch()); + assertEquals(otherNodeKey, votedState.votedKey().get()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testObserverUnattachedToFollower(KRaftVersion kraftVersion) { + boolean withDirectoryId = kraftVersion.featureLevel() > 0; + int node1 = 1; + int node2 = 2; + VoterSet voters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(node1, node2), withDirectoryId) + ); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + assertTrue(state.isObserver()); + + state.transitionToUnattached(2, OptionalInt.empty()); + state.transitionToFollower(3, node1, voters.listeners(node1)); + assertTrue(state.isFollower()); + FollowerState followerState = state.followerStateOrThrow(); + assertEquals(3, followerState.epoch()); + assertEquals( + voters.listeners(node1), + followerState.leaderEndpoints() + ); + 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.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, 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 testObserverUnattachedToCandidateOrLeaderOrResigned(KRaftVersion kraftVersion) { boolean withDirectoryId = kraftVersion.featureLevel() > 0; int otherNodeId = 1; VoterSet voters = VoterSetTest.voterSet( @@ -1504,26 +2621,12 @@ public class QuorumStateTest { assertTrue(state.isObserver()); assertThrows(IllegalStateException.class, state::transitionToCandidate); assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); + assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } - @ParameterizedTest - @EnumSource(value = KRaftVersion.class) - public void testObserverWithIdCanVote(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, logEndEpoch)); - assertTrue(state.isObserver()); - - state.transitionToUnattachedVotedState(5, otherNodeKey); - assertTrue(state.isUnattachedAndVoted()); - - UnattachedState votedState = state.unattachedStateOrThrow(); - assertEquals(5, votedState.epoch()); - assertEquals(otherNodeKey, votedState.votedKey().get()); - } - + /** + * Test transitions from Observer as Follower + */ @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testObserverFollowerToUnattached(KRaftVersion kraftVersion) { @@ -1542,7 +2645,7 @@ public class QuorumStateTest { node1, voters.listeners(node1) ); - state.transitionToUnattached(3); + state.transitionToUnattached(3, OptionalInt.empty()); assertTrue(state.isUnattached()); UnattachedState unattachedState = state.unattachedStateOrThrow(); assertEquals(3, unattachedState.epoch()); @@ -1553,161 +2656,20 @@ public class QuorumStateTest { @ParameterizedTest @EnumSource(value = KRaftVersion.class) - public void testObserverUnattachedToFollower(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 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)); + + state.transitionToFollower(logEndEpoch, voter1, voters.listeners(voter1)); assertTrue(state.isObserver()); + assertEquals(fetchTimeoutMs, state.followerStateOrThrow().remainingFetchTimeMs(time.milliseconds())); - state.transitionToUnattached(2); - state.transitionToFollower(3, node1, voters.listeners(node1)); - assertTrue(state.isFollower()); - FollowerState followerState = state.followerStateOrThrow(); - assertEquals(3, followerState.epoch()); - assertEquals( - voters.listeners(node1), - followerState.leaderEndpoints() - ); - assertEquals(fetchTimeoutMs, followerState.remainingFetchTimeMs(time.milliseconds())); - } - - @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 - ); - - 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.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.transitionToUnattachedVotedState(state.epoch() + 1, otherNodeKey); - assertFalse(state.hasRemoteLeader()); - - state.transitionToFollower( - state.epoch() + 1, - 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.transitionToUnattachedVotedState(7, otherNodeKey); - assertEquals(highWatermark, state.highWatermark()); - - state.transitionToCandidate(); - assertEquals(highWatermark, state.highWatermark()); - - CandidateState candidateState = state.candidateStateOrThrow(); - candidateState.recordGrantedVote(otherNodeKey.id()); - assertTrue(candidateState.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)); - - assertTrue(state.isObserver()); - assertFalse(state.isVoter()); - + assertThrows(IllegalStateException.class, state::transitionToProspective); assertThrows(IllegalStateException.class, state::transitionToCandidate); - assertThrows( - IllegalStateException.class, - () -> state.transitionToUnattachedVotedState(1, ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID)) - ); - 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 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) - ); - - 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))); + 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 86dc81e39c2..ee840ff59ed 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.IntStream; 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 final class RaftClientTestContext { 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(); @@ -199,7 +200,7 @@ public final class RaftClientTestContext { 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; @@ -292,8 +293,16 @@ public final class RaftClientTestContext { 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; } @@ -302,6 +311,14 @@ public final class RaftClientTestContext { return this; } + Builder withStartingVoters(VoterSet voters, KRaftVersion kraftVersion) { + if (kraftVersion.isReconfigSupported()) { + return withBootstrapSnapshot(Optional.of(voters)); + } else { + return withStaticVoters(voters.voterIds()); + } + } + Builder withStaticVoters(Set staticVoters) { Map staticVoterAddressMap = staticVoters .stream() @@ -330,7 +347,7 @@ public final class RaftClientTestContext { isStartingVotersStatic = false; if (voters.isPresent()) { - kraftVersion = KRaftVersion.KRAFT_VERSION_1; + kraftVersion = KRaftVersion.LATEST_PRODUCTION; RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() .setRawSnapshotWriter( @@ -450,7 +467,7 @@ public final class RaftClientTestContext { .limit(bootstrapServers.map(List::size).orElse(0)) .boxed() .collect(Collectors.toSet()), - kip853Rpc, + raftProtocol, alwaysFlush, metrics, listener @@ -478,7 +495,7 @@ public final class RaftClientTestContext { QuorumStateStore quorumStateStore, VoterSet startingVoters, Set bootstrapIds, - boolean kip853Rpc, + RaftProtocol raftProtocol, boolean alwaysFlush, Metrics metrics, MockListener listener @@ -495,7 +512,7 @@ public final class 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; @@ -545,23 +562,14 @@ public final class RaftClientTestContext { return builder.build(); } - static RaftClientTestContext initializeAsLeader(int localId, Set voters, int epoch) throws Exception { - if (epoch <= 0) { - throw new IllegalArgumentException("Cannot become leader in epoch " + epoch); - } - - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withUnknownLeader(epoch - 1) - .build(); - - context.assertUnknownLeader(epoch - 1); - context.becomeLeader(); - return context; + public void unattachedToCandidate() throws Exception { + time.sleep(electionTimeoutMs * 2L); + expectAndGrantPreVotes(currentEpoch()); } - public void becomeLeader() throws Exception { + public void unattachedToLeader() throws Exception { int currentEpoch = currentEpoch(); - time.sleep(electionTimeoutMs * 2L); + unattachedToCandidate(); expectAndGrantVotes(currentEpoch + 1); expectBeginEpoch(currentEpoch + 1); } @@ -594,12 +602,38 @@ public final class RaftClientTestContext { 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() { return localId.orElseThrow(() -> new AssertionError("Required local id is not defined")); } public ReplicaKey localReplicaKey() { - return kip853Rpc ? + return raftProtocol.isReconfigSupported() ? ReplicaKey.of(localIdOrThrow(), localDirectoryId) : ReplicaKey.of(localIdOrThrow(), ReplicaKey.NO_DIRECTORY_ID); } @@ -629,40 +663,67 @@ public final class RaftClientTestContext { } 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() + persistedVotedKey(candidateKey, kraftVersion), + 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, Optional.empty(), expectedVoters()), quorumStateStore.readElectionState().get() ); } - void assertUnknownLeader(int epoch) { + public void assertElectedLeaderAndVotedKey(int epoch, int leaderId, ReplicaKey candidateKey) { assertEquals( - ElectionState.withUnknownLeader(epoch, startingVoters.voterIds()), + ElectionState.withElectedLeader( + epoch, + leaderId, + Optional.of(persistedVotedKey(candidateKey, kraftVersion)), + expectedVoters() + ), quorumStateStore.readElectionState().get() ); } + private static ReplicaKey persistedVotedKey(ReplicaKey replicaKey, KRaftVersion kraftVersion) { + if (kraftVersion.isReconfigSupported()) { + return replicaKey; + } + + return ReplicaKey.of(replicaKey.id(), ReplicaKey.NO_DIRECTORY_ID); + } + + void assertUnknownLeaderAndNoVotedCandidate(int epoch) { + assertEquals( + 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, Optional.empty(), 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()); @@ -737,6 +798,12 @@ public final class RaftClientTestContext { assertEquals(expectedResponse, response); } + RaftRequest.Outbound assertSentPreVoteRequest(int epoch, int lastEpoch, long lastEpochOffset, int numVoteReceivers) { + List voteRequests = collectPreVoteRequests(epoch, lastEpoch, lastEpochOffset); + assertEquals(numVoteReceivers, voteRequests.size()); + return voteRequests.iterator().next(); + } + RaftRequest.Outbound assertSentVoteRequest(int epoch, int lastEpoch, long lastEpochOffset, int numVoteReceivers) { List voteRequests = collectVoteRequests(epoch, lastEpoch, lastEpochOffset); assertEquals(numVoteReceivers, voteRequests.size()); @@ -775,7 +842,7 @@ public final class RaftClientTestContext { 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(), @@ -786,6 +853,28 @@ public final class RaftClientTestContext { } } + 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, @@ -796,6 +885,7 @@ public final class RaftClientTestContext { if (raftMessage.data() instanceof VoteRequestData request) { VoteRequestData.PartitionData partitionRequest = unwrap(request); + assertFalse(partitionRequest.preVote()); assertEquals(epoch, partitionRequest.replicaEpoch()); assertEquals(localIdOrThrow(), partitionRequest.replicaId()); assertEquals(lastEpoch, partitionRequest.lastOffsetEpoch()); @@ -890,7 +980,7 @@ public final class RaftClientTestContext { .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( @@ -927,7 +1017,7 @@ public final class RaftClientTestContext { " 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(), @@ -964,7 +1054,7 @@ public final class RaftClientTestContext { .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( @@ -996,7 +1086,7 @@ public final class RaftClientTestContext { 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(), @@ -1040,7 +1130,7 @@ public final class RaftClientTestContext { 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( @@ -1130,7 +1220,7 @@ public final class RaftClientTestContext { 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( @@ -1367,7 +1457,7 @@ public final class RaftClientTestContext { } 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); @@ -1375,7 +1465,7 @@ public final class RaftClientTestContext { } 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); @@ -1467,7 +1557,7 @@ public final class RaftClientTestContext { long lastEpochOffset, boolean preVote ) { - ReplicaKey localReplicaKey = kip853Rpc ? + ReplicaKey localReplicaKey = raftProtocol.isReconfigSupported() ? ReplicaKey.of(localIdOrThrow(), localDirectoryId) : ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID); @@ -1504,12 +1594,20 @@ public final class RaftClientTestContext { } VoteResponseData voteResponse(boolean voteGranted, OptionalInt leaderId, int epoch) { + return voteResponse(Errors.NONE, voteGranted, leaderId, epoch, voteRpcVersion()); + } + + 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(), - voteRpcVersion(), + version, Errors.NONE, metadataPartition, - Errors.NONE, + error, epoch, leaderId.orElse(-1), voteGranted, @@ -1637,7 +1735,7 @@ public final class RaftClientTestContext { .setCurrentLeaderEpoch(epoch) .setLastFetchedEpoch(lastFetchedEpoch) .setFetchOffset(fetchOffset); - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { fetchPartition .setReplicaDirectoryId(replicaKey.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)); } @@ -1822,7 +1920,7 @@ public final class RaftClientTestContext { } private short fetchRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 17; } else { return 16; @@ -1830,7 +1928,7 @@ public final class RaftClientTestContext { } private short fetchSnapshotRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 1; } else { return 0; @@ -1838,15 +1936,17 @@ public final class RaftClientTestContext { } short voteRpcVersion() { - if (kip853Rpc) { - return ApiKeys.VOTE.latestVersion(); + if (raftProtocol.isPreVoteSupported()) { + return 2; + } else if (raftProtocol.isReconfigSupported()) { + return 1; } else { return 0; } } private short beginQuorumEpochRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 1; } else { return 0; @@ -1854,7 +1954,7 @@ public final class RaftClientTestContext { } private short endQuorumEpochRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 1; } else { return 0; @@ -1862,7 +1962,7 @@ public final class RaftClientTestContext { } private short describeQuorumRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 2; } else { return 1; @@ -1870,26 +1970,26 @@ public final class RaftClientTestContext { } private short addVoterRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 0; } else { - throw new IllegalStateException("Reconfiguration must be enabled by calling withKip853Rpc(true)"); + throw new IllegalStateException("Reconfiguration must be enabled by calling withRaftProtocol(KIP_853_PROTOCOL)"); } } private short removeVoterRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 0; } else { - throw new IllegalStateException("Reconfiguration must be enabled by calling withKip853Rpc(true)"); + throw new IllegalStateException("Reconfiguration must be enabled by calling withRaftProtocol(KIP_853_PROTOCOL)"); } } private short updateVoterRpcVersion() { - if (kip853Rpc) { + if (raftProtocol.isReconfigSupported()) { return 0; } else { - throw new IllegalStateException("Reconfiguration must be enabled by calling withKip853Rpc(true)"); + throw new IllegalStateException("Reconfiguration must be enabled by calling withRaftProtocol(KIP_853_PROTOCOL)"); } } @@ -2113,4 +2213,33 @@ public final class RaftClientTestContext { 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 isKRaftSupported() { + return isAtLeast(KIP_595_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; + } + } } 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 e99fde7a0da..48d08378c4f 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,111 @@ public class RaftEventSimulationTest { scheduler.runUntil(() -> cluster.allReachedHighWatermark(2 * restoredLogEndOffset)); } + @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY) + void leadershipAssignedOnlyOnceWithNetworkPartitionIfThereExistsMajority( + @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 StableLeadership(cluster)); + + // Create network partition which would result in ping-pong of leadership between nodes 2 and 3 without PreVote + // Scenario explained in detail in KIP-996 + // 0 1 + // | | + // 2 - 3 + // \ / + // 4 + router.filter( + 0, + new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(1, 3, 4))) + ); + router.filter( + 1, + new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(0, 2, 4))) + ); + router.filter(2, new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(1)))); + router.filter(3, new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(0)))); + router.filter(4, new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(0, 1)))); + + // Start cluster + 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); + + // Check that leadership remains stable after majority processes some data + int leaderId = cluster.latestLeader().getAsInt(); + // Determine the voters in the majority based on the leader + Set majority = new HashSet<>(Set.of(0, 1, 2, 3, 4)); + switch (leaderId) { + case 2 -> majority.remove(1); + case 3 -> majority.remove(0); + case 4 -> { + majority.remove(0); + majority.remove(1); + } + default -> throw new IllegalStateException("Unexpected leader: " + leaderId); + } + scheduler.runUntil(() -> cluster.allReachedHighWatermark(20, majority)); + } + + @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY) + void leadershipWillNotChangeDuringNetworkPartitionIfMajorityStillReachable( + @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 StableLeadership(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") + ); + + // Create network partition which would result in ping-pong of leadership between nodes C and D without PreVote + // Scenario explained in detail in KIP-996 + // 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 remains stable + scheduler.runUntil(() -> cluster.allReachedHighWatermark(20, Set.of(nodeA, leaderId, nodeD, nodeE))); + } + @Property(tries = 100, afterFailure = AfterFailureMode.SAMPLE_ONLY) void canMakeProgressAfterBackToBackLeaderFailures( @ForAll int seed, @@ -651,14 +756,18 @@ public class RaftEventSimulationTest { return false; RaftNode first = iter.next(); - ElectionState election = first.store.readElectionState().get(); - if (!election.hasLeader()) + OptionalInt firstLeaderId = first.store.readElectionState().get().optionalLeaderId(); + int firstEpoch = first.store.readElectionState().get().epoch(); + if (firstLeaderId.isEmpty()) return false; while (iter.hasNext()) { RaftNode next = iter.next(); - if (!election.equals(next.store.readElectionState().get())) + OptionalInt nextLeaderId = next.store.readElectionState().get().optionalLeaderId(); + int nextEpoch = next.store.readElectionState().get().epoch(); + if (!firstLeaderId.equals(nextLeaderId) || firstEpoch != nextEpoch) { return false; + } } return true; @@ -1056,6 +1165,45 @@ public class RaftEventSimulationTest { } } + /** + * This invariant currently checks that the leader does not change after the first successful election + * and should only be applied to tests where we expect leadership not to change (e.g. non-impactful + * routing filter changes, no network jitter) + */ + private static class StableLeadership implements Invariant { + final Cluster cluster; + OptionalInt epochWithFirstLeader = OptionalInt.empty(); + OptionalInt firstLeaderId = OptionalInt.empty(); + + private StableLeadership(Cluster cluster) { + this.cluster = cluster; + } + + @Override + public void verify() { + // KAFKA-18439: 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; 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 ba867a01e0d..f4fc143e2eb 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 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; @@ -67,7 +68,7 @@ class ResignedStateTest { 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 @@ class ResignedStateTest { 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 d4131e77c05..5e9c68d2d05 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,13 @@ */ 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 +39,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 +59,17 @@ public class UnattachedStateTest { ); } - @Test - public void testElectionTimeout() { - Set voters = Set.of(1, 2, 3); + @ParameterizedTest + @CsvSource({ "true,false", "false,true", "false,false" }) + 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); - UnattachedState state = newUnattachedState(voters, OptionalInt.empty()); - - assertEquals(epoch, state.epoch()); - - 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,59 +84,109 @@ public class UnattachedStateTest { @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(voter1Key, isLogUpToDate, false) + ); + 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(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) - ); assertEquals( isLogUpToDate, state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) ); + + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); assertEquals( isLogUpToDate, state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) ); - } - @Test - void testLeaderEndpoints() { - UnattachedState state = newUnattachedState(Set.of(1, 2, 3), OptionalInt.empty()); - - assertEquals(Endpoints.empty(), state.leaderEndpoints()); + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + ); + assertEquals( + isLogUpToDate, + state.canGrantVote(ReplicaKey.of(10, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false) + ); } @ParameterizedTest @ValueSource(booleans = {true, false}) - void testUnattachedWithLeader(boolean isLogUpToDate) { - int leaderId = 3; - Set voters = Set.of(1, 2, leaderId); + public void testCanGrantVoteWithVotedKey(boolean isLogUpToDate) { + UnattachedState state = newUnattachedState(OptionalInt.empty(), Optional.of(votedKey)); - UnattachedState state = newUnattachedState(voters, OptionalInt.of(leaderId)); + // Same voterKey + // 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) + ); + assertTrue(state.canGrantVote(votedKey, isLogUpToDate, false)); + + // Different directoryId + // 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, + 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 + @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, 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 assertEquals( isLogUpToDate, - state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true) + state.canGrantVote(voter1Key, isLogUpToDate, true) ); assertEquals( isLogUpToDate, @@ -137,12 +194,24 @@ public class UnattachedStateTest { ); 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, + 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(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 c7ec0292e40..00000000000 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java +++ /dev/null @@ -1,156 +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 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 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)), - Collections.emptySet(), - 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, Collections.emptySet()), - 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}) - 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), true, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(votedId + 1, ReplicaKey.NO_DIRECTORY_ID), true, false)); - } - - @Test - void testLeaderEndpoints() { - Uuid votedDirectoryId = Uuid.randomUuid(); - UnattachedState state = newUnattachedVotedState(votedDirectoryId); - - assertEquals(Endpoints.empty(), state.leaderEndpoints()); - } -} 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 0c6b8bfb57e..f8a0ffc8c35 100644 --- a/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java @@ -359,7 +359,7 @@ public final class VoterSetTest { } public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) { - return new VoterSet.VoterNode( + return voterNode( replicaKey, Endpoints.fromInetSocketAddresses( Collections.singletonMap( @@ -369,7 +369,14 @@ public final class VoterSetTest { 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 fc632672f98..f0c6710cd60 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 @@ public class KafkaRaftMetricsTest { 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,9 +132,22 @@ public class KafkaRaftMetricsTest { 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) -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() @@ -141,17 +155,59 @@ public class KafkaRaftMetricsTest { 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) 0, 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) 0, 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) 0, 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) 5L, getMetric(metrics, "high-watermark").metricValue()); + // leader state.candidateStateOrThrow().recordGrantedVote(1); state.transitionToLeader(2L, accumulator); assertEquals("leader", getMetric(metrics, "current-state").metricValue()); @@ -162,16 +218,18 @@ public class KafkaRaftMetricsTest { 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, "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()); @@ -181,25 +239,25 @@ public class KafkaRaftMetricsTest { 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) 10L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToUnattachedVotedState(3, ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID)); - assertEquals("voted", getMetric(metrics, "current-state").metricValue()); + // unattached with votedKey + 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()); 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) 10L, getMetric(metrics, "high-watermark").metricValue()); - state.transitionToUnattached(4); + // 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()); + assertEquals((double) 1, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) -1, getMetric(metrics, "current-vote").metricValue()); assertEquals( Uuid.ZERO_UUID.toString(), @@ -244,7 +302,7 @@ public class KafkaRaftMetricsTest { 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());