jsancio commented on code in PR #18240:
URL: https://github.com/apache/kafka/pull/18240#discussion_r1890554341
##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -926,29 +973,16 @@ private boolean handleVoteResponse(
if (quorum.isLeader()) {
logger.debug("Ignoring vote response {} since we already
became leader for epoch {}",
partitionResponse, quorum.epoch());
- } else if (quorum.isCandidate()) {
- CandidateState state = quorum.candidateStateOrThrow();
- if (partitionResponse.voteGranted()) {
- state.recordGrantedVote(remoteNodeId);
- maybeTransitionToLeader(state, currentTimeMs);
- } else {
- state.recordRejectedVote(remoteNodeId);
-
- // If our vote is rejected, we go immediately to the
random backoff. This
- // ensures that we are not stuck waiting for the election
timeout when the
- // vote has become gridlocked.
- if (state.isVoteRejected() && !state.isBackingOff()) {
- logger.info("Insufficient remaining votes to become
leader (rejected by {}). " +
- "We will backoff before retrying election again",
state.rejectingVoters());
-
- state.startBackingOff(
- currentTimeMs,
- binaryExponentialElectionBackoffMs(state.retries())
- );
- }
- }
+ } else if (quorum.isVotingState()) {
+ NomineeState state = quorum.votingStateOrThrow();
Review Comment:
Let's use the same word you used in the `NomineeState`: `isNomineeState()`
and `nomineeStateOrThrow()`.
##########
raft/src/main/java/org/apache/kafka/raft/EpochElection.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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;
Review Comment:
Can we try moving this to the internal module? Anything public in this
package can be used outside the `raft` module.
##########
raft/src/main/java/org/apache/kafka/raft/EpochElection.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class EpochElection {
+ private Map<Integer, VoterState> voterStates;
+
+ EpochElection(Set<ReplicaKey> voters) {
+ this.voterStates = voters.stream()
+ .collect(Collectors.toMap(
+ ReplicaKey::id,
+ VoterState::new
+ ));
+ }
+
+ VoterState getVoterStateOrThrow(int voterId) {
+ VoterState voterState = voterStates.get(voterId);
+ if (voterState == null) {
+ throw new IllegalArgumentException("Attempt to access voter state
of non-voter " + voterId);
+ }
+ return voterState;
+ }
+
+ boolean recordVote(int voterId, boolean isGranted) {
Review Comment:
Please write Java doc for all public methods.
##########
raft/src/main/java/org/apache/kafka/raft/NomineeState.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+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 rejected vote had not been previously recorded
+ * @throws IllegalArgumentException
+ */
+ boolean recordRejectedVote(int remoteNodeId);
+
+ boolean hasElectionTimeoutExpired(long currentTimeMs);
+
+ long remainingElectionTimeMs(long currentTimeMs);
Review Comment:
Let's write Java doc for all public methods.
##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -883,6 +923,13 @@ private boolean handleVoteResponse(
VoteResponseData response = (VoteResponseData) responseMetadata.data();
Errors topLevelError = Errors.forCode(response.errorCode());
if (topLevelError != Errors.NONE) {
+ if (topLevelError == Errors.UNSUPPORTED_VERSION &&
quorum.isProspective()) {
+ logger.warn("Prospective received unsupported version error in
vote response in epoch {}, " +
+ "transitioning to Candidate state immediately since
entire quorum does not support PreVote.",
+ quorum.epoch());
Review Comment:
This is not a warning. This is a valid state or condition. We should be able
to log this message at INFO level. It should be rare because there are
backoff/timeout logic in Candidate, Follower and Unattached which limit how
quickly a replica transitions to Prospective. What do you think?
##########
raft/src/main/java/org/apache/kafka/raft/EpochElection.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class EpochElection {
Review Comment:
Can we explicitly mark a private any method that is not used outside of this
class?
##########
raft/src/main/java/org/apache/kafka/raft/ProspectiveState.java:
##########
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.raft;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.OptionalInt;
+
+public class ProspectiveState implements NomineeState {
+ private final int localId;
+ private final int epoch;
+ private final OptionalInt leaderId;
+ private final Optional<Endpoints> leaderEndpoints;
+ private final Optional<ReplicaKey> votedKey;
+ private final VoterSet voters;
+ private final EpochElection epochElection;
+ private final Optional<LogOffsetMetadata> highWatermark;
+ private final Timer electionTimer;
+ private final Logger log;
+
+ /**
+ * The lifetime of a prospective state is the following.
+ *
+ * 1. Once started, it will keep record of the received votes and continue
to fetch from bootstrap voters.
+ * 2. If it receives a fetch response denoting a leader with a higher
epoch, it will transition to follower state.
+ * 3. If majority votes granted, it will transition to leader state.
+ * 4. If majority votes rejected or election times out, it will enter a
backing off phase;
+ * after the backoff phase completes, it will send out another round
of PreVote requests.
+ */
+ public ProspectiveState(
+ Time time,
+ int localId,
+ int epoch,
+ OptionalInt leaderId,
+ Optional<Endpoints> leaderEndpoints,
+ Optional<ReplicaKey> votedKey,
+ VoterSet voters,
+ Optional<LogOffsetMetadata> highWatermark,
+ 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.electionTimer = time.timer(electionTimeoutMs);
+ this.log = logContext.logger(ProspectiveState.class);
+
+ this.epochElection = new EpochElection(voters.voterKeys());
+ epochElection.recordVote(localId, true);
//voterStates().get(localId).setState(State.GRANTED);
+ }
+
+ public int localId() {
+ return localId;
+ }
+
+ public Optional<ReplicaKey> votedKey() {
+ return votedKey;
+ }
+
+ @Override
+ public EpochElection epochElection() {
+ return epochElection;
+ }
+
+ @Override
+ public boolean recordGrantedVote(int remoteNodeId) {
+ return epochElection().recordVote(remoteNodeId, true);
+ }
+
+ @Override
+ public boolean recordRejectedVote(int remoteNodeId) {
+ if (remoteNodeId == localId) {
+ throw new IllegalStateException("Attempted to reject vote from
ourselves");
Review Comment:
`IllegalArgumentException` seems like a better exception type.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]