ahuang98 commented on code in PR #16735: URL: https://github.com/apache/kafka/pull/16735#discussion_r1702357266
########## raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java: ########## @@ -2153,6 +2175,84 @@ private CompletableFuture<RemoveRaftVoterResponseData> handleRemoveVoterRequest( ); } + private CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest( + RaftRequest.Inbound requestMetadata, + long currentTimeMs + ) { + UpdateRaftVoterRequestData data = (UpdateRaftVoterRequestData) requestMetadata.data(); + + if (!hasValidClusterId(data.clusterId())) { + return completedFuture( + RaftUtil.updateVoterResponse( + Errors.INCONSISTENT_CLUSTER_ID, + requestMetadata.listenerName(), + quorum.leaderAndEpoch(), + quorum.leaderEndpoints() + ) + ); + } + + Optional<Errors> leaderValidation = validateLeaderOnlyRequest(data.currentLeaderEpoch()); + if (leaderValidation.isPresent()) { + return completedFuture( + RaftUtil.updateVoterResponse( + leaderValidation.get(), + requestMetadata.listenerName(), + quorum.leaderAndEpoch(), + quorum.leaderEndpoints() + ) + ); + } + + Optional<ReplicaKey> voter = RaftUtil.updateVoterRequestVoterKey(data); + if (!voter.isPresent() || !voter.get().directoryId().isPresent()) { + return completedFuture( + RaftUtil.updateVoterResponse( + Errors.INVALID_REQUEST, + requestMetadata.listenerName(), + quorum.leaderAndEpoch(), + quorum.leaderEndpoints() + ) + ); + } + + Endpoints voterEndpoints = Endpoints.fromUpdateVoterRequest(data.listeners()); Review Comment: do want to catch any illegalArgumentExceptions that might be thrown from this method? ########## raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java: ########## @@ -95,6 +95,21 @@ public Optional<Node> voterNode(int voterId, ListenerName listenerName) { .map(address -> new Node(voterId, address.getHostString(), address.getPort())); } + /** + * Return true the provided voter node is a voter and would cause a change in the voter set. Review Comment: nit: `Return true if the provided voter node ...` ########## raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java: ########## @@ -2153,6 +2175,84 @@ private CompletableFuture<RemoveRaftVoterResponseData> handleRemoveVoterRequest( ); } + private CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest( + RaftRequest.Inbound requestMetadata, + long currentTimeMs + ) { + UpdateRaftVoterRequestData data = (UpdateRaftVoterRequestData) requestMetadata.data(); + + if (!hasValidClusterId(data.clusterId())) { + return completedFuture( + RaftUtil.updateVoterResponse( + Errors.INCONSISTENT_CLUSTER_ID, + requestMetadata.listenerName(), + quorum.leaderAndEpoch(), + quorum.leaderEndpoints() + ) + ); + } + + Optional<Errors> leaderValidation = validateLeaderOnlyRequest(data.currentLeaderEpoch()); Review Comment: nit: I know this is the convention handleFetchSnapshotRequest uses too, but could we rename to `leaderValidationError`? ########## raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java: ########## @@ -2153,6 +2175,84 @@ private CompletableFuture<RemoveRaftVoterResponseData> handleRemoveVoterRequest( ); } + private CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest( + RaftRequest.Inbound requestMetadata, + long currentTimeMs + ) { + UpdateRaftVoterRequestData data = (UpdateRaftVoterRequestData) requestMetadata.data(); + + if (!hasValidClusterId(data.clusterId())) { + return completedFuture( + RaftUtil.updateVoterResponse( + Errors.INCONSISTENT_CLUSTER_ID, + requestMetadata.listenerName(), + quorum.leaderAndEpoch(), + quorum.leaderEndpoints() + ) + ); + } + + Optional<Errors> leaderValidation = validateLeaderOnlyRequest(data.currentLeaderEpoch()); + if (leaderValidation.isPresent()) { + return completedFuture( + RaftUtil.updateVoterResponse( + leaderValidation.get(), + requestMetadata.listenerName(), + quorum.leaderAndEpoch(), + quorum.leaderEndpoints() + ) + ); + } + + Optional<ReplicaKey> voter = RaftUtil.updateVoterRequestVoterKey(data); + if (!voter.isPresent() || !voter.get().directoryId().isPresent()) { + return completedFuture( + RaftUtil.updateVoterResponse( + Errors.INVALID_REQUEST, + requestMetadata.listenerName(), + quorum.leaderAndEpoch(), + quorum.leaderEndpoints() + ) + ); + } + + Endpoints voterEndpoints = Endpoints.fromUpdateVoterRequest(data.listeners()); + if (!voterEndpoints.address(channel.listenerName()).isPresent()) { + return completedFuture( + RaftUtil.updateVoterResponse( + Errors.INVALID_REQUEST, Review Comment: wonder if we might benefit from the update voter response schema including an error message field. ########## raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java: ########## @@ -543,6 +555,15 @@ public void initialize( quorumConfig.requestTimeoutMs(), logContext ); + + // Specialized remove voter handler Review Comment: nit: specialized update voter handler ########## raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java: ########## @@ -543,6 +555,15 @@ public void initialize( quorumConfig.requestTimeoutMs(), logContext ); + + // Specialized remove voter handler + this.updateVoterHandler = new UpdateVoterHandler( + nodeId, + partitionState, + channel.listenerName(), + time, + quorumConfig.requestTimeoutMs() Review Comment: does this need to be passed in vs imported in UpdateVoterHandler? ########## raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java: ########## @@ -0,0 +1,233 @@ +/* + * 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.common.feature.SupportedVersionRange; +import org.apache.kafka.common.message.UpdateRaftVoterRequestData; +import org.apache.kafka.common.message.UpdateRaftVoterResponseData; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.raft.Endpoints; +import org.apache.kafka.raft.LeaderAndEpoch; +import org.apache.kafka.raft.LeaderState; +import org.apache.kafka.raft.LogOffsetMetadata; +import org.apache.kafka.raft.RaftUtil; +import org.apache.kafka.server.common.KRaftVersion; + +import java.util.Optional; +import java.util.OptionalInt; +import java.util.concurrent.CompletableFuture; + +/** + * This type implements the protocol for updating a voter from a KRaft partition. + * + * 1. Check that the leader has fenced the previous leader(s) by checking that the HWM is known, + * otherwise return the REQUEST_TIMED_OUT error. + * 2. Check that the cluster supports kraft.version 1, otherwise return the UNSUPPORTED_VERSION error. + * 3. Check that there are no uncommitted voter changes, otherwise return the REQUEST_TIMED_OUT error. + * 4. Check that the updated voter still supports the currently finalized kraft.version, otherwise + * return the INVALID_REQUEST error. + * 5. Check that the updated voter is still listening on the default listener. + * 6. Append the updated VotersRecord to the log. The KRaft internal listener will read this + * uncommitted record from the log and update the voter in the set of voters. + * 7. Wait for the VotersRecord to commit using the majority of the voters. Return a + * REQUEST_TIMED_OUT error if it doesn't commit in time. + * 8. Send the UpdateVoter successful response to the voter. + * + * KAFKA-16538 is going to add support for handling this RPC when the kraft.version is 0. + */ +public final class UpdateVoterHandler { + private final OptionalInt localId; + private final KRaftControlRecordStateMachine partitionState; + private final ListenerName defaultListenerName; + private final Time time; + private final long requestTimeoutMs; + + public UpdateVoterHandler( + OptionalInt localId, + KRaftControlRecordStateMachine partitionState, + ListenerName defaultListenerName, + Time time, + long requestTimeoutMs + ) { + this.localId = localId; + this.partitionState = partitionState; + this.defaultListenerName = defaultListenerName; + this.time = time; + this.requestTimeoutMs = requestTimeoutMs; + } + + public CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest( + LeaderState<?> leaderState, + ListenerName requestListenerName, + ReplicaKey voterKey, + Endpoints voterEndpoints, + UpdateRaftVoterRequestData.KRaftVersionFeature supportedKraftVersions, + long currentTimeMs + ) { + // Check if there are any pending voter change requests + if (leaderState.isOperationPending(currentTimeMs)) { + return CompletableFuture.completedFuture( + RaftUtil.updateVoterResponse( + Errors.REQUEST_TIMED_OUT, Review Comment: another place where it would be nice if there was an error msg (or at least logging) -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org