junrao commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579438984



##########
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##########
@@ -0,0 +1,894 @@
+/*
+ * 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.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+    static class TopicControlInfo {
+        private final Uuid id;
+        private final TimelineHashMap<Integer, PartitionControlInfo> parts;
+
+        TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+            this.id = id;
+            this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+        }
+    }
+
+    static class PartitionControlInfo {
+        private final int[] replicas;
+        private final int[] isr;
+        private final int[] removingReplicas;
+        private final int[] addingReplicas;
+        private final int leader;
+        private final int leaderEpoch;
+        private final int partitionEpoch;
+
+        PartitionControlInfo(PartitionRecord record) {
+            this(Replicas.toArray(record.replicas()),
+                Replicas.toArray(record.isr()),
+                Replicas.toArray(record.removingReplicas()),
+                Replicas.toArray(record.addingReplicas()),
+                record.leader(),
+                record.leaderEpoch(),
+                record.partitionEpoch());
+        }
+
+        PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+                             int[] addingReplicas, int leader, int leaderEpoch,
+                             int partitionEpoch) {
+            this.replicas = replicas;
+            this.isr = isr;
+            this.removingReplicas = removingReplicas;
+            this.addingReplicas = addingReplicas;
+            this.leader = leader;
+            this.leaderEpoch = leaderEpoch;
+            this.partitionEpoch = partitionEpoch;
+        }
+
+        PartitionControlInfo merge(PartitionChangeRecord record) {
+            int[] newIsr = (record.isr() == null) ? isr : 
Replicas.toArray(record.isr());
+            int newLeader;
+            int newLeaderEpoch;
+            if (record.leader() == Integer.MIN_VALUE) {
+                newLeader = leader;
+                newLeaderEpoch = leaderEpoch;
+            } else {
+                newLeader = record.leader();
+                newLeaderEpoch = leaderEpoch + 1;
+            }
+            return new PartitionControlInfo(replicas,
+                newIsr,
+                removingReplicas,
+                addingReplicas,
+                newLeader,
+                newLeaderEpoch,
+                partitionEpoch + 1);
+        }
+
+        String diff(PartitionControlInfo prev) {
+            StringBuilder builder = new StringBuilder();
+            String prefix = "";
+            if (!Arrays.equals(replicas, prev.replicas)) {
+                
builder.append(prefix).append("oldReplicas=").append(Arrays.toString(prev.replicas));
+                prefix = ", ";
+                
builder.append(prefix).append("newReplicas=").append(Arrays.toString(replicas));
+            }
+            if (!Arrays.equals(isr, prev.isr)) {
+                
builder.append(prefix).append("oldIsr=").append(Arrays.toString(prev.isr));
+                prefix = ", ";
+                
builder.append(prefix).append("newIsr=").append(Arrays.toString(isr));
+            }
+            if (!Arrays.equals(removingReplicas, prev.removingReplicas)) {
+                builder.append(prefix).append("oldRemovingReplicas=").
+                    append(Arrays.toString(prev.removingReplicas));
+                prefix = ", ";
+                builder.append(prefix).append("newRemovingReplicas=").
+                    append(Arrays.toString(removingReplicas));
+            }
+            if (!Arrays.equals(addingReplicas, prev.addingReplicas)) {
+                builder.append(prefix).append("oldAddingReplicas=").
+                    append(Arrays.toString(prev.addingReplicas));
+                prefix = ", ";
+                builder.append(prefix).append("newAddingReplicas=").
+                    append(Arrays.toString(addingReplicas));
+            }
+            if (leader != prev.leader) {
+                
builder.append(prefix).append("oldLeader=").append(prev.leader);
+                prefix = ", ";
+                builder.append(prefix).append("newLeader=").append(leader);
+            }
+            if (leaderEpoch != prev.leaderEpoch) {
+                
builder.append(prefix).append("oldLeaderEpoch=").append(prev.leaderEpoch);
+                prefix = ", ";
+                
builder.append(prefix).append("newLeaderEpoch=").append(leaderEpoch);
+            }
+            if (partitionEpoch != prev.partitionEpoch) {
+                
builder.append(prefix).append("oldPartitionEpoch=").append(prev.partitionEpoch);
+                prefix = ", ";
+                
builder.append(prefix).append("newPartitionEpoch=").append(partitionEpoch);
+            }
+            return builder.toString();
+        }
+
+        int preferredReplica() {
+            if (replicas.length == 0) return -1;
+            return replicas[0];
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(replicas, isr, removingReplicas, 
addingReplicas, leader,
+                leaderEpoch, partitionEpoch);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (!(o instanceof PartitionControlInfo)) return false;
+            PartitionControlInfo other = (PartitionControlInfo) o;
+            return diff(other).isEmpty();
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder builder = new StringBuilder("PartitionControlInfo(");
+            builder.append("replicas=").append(Arrays.toString(replicas));
+            builder.append(", isr=").append(Arrays.toString(isr));
+            builder.append(", 
removingReplicas=").append(Arrays.toString(removingReplicas));
+            builder.append(", 
addingReplicas=").append(Arrays.toString(addingReplicas));
+            builder.append(", leader=").append(leader);
+            builder.append(", leaderEpoch=").append(leaderEpoch);
+            builder.append(", partitionEpoch=").append(partitionEpoch);
+            builder.append(")");
+            return builder.toString();
+        }
+    }
+
+    private final SnapshotRegistry snapshotRegistry;
+    private final Logger log;
+
+    /**
+     * The random number generator used by this object.
+     */
+    private final Random random;
+
+    /**
+     * The KIP-464 default replication factor that is used if a CreateTopics 
request does
+     * not specify one.
+     */
+    private final short defaultReplicationFactor;
+
+    /**
+     * The KIP-464 default number of partitions that is used if a CreateTopics 
request does
+     * not specify a number of partitions.
+     */
+    private final int defaultNumPartitions;
+
+    /**
+     * A reference to the controller's configuration control manager.
+     */
+    private final ConfigurationControlManager configurationControl;
+
+    /**
+     * A reference to the controller's cluster control manager.
+     */
+    final ClusterControlManager clusterControl;
+
+    /**
+     * Maps topic names to topic UUIDs.
+     */
+    private final TimelineHashMap<String, Uuid> topicsByName;
+
+    /**
+     * Maps topic UUIDs to structures containing topic information, including 
partitions.
+     */
+    private final TimelineHashMap<Uuid, TopicControlInfo> topics;
+
+    /**
+     * A map of broker IDs to the partitions that the broker is in the ISR for.
+     */
+    private final BrokersToIsrs brokersToIsrs;
+
+    ReplicationControlManager(SnapshotRegistry snapshotRegistry,
+                              LogContext logContext,
+                              Random random,
+                              short defaultReplicationFactor,
+                              int defaultNumPartitions,
+                              ConfigurationControlManager configurationControl,
+                              ClusterControlManager clusterControl) {
+        this.snapshotRegistry = snapshotRegistry;
+        this.log = logContext.logger(ReplicationControlManager.class);
+        this.random = random;
+        this.defaultReplicationFactor = defaultReplicationFactor;
+        this.defaultNumPartitions = defaultNumPartitions;
+        this.configurationControl = configurationControl;
+        this.clusterControl = clusterControl;
+        this.topicsByName = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.topics = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.brokersToIsrs = new BrokersToIsrs(snapshotRegistry);
+    }
+
+    public void replay(TopicRecord record) {
+        topicsByName.put(record.name(), record.topicId());
+        topics.put(record.topicId(), new TopicControlInfo(snapshotRegistry, 
record.topicId()));
+        log.info("Created topic {} with ID {}.", record.name(), 
record.topicId());
+    }
+
+    public void replay(PartitionRecord record) {
+        TopicControlInfo topicInfo = topics.get(record.topicId());
+        if (topicInfo == null) {
+            throw new RuntimeException("Tried to create partition " + 
record.topicId() +
+                ":" + record.partitionId() + ", but no topic with that ID was 
found.");
+        }
+        PartitionControlInfo newPartInfo = new PartitionControlInfo(record);
+        PartitionControlInfo prevPartInfo = 
topicInfo.parts.get(record.partitionId());
+        if (prevPartInfo == null) {
+            log.info("Created partition {}:{} with {}.", record.topicId(),
+                record.partitionId(), newPartInfo.toString());
+            topicInfo.parts.put(record.partitionId(), newPartInfo);
+            brokersToIsrs.update(record.topicId(), record.partitionId(), null,
+                newPartInfo.isr, -1, newPartInfo.leader);
+        } else {
+            String diff = newPartInfo.diff(prevPartInfo);
+            if (!diff.isEmpty()) {
+                log.info("Modified partition {}:{}: {}.", record.topicId(),
+                    record.partitionId(), diff);
+                topicInfo.parts.put(record.partitionId(), newPartInfo);
+                brokersToIsrs.update(record.topicId(), record.partitionId(),
+                    prevPartInfo.isr, newPartInfo.isr, prevPartInfo.leader,
+                    newPartInfo.leader);
+            }
+        }
+    }
+
+    public void replay(PartitionChangeRecord record) {
+        TopicControlInfo topicInfo = topics.get(record.topicId());
+        if (topicInfo == null) {
+            throw new RuntimeException("Tried to create partition " + 
record.topicId() +
+                ":" + record.partitionId() + ", but no topic with that ID was 
found.");
+        }
+        PartitionControlInfo prevPartitionInfo = 
topicInfo.parts.get(record.partitionId());
+        if (prevPartitionInfo == null) {
+            throw new RuntimeException("Tried to create partition " + 
record.topicId() +
+                ":" + record.partitionId() + ", but no partition with that id 
was found.");
+        }
+        PartitionControlInfo newPartitionInfo = 
prevPartitionInfo.merge(record);

Review comment:
       Hmm, merge bumps up the leaderEpoch. It seems that this needs to be 
persisted in the metadata log?




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

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to