junrao commented on code in PR #15859:
URL: https://github.com/apache/kafka/pull/15859#discussion_r1593188110


##########
raft/src/main/java/org/apache/kafka/raft/ElectionState.java:
##########
@@ -16,46 +16,44 @@
  */
 package org.apache.kafka.raft;
 
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
 import java.util.OptionalInt;
 import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.raft.generated.QuorumStateData;
+import org.apache.kafka.raft.internals.ReplicaKey;
 
 /**
  * Encapsulate election state stored on disk after every state change.
  */
-public class ElectionState {
-    public final int epoch;
-    public final OptionalInt leaderIdOpt;
-    public final OptionalInt votedIdOpt;
+final public class ElectionState {
+    private static int unknownLeaderId = -1;

Review Comment:
   For constants, should we follow the convention of using all capital letters 
plus underscore?



##########
raft/src/main/java/org/apache/kafka/raft/ElectionState.java:
##########
@@ -115,15 +166,51 @@ public boolean equals(Object o) {
         ElectionState that = (ElectionState) o;
 
         if (epoch != that.epoch) return false;
-        if (!leaderIdOpt.equals(that.leaderIdOpt)) return false;
-        return votedIdOpt.equals(that.votedIdOpt);
+        if (!leaderId.equals(that.leaderId)) return false;
+        if (!votedKey.equals(that.votedKey)) return false;
+
+        return voters.equals(that.voters);
     }
 
     @Override
     public int hashCode() {
-        int result = epoch;
-        result = 31 * result + leaderIdOpt.hashCode();
-        result = 31 * result + votedIdOpt.hashCode();
-        return result;
+        return Objects.hash(epoch, leaderId, votedKey, voters);
+    }
+
+    public static ElectionState withVotedCandidate(int epoch, ReplicaKey 
votedKey, Set<Integer> voters) {
+        if (votedKey.id() < 0) {
+            throw new IllegalArgumentException("Illegal voted Id " + 
votedKey.id() + ": must be non-negative");
+        }
+
+        return new ElectionState(epoch, OptionalInt.empty(), 
Optional.of(votedKey), voters);
+    }
+
+    public static ElectionState withElectedLeader(int epoch, int leaderId, 
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);
+    }
+
+    public static ElectionState withUnknownLeader(int epoch, Set<Integer> 
voters) {
+        return new ElectionState(epoch, OptionalInt.empty(), Optional.empty(), 
voters);
+    }
+
+    public static ElectionState fromQuorumStateData(QuorumStateData data) {
+        Optional<Uuid> votedDirectoryId = 
data.votedDirectoryId().equals(noVotedDirectoryId) ?
+            Optional.empty() :
+            Optional.of(data.votedDirectoryId());
+
+        Optional<ReplicaKey> voterKey = data.votedId() == notVoted ?

Review Comment:
   voterKey => votedKey



##########
raft/src/main/java/org/apache/kafka/raft/ElectionState.java:
##########
@@ -64,47 +62,100 @@ public boolean isLeader(int nodeId) {
         return leaderIdOrSentinel() == nodeId;
     }
 
-    public boolean isVotedCandidate(int nodeId) {
-        if (nodeId < 0)
-            throw new IllegalArgumentException("Invalid negative nodeId: " + 
nodeId);
-        return votedIdOpt.orElse(-1) == nodeId;
+    /**
+     * Return if the replica has voted for the given candidate.
+     *
+     * A replica has voted for a candidate if all of the following are true:
+     * 1. the node's id and voted id match and
+     * 2. if the voted directory id is set, it matches the node's directory id
+     *
+     * @param nodeKey the id and directory id of the replica
+     * @return true when the arguments match, otherwise false
+     */
+    public boolean isVotedCandidate(ReplicaKey nodeKey) {
+        if (nodeKey.id() < 0) {
+            throw new IllegalArgumentException("Invalid node key " + nodeKey);
+        } else if (!votedKey.isPresent()) {
+            return false;
+        } else if (votedKey.get().id() != nodeKey.id()) {
+            return false;
+        } else if (!votedKey.get().directoryId().isPresent()) {
+            // when the persisted voted uuid is not present assume that we 
voted for this candidate;
+            // this happends when the kraft version is 0.

Review Comment:
   typo happends



##########
raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java:
##########
@@ -79,20 +81,79 @@ void testRemoveVoter() {
         );
     }
 
+    @Test
+    void testIsVoterWithDirectoryId() {
+        Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(Arrays.asList(1, 
2, 3), true);
+        VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
+
+        assertTrue(voterSet.isVoter(aVoterMap.get(1).voterKey()));
+        assertFalse(voterSet.isVoter(ReplicaKey.of(1, 
Optional.of(Uuid.randomUuid()))));
+        assertFalse(voterSet.isVoter(ReplicaKey.of(1, Optional.empty())));
+        assertFalse(
+            voterSet.isVoter(ReplicaKey.of(2, 
aVoterMap.get(1).voterKey().directoryId()))
+        );
+        assertFalse(
+            voterSet.isVoter(ReplicaKey.of(4, 
aVoterMap.get(1).voterKey().directoryId()))
+        );
+        assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.empty())));
+    }
+
+    @Test
+    void testIsVoterWithoutDirectoryId() {
+        Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(Arrays.asList(1, 
2, 3), false);
+        VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
+
+        assertTrue(voterSet.isVoter(ReplicaKey.of(1, Optional.empty())));
+        assertTrue(voterSet.isVoter(ReplicaKey.of(1, 
Optional.of(Uuid.randomUuid()))));
+        assertFalse(voterSet.isVoter(ReplicaKey.of(4, 
Optional.of(Uuid.randomUuid()))));
+        assertFalse(voterSet.isVoter(ReplicaKey.of(4, Optional.empty())));
+    }
+
+    @Test
+    void testStandaloneAndOnlyVoter() {
+        Map<Integer, VoterSet.VoterNode> aVoterMap = 
voterMap(Arrays.asList(1), true);
+        VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap));
+
+        assertTrue(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey()));
+        assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, 
Optional.of(Uuid.randomUuid()))));
+        assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Optional.empty())));
+        assertFalse(
+            voterSet.isOnlyVoter(ReplicaKey.of(4, 
aVoterMap.get(1).voterKey().directoryId()))
+        );
+        assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(4, Optional.empty())));
+    }
+
+    @Test
+    void testOnlyVoter() {

Review Comment:
   The test name seems inaccurate since the test has more than one voter.



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -1700,16 +1710,16 @@ private void handleResponse(RaftResponse.Inbound 
response, long currentTimeMs) {
     }
 
     /**
-     * Validate a request which is only valid between voters. If an error is
-     * present in the returned value, it should be returned in the response.
+     * Validate common state for requests to establish leadership.
+     *
+     * These include the Vote, BeginQuorumEpoch rnd EndQuorumEpoch RPCs. If an 
error is present in

Review Comment:
   typo rnd



##########
raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java:
##########
@@ -64,6 +64,43 @@ public Optional<InetSocketAddress> voterAddress(int voter, 
String listener) {
             .flatMap(voterNode -> voterNode.address(listener));
     }
 
+    /**
+     * Returns if the node is a voter in the set of voters.
+     *
+     * If the voter set includes the directory id, the {@code nodeKey} 
directory id must match the
+     * directory id specified by the voter set.
+     *
+     * If the voter set doesn't include the directory id ({@code 
Optional.empty()}), a node is in
+     * the voter set as long as the node id matches. The directory id is not 
checked.
+     *
+     * @param nodeKey the node's id and directory id
+     * @return true if the node is a voter in the voter set, otherwise false
+     */
+    public boolean isVoter(ReplicaKey nodeKey) {
+        VoterNode node = voters.get(nodeKey.id());
+        if (node != null) {
+            if (node.voterKey().directoryId().isPresent()) {
+                return 
node.voterKey().directoryId().equals(nodeKey.directoryId());
+            } else {
+                // configured voter set doesn't an uuid so it is a voter as 
long as the node id

Review Comment:
   uuid => directory id ?



##########
raft/src/main/java/org/apache/kafka/raft/QuorumState.java:
##########
@@ -481,7 +517,7 @@ private void durableTransitionTo(EpochState state) {
             }
         }
 
-        this.store.writeElectionState(state.election());
+        this.store.writeElectionState(state.election(), 
latestKraftVersion.get());

Review Comment:
   We refer to an instance val sometimes with `this` and some other times 
without. What's the convention?



##########
raft/src/main/java/org/apache/kafka/raft/CandidateState.java:
##########
@@ -51,14 +54,27 @@ public class CandidateState implements EpochState {
     protected CandidateState(
         Time time,
         int localId,
+        Uuid localDirectoryId,

Review Comment:
   Should we include `localDirectoryId` in `toString() `?



##########
raft/src/main/java/org/apache/kafka/raft/EpochState.java:
##########
@@ -26,15 +27,16 @@ default Optional<LogOffsetMetadata> highWatermark() {
     }
 
     /**
-     * Decide whether to grant a vote to a candidate, it is the responsibility 
of the caller to invoke
+     * Decide whether to grant a vote to a candidate.
+     *
+     * It is the responsibility of the caller to invoke

Review Comment:
   This might be an existing issue, but `QuorumState#transitionToVoted` doesn't 
exist.



##########
raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.raft.generated.QuorumStateData;
+import org.apache.kafka.raft.internals.ReplicaKey;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+final class ElectionStateTest {
+    @Test
+    void testVotedCandidateWithoutVotedId() {
+        ElectionState electionState = ElectionState.withUnknownLeader(5, 
Collections.emptySet());
+        assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, 
Optional.empty())));
+    }
+
+    @Test
+    void testVotedCandidateWithoutVotedDirectoryId() {
+        ElectionState electionState = ElectionState.withVotedCandidate(
+            5,
+            ReplicaKey.of(1, Optional.empty()),
+            Collections.emptySet()
+        );
+        assertTrue(electionState.isVotedCandidate(ReplicaKey.of(1, 
Optional.empty())));
+        assertTrue(
+            electionState.isVotedCandidate(ReplicaKey.of(1, 
Optional.of(Uuid.randomUuid())))
+        );
+    }
+
+    @Test
+    void testVotedCandidateWithVotedDirectoryId() {
+        ReplicaKey votedKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
+        ElectionState electionState = ElectionState.withVotedCandidate(
+            5,
+            votedKey,
+            Collections.emptySet()
+        );
+        assertFalse(electionState.isVotedCandidate(ReplicaKey.of(1, 
Optional.empty())));
+        assertTrue(electionState.isVotedCandidate(votedKey));
+    }
+
+    @ParameterizedTest
+    @ValueSource(shorts = {0, 1})
+    void testQuorumStateDataRoundTrip(short version) {
+        ReplicaKey votedKey = ReplicaKey.of(1, Optional.of(Uuid.randomUuid()));
+        List<ElectionState> electionStates = Arrays.asList(
+            ElectionState.withUnknownLeader(5, Utils.mkSet(1, 2, 3)),
+            ElectionState.withElectedLeader(5, 1, Utils.mkSet(1, 2, 3)),
+            ElectionState.withVotedCandidate(5, votedKey, Utils.mkSet(1, 2, 3))
+        );
+
+        final List<ElectionState> expected;
+        if (version == 0) {
+            expected = Arrays.asList(
+                ElectionState.withUnknownLeader(5, Utils.mkSet(1, 2, 3)),
+                ElectionState.withElectedLeader(5, 1, Utils.mkSet(1, 2, 3)),
+                ElectionState.withVotedCandidate(
+                    5,
+                    ReplicaKey.of(1, Optional.empty()),
+                    Utils.mkSet(1, 2, 3)
+                )
+            );
+        } else {
+            expected = Arrays.asList(
+                ElectionState.withUnknownLeader(5, Collections.emptySet()),
+                ElectionState.withElectedLeader(5, 1, Collections.emptySet()),
+                ElectionState.withVotedCandidate(5, votedKey, 
Collections.emptySet())

Review Comment:
   It seems inconsistent that we fill the voters in version 0, but not in 
version 1.



##########
raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java:
##########
@@ -63,7 +66,9 @@ public void tearDown() {
     private QuorumState buildQuorumState(Set<Integer> voters) {
         return new QuorumState(
             OptionalInt.of(localId),
-            voters,
+            localDirectoryId,
+            () -> VoterSetTest.voterSet(VoterSetTest.voterMap(voters, false)),
+            () -> (short) 0,

Review Comment:
   Do we have test with `latestKraftVersion` as 1?



##########
raft/src/main/java/org/apache/kafka/raft/FileQuorumStateStore.java:
##########
@@ -59,17 +54,18 @@
  *   "data_version":0}
  * </pre>
  * */
-public class FileBasedStateStore implements QuorumStateStore {
-    private static final Logger log = 
LoggerFactory.getLogger(FileBasedStateStore.class);
+public class FileQuorumStateStore implements QuorumStateStore {

Review Comment:
   Could we update the file layout in the javadoc?



##########
raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java:
##########
@@ -850,14 +846,10 @@ public String toString() {
     }
 
     private static class InflightRequest {
-        final int correlationId;
         final int sourceId;
-        final int destinationId;
 
         private InflightRequest(int correlationId, int sourceId, int 
destinationId) {
-            this.correlationId = correlationId;

Review Comment:
   Need to remove the unused input params.



-- 
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

Reply via email to