mirror of https://github.com/apache/kafka.git
KAFKA-17642: PreVote response handling and ProspectiveState (#18240)
This PR implements the second part of KIP-996 and KAFKA-16164 (tasks KAFKA-16607, KAFKA-17642, KAFKA-17643, KAFKA-17675) which encompass the response handling of PreVotes, addition of new ProspectiveState, update to metrics, and addition of Raft simulation tests. Voters now transition to ProspectiveState first before CandidateState to prevent unnecessary epoch bumps. Voters in ProspectiveState send PreVotes requests which are Vote requests with PreVote set to true. Follower grants PreVotes if it has not yet fetched successfully from leader. Leader denies all PreVotes. Unattached, Prospective, Candidate, and Resigned will grant PreVotes if the requesting replica's log is at least as long as theirs. Granted PreVotes are not persisted like standard votes. It is possible for a voter to grant several PreVotes in the same epoch. The only state which is allowed to transition directly to CandidateState is ProspectiveState. This happens on reception of majority of granted PreVotes or if at least one voter doesn't support PreVote requests. Prospective will transition to Follower after election loss/timeout if it was already aware of last known leader and the leader's endpoint, or at any point if it discovers the leader. Prospective will transition to Unattached after election loss/timeout if it does not know the leader endpoints. After electionTimeout, Resigned now always transitions to Unattached and increases the epoch. Prospective grants standard votes if it has not already granted a standard vote (no votedKey), has no leaderId, and the recipient's log is current enough Candidate no longer backs off after election timeout. Candidate still backs off after election loss. Reviewers: José Armando García Sancio <jsancio@apache.org>
This commit is contained in:
parent
3191fe56fc
commit
4583b033f0
|
@ -58,7 +58,7 @@
|
|||
<suppress id="dontUseSystemExit"
|
||||
files="Exit.java"/>
|
||||
<suppress checks="ClassFanOutComplexity"
|
||||
files="(AbstractFetch|Sender|SenderTest|ConsumerCoordinator|KafkaConsumer|KafkaProducer|Utils|TransactionManager|TransactionManagerTest|KafkaAdminClient|NetworkClient|Admin|KafkaRaftClient|KafkaRaftClientTest|RaftClientTestContext|TestingMetricsInterceptingAdminClient).java"/>
|
||||
files="(AbstractFetch|Sender|SenderTest|ConsumerCoordinator|KafkaConsumer|KafkaProducer|Utils|TransactionManager|TransactionManagerTest|KafkaAdminClient|NetworkClient|Admin|RaftClientTestContext|TestingMetricsInterceptingAdminClient).java"/>
|
||||
<suppress checks="ClassFanOutComplexity"
|
||||
files="(SaslServerAuthenticator|SaslAuthenticatorTest).java"/>
|
||||
<suppress checks="NPath"
|
||||
|
@ -76,7 +76,7 @@
|
|||
files="(KerberosLogin|RequestResponseTest|ConnectMetricsRegistry|KafkaConsumer|AbstractStickyAssignor|AbstractRequest|AbstractResponse).java"/>
|
||||
|
||||
<suppress checks="ParameterNumber"
|
||||
files="(NetworkClient|FieldSpec|KafkaRaftClient|KafkaProducer).java"/>
|
||||
files="(NetworkClient|FieldSpec|KafkaProducer).java"/>
|
||||
<suppress checks="ParameterNumber"
|
||||
files="(KafkaConsumer|ConsumerCoordinator).java"/>
|
||||
<suppress checks="ParameterNumber"
|
||||
|
@ -91,7 +91,7 @@
|
|||
files="ClientUtils.java"/>
|
||||
|
||||
<suppress checks="ClassDataAbstractionCoupling"
|
||||
files="(KafkaConsumer|ConsumerCoordinator|AbstractFetch|KafkaProducer|AbstractRequest|AbstractResponse|TransactionManager|Admin|KafkaAdminClient|MockAdminClient|KafkaRaftClient|KafkaRaftClientTest|KafkaNetworkChannelTest).java"/>
|
||||
files="(KafkaConsumer|ConsumerCoordinator|AbstractFetch|KafkaProducer|AbstractRequest|AbstractResponse|TransactionManager|Admin|KafkaAdminClient|MockAdminClient|KafkaNetworkChannelTest).java"/>
|
||||
<suppress checks="ClassDataAbstractionCoupling"
|
||||
files="(Errors|SaslAuthenticatorTest|AgentTest|CoordinatorTest|NetworkClientTest).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"/>
|
||||
|
||||
<suppress checks="JavaNCSS"
|
||||
files="(AbstractRequest|AbstractResponse|KerberosLogin|WorkerSinkTaskTest|TransactionManagerTest|SenderTest|KafkaAdminClient|ConsumerCoordinatorTest|KafkaAdminClientTest|KafkaRaftClientTest).java"/>
|
||||
files="(AbstractRequest|AbstractResponse|KerberosLogin|WorkerSinkTaskTest|TransactionManagerTest|SenderTest|KafkaAdminClient|ConsumerCoordinatorTest|KafkaAdminClientTest).java"/>
|
||||
|
||||
<suppress checks="NPathComplexity"
|
||||
files="(ConsumerCoordinator|BufferPool|MetricName|Node|ConfigDef|RecordBatch|SslFactory|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|TokenInformation|Agent|PluginUtils|MiniTrogdorCluster|TasksRequest|KafkaProducer|AbstractStickyAssignor|KafkaRaftClient|Authorizer|FetchSessionHandler|RecordAccumulator|Shell).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"/>
|
||||
|
||||
<suppress checks="(JavaNCSS|CyclomaticComplexity|MethodLength)"
|
||||
files="CoordinatorClient.java"/>
|
||||
|
@ -187,6 +187,9 @@
|
|||
<suppress checks="NPathComplexity"
|
||||
files="(DynamicVoter|RecordsIterator).java"/>
|
||||
|
||||
<suppress checks="JavaNCSS"
|
||||
files="(KafkaRaftClientTest).java"/>
|
||||
|
||||
<!-- Streams -->
|
||||
<suppress checks="ClassFanOutComplexity"
|
||||
files="(KafkaStreams|KStreamImpl|KTableImpl|InternalTopologyBuilder|StreamsPartitionAssignor|StreamThread|IQv2StoreIntegrationTest|KStreamImplTest|RocksDBStore|StreamTask|TaskManager).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))
|
||||
)));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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<Integer, VoterState> voteStates = new HashMap<>();
|
||||
private final EpochElection epochElection;
|
||||
private final Optional<LogOffsetMetadata> 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<ReplicaKey> 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<Integer> 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<Integer> rejectingVoters() {
|
||||
return votersInState(State.REJECTED).map(ReplicaKey::id).collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
private Stream<ReplicaKey> 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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<Integer> voters) {
|
||||
public static ElectionState withElectedLeader(
|
||||
int epoch,
|
||||
int leaderId,
|
||||
Optional<ReplicaKey> votedKey,
|
||||
Set<Integer> 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<Integer> voters) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -34,6 +34,7 @@ public class FollowerState implements EpochState {
|
|||
private final int epoch;
|
||||
private final int leaderId;
|
||||
private final Endpoints leaderEndpoints;
|
||||
private final Optional<ReplicaKey> votedKey;
|
||||
private final Set<Integer> 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<ReplicaKey> votedKey,
|
||||
Set<Integer> voters,
|
||||
Optional<LogOffsetMetadata> 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
|
||||
|
|
|
@ -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<T> implements RaftClient<T> {
|
||||
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<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
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<T> implements RaftClient<T> {
|
|||
}
|
||||
}
|
||||
|
||||
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<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
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<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<T> implements RaftClient<T> {
|
|||
* - 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<Boolean> maybeHandleCommonResponse(
|
||||
Errors error,
|
||||
|
@ -2445,7 +2524,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
);
|
||||
}
|
||||
|
||||
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<T> implements RaftClient<T> {
|
|||
remoteVoter,
|
||||
endOffset.epoch(),
|
||||
endOffset.offset(),
|
||||
false
|
||||
preVote
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -2929,18 +3008,16 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
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<T> implements RaftClient<T> {
|
|||
)
|
||||
)
|
||||
),
|
||||
this::buildVoteRequest
|
||||
voterId -> buildVoteRequest(voterId, preVote)
|
||||
);
|
||||
}
|
||||
return Long.MAX_VALUE;
|
||||
|
@ -3015,7 +3093,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
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<T> implements RaftClient<T> {
|
|||
// 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<T> implements RaftClient<T> {
|
|||
// 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<T> implements RaftClient<T> {
|
|||
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()) {
|
||||
|
|
|
@ -409,7 +409,7 @@ public class LeaderState<T> 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
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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<ReplicaKey> votedKey;
|
||||
private final VoterSet voters;
|
||||
private final EpochElection epochElection;
|
||||
private final Optional<LogOffsetMetadata> 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<ReplicaKey> votedKey,
|
||||
VoterSet voters,
|
||||
Optional<LogOffsetMetadata> 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<ReplicaKey> 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<LogOffsetMetadata> 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() {}
|
||||
}
|
|
@ -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<ReplicaKey> 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<ReplicaKey> 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<ReplicaKey> 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 <T> LeaderState<T> transitionToLeader(long epochStartOffset, BatchAccumulator<T> 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 <T> LeaderState<T> leaderStateOrThrow() {
|
||||
|
@ -683,12 +801,42 @@ public class QuorumState {
|
|||
throw new IllegalStateException("Expected to be Resigned, but current state is " + state);
|
||||
}
|
||||
|
||||
public Optional<ProspectiveState> 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<ReplicaKey> 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<Integer, VoterState> voterStates;
|
||||
|
||||
public EpochElection(Set<ReplicaKey> 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<Integer> 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<ReplicaKey> 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<Integer> 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<Integer> 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<ReplicaKey> 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
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -71,10 +71,14 @@ public class KafkaRaftMetrics implements AutoCloseable {
|
|||
Gauge<String> 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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<ElectionState> 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<ElectionState> 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())
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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<ReplicaKey> 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);
|
||||
}
|
||||
}
|
|
@ -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<Integer> voters = Set.of(voter1, voter2, voter3);
|
||||
|
||||
stateStore.writeElectionState(
|
||||
ElectionState.withElectedLeader(epoch, voter1, voters),
|
||||
ElectionState.withElectedLeader(epoch, voter1, Optional.of(votedKey), voters),
|
||||
kraftVersion
|
||||
);
|
||||
|
||||
final Optional<ElectionState> 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());
|
||||
|
|
|
@ -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<Integer> voters,
|
||||
Optional<LogOffsetMetadata> highWatermark
|
||||
Set<Integer> 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());
|
||||
|
|
|
@ -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 ->
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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);
|
||||
|
|
|
@ -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<String> 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<String> 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:
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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<ReplicaKey> 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<ReplicaKey> remoteVoterKeys = remoteVoterIds
|
||||
.boxed()
|
||||
.map(id -> replicaKey(id, withDirectoryId));
|
||||
|
||||
return voterSetWithLocal(remoteVoterKeys, withDirectoryId);
|
||||
}
|
||||
|
||||
private VoterSet voterSetWithLocal(Stream<ReplicaKey> remoteVoterKeys, boolean withDirectoryId) {
|
||||
ReplicaKey actualLocalVoter = withDirectoryId ?
|
||||
localReplicaKey :
|
||||
ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID);
|
||||
|
||||
return VoterSetTest.voterSet(
|
||||
Stream.concat(Stream.of(actualLocalVoter), remoteVoterKeys)
|
||||
);
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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<Integer> 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<List<InetSocketAddress>> 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<Integer> staticVoters) {
|
||||
Map<Integer, InetSocketAddress> 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<Integer> 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<Integer> 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<RaftRequest.Outbound> 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<Integer> 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<Integer> expectedVoters() {
|
||||
return kraftVersion.isReconfigSupported() ? Collections.emptySet() : startingVoters.voterIds();
|
||||
}
|
||||
|
||||
DescribeQuorumResponseData collectDescribeQuorumResponse() {
|
||||
List<RaftResponse.Outbound> 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<RaftRequest.Outbound> 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<RaftRequest.Outbound> 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<RaftRequest.Outbound> collectPreVoteRequests(
|
||||
int epoch,
|
||||
int lastEpoch,
|
||||
long lastEpochOffset
|
||||
) {
|
||||
List<RaftRequest.Outbound> 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<RaftRequest.Outbound> 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<FetchSnapshotResponseData.PartitionSnapshot> 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<Integer> 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<Integer, PersistentState> nodeEntry : cluster.nodes.entrySet()) {
|
||||
PersistentState state = nodeEntry.getValue();
|
||||
Optional<ElectionState> 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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<Integer> voters = Set.of(1, 2, 3);
|
||||
private final ReplicaKey voter1Key = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
private final ReplicaKey votedKey = voter1Key;
|
||||
|
||||
private UnattachedState newUnattachedState(
|
||||
Set<Integer> voters,
|
||||
OptionalInt leaderId
|
||||
OptionalInt leaderId,
|
||||
Optional<ReplicaKey> 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<Integer> 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<ReplicaKey> 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<Integer> 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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
|
|
|
@ -116,6 +116,7 @@ public class KafkaRaftMetricsTest {
|
|||
public void shouldRecordVoterQuorumState(KRaftVersion kraftVersion) {
|
||||
boolean withDirectoryId = kraftVersion.featureLevel() > 0;
|
||||
Map<Integer, VoterSet.VoterNode> 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());
|
||||
|
|
Loading…
Reference in New Issue