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());