hachikuji commented on code in PR #15671:
URL: https://github.com/apache/kafka/pull/15671#discussion_r1583487039


##########
core/src/main/scala/kafka/raft/RaftManager.scala:
##########
@@ -181,20 +181,12 @@ class KafkaRaftManager[T](
   private val clientDriver = new KafkaRaftClientDriver[T](client, 
threadNamePrefix, fatalFaultHandler, logContext)
 
   def startup(): Unit = {
-    // Update the voter endpoints (if valid) with what's in RaftConfig
-    val voterAddresses: util.Map[Integer, AddressSpec] = 
controllerQuorumVotersFuture.get()
-    for (voterAddressEntry <- voterAddresses.entrySet.asScala) {
-      voterAddressEntry.getValue match {
-        case spec: InetAddressSpec =>
-          netChannel.updateEndpoint(voterAddressEntry.getKey, spec)
-        case _: UnknownAddressSpec =>
-          info(s"Skipping channel update for destination ID: 
${voterAddressEntry.getKey} " +
-            s"because of non-routable endpoint: 
${NON_ROUTABLE_ADDRESS.toString}")
-        case invalid: AddressSpec =>
-          warn(s"Unexpected address spec (type: ${invalid.getClass}) for 
channel update for " +
-            s"destination ID: ${voterAddressEntry.getKey}")
-      }
-    }
+    client.initialize(
+      controllerQuorumVotersFuture.get(),
+      config.controllerListenerNames.head,
+      new FileBasedStateStore(new File(dataDir, "quorum-state")),

Review Comment:
   Maybe we can have a constant for the file name. Not sure if it is used 
elsewhere, but it would be nice to have a nice descriptive name.



##########
raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java:
##########
@@ -0,0 +1,375 @@
+/*
+ * 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.util.Arrays;
+import java.util.Optional;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.KRaftVersionRecord;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.utils.BufferSupplier;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.raft.MockLog;
+import org.apache.kafka.raft.OffsetAndEpoch;
+import org.apache.kafka.server.common.serialization.RecordSerde;
+import org.apache.kafka.snapshot.RecordsSnapshotWriter;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+final class KRaftControlRecordStateMachineTest {
+    private static final RecordSerde<String> STRING_SERDE = new StringSerde();
+
+    private static MockLog buildLog() {
+        return new MockLog(new TopicPartition("partition", 0), 
Uuid.randomUuid(), new LogContext());
+    }
+
+    private static KRaftControlRecordStateMachine 
buildPartitionListener(MockLog log, Optional<VoterSet> staticVoterSet) {
+        return new KRaftControlRecordStateMachine(
+            staticVoterSet,
+            log,
+            STRING_SERDE,
+            BufferSupplier.NO_CACHING,
+            1024,
+            new LogContext()
+        );
+    }
+
+    @Test
+    void testEmptyParition() {

Review Comment:
   nit: typo Par**t**ition



##########
raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java:
##########
@@ -206,8 +151,16 @@ private static Integer parseVoterId(String idString) {
         }
     }
 
-    public static Map<Integer, AddressSpec> parseVoterConnections(List<String> 
voterEntries) {
-        Map<Integer, AddressSpec> voterMap = new HashMap<>();
+    public static Map<Integer, InetSocketAddress> 
parseVoterConnections(List<String> voterEntries) {
+        return parseVoterConnections(voterEntries, true);
+    }
+
+    public static Set<Integer> parseVoterIds(List<String> voterEntries) {
+        return parseVoterConnections(voterEntries, false).keySet();
+    }
+
+    private static Map<Integer, InetSocketAddress> 
parseVoterConnections(List<String> voterEntries, boolean routableOnly) {

Review Comment:
   nit: I think `requireRoutableAddresses` might convey the expectation more 
clearly.



##########
raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java:
##########
@@ -213,20 +213,21 @@ private void completeCurrentBatch() {
      *
      * @param valueCreator a function that uses the passed buffer to create 
the control
      *        batch that will be appended. The memory records returned must 
contain one
-     *        control batch and that control batch have one record.
+     *        control batch and that control batch have at least one record.
      */
-    private void appendControlMessage(Function<ByteBuffer, MemoryRecords> 
valueCreator) {
+    public void appendControlMessages(Function<ByteBuffer, CreatedRecords> 
valueCreator) {

Review Comment:
   Since we have made this public, I wonder if it would be useful to validate 
that the created batch is in fact a control batch.



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -159,67 +165,76 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
     private final MemoryPool memoryPool;
     private final RaftMessageQueue messageQueue;
     private final QuorumConfig quorumConfig;
-    private final KafkaRaftMetrics kafkaRaftMetrics;
-    private final QuorumState quorum;
-    private final RequestManager requestManager;
     private final RaftMetadataLogCleanerManager snapshotCleaner;
 
     private final Map<Listener<T>, ListenerContext> listenerContexts = new 
IdentityHashMap<>();
     private final ConcurrentLinkedQueue<Registration<T>> pendingRegistrations 
= new ConcurrentLinkedQueue<>();
 
+    // These components need to be initialized by the method initialize() 
because they depend on the voter set
+    /*
+     * The key invariant for the kraft control record state machine is that it 
has always read to the LEO. This is achived by:

Review Comment:
   nit: achi**e**ved



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -370,8 +363,52 @@ private void maybeFireLeaderChange() {
         }
     }
 
-    @Override
-    public void initialize() {
+    public void initialize(
+        Map<Integer, InetSocketAddress> voterAddresses,
+        String listenerName,
+        QuorumStateStore quorumStateStore,
+        Metrics metrics

Review Comment:
   Why do we need to delay initialization of metrics?
   
   I can feel that the parameters for this method might grow over time. Perhaps 
we should package it up into some kind of initialization object.



##########
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();

Review Comment:
   That is a higher level invariant. It depends on other parts of the system. 
Should there be a local invariant as well which does not depend on that?



##########
raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java:
##########
@@ -261,7 +260,7 @@ default long truncateToEndOffset(OffsetAndEpoch endOffset) {
      * @param snapshotId the end offset and epoch that identifies the snapshot
      * @return a writable snapshot if it doesn't already exist
      */
-    Optional<RawSnapshotWriter> storeSnapshot(OffsetAndEpoch snapshotId);
+    Optional<RawSnapshotWriter> createNewSnapshotUnchecked(OffsetAndEpoch 
snapshotId);

Review Comment:
   Do we need the method? What would happen if the leader did violate the 
expected invariant? Would this data structure update itself to remain 
consistent?



##########
raft/src/main/java/org/apache/kafka/raft/internals/TreeMapHistory.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.util.NavigableMap;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.Map;
+
+/**
+ * A implementation for {@code History} which uses a red-black tree to store 
values sorted by offset.
+ */
+final public class TreeMapHistory<T> implements History<T> {
+    private final NavigableMap<Long, T> history = new TreeMap<>();
+
+    @Override
+    public void addAt(long offset, T value) {
+        if (offset < 0) {
+            throw new IllegalArgumentException(
+                String.format("Next offset %d must be greater than or equal to 
0", offset)
+            );
+        }
+
+        Map.Entry<Long, ?> lastEntry = history.lastEntry();

Review Comment:
   Fair enough. Though I'm not sure it matters too much given the infrequency 
of these records.



##########
raft/src/main/java/org/apache/kafka/raft/internals/History.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.util.Objects;
+import java.util.Optional;
+
+/**
+ * A object tracks values of {@code T} at different offsets.
+ */
+public interface History<T> {

Review Comment:
   Ok. In that case, perhaps we can qualify the general `History` name. Perhaps 
it should be `RecordHistory` or `LogHistory` or something like that?



##########
raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.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.internals;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+final public class VoterSetHistoryTest {

Review Comment:
   Should we have some cases where we try to add a non-overlapping voter set?



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -370,8 +363,52 @@ private void maybeFireLeaderChange() {
         }
     }
 
-    @Override
-    public void initialize() {
+    public void initialize(
+        Map<Integer, InetSocketAddress> voterAddresses,
+        String listenerName,
+        QuorumStateStore quorumStateStore,
+        Metrics metrics
+    ) {
+        partitionState = new KRaftControlRecordStateMachine(
+            Optional.of(VoterSet.fromAddressSpecs(listenerName, 
voterAddresses)),

Review Comment:
   nit: maybe we should rename the method `fromInetSocketAddresses` or 
something like that?



##########
raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java:
##########
@@ -216,4 +140,132 @@ private void appendBatches(List<CompletedBatch<T>> 
batches) {
             batches.forEach(CompletedBatch::release);
         }
     }
+
+    final public static class Builder {
+        private long lastContainedLogTimestamp = 0;
+        private CompressionType compressionType = CompressionType.NONE;
+        private Time time = Time.SYSTEM;
+        private int maxBatchSize = 1024;
+        private MemoryPool memoryPool = MemoryPool.NONE;
+        private short kraftVersion = 0;

Review Comment:
   I wonder if we should force this to be set explicitly. Perhaps it could be 
in the constructor. Or perhaps static methods `newBuilder(version)` and 
`newBuilderWithLatestVersion`.



##########
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java:
##########
@@ -2415,17 +2476,30 @@ public void resign(int epoch) {
     @Override
     public Optional<SnapshotWriter<T>> createSnapshot(
         OffsetAndEpoch snapshotId,
-        long lastContainedLogTime
+        long lastContainedLogTimestamp
     ) {
-        return RecordsSnapshotWriter.createWithHeader(
-                () -> log.createNewSnapshot(snapshotId),
-                MAX_BATCH_SIZE_BYTES,
-                memoryPool,
-                time,
-                lastContainedLogTime,
-                CompressionType.NONE,
-                serde
-            );
+        if (!isInitialized()) {
+            throw new IllegalStateException("Cannot create snapshot before the 
replica has been initialized");
+        }
+
+        return log.createNewSnapshot(snapshotId).map(writer -> {
+            long lastContainedLogOffset = snapshotId.offset() - 1;

Review Comment:
   nit: maybe we could have a helper for this logic



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

Review Comment:
   Hmm, the empty case still seems ambiguous to me. Seems worth clarifying 
explicitly. Either that or maybe we can use `VoterKey` directly. 



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