jsancio commented on code in PR #15671: URL: https://github.com/apache/kafka/pull/15671#discussion_r1581410429
########## raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java: ########## @@ -0,0 +1,327 @@ +/* + * 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 java.net.InetSocketAddress; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.VotersRecord; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.feature.SupportedVersionRange; + +/** + * A type for representing the set of voters for a topic partition. + * + * It encapsulates static information like a voter's endpoint and their supported kraft.version. + * + * It providees functionality for converting to and from {@code VotersRecord} and for converting + * from the static configuration. + */ +final public class VoterSet { + private final Map<Integer, VoterNode> voters; + + VoterSet(Map<Integer, VoterNode> voters) { + if (voters.isEmpty()) { + throw new IllegalArgumentException("Voters cannot be empty"); + } + + this.voters = voters; + } + + /** + * Returns the socket address for a given voter at a given listener. + * + * @param voter the id of the voter + * @param listener the name of the listener + * @return the socket address if it exist, otherwise {@code Optional.empty()} + */ + public Optional<InetSocketAddress> voterAddress(int voter, String listener) { + return Optional.ofNullable(voters.get(voter)) + .flatMap(voterNode -> voterNode.address(listener)); + } + + /** + * Returns all of the voter ids. + */ + public Set<Integer> voterIds() { + return voters.keySet(); + } + + /** + * Adds a voter to the voter set. + * + * This object is immutable. A new voter set is returned if the voter was added. + * + * A new voter can be added to a voter set if its id doesn't already exist in the voter set. + * + * @param voter the new voter to add + * @return a new voter set if the voter was added, otherwise {@code Optional.empty()} + */ + public Optional<VoterSet> addVoter(VoterNode voter) { + if (voters.containsKey(voter.id())) { + return Optional.empty(); + } + + HashMap<Integer, VoterNode> newVoters = new HashMap<>(voters); + newVoters.put(voter.id(), voter); + + return Optional.of(new VoterSet(newVoters)); + } + + /** + * Removew a voter from the voter set. + * + * This object is immutable. A new voter set is returned if the voter was removed. + * + * A voter can be removed from the voter set if its id and uuid match. + * + * @param voterId the voter id + * @param voterUuid the voter uuid + * @return a new voter set if the voter was remove, otherwise {@code Optional.empty()} + */ + public Optional<VoterSet> removeVoter(int voterId, Optional<Uuid> voterUuid) { + VoterNode oldVoter = voters.get(voterId); + if (oldVoter != null && Objects.equals(oldVoter.uuid(), voterUuid)) { + HashMap<Integer, VoterNode> newVoters = new HashMap<>(voters); + newVoters.remove(voterId); + + return Optional.of(new VoterSet(newVoters)); + } + + return Optional.empty(); + } + + /** + * Converts a voter set to a voters record for a given version. + * + * @param version the version of the voters record + */ + public VotersRecord toVotersRecord(short version) { + return new VotersRecord() + .setVersion(version) + .setVoters( + voters + .values() + .stream() + .map(voter -> { + Iterator<VotersRecord.Endpoint> endpoints = voter + .listeners() Review Comment: Fix. -- 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