ahuang98 commented on code in PR #19416: URL: https://github.com/apache/kafka/pull/19416#discussion_r2052945464
########## raft/src/main/java/org/apache/kafka/raft/internals/KRaftVersionUpgrade.java: ########## @@ -0,0 +1,65 @@ +/* + * 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.VoterSet; +import org.apache.kafka.server.common.KRaftVersion; + +import java.util.Optional; + +/** + * KRaftVersionUpgrade is a sum types for coordinating the upgrade of the kraft version. + * + * {@code Voters} is used to stored in-memory the latest voter set. {@code Version} is used to + * stored in-memory the upgraded kraft version. Review Comment: nit: `store` ########## raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java: ########## @@ -104,26 +111,39 @@ public CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest( ); } - // KAFKA-16538 will implement the case when the kraft.version is 0 - // Check that the cluster supports kraft.version >= 1 + // Read the voter set from the log or leader state KRaftVersion kraftVersion = partitionState.lastKraftVersion(); - if (!kraftVersion.isReconfigSupported()) { - return CompletableFuture.completedFuture( - RaftUtil.updateVoterResponse( - Errors.UNSUPPORTED_VERSION, - requestListenerName, - new LeaderAndEpoch( - localId, - leaderState.epoch() - ), - leaderState.leaderEndpoints() - ) - ); - } + final Optional<KRaftVersionUpgrade.Voters> inMemoryVoters; + final Optional<VoterSet> voters; Review Comment: nit, why not instantiate these both as empty optionals to remove need for lines L119 & L124 ########## raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java: ########## @@ -104,26 +111,39 @@ public CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest( ); } - // KAFKA-16538 will implement the case when the kraft.version is 0 - // Check that the cluster supports kraft.version >= 1 + // Read the voter set from the log or leader state KRaftVersion kraftVersion = partitionState.lastKraftVersion(); - if (!kraftVersion.isReconfigSupported()) { - return CompletableFuture.completedFuture( - RaftUtil.updateVoterResponse( - Errors.UNSUPPORTED_VERSION, - requestListenerName, - new LeaderAndEpoch( - localId, - leaderState.epoch() - ), - leaderState.leaderEndpoints() - ) - ); - } + final Optional<KRaftVersionUpgrade.Voters> inMemoryVoters; + final Optional<VoterSet> voters; + if (kraftVersion.isReconfigSupported()) { + inMemoryVoters = Optional.empty(); - // Check that there are no uncommitted VotersRecord - Optional<LogHistory.Entry<VoterSet>> votersEntry = partitionState.lastVoterSetEntry(); - if (votersEntry.isEmpty() || votersEntry.get().offset() >= highWatermark.get()) { + // Check that there are no uncommitted VotersRecord + Optional<LogHistory.Entry<VoterSet>> votersEntry = partitionState.lastVoterSetEntry(); + if (votersEntry.isEmpty() || votersEntry.get().offset() >= highWatermark.get()) { + voters = Optional.empty(); + } else { + voters = votersEntry.map(LogHistory.Entry::value); + } + } else { + inMemoryVoters = leaderState.volatileVoters(); + if (inMemoryVoters.isEmpty()) { Review Comment: this would be unexpected right? wonder what the rationale is for returning `REQUEST_TIMED_OUT` and should we log an error here? ########## raft/src/main/java/org/apache/kafka/raft/VoterSet.java: ########## @@ -250,6 +251,26 @@ public Optional<VoterSet> updateVoter(VoterNode voter) { return Optional.empty(); } + /** + * Update a voter by only comparing the node id. + * + * This update voter operation doesn't compare the directory id. This is useful when upgrading + * from a voter set that doesn't support directory id to one that supports directory ids. Review Comment: doesn't `isVoter` already take care of this? ``` * If the voter node doesn't include the directory id ({@code Optional.empty()}), a replica * is the voter as long as the node id matches. The directory id is not checked. ``` It seems like we might not need a separate `updateVoterIgnoringDirectoryId` method -- 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