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



##########
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##########
@@ -0,0 +1,917 @@
+/*
+ * 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.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+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.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+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.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+    /**
+     * A builder class which creates the QuorumController.
+     */
+    static public class Builder {
+        private final int nodeId;
+        private Time time = Time.SYSTEM;
+        private String threadNamePrefix = null;
+        private LogContext logContext = null;
+        private Map<ConfigResource.Type, ConfigDef> configDefs = 
Collections.emptyMap();
+        private MetaLogManager logManager = null;
+        private Map<String, VersionRange> supportedFeatures = 
Collections.emptyMap();
+        private short defaultReplicationFactor = 3;
+        private int defaultNumPartitions = 1;
+        private ReplicaPlacementPolicy replicaPlacementPolicy =
+            new SimpleReplicaPlacementPolicy(new Random());
+        private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+        public Builder(int nodeId) {
+            this.nodeId = nodeId;
+        }
+
+        public Builder setTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder setThreadNamePrefix(String threadNamePrefix) {
+            this.threadNamePrefix = threadNamePrefix;
+            return this;
+        }
+
+        public Builder setLogContext(LogContext logContext) {
+            this.logContext = logContext;
+            return this;
+        }
+
+        public Builder setConfigDefs(Map<ConfigResource.Type, ConfigDef> 
configDefs) {
+            this.configDefs = configDefs;
+            return this;
+        }
+
+        public Builder setLogManager(MetaLogManager logManager) {
+            this.logManager = logManager;
+            return this;
+        }
+
+        public Builder setSupportedFeatures(Map<String, VersionRange> 
supportedFeatures) {
+            this.supportedFeatures = supportedFeatures;
+            return this;
+        }
+
+        public Builder setDefaultReplicationFactor(short 
defaultReplicationFactor) {
+            this.defaultReplicationFactor = defaultReplicationFactor;
+            return this;
+        }
+
+        public Builder setDefaultNumPartitions(int defaultNumPartitions) {
+            this.defaultNumPartitions = defaultNumPartitions;
+            return this;
+        }
+
+        public Builder setReplicaPlacementPolicy(ReplicaPlacementPolicy 
replicaPlacementPolicy) {
+            this.replicaPlacementPolicy = replicaPlacementPolicy;
+            return this;
+        }
+
+        public Builder setSessionTimeoutNs(long sessionTimeoutNs) {
+            this.sessionTimeoutNs = sessionTimeoutNs;
+            return this;
+        }
+
+        public QuorumController build() throws Exception {
+            if (logManager == null) {
+                throw new RuntimeException("You must set a metadata log 
manager.");
+            }
+            if (threadNamePrefix == null) {
+                threadNamePrefix = String.format("Node%d_", nodeId);
+            }
+            if (logContext == null) {
+                logContext = new LogContext(String.format("[Controller %d] ", 
nodeId));
+            }
+            KafkaEventQueue queue = null;
+            try {
+                queue = new KafkaEventQueue(time, logContext, 
threadNamePrefix);
+                return new QuorumController(logContext, nodeId, queue, time, 
configDefs,
+                        logManager, supportedFeatures, 
defaultReplicationFactor,
+                        defaultNumPartitions, replicaPlacementPolicy, 
sessionTimeoutNs);
+            } catch (Exception e) {
+                Utils.closeQuietly(queue, "event queue");
+                throw e;
+            }
+        }
+    }
+
+    private static final String ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX =
+        "The active controller appears to be node ";
+
+    private NotControllerException newNotControllerException() {

Review comment:
       I normally do group private methods after internal classes, but in this 
case, it seemed better to keep them together.  Otherwise you'd have to jump 
around a lot when reading the code.  What do you think?




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