You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/02/11 23:34:01 UTC

[GitHub] [kafka] junrao commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

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



##########
File path: metadata/src/main/java/org/apache/kafka/controller/Controller.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+import org.apache.kafka.common.config.ConfigResource;
+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.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+
+public interface Controller extends AutoCloseable {
+    /**
+     * Change partition ISRs.
+     *
+     * @param request       The AlterIsrRequest data.
+     *
+     * @return              A future yielding the response.
+     */
+    CompletableFuture<AlterIsrResponseData> alterIsr(AlterIsrRequestData request);
+
+    /**
+     * Create a batch of topics.
+     *
+     * @param request       The CreateTopicsRequest data.
+     *
+     * @return              A future yielding the response.
+     */
+    CompletableFuture<CreateTopicsResponseData>
+        createTopics(CreateTopicsRequestData request);
+
+    /**
+     * Decommission a broker.
+     *
+     * @param brokerId      The broker id to decommission.
+     *
+     * @return              A future that is completed successfully hwne the broker is
+     *                      decommissioned, or if it is not registered in the first place.
+     */
+    CompletableFuture<Void> decommissionBroker(int brokerId);
+
+    /**
+     * Describe the current configuration of various resources.
+     *
+     * @param resources     A map from resources to the collection of config keys that we
+     *                      want to describe for each.  If the collection is empty, then
+     *                      all configuration keys will be described.
+     *
+     * @return
+     */
+    CompletableFuture<Map<ConfigResource, ResultOrError<Map<String, String>>>>
+        describeConfigs(Map<ConfigResource, Collection<String>> resources);
+
+    /**
+     * Elect new partition leaders.
+     *
+     * @param request       The request.
+     *
+     * @return              A future yielding the elect leaders response.
+     */
+    CompletableFuture<ElectLeadersResponseData> electLeaders(ElectLeadersRequestData request);
+
+    /**
+     * Get the current finalized feature ranges for each feature.
+     *
+     * @return              A future yielding the feature ranges.
+     */
+    CompletableFuture<FeatureManager.FinalizedFeaturesAndEpoch> finalizedFeatures();
+
+    /**
+     * Perform some incremental configuration changes.
+     *
+     * @param configChanges The changes.
+     * @param validateOnly  True if we should validate the changes but not apply them.
+     *
+     * @return              A future yielding a map from partitions to error results.

Review comment:
       The returned map is not keyed on partition.

##########
File path: metadata/src/main/java/org/apache/kafka/controller/Controller.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+import org.apache.kafka.common.config.ConfigResource;
+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.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+
+public interface Controller extends AutoCloseable {
+    /**
+     * Change partition ISRs.
+     *
+     * @param request       The AlterIsrRequest data.
+     *
+     * @return              A future yielding the response.
+     */
+    CompletableFuture<AlterIsrResponseData> alterIsr(AlterIsrRequestData request);
+
+    /**
+     * Create a batch of topics.
+     *
+     * @param request       The CreateTopicsRequest data.
+     *
+     * @return              A future yielding the response.
+     */
+    CompletableFuture<CreateTopicsResponseData>
+        createTopics(CreateTopicsRequestData request);
+
+    /**
+     * Decommission a broker.
+     *
+     * @param brokerId      The broker id to decommission.
+     *
+     * @return              A future that is completed successfully hwne the broker is

Review comment:
       typo hwne

##########
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:
       Could we move those private methods after all the internal classes?

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##########
@@ -0,0 +1,712 @@
+/*
+ * 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.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.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.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+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.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.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;
+
+
+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(IsrChangeRecord record) {
+            return new PartitionControlInfo(replicas,
+                Replicas.toArray(record.isr()),
+                removingReplicas,
+                addingReplicas,
+                record.leader(),
+                record.leaderEpoch(),
+                record.partitionEpoch());
+        }
+
+        String diff(PartitionControlInfo prev) {
+            StringBuilder builder = new StringBuilder();
+            String prefix = "";
+            if (!Arrays.equals(replicas, prev.replicas)) {
+                builder.append("replicas=").append(Arrays.toString(replicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(isr, prev.isr)) {
+                builder.append(prefix).append("isr=").append(Arrays.toString(isr));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(removingReplicas, prev.removingReplicas)) {
+                builder.append(prefix).append("removingReplicas=").
+                    append(Arrays.toString(removingReplicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(addingReplicas, prev.addingReplicas)) {
+                builder.append(prefix).append("addingReplicas=").
+                    append(Arrays.toString(addingReplicas));
+                prefix = ", ";
+            }
+            if (leader != prev.leader) {
+                builder.append(prefix).append("leader=").append(leader);
+                prefix = ", ";
+            }
+            if (leaderEpoch != prev.leaderEpoch) {
+                builder.append(prefix).append("leaderEpoch=").append(leaderEpoch);
+            }
+            if (partitionEpoch != prev.partitionEpoch) {
+                builder.append(prefix).append("partitionEpoch=").append(partitionEpoch);
+            }
+            return builder.toString();
+        }
+
+        @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();
+        }
+
+        int chooseNewLeader(int[] newIsr, boolean unclean) {
+            for (int i = 0; i < replicas.length; i++) {
+                int replica = replicas[i];
+                if (Replicas.contains(newIsr, replica)) {
+                    return replica;
+                }
+            }
+            if (unclean && replicas.length > 0) {
+                return replicas[0];
+            }
+            return -1;
+        }
+    }
+
+    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(IsrChangeRecord 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);
+        topicInfo.parts.put(record.partitionId(), newPartitionInfo);
+        brokersToIsrs.update(record.topicId(), record.partitionId(),
+            prevPartitionInfo.isr, newPartitionInfo.isr, prevPartitionInfo.leader,
+            newPartitionInfo.leader);
+        log.debug("Applied ISR change record: {}", record.toString());
+    }
+
+    ControllerResult<CreateTopicsResponseData>
+            createTopics(CreateTopicsRequestData request) {
+        Map<String, ApiError> topicErrors = new HashMap<>();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+
+        // Check the topic names.
+        validateNewTopicNames(topicErrors, request.topics());
+
+        // Identify topics that already exist and mark them with the appropriate error
+        request.topics().stream().filter(creatableTopic -> topicsByName.containsKey(creatableTopic.name()))
+                .forEach(t -> topicErrors.put(t.name(), new ApiError(Errors.TOPIC_ALREADY_EXISTS)));
+
+        // Verify that the configurations for the new topics are OK, and figure out what
+        // ConfigRecords should be created.
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges =
+            computeConfigChanges(topicErrors, request.topics());
+        ControllerResult<Map<ConfigResource, ApiError>> configResult =
+            configurationControl.incrementalAlterConfigs(configChanges);
+        for (Entry<ConfigResource, ApiError> entry : configResult.response().entrySet()) {
+            if (entry.getValue().isFailure()) {
+                topicErrors.put(entry.getKey().name(), entry.getValue());
+            }
+        }
+        records.addAll(configResult.records());
+
+        // Try to create whatever topics are needed.
+        Map<String, CreatableTopicResult> successes = new HashMap<>();
+        for (CreatableTopic topic : request.topics()) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            ApiError error = createTopic(topic, records, successes);
+            if (error.isFailure()) {
+                topicErrors.put(topic.name(), error);
+            }
+        }
+
+        // Create responses for all topics.
+        CreateTopicsResponseData data = new CreateTopicsResponseData();
+        StringBuilder resultsBuilder = new StringBuilder();
+        String resultsPrefix = "";
+        for (CreatableTopic topic : request.topics()) {
+            ApiError error = topicErrors.get(topic.name());
+            if (error != null) {
+                data.topics().add(new CreatableTopicResult().
+                    setName(topic.name()).
+                    setErrorCode(error.error().code()).
+                    setErrorMessage(error.message()));
+                resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                    append(error.error()).append(" (").append(error.message()).append(")");
+                resultsPrefix = ", ";
+                continue;
+            }
+            CreatableTopicResult result = successes.get(topic.name());
+            data.topics().add(result);
+            resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                append("SUCCESS");
+            resultsPrefix = ", ";
+        }
+        log.info("createTopics result(s): {}", resultsBuilder.toString());
+        return new ControllerResult<>(records, data);
+    }
+
+    private ApiError createTopic(CreatableTopic topic,
+                                 List<ApiMessageAndVersion> records,
+                                 Map<String, CreatableTopicResult> successes) {
+        Map<Integer, PartitionControlInfo> newParts = new HashMap<>();
+        if (!topic.assignments().isEmpty()) {
+            if (topic.replicationFactor() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but replication " +
+                    "factor was not set to -1.");
+            }
+            if (topic.numPartitions() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but numPartitions " +
+                        "was not set to -1.");
+            }
+            for (CreatableReplicaAssignment assignment : topic.assignments()) {
+                if (newParts.containsKey(assignment.partitionIndex())) {
+                    return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                        "Found multiple manual partition assignments for partition " +
+                            assignment.partitionIndex());
+                }
+                HashSet<Integer> brokerIds = new HashSet<>();
+                for (int brokerId : assignment.brokerIds()) {
+                    if (!brokerIds.add(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment specifies the same node " +
+                                "id more than once.");
+                    } else if (!clusterControl.unfenced(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment contains node " + brokerId +
+                                ", but that node is not usable.");
+                    }
+                }
+                int[] replicas = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    replicas[i] = assignment.brokerIds().get(i);
+                }
+                int[] isr = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    isr[i] = assignment.brokerIds().get(i);
+                }
+                newParts.put(assignment.partitionIndex(),
+                    new PartitionControlInfo(replicas, isr, null, null, isr[0], 0, 0));
+            }
+        } else if (topic.replicationFactor() < -1 || topic.replicationFactor() == 0) {
+            return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                "Replication factor was set to an invalid non-positive value.");
+        } else if (!topic.assignments().isEmpty()) {
+            return new ApiError(Errors.INVALID_REQUEST,
+                "Replication factor was not set to -1 but a manual partition " +
+                    "assignment was specified.");
+        } else if (topic.numPartitions() < -1 || topic.numPartitions() == 0) {
+            return new ApiError(Errors.INVALID_PARTITIONS,
+                "Number of partitions was set to an invalid non-positive value.");
+        } else {
+            int numPartitions = topic.numPartitions() == -1 ?
+                defaultNumPartitions : topic.numPartitions();
+            short replicationFactor = topic.replicationFactor() == -1 ?
+                defaultReplicationFactor : topic.replicationFactor();
+            try {
+                List<List<Integer>> replicas = clusterControl.
+                    placeReplicas(numPartitions, replicationFactor);
+                for (int partitionId = 0; partitionId < replicas.size(); partitionId++) {
+                    int[] r = Replicas.toArray(replicas.get(partitionId));
+                    newParts.put(partitionId,
+                        new PartitionControlInfo(r, r, null, null, r[0], 0, 0));
+                }
+            } catch (InvalidReplicationFactorException e) {
+                return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                    "Unable to replicate the partition " + replicationFactor +
+                        " times: " + e.getMessage());
+            }
+        }
+        Uuid topicId = new Uuid(random.nextLong(), random.nextLong());
+        successes.put(topic.name(), new CreatableTopicResult().
+            setName(topic.name()).
+            setTopicId(topicId).
+            setErrorCode((short) 0).
+            setErrorMessage(null).
+            setNumPartitions(newParts.size()).
+            setReplicationFactor((short) newParts.get(0).replicas.length));
+        records.add(new ApiMessageAndVersion(new TopicRecord().
+            setName(topic.name()).
+            setTopicId(topicId), (short) 0));
+        for (Entry<Integer, PartitionControlInfo> partEntry : newParts.entrySet()) {
+            int partitionIndex = partEntry.getKey();
+            PartitionControlInfo info = partEntry.getValue();
+            records.add(new ApiMessageAndVersion(new PartitionRecord().
+                setPartitionId(partitionIndex).
+                setTopicId(topicId).
+                setReplicas(Replicas.toList(info.replicas)).
+                setIsr(Replicas.toList(info.isr)).
+                setRemovingReplicas(null).
+                setAddingReplicas(null).
+                setLeader(info.leader).
+                setLeaderEpoch(info.leaderEpoch).
+                setPartitionEpoch(0), (short) 0));
+        }
+        return ApiError.NONE;
+    }
+
+    static void validateNewTopicNames(Map<String, ApiError> topicErrors,
+                                      CreatableTopicCollection topics) {
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            try {
+                Topic.validate(topic.name());
+            } catch (InvalidTopicException e) {
+                topicErrors.put(topic.name(),
+                    new ApiError(Errors.INVALID_TOPIC_EXCEPTION, e.getMessage()));
+            }
+        }
+    }
+
+    static Map<ConfigResource, Map<String, Entry<OpType, String>>>
+            computeConfigChanges(Map<String, ApiError> topicErrors,
+                                 CreatableTopicCollection topics) {
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges = new HashMap<>();
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            Map<String, Entry<OpType, String>> topicConfigs = new HashMap<>();
+            for (CreateTopicsRequestData.CreateableTopicConfig config : topic.configs()) {
+                topicConfigs.put(config.name(), new SimpleImmutableEntry<>(SET, config.value()));
+            }
+            if (!topicConfigs.isEmpty()) {
+                configChanges.put(new ConfigResource(TOPIC, topic.name()), topicConfigs);
+            }
+        }
+        return configChanges;
+    }
+
+    // VisibleForTesting
+    PartitionControlInfo getPartition(Uuid topicId, int partitionId) {
+        TopicControlInfo topic = topics.get(topicId);
+        if (topic == null) {
+            return null;
+        }
+        return topic.parts.get(partitionId);
+    }
+
+    // VisibleForTesting
+    BrokersToIsrs brokersToIsrs() {
+        return brokersToIsrs;
+    }
+
+    ControllerResult<AlterIsrResponseData> alterIsr(AlterIsrRequestData request) {
+        clusterControl.checkBrokerEpoch(request.brokerId(), request.brokerEpoch());

Review comment:
       This can throw StaleBrokerEpochException. It would be useful for KafkaEventQueue.run() to log the event associated with the exception. 

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##########
@@ -0,0 +1,712 @@
+/*
+ * 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.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.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.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+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.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.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;
+
+
+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(IsrChangeRecord record) {
+            return new PartitionControlInfo(replicas,
+                Replicas.toArray(record.isr()),
+                removingReplicas,
+                addingReplicas,
+                record.leader(),
+                record.leaderEpoch(),
+                record.partitionEpoch());
+        }
+
+        String diff(PartitionControlInfo prev) {
+            StringBuilder builder = new StringBuilder();
+            String prefix = "";
+            if (!Arrays.equals(replicas, prev.replicas)) {
+                builder.append("replicas=").append(Arrays.toString(replicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(isr, prev.isr)) {
+                builder.append(prefix).append("isr=").append(Arrays.toString(isr));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(removingReplicas, prev.removingReplicas)) {
+                builder.append(prefix).append("removingReplicas=").
+                    append(Arrays.toString(removingReplicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(addingReplicas, prev.addingReplicas)) {
+                builder.append(prefix).append("addingReplicas=").
+                    append(Arrays.toString(addingReplicas));
+                prefix = ", ";
+            }
+            if (leader != prev.leader) {
+                builder.append(prefix).append("leader=").append(leader);
+                prefix = ", ";
+            }
+            if (leaderEpoch != prev.leaderEpoch) {
+                builder.append(prefix).append("leaderEpoch=").append(leaderEpoch);
+            }
+            if (partitionEpoch != prev.partitionEpoch) {
+                builder.append(prefix).append("partitionEpoch=").append(partitionEpoch);
+            }
+            return builder.toString();
+        }
+
+        @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();
+        }
+
+        int chooseNewLeader(int[] newIsr, boolean unclean) {
+            for (int i = 0; i < replicas.length; i++) {
+                int replica = replicas[i];
+                if (Replicas.contains(newIsr, replica)) {
+                    return replica;
+                }
+            }
+            if (unclean && replicas.length > 0) {
+                return replicas[0];
+            }
+            return -1;
+        }
+    }
+
+    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(IsrChangeRecord 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);
+        topicInfo.parts.put(record.partitionId(), newPartitionInfo);
+        brokersToIsrs.update(record.topicId(), record.partitionId(),
+            prevPartitionInfo.isr, newPartitionInfo.isr, prevPartitionInfo.leader,
+            newPartitionInfo.leader);
+        log.debug("Applied ISR change record: {}", record.toString());
+    }
+
+    ControllerResult<CreateTopicsResponseData>
+            createTopics(CreateTopicsRequestData request) {
+        Map<String, ApiError> topicErrors = new HashMap<>();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+
+        // Check the topic names.
+        validateNewTopicNames(topicErrors, request.topics());
+
+        // Identify topics that already exist and mark them with the appropriate error
+        request.topics().stream().filter(creatableTopic -> topicsByName.containsKey(creatableTopic.name()))
+                .forEach(t -> topicErrors.put(t.name(), new ApiError(Errors.TOPIC_ALREADY_EXISTS)));
+
+        // Verify that the configurations for the new topics are OK, and figure out what
+        // ConfigRecords should be created.
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges =
+            computeConfigChanges(topicErrors, request.topics());
+        ControllerResult<Map<ConfigResource, ApiError>> configResult =
+            configurationControl.incrementalAlterConfigs(configChanges);
+        for (Entry<ConfigResource, ApiError> entry : configResult.response().entrySet()) {
+            if (entry.getValue().isFailure()) {
+                topicErrors.put(entry.getKey().name(), entry.getValue());
+            }
+        }
+        records.addAll(configResult.records());
+
+        // Try to create whatever topics are needed.
+        Map<String, CreatableTopicResult> successes = new HashMap<>();
+        for (CreatableTopic topic : request.topics()) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            ApiError error = createTopic(topic, records, successes);
+            if (error.isFailure()) {
+                topicErrors.put(topic.name(), error);
+            }
+        }
+
+        // Create responses for all topics.
+        CreateTopicsResponseData data = new CreateTopicsResponseData();
+        StringBuilder resultsBuilder = new StringBuilder();
+        String resultsPrefix = "";
+        for (CreatableTopic topic : request.topics()) {
+            ApiError error = topicErrors.get(topic.name());
+            if (error != null) {
+                data.topics().add(new CreatableTopicResult().
+                    setName(topic.name()).
+                    setErrorCode(error.error().code()).
+                    setErrorMessage(error.message()));
+                resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                    append(error.error()).append(" (").append(error.message()).append(")");
+                resultsPrefix = ", ";
+                continue;
+            }
+            CreatableTopicResult result = successes.get(topic.name());
+            data.topics().add(result);
+            resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                append("SUCCESS");
+            resultsPrefix = ", ";
+        }
+        log.info("createTopics result(s): {}", resultsBuilder.toString());
+        return new ControllerResult<>(records, data);
+    }
+
+    private ApiError createTopic(CreatableTopic topic,
+                                 List<ApiMessageAndVersion> records,
+                                 Map<String, CreatableTopicResult> successes) {
+        Map<Integer, PartitionControlInfo> newParts = new HashMap<>();
+        if (!topic.assignments().isEmpty()) {
+            if (topic.replicationFactor() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but replication " +
+                    "factor was not set to -1.");
+            }
+            if (topic.numPartitions() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but numPartitions " +
+                        "was not set to -1.");
+            }
+            for (CreatableReplicaAssignment assignment : topic.assignments()) {
+                if (newParts.containsKey(assignment.partitionIndex())) {
+                    return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                        "Found multiple manual partition assignments for partition " +
+                            assignment.partitionIndex());
+                }
+                HashSet<Integer> brokerIds = new HashSet<>();
+                for (int brokerId : assignment.brokerIds()) {
+                    if (!brokerIds.add(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment specifies the same node " +
+                                "id more than once.");
+                    } else if (!clusterControl.unfenced(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment contains node " + brokerId +
+                                ", but that node is not usable.");
+                    }
+                }
+                int[] replicas = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    replicas[i] = assignment.brokerIds().get(i);
+                }
+                int[] isr = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    isr[i] = assignment.brokerIds().get(i);
+                }
+                newParts.put(assignment.partitionIndex(),
+                    new PartitionControlInfo(replicas, isr, null, null, isr[0], 0, 0));
+            }
+        } else if (topic.replicationFactor() < -1 || topic.replicationFactor() == 0) {
+            return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                "Replication factor was set to an invalid non-positive value.");
+        } else if (!topic.assignments().isEmpty()) {
+            return new ApiError(Errors.INVALID_REQUEST,
+                "Replication factor was not set to -1 but a manual partition " +
+                    "assignment was specified.");
+        } else if (topic.numPartitions() < -1 || topic.numPartitions() == 0) {
+            return new ApiError(Errors.INVALID_PARTITIONS,
+                "Number of partitions was set to an invalid non-positive value.");
+        } else {
+            int numPartitions = topic.numPartitions() == -1 ?
+                defaultNumPartitions : topic.numPartitions();
+            short replicationFactor = topic.replicationFactor() == -1 ?
+                defaultReplicationFactor : topic.replicationFactor();
+            try {
+                List<List<Integer>> replicas = clusterControl.
+                    placeReplicas(numPartitions, replicationFactor);
+                for (int partitionId = 0; partitionId < replicas.size(); partitionId++) {
+                    int[] r = Replicas.toArray(replicas.get(partitionId));
+                    newParts.put(partitionId,
+                        new PartitionControlInfo(r, r, null, null, r[0], 0, 0));
+                }
+            } catch (InvalidReplicationFactorException e) {
+                return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                    "Unable to replicate the partition " + replicationFactor +
+                        " times: " + e.getMessage());
+            }
+        }
+        Uuid topicId = new Uuid(random.nextLong(), random.nextLong());
+        successes.put(topic.name(), new CreatableTopicResult().
+            setName(topic.name()).
+            setTopicId(topicId).
+            setErrorCode((short) 0).
+            setErrorMessage(null).
+            setNumPartitions(newParts.size()).
+            setReplicationFactor((short) newParts.get(0).replicas.length));
+        records.add(new ApiMessageAndVersion(new TopicRecord().
+            setName(topic.name()).
+            setTopicId(topicId), (short) 0));
+        for (Entry<Integer, PartitionControlInfo> partEntry : newParts.entrySet()) {
+            int partitionIndex = partEntry.getKey();
+            PartitionControlInfo info = partEntry.getValue();
+            records.add(new ApiMessageAndVersion(new PartitionRecord().
+                setPartitionId(partitionIndex).
+                setTopicId(topicId).
+                setReplicas(Replicas.toList(info.replicas)).
+                setIsr(Replicas.toList(info.isr)).
+                setRemovingReplicas(null).
+                setAddingReplicas(null).
+                setLeader(info.leader).
+                setLeaderEpoch(info.leaderEpoch).
+                setPartitionEpoch(0), (short) 0));
+        }
+        return ApiError.NONE;
+    }
+
+    static void validateNewTopicNames(Map<String, ApiError> topicErrors,
+                                      CreatableTopicCollection topics) {
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            try {
+                Topic.validate(topic.name());
+            } catch (InvalidTopicException e) {
+                topicErrors.put(topic.name(),
+                    new ApiError(Errors.INVALID_TOPIC_EXCEPTION, e.getMessage()));
+            }
+        }
+    }
+
+    static Map<ConfigResource, Map<String, Entry<OpType, String>>>
+            computeConfigChanges(Map<String, ApiError> topicErrors,
+                                 CreatableTopicCollection topics) {
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges = new HashMap<>();
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            Map<String, Entry<OpType, String>> topicConfigs = new HashMap<>();
+            for (CreateTopicsRequestData.CreateableTopicConfig config : topic.configs()) {
+                topicConfigs.put(config.name(), new SimpleImmutableEntry<>(SET, config.value()));
+            }
+            if (!topicConfigs.isEmpty()) {
+                configChanges.put(new ConfigResource(TOPIC, topic.name()), topicConfigs);
+            }
+        }
+        return configChanges;
+    }
+
+    // VisibleForTesting
+    PartitionControlInfo getPartition(Uuid topicId, int partitionId) {
+        TopicControlInfo topic = topics.get(topicId);
+        if (topic == null) {
+            return null;
+        }
+        return topic.parts.get(partitionId);
+    }
+
+    // VisibleForTesting
+    BrokersToIsrs brokersToIsrs() {
+        return brokersToIsrs;
+    }
+
+    ControllerResult<AlterIsrResponseData> alterIsr(AlterIsrRequestData request) {
+        clusterControl.checkBrokerEpoch(request.brokerId(), request.brokerEpoch());
+        AlterIsrResponseData response = new AlterIsrResponseData();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        for (AlterIsrRequestData.TopicData topicData : request.topics()) {
+            AlterIsrResponseData.TopicData responseTopicData =
+                new AlterIsrResponseData.TopicData().setName(topicData.name());
+            response.topics().add(responseTopicData);
+            Uuid topicId = topicsByName.get(topicData.name());
+            if (topicId == null || !topics.containsKey(topicId)) {
+                for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()));
+                }
+                continue;
+            }
+            TopicControlInfo topic = topics.get(topicId);
+            for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) {
+                PartitionControlInfo partition = topic.parts.get(partitionData.partitionIndex());
+                if (partition == null) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()));
+                    continue;
+                }
+                if (partitionData.leaderEpoch() != partition.leaderEpoch) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.FENCED_LEADER_EPOCH.code()));
+                    continue;
+                }
+                if (partitionData.currentIsrVersion() != partition.partitionEpoch) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.INVALID_UPDATE_VERSION.code()));
+                    continue;
+                }
+                int[] newIsr = Replicas.toArray(partitionData.newIsr());
+                if (!Replicas.validateIsr(partition.replicas, newIsr)) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.INVALID_REQUEST.code()));
+                }
+                int newLeader, newLeaderEpoch;
+                if (Replicas.contains(newIsr, partition.leader)) {
+                    newLeader = partition.leader;
+                    newLeaderEpoch = partition.leaderEpoch;
+                } else {
+                    newLeader = partition.chooseNewLeader(newIsr, false);
+                    newLeaderEpoch = partition.leaderEpoch + 1;
+                }
+                records.add(new ApiMessageAndVersion(new IsrChangeRecord().
+                    setPartitionId(partitionData.partitionIndex()).
+                    setTopicId(topic.id).
+                    setIsr(partitionData.newIsr()).
+                    setLeader(newLeader).
+                    setLeaderEpoch(newLeaderEpoch).
+                    setPartitionEpoch(partition.partitionEpoch + 1), (short) 0));
+            }
+        }
+        return new ControllerResult<>(records, response);
+    }
+
+    void removeFromIsr(int brokerId, List<ApiMessageAndVersion> records) {

Review comment:
       We probably should name this sth like removeFromIsrAndMaybeChooseLeader.

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##########
@@ -0,0 +1,712 @@
+/*
+ * 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.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.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.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+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.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.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;
+
+
+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(IsrChangeRecord record) {
+            return new PartitionControlInfo(replicas,
+                Replicas.toArray(record.isr()),
+                removingReplicas,
+                addingReplicas,
+                record.leader(),
+                record.leaderEpoch(),
+                record.partitionEpoch());
+        }
+
+        String diff(PartitionControlInfo prev) {
+            StringBuilder builder = new StringBuilder();
+            String prefix = "";
+            if (!Arrays.equals(replicas, prev.replicas)) {
+                builder.append("replicas=").append(Arrays.toString(replicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(isr, prev.isr)) {
+                builder.append(prefix).append("isr=").append(Arrays.toString(isr));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(removingReplicas, prev.removingReplicas)) {
+                builder.append(prefix).append("removingReplicas=").
+                    append(Arrays.toString(removingReplicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(addingReplicas, prev.addingReplicas)) {
+                builder.append(prefix).append("addingReplicas=").
+                    append(Arrays.toString(addingReplicas));
+                prefix = ", ";
+            }
+            if (leader != prev.leader) {
+                builder.append(prefix).append("leader=").append(leader);
+                prefix = ", ";
+            }
+            if (leaderEpoch != prev.leaderEpoch) {
+                builder.append(prefix).append("leaderEpoch=").append(leaderEpoch);
+            }
+            if (partitionEpoch != prev.partitionEpoch) {
+                builder.append(prefix).append("partitionEpoch=").append(partitionEpoch);
+            }
+            return builder.toString();
+        }
+
+        @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();
+        }
+
+        int chooseNewLeader(int[] newIsr, boolean unclean) {
+            for (int i = 0; i < replicas.length; i++) {
+                int replica = replicas[i];
+                if (Replicas.contains(newIsr, replica)) {
+                    return replica;
+                }
+            }
+            if (unclean && replicas.length > 0) {
+                return replicas[0];
+            }
+            return -1;
+        }
+    }
+
+    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(IsrChangeRecord 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);
+        topicInfo.parts.put(record.partitionId(), newPartitionInfo);
+        brokersToIsrs.update(record.topicId(), record.partitionId(),
+            prevPartitionInfo.isr, newPartitionInfo.isr, prevPartitionInfo.leader,
+            newPartitionInfo.leader);
+        log.debug("Applied ISR change record: {}", record.toString());
+    }
+
+    ControllerResult<CreateTopicsResponseData>
+            createTopics(CreateTopicsRequestData request) {
+        Map<String, ApiError> topicErrors = new HashMap<>();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+
+        // Check the topic names.
+        validateNewTopicNames(topicErrors, request.topics());
+
+        // Identify topics that already exist and mark them with the appropriate error
+        request.topics().stream().filter(creatableTopic -> topicsByName.containsKey(creatableTopic.name()))
+                .forEach(t -> topicErrors.put(t.name(), new ApiError(Errors.TOPIC_ALREADY_EXISTS)));
+
+        // Verify that the configurations for the new topics are OK, and figure out what
+        // ConfigRecords should be created.
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges =
+            computeConfigChanges(topicErrors, request.topics());
+        ControllerResult<Map<ConfigResource, ApiError>> configResult =
+            configurationControl.incrementalAlterConfigs(configChanges);
+        for (Entry<ConfigResource, ApiError> entry : configResult.response().entrySet()) {
+            if (entry.getValue().isFailure()) {
+                topicErrors.put(entry.getKey().name(), entry.getValue());
+            }
+        }
+        records.addAll(configResult.records());
+
+        // Try to create whatever topics are needed.
+        Map<String, CreatableTopicResult> successes = new HashMap<>();
+        for (CreatableTopic topic : request.topics()) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            ApiError error = createTopic(topic, records, successes);
+            if (error.isFailure()) {
+                topicErrors.put(topic.name(), error);
+            }
+        }
+
+        // Create responses for all topics.
+        CreateTopicsResponseData data = new CreateTopicsResponseData();
+        StringBuilder resultsBuilder = new StringBuilder();
+        String resultsPrefix = "";
+        for (CreatableTopic topic : request.topics()) {
+            ApiError error = topicErrors.get(topic.name());
+            if (error != null) {
+                data.topics().add(new CreatableTopicResult().
+                    setName(topic.name()).
+                    setErrorCode(error.error().code()).
+                    setErrorMessage(error.message()));
+                resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                    append(error.error()).append(" (").append(error.message()).append(")");
+                resultsPrefix = ", ";
+                continue;
+            }
+            CreatableTopicResult result = successes.get(topic.name());
+            data.topics().add(result);
+            resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                append("SUCCESS");
+            resultsPrefix = ", ";
+        }
+        log.info("createTopics result(s): {}", resultsBuilder.toString());
+        return new ControllerResult<>(records, data);
+    }
+
+    private ApiError createTopic(CreatableTopic topic,
+                                 List<ApiMessageAndVersion> records,
+                                 Map<String, CreatableTopicResult> successes) {
+        Map<Integer, PartitionControlInfo> newParts = new HashMap<>();
+        if (!topic.assignments().isEmpty()) {
+            if (topic.replicationFactor() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but replication " +
+                    "factor was not set to -1.");
+            }
+            if (topic.numPartitions() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but numPartitions " +
+                        "was not set to -1.");
+            }
+            for (CreatableReplicaAssignment assignment : topic.assignments()) {
+                if (newParts.containsKey(assignment.partitionIndex())) {
+                    return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                        "Found multiple manual partition assignments for partition " +
+                            assignment.partitionIndex());
+                }
+                HashSet<Integer> brokerIds = new HashSet<>();
+                for (int brokerId : assignment.brokerIds()) {
+                    if (!brokerIds.add(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment specifies the same node " +
+                                "id more than once.");
+                    } else if (!clusterControl.unfenced(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment contains node " + brokerId +
+                                ", but that node is not usable.");
+                    }
+                }
+                int[] replicas = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    replicas[i] = assignment.brokerIds().get(i);
+                }
+                int[] isr = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    isr[i] = assignment.brokerIds().get(i);
+                }
+                newParts.put(assignment.partitionIndex(),
+                    new PartitionControlInfo(replicas, isr, null, null, isr[0], 0, 0));
+            }
+        } else if (topic.replicationFactor() < -1 || topic.replicationFactor() == 0) {
+            return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                "Replication factor was set to an invalid non-positive value.");
+        } else if (!topic.assignments().isEmpty()) {
+            return new ApiError(Errors.INVALID_REQUEST,
+                "Replication factor was not set to -1 but a manual partition " +
+                    "assignment was specified.");
+        } else if (topic.numPartitions() < -1 || topic.numPartitions() == 0) {
+            return new ApiError(Errors.INVALID_PARTITIONS,
+                "Number of partitions was set to an invalid non-positive value.");
+        } else {
+            int numPartitions = topic.numPartitions() == -1 ?
+                defaultNumPartitions : topic.numPartitions();
+            short replicationFactor = topic.replicationFactor() == -1 ?
+                defaultReplicationFactor : topic.replicationFactor();
+            try {
+                List<List<Integer>> replicas = clusterControl.
+                    placeReplicas(numPartitions, replicationFactor);
+                for (int partitionId = 0; partitionId < replicas.size(); partitionId++) {
+                    int[] r = Replicas.toArray(replicas.get(partitionId));
+                    newParts.put(partitionId,
+                        new PartitionControlInfo(r, r, null, null, r[0], 0, 0));
+                }
+            } catch (InvalidReplicationFactorException e) {
+                return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                    "Unable to replicate the partition " + replicationFactor +
+                        " times: " + e.getMessage());
+            }
+        }
+        Uuid topicId = new Uuid(random.nextLong(), random.nextLong());
+        successes.put(topic.name(), new CreatableTopicResult().
+            setName(topic.name()).
+            setTopicId(topicId).
+            setErrorCode((short) 0).
+            setErrorMessage(null).
+            setNumPartitions(newParts.size()).
+            setReplicationFactor((short) newParts.get(0).replicas.length));
+        records.add(new ApiMessageAndVersion(new TopicRecord().
+            setName(topic.name()).
+            setTopicId(topicId), (short) 0));
+        for (Entry<Integer, PartitionControlInfo> partEntry : newParts.entrySet()) {
+            int partitionIndex = partEntry.getKey();
+            PartitionControlInfo info = partEntry.getValue();
+            records.add(new ApiMessageAndVersion(new PartitionRecord().
+                setPartitionId(partitionIndex).
+                setTopicId(topicId).
+                setReplicas(Replicas.toList(info.replicas)).
+                setIsr(Replicas.toList(info.isr)).
+                setRemovingReplicas(null).
+                setAddingReplicas(null).
+                setLeader(info.leader).
+                setLeaderEpoch(info.leaderEpoch).
+                setPartitionEpoch(0), (short) 0));
+        }
+        return ApiError.NONE;
+    }
+
+    static void validateNewTopicNames(Map<String, ApiError> topicErrors,
+                                      CreatableTopicCollection topics) {
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            try {
+                Topic.validate(topic.name());
+            } catch (InvalidTopicException e) {
+                topicErrors.put(topic.name(),
+                    new ApiError(Errors.INVALID_TOPIC_EXCEPTION, e.getMessage()));
+            }
+        }
+    }
+
+    static Map<ConfigResource, Map<String, Entry<OpType, String>>>
+            computeConfigChanges(Map<String, ApiError> topicErrors,
+                                 CreatableTopicCollection topics) {
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges = new HashMap<>();
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            Map<String, Entry<OpType, String>> topicConfigs = new HashMap<>();
+            for (CreateTopicsRequestData.CreateableTopicConfig config : topic.configs()) {
+                topicConfigs.put(config.name(), new SimpleImmutableEntry<>(SET, config.value()));
+            }
+            if (!topicConfigs.isEmpty()) {
+                configChanges.put(new ConfigResource(TOPIC, topic.name()), topicConfigs);
+            }
+        }
+        return configChanges;
+    }
+
+    // VisibleForTesting
+    PartitionControlInfo getPartition(Uuid topicId, int partitionId) {
+        TopicControlInfo topic = topics.get(topicId);
+        if (topic == null) {
+            return null;
+        }
+        return topic.parts.get(partitionId);
+    }
+
+    // VisibleForTesting
+    BrokersToIsrs brokersToIsrs() {
+        return brokersToIsrs;
+    }
+
+    ControllerResult<AlterIsrResponseData> alterIsr(AlterIsrRequestData request) {
+        clusterControl.checkBrokerEpoch(request.brokerId(), request.brokerEpoch());
+        AlterIsrResponseData response = new AlterIsrResponseData();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        for (AlterIsrRequestData.TopicData topicData : request.topics()) {
+            AlterIsrResponseData.TopicData responseTopicData =
+                new AlterIsrResponseData.TopicData().setName(topicData.name());
+            response.topics().add(responseTopicData);
+            Uuid topicId = topicsByName.get(topicData.name());
+            if (topicId == null || !topics.containsKey(topicId)) {
+                for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()));
+                }
+                continue;
+            }
+            TopicControlInfo topic = topics.get(topicId);
+            for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) {
+                PartitionControlInfo partition = topic.parts.get(partitionData.partitionIndex());
+                if (partition == null) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()));
+                    continue;
+                }
+                if (partitionData.leaderEpoch() != partition.leaderEpoch) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.FENCED_LEADER_EPOCH.code()));
+                    continue;
+                }
+                if (partitionData.currentIsrVersion() != partition.partitionEpoch) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.INVALID_UPDATE_VERSION.code()));
+                    continue;
+                }
+                int[] newIsr = Replicas.toArray(partitionData.newIsr());
+                if (!Replicas.validateIsr(partition.replicas, newIsr)) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.INVALID_REQUEST.code()));
+                }
+                int newLeader, newLeaderEpoch;
+                if (Replicas.contains(newIsr, partition.leader)) {
+                    newLeader = partition.leader;
+                    newLeaderEpoch = partition.leaderEpoch;
+                } else {
+                    newLeader = partition.chooseNewLeader(newIsr, false);
+                    newLeaderEpoch = partition.leaderEpoch + 1;
+                }
+                records.add(new ApiMessageAndVersion(new IsrChangeRecord().
+                    setPartitionId(partitionData.partitionIndex()).
+                    setTopicId(topic.id).
+                    setIsr(partitionData.newIsr()).
+                    setLeader(newLeader).
+                    setLeaderEpoch(newLeaderEpoch).
+                    setPartitionEpoch(partition.partitionEpoch + 1), (short) 0));
+            }
+        }
+        return new ControllerResult<>(records, response);
+    }
+
+    void removeFromIsr(int brokerId, List<ApiMessageAndVersion> records) {
+        Iterator<TopicPartition> iterator = brokersToIsrs.iterator(brokerId, false);
+        while (iterator.hasNext()) {
+            TopicPartition topicPartition = iterator.next();
+            TopicControlInfo topic = topics.get(topicPartition.topicId());
+            if (topic == null) {
+                throw new RuntimeException("Topic ID " + topicPartition.topicId() + " existed in " +
+                    "isrMembers, but not in the topics map.");
+            }
+            PartitionControlInfo partition = topic.parts.get(topicPartition.partitionId());
+            if (partition == null) {
+                throw new RuntimeException("Partition " + topicPartition +
+                    " existed in isrMembers, but not in the partitions map.");
+            }
+            int[] newIsr = Replicas.copyWithout(partition.isr, brokerId);
+            int newLeader, newLeaderEpoch;
+            if (partition.leader == brokerId) {
+                newLeader = partition.chooseNewLeader(newIsr, false);
+                newLeaderEpoch = partition.leaderEpoch + 1;
+            } else {
+                newLeader = partition.leader;
+                newLeaderEpoch = partition.leaderEpoch;
+            }
+            records.add(new ApiMessageAndVersion(new IsrChangeRecord().
+                setPartitionId(topicPartition.partitionId()).
+                setTopicId(topic.id).
+                setIsr(Replicas.toList(newIsr)).
+                setLeader(newLeader).
+                setLeaderEpoch(newLeaderEpoch).
+                setPartitionEpoch(partition.partitionEpoch + 1), (short) 0));
+        }
+    }
+
+    List<ApiMessageAndVersion> removeLeaderships(int brokerId) {
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        Iterator<TopicPartition> iterator = brokersToIsrs.iterator(brokerId, true);
+        while (iterator.hasNext()) {
+            TopicPartition topicPartition = iterator.next();
+            TopicControlInfo topic = topics.get(topicPartition.topicId());
+            if (topic == null) {
+                throw new RuntimeException("Topic ID " + topicPartition.topicId() + " existed in " +
+                    "isrMembers, but not in the topics map.");
+            }
+            PartitionControlInfo partition = topic.parts.get(topicPartition.partitionId());
+            if (partition == null) {
+                throw new RuntimeException("Partition " + topicPartition +
+                    " existed in isrMembers, but not in the partitions map.");
+            }
+            int[] isrWithoutCurLeader = Replicas.copyWithout(partition.isr, brokerId);
+            int newLeader = partition.chooseNewLeader(isrWithoutCurLeader, false);

Review comment:
       Hmm, not all partitions with ISR containing the shutting down need to change the leader.

##########
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() {
+        int latestController = logManager.leader().nodeId();
+        if (latestController < 0) {
+            return new NotControllerException("No controller appears to be active.");
+        } else {
+            return new NotControllerException(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX +
+                latestController);
+        }
+    }
+
+    public static int exceptionToApparentController(NotControllerException e) {
+        if (e.getMessage().startsWith(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX)) {
+            return Integer.parseInt(e.getMessage().substring(
+                ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX.length()));
+        } else {
+            return -1;
+        }
+    }
+
+    private void handleEventEnd(String name, long startProcessingTimeNs) {
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs;
+        log.debug("Processed {} in {} us", name,

Review comment:
       Do we also need to update the metric for processing time?

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ControllerPurgatory.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.TreeMap;
+
+/**
+ * The purgatory which holds events that have been started, but not yet completed.
+ * We wait for the high water mark of the metadata log to advance before completing
+ * them.
+ */
+class ControllerPurgatory {
+    /**
+     * A map from log offsets to events.  Each event will be completed once the log
+     * advances past its offset.
+     */
+    private final TreeMap<Long, List<DeferredEvent>> pending = new TreeMap<>();
+
+    /**
+     * Complete some purgatory entries.
+     *
+     * @param offset        The offset which the high water mark has advanced to.
+     */
+    void completeUpTo(long offset) {
+        Iterator<Entry<Long, List<DeferredEvent>>> iter = pending.entrySet().iterator();
+        while (iter.hasNext()) {
+            Entry<Long, List<DeferredEvent>> entry = iter.next();
+            if (entry.getKey() > offset) {
+                break;
+            }
+            for (DeferredEvent event : entry.getValue()) {
+                event.complete(null);
+            }
+            iter.remove();
+        }
+    }
+
+    /**
+     * Fail all the pending purgatory entries.
+     *
+     * @param exception     The exception to fail the entries with.
+     */
+    void failAll(Exception exception) {
+        Iterator<Entry<Long, List<DeferredEvent>>> iter = pending.entrySet().iterator();
+        while (iter.hasNext()) {
+            Entry<Long, List<DeferredEvent>> entry = iter.next();
+            for (DeferredEvent event : entry.getValue()) {
+                event.complete(exception);
+            }
+            iter.remove();
+        }
+    }
+
+    /**
+     * Add a new purgatory event.
+     *
+     * @param offset        The offset to add the new event at.
+     * @param event         The new event.
+     */
+    void add(long offset, DeferredEvent event) {
+        if (!pending.isEmpty()) {
+            long lastKey = pending.lastKey();
+            if (offset < lastKey) {
+                throw new RuntimeException("There is already a purgatory event with " +
+                    "offset " + offset + ".  We should not add one with an offset which " +
+                    "is lower than or equal to that.");

Review comment:
       Could we log lastKey too?

##########
File path: metadata/src/main/java/org/apache/kafka/controller/Controller.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+import org.apache.kafka.common.config.ConfigResource;
+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.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+
+public interface Controller extends AutoCloseable {
+    /**
+     * Change partition ISRs.
+     *
+     * @param request       The AlterIsrRequest data.
+     *
+     * @return              A future yielding the response.
+     */
+    CompletableFuture<AlterIsrResponseData> alterIsr(AlterIsrRequestData request);
+
+    /**
+     * Create a batch of topics.
+     *
+     * @param request       The CreateTopicsRequest data.
+     *
+     * @return              A future yielding the response.
+     */
+    CompletableFuture<CreateTopicsResponseData>
+        createTopics(CreateTopicsRequestData request);
+
+    /**
+     * Decommission a broker.
+     *
+     * @param brokerId      The broker id to decommission.
+     *
+     * @return              A future that is completed successfully hwne the broker is
+     *                      decommissioned, or if it is not registered in the first place.
+     */
+    CompletableFuture<Void> decommissionBroker(int brokerId);
+
+    /**
+     * Describe the current configuration of various resources.
+     *
+     * @param resources     A map from resources to the collection of config keys that we
+     *                      want to describe for each.  If the collection is empty, then
+     *                      all configuration keys will be described.
+     *
+     * @return
+     */
+    CompletableFuture<Map<ConfigResource, ResultOrError<Map<String, String>>>>
+        describeConfigs(Map<ConfigResource, Collection<String>> resources);
+
+    /**
+     * Elect new partition leaders.
+     *
+     * @param request       The request.
+     *
+     * @return              A future yielding the elect leaders response.
+     */
+    CompletableFuture<ElectLeadersResponseData> electLeaders(ElectLeadersRequestData request);
+
+    /**
+     * Get the current finalized feature ranges for each feature.
+     *
+     * @return              A future yielding the feature ranges.
+     */
+    CompletableFuture<FeatureManager.FinalizedFeaturesAndEpoch> finalizedFeatures();
+
+    /**
+     * Perform some incremental configuration changes.
+     *
+     * @param configChanges The changes.
+     * @param validateOnly  True if we should validate the changes but not apply them.
+     *
+     * @return              A future yielding a map from partitions to error results.
+     */
+    CompletableFuture<Map<ConfigResource, ApiError>> incrementalAlterConfigs(
+        Map<ConfigResource, Map<String, Map.Entry<AlterConfigOp.OpType, String>>> configChanges,
+        boolean validateOnly);
+
+    /**
+     * Perform some configuration changes using the legacy API.
+     *
+     * @param newConfigs    The new configuration maps to apply.
+     * @param validateOnly  True if we should validate the changes but not apply them.
+     *
+     * @return              A future yielding a map from partitions to error results.

Review comment:
       The returned map is not keyed on partition.

##########
File path: metadata/src/main/java/org/apache/kafka/metalog/LocalLogManager.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metalog;
+
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+
+
+/**
+ * The LocalLogManager is a test implementation that relies on the contents of memory.
+ */
+public final class LocalLogManager implements MetaLogManager, AutoCloseable {
+    interface LocalBatch {
+        int size();
+    }
+
+    static class LeaderChangeBatch implements LocalBatch {
+        private final MetaLogLeader newLeader;
+
+        LeaderChangeBatch(MetaLogLeader newLeader) {
+            this.newLeader = newLeader;
+        }
+
+        @Override
+        public int size() {
+            return 1;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (!(o instanceof LeaderChangeBatch)) return false;
+            LeaderChangeBatch other = (LeaderChangeBatch) o;
+            if (!other.newLeader.equals(newLeader)) return false;
+            return true;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(newLeader);
+        }
+
+        @Override
+        public String toString() {
+            return "LeaderChangeBatch(newLeader=" + newLeader + ")";
+        }
+    }
+
+    static class LocalRecordBatch implements LocalBatch {
+        private final List<ApiMessage> records;
+
+        LocalRecordBatch(List<ApiMessage> records) {
+            this.records = records;
+        }
+
+        @Override
+        public int size() {
+            return records.size();
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (!(o instanceof LocalRecordBatch)) return false;
+            LocalRecordBatch other = (LocalRecordBatch) o;
+            if (!other.records.equals(records)) return false;
+            return true;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(records);
+        }
+
+        @Override
+        public String toString() {
+            return "LocalRecordBatch(records=" + records + ")";
+        }
+    }
+
+    public static class SharedLogData {
+        private final Logger log = LoggerFactory.getLogger(SharedLogData.class);
+        private final HashMap<Integer, LocalLogManager> logManagers = new HashMap<>();
+        private final TreeMap<Long, LocalBatch> batches = new TreeMap<>();

Review comment:
       Could we add a bit comment explaining logManagers and batches?

##########
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() {
+        int latestController = logManager.leader().nodeId();
+        if (latestController < 0) {
+            return new NotControllerException("No controller appears to be active.");
+        } else {
+            return new NotControllerException(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX +
+                latestController);
+        }
+    }
+
+    public static int exceptionToApparentController(NotControllerException e) {
+        if (e.getMessage().startsWith(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX)) {
+            return Integer.parseInt(e.getMessage().substring(
+                ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX.length()));
+        } else {
+            return -1;
+        }
+    }
+
+    private void handleEventEnd(String name, long startProcessingTimeNs) {
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs;
+        log.debug("Processed {} in {} us", name,
+            TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS));
+    }
+
+    private Throwable handleEventException(String name,
+                                           Optional<Long> startProcessingTimeNs,
+                                           Throwable exception) {
+        if (!startProcessingTimeNs.isPresent()) {
+            log.debug("unable to start processing {} because of {}.", name,
+                exception.getClass().getSimpleName());
+            if (exception instanceof ApiException) {
+                return exception;
+            } else {
+                return new UnknownServerException(exception);
+            }
+        }
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs.get();
+        long deltaUs = TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS);
+        if (exception instanceof ApiException) {
+            log.debug("{}: failed with {} in {} us", name,
+                exception.getClass().getSimpleName(), deltaUs);
+            return exception;
+        }
+        log.warn("{}: failed with unknown server exception {} at epoch {} in {} us.  " +
+            "Reverting to last committed offset {}.",
+            this, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs,
+            lastCommittedOffset, exception);
+        renounce();
+        return new UnknownServerException(exception);
+    }
+
+    /**
+     * A controller event for handling internal state changes, such as Raft inputs.
+     */
+    class ControlEvent implements EventQueue.Event {
+        private final String name;
+        private final Runnable handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControlEvent(String name, Runnable handler) {
+            this.name = name;
+            this.handler = handler;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            log.debug("Executing {}.", this);
+            handler.run();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            handleEventException(name, startProcessingTimeNs, exception);
+        }
+
+        @Override
+        public String toString() {
+            return name;
+        }
+    }
+
+    private void appendControlEvent(String name, Runnable handler) {
+        ControlEvent event = new ControlEvent(name, handler);
+        queue.append(event);
+    }
+
+    /**
+     * A controller event that reads the committed internal state in order to expose it
+     * to an API.
+     */
+    class ControllerReadEvent<T> implements EventQueue.Event {
+        private final String name;
+        private final CompletableFuture<T> future;
+        private final Supplier<T> handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControllerReadEvent(String name, Supplier<T> handler) {
+            this.name = name;
+            this.future = new CompletableFuture<T>();
+            this.handler = handler;
+        }
+
+        CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            T value = handler.get();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+            future.complete(value);
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            future.completeExceptionally(
+                handleEventException(name, startProcessingTimeNs, exception));
+        }
+
+        @Override
+        public String toString() {
+            return name + "(" + System.identityHashCode(this) + ")";
+        }
+    }
+
+    // VisibleForTesting
+    ReplicationControlManager replicationControl() {
+        return replicationControl;
+    }
+
+    // VisibleForTesting
+    <T> CompletableFuture<T> appendReadEvent(String name, Supplier<T> handler) {
+        ControllerReadEvent<T> event = new ControllerReadEvent<T>(name, handler);
+        queue.append(event);
+        return event.future();
+    }
+
+    interface ControllerWriteOperation<T> {
+        /**
+         * Generate the metadata records needed to implement this controller write
+         * operation.  In general, this operation should not modify the "hard state" of
+         * the controller.  That modification will happen later on, when we replay the
+         * records generated by this function.
+         *
+         * There are cases where this function modifies the "soft state" of the
+         * controller.  Mainly, this happens when we process cluster heartbeats.
+         *
+         * This function also generates an RPC result.  In general, if the RPC resulted in
+         * an error, the RPC result will be an error, and the generated record list will
+         * be empty.  This would happen if we tried to create a topic with incorrect
+         * parameters, for example.  Of course, partial errors are possible for batch
+         * operations.
+         *
+         * @return              A tuple containing a list of records, and an RPC result.

Review comment:
       The return type is not a tuple.

##########
File path: metadata/src/main/java/org/apache/kafka/metalog/LocalLogManager.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metalog;
+
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+
+
+/**
+ * The LocalLogManager is a test implementation that relies on the contents of memory.
+ */
+public final class LocalLogManager implements MetaLogManager, AutoCloseable {
+    interface LocalBatch {
+        int size();
+    }
+
+    static class LeaderChangeBatch implements LocalBatch {
+        private final MetaLogLeader newLeader;
+
+        LeaderChangeBatch(MetaLogLeader newLeader) {
+            this.newLeader = newLeader;
+        }
+
+        @Override
+        public int size() {
+            return 1;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (!(o instanceof LeaderChangeBatch)) return false;
+            LeaderChangeBatch other = (LeaderChangeBatch) o;
+            if (!other.newLeader.equals(newLeader)) return false;
+            return true;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(newLeader);
+        }
+
+        @Override
+        public String toString() {
+            return "LeaderChangeBatch(newLeader=" + newLeader + ")";
+        }
+    }
+
+    static class LocalRecordBatch implements LocalBatch {
+        private final List<ApiMessage> records;
+
+        LocalRecordBatch(List<ApiMessage> records) {
+            this.records = records;
+        }
+
+        @Override
+        public int size() {
+            return records.size();
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (!(o instanceof LocalRecordBatch)) return false;
+            LocalRecordBatch other = (LocalRecordBatch) o;
+            if (!other.records.equals(records)) return false;
+            return true;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(records);
+        }
+
+        @Override
+        public String toString() {
+            return "LocalRecordBatch(records=" + records + ")";
+        }
+    }
+
+    public static class SharedLogData {
+        private final Logger log = LoggerFactory.getLogger(SharedLogData.class);
+        private final HashMap<Integer, LocalLogManager> logManagers = new HashMap<>();
+        private final TreeMap<Long, LocalBatch> batches = new TreeMap<>();
+        private MetaLogLeader leader = new MetaLogLeader(-1, -1);
+        private long prevOffset = -1;
+
+        synchronized void registerLogManager(LocalLogManager logManager) {
+            if (logManagers.put(logManager.nodeId(), logManager) != null) {
+                throw new RuntimeException("Can't have multiple LocalLogManagers " +
+                    "with id " + logManager.nodeId());
+            }
+            electLeaderIfNeeded();
+        }
+
+        synchronized void unregisterLogManager(LocalLogManager logManager) {
+            if (!logManagers.remove(logManager.nodeId(), logManager)) {
+                throw new RuntimeException("Log manager " + logManager.nodeId() +
+                    " was not found.");
+            }
+        }
+
+        synchronized long tryAppend(int nodeId, long epoch, LocalBatch batch) {
+            if (epoch != leader.epoch()) {
+                log.trace("tryAppend(nodeId={}, epoch={}): the provided epoch does not " +
+                    "match the current leader epoch of {}.", nodeId, epoch, leader.epoch());
+                return Long.MAX_VALUE;
+            }
+            if (nodeId != leader.nodeId()) {
+                log.trace("tryAppend(nodeId={}, epoch={}): the given node id does not " +
+                    "match the current leader id of {}.", nodeId, leader.nodeId());
+                return Long.MAX_VALUE;
+            }
+            log.trace("tryAppend(nodeId={}): appending {}.", nodeId, batch);
+            long offset = append(batch);
+            electLeaderIfNeeded();
+            return offset;
+        }
+
+        synchronized long append(LocalBatch batch) {
+            prevOffset += batch.size();
+            log.debug("append(batch={}, prevOffset={})", batch, prevOffset);
+            batches.put(prevOffset, batch);
+            if (batch instanceof LeaderChangeBatch) {
+                LeaderChangeBatch leaderChangeBatch = (LeaderChangeBatch) batch;
+                leader = leaderChangeBatch.newLeader;
+            }
+            for (LocalLogManager logManager : logManagers.values()) {
+                logManager.scheduleLogCheck();
+            }
+            return prevOffset;
+        }
+
+        synchronized void electLeaderIfNeeded() {
+            if (leader.nodeId() != -1 || logManagers.isEmpty()) {
+                return;
+            }
+            int nextLeaderIndex = ThreadLocalRandom.current().nextInt(logManagers.size());
+            Iterator<Integer> iter = logManagers.keySet().iterator();
+            Integer nextLeaderNode = null;
+            for (int i = 0; i <= nextLeaderIndex; i++) {
+                nextLeaderNode = iter.next();
+            }
+            MetaLogLeader newLeader = new MetaLogLeader(nextLeaderNode, leader.epoch() + 1);
+            log.info("Elected new leader: {}.", newLeader);
+            append(new LeaderChangeBatch(newLeader));
+        }
+
+        synchronized Entry<Long, LocalBatch> nextBatch(long offset) {
+            Entry<Long, LocalBatch> entry = batches.higherEntry(offset);
+            if (entry == null) {
+                return null;
+            }
+            return new SimpleImmutableEntry<>(entry.getKey(), entry.getValue());
+        }
+    }
+
+    private static class MetaLogListenerData {
+        private long offset = -1;
+        private final MetaLogListener listener;
+
+        MetaLogListenerData(MetaLogListener listener) {
+            this.listener = listener;
+        }
+    }
+
+    private final Logger log;
+
+    private final int nodeId;
+
+    private final SharedLogData shared;
+
+    private final EventQueue eventQueue;
+
+    private boolean initialized = false;
+
+    private boolean shutdown = false;
+
+    private long maxReadOffset = Long.MAX_VALUE;

Review comment:
       Could we add a comment for maxReadOffset? Is it the committed offset?

##########
File path: metadata/src/main/java/org/apache/kafka/metalog/LocalLogManager.java
##########
@@ -0,0 +1,379 @@
+/*
+ * 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.metalog;
+
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+
+
+/**
+ * The LocalLogManager is a test implementation that relies on the contents of memory.

Review comment:
       Is this used only for test?

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+    class ReadyBrokersFuture {
+        private final CompletableFuture<Void> future;
+        private final int minBrokers;
+
+        ReadyBrokersFuture(CompletableFuture<Void> future, int minBrokers) {
+            this.future = future;
+            this.minBrokers = minBrokers;
+        }
+
+        boolean check() {
+            int numUnfenced = 0;
+            for (BrokerRegistration registration : brokerRegistrations.values()) {
+                if (!registration.fenced()) {
+                    numUnfenced++;
+                }
+                if (numUnfenced >= minBrokers) {
+                    return true;
+                }
+            }
+            return false;
+        }
+    }
+
+    /**
+     * The SLF4J log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The SLF4J log object.
+     */
+    private final Logger log;
+
+    /**
+     * The Kafka clock object to use.
+     */
+    private final Time time;
+
+    /**
+     * How long sessions should last, in nanoseconds.
+     */
+    private final long sessionTimeoutNs;
+
+    /**
+     * The replica placement policy to use.
+     */
+    private final ReplicaPlacementPolicy placementPolicy;
+
+    /**
+     * Maps broker IDs to broker registrations.
+     */
+    private final TimelineHashMap<Integer, BrokerRegistration> brokerRegistrations;
+
+    /**
+     * The broker heartbeat manager, or null if this controller is on standby.
+     */
+    private BrokerHeartbeatManager heartbeatManager;
+
+    /**
+     * A future which is completed as soon as we have the given number of brokers
+     * ready.
+     */
+    private Optional<ReadyBrokersFuture> readyBrokersFuture;
+
+    ClusterControlManager(LogContext logContext,
+                          Time time,
+                          SnapshotRegistry snapshotRegistry,
+                          long sessionTimeoutNs,
+                          ReplicaPlacementPolicy placementPolicy) {
+        this.logContext = logContext;
+        this.log = logContext.logger(ClusterControlManager.class);
+        this.time = time;
+        this.sessionTimeoutNs = sessionTimeoutNs;
+        this.placementPolicy = placementPolicy;
+        this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.heartbeatManager = null;
+        this.readyBrokersFuture = Optional.empty();
+    }
+
+    /**
+     * Transition this ClusterControlManager to active.
+     */
+    public void activate() {
+        heartbeatManager = new BrokerHeartbeatManager(logContext, time, sessionTimeoutNs);
+        for (BrokerRegistration registration : brokerRegistrations.values()) {
+            heartbeatManager.touch(registration.id(), registration.fenced(), -1);
+        }
+    }
+
+    /**
+     * Transition this ClusterControlManager to standby.
+     */
+    public void deactivate() {
+        heartbeatManager = null;
+    }
+
+    // VisibleForTesting
+    Map<Integer, BrokerRegistration> brokerRegistrations() {
+        return brokerRegistrations;
+    }
+
+    /**
+     * Process an incoming broker registration request.
+     */
+    public ControllerResult<BrokerRegistrationReply> registerBroker(
+            BrokerRegistrationRequestData request, long brokerEpoch,
+            FeatureManager.FinalizedFeaturesAndEpoch finalizedFeaturesAndEpoch) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        int brokerId = request.brokerId();
+        BrokerRegistration existing = brokerRegistrations.get(brokerId);
+        if (existing != null) {
+            if (heartbeatManager.hasValidSession(brokerId)) {
+                if (!existing.incarnationId().equals(request.incarnationId())) {
+                    throw new DuplicateBrokerRegistrationException("Another broker is " +
+                        "registered with that broker id.");
+                }
+            } else {
+                if (!existing.incarnationId().equals(request.incarnationId())) {
+                    // Remove any existing session for the old broker incarnation.
+                    heartbeatManager.remove(brokerId);
+                    existing = null;
+                }
+            }
+        }
+
+        RegisterBrokerRecord record = new RegisterBrokerRecord().setBrokerId(brokerId).
+            setIncarnationId(request.incarnationId()).
+            setBrokerEpoch(brokerEpoch).
+            setRack(request.rack());
+        for (BrokerRegistrationRequestData.Listener listener : request.listeners()) {
+            record.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint().
+                setHost(listener.host()).
+                setName(listener.name()).
+                setPort(listener.port()).
+                setSecurityProtocol(listener.securityProtocol()));
+        }
+        for (BrokerRegistrationRequestData.Feature feature : request.features()) {
+            VersionRange supported = finalizedFeaturesAndEpoch.finalizedFeatures().
+                getOrDefault(feature.name(), VersionRange.ALL);
+            if (!supported.contains(new VersionRange(feature.minSupportedVersion(),
+                    feature.maxSupportedVersion()))) {
+                throw new UnsupportedVersionException("Unable to register because " +
+                    "the broker has an unsupported version of " + feature.name());
+            }
+            record.features().add(new RegisterBrokerRecord.BrokerFeature().
+                setName(feature.name()).
+                setMinSupportedVersion(feature.minSupportedVersion()).
+                setMaxSupportedVersion(feature.maxSupportedVersion()));
+        }
+
+        if (existing == null) {
+            heartbeatManager.touch(brokerId, true, -1);
+        } else {
+            heartbeatManager.touch(brokerId, existing.fenced(), -1);
+        }
+
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        records.add(new ApiMessageAndVersion(record, (short) 0));
+        return new ControllerResult<>(records, new BrokerRegistrationReply(brokerEpoch));
+    }
+
+    public ControllerResult<Void> decommissionBroker(int brokerId) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        BrokerRegistration existing = brokerRegistrations.get(brokerId);
+        if (existing == null) {
+            return new ControllerResult<>(new ArrayList<>(), null);

Review comment:
       This is in response to a heartBeat request. So, it should generate a response in ControllerResult?

##########
File path: metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##########
@@ -0,0 +1,920 @@
+/*
+ * 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() {
+        int latestController = logManager.leader().nodeId();
+        if (latestController < 0) {
+            return new NotControllerException("No controller appears to be active.");
+        } else {
+            return new NotControllerException(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX +
+                latestController);
+        }
+    }
+
+    public static int exceptionToApparentController(NotControllerException e) {
+        if (e.getMessage().startsWith(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX)) {
+            return Integer.parseInt(e.getMessage().substring(
+                ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX.length()));
+        } else {
+            return -1;
+        }
+    }
+
+    private void handleEventEnd(String name, long startProcessingTimeNs) {
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs;
+        log.debug("Processed {} in {} us", name,
+            TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS));
+    }
+
+    private Throwable handleEventException(String name,
+                                           Optional<Long> startProcessingTimeNs,
+                                           Throwable exception) {
+        if (!startProcessingTimeNs.isPresent()) {
+            log.debug("unable to start processing {} because of {}.", name,
+                exception.getClass().getSimpleName());
+            if (exception instanceof ApiException) {
+                return exception;
+            } else {
+                return new UnknownServerException(exception);
+            }
+        }
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs.get();
+        long deltaUs = TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS);
+        if (exception instanceof ApiException) {
+            log.debug("{}: failed with {} in {} us", name,
+                exception.getClass().getSimpleName(), deltaUs);
+            return exception;
+        }
+        log.warn("{}: failed with unknown server exception {} at epoch {} in {} us.  " +
+            "Reverting to last committed offset {}.",
+            this, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs,
+            lastCommittedOffset, exception);
+        renounce();
+        return new UnknownServerException(exception);
+    }
+
+    /**
+     * A controller event for handling internal state changes, such as Raft inputs.
+     */
+    class ControlEvent implements EventQueue.Event {
+        private final String name;
+        private final Runnable handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControlEvent(String name, Runnable handler) {
+            this.name = name;
+            this.handler = handler;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            log.debug("Executing {}.", this);
+            handler.run();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            handleEventException(name, startProcessingTimeNs, exception);
+        }
+
+        @Override
+        public String toString() {
+            return name;
+        }
+    }
+
+    private void appendControlEvent(String name, Runnable handler) {
+        ControlEvent event = new ControlEvent(name, handler);
+        queue.append(event);
+    }
+
+    /**
+     * A controller event that reads the committed internal state in order to expose it
+     * to an API.
+     */
+    class ControllerReadEvent<T> implements EventQueue.Event {
+        private final String name;
+        private final CompletableFuture<T> future;
+        private final Supplier<T> handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControllerReadEvent(String name, Supplier<T> handler) {
+            this.name = name;
+            this.future = new CompletableFuture<T>();
+            this.handler = handler;
+        }
+
+        CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            T value = handler.get();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+            future.complete(value);
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            future.completeExceptionally(
+                handleEventException(name, startProcessingTimeNs, exception));
+        }
+
+        @Override
+        public String toString() {
+            return name + "(" + System.identityHashCode(this) + ")";
+        }
+    }
+
+    // VisibleForTesting
+    ReplicationControlManager replicationControl() {
+        return replicationControl;
+    }
+
+    // VisibleForTesting
+    <T> CompletableFuture<T> appendReadEvent(String name, Supplier<T> handler) {
+        ControllerReadEvent<T> event = new ControllerReadEvent<T>(name, handler);
+        queue.append(event);
+        return event.future();
+    }
+
+    interface ControllerWriteOperation<T> {
+        /**
+         * Generate the metadata records needed to implement this controller write
+         * operation.  In general, this operation should not modify the "hard state" of
+         * the controller.  That modification will happen later on, when we replay the
+         * records generated by this function.
+         *
+         * There are cases where this function modifies the "soft state" of the
+         * controller.  Mainly, this happens when we process cluster heartbeats.
+         *
+         * This function also generates an RPC result.  In general, if the RPC resulted in
+         * an error, the RPC result will be an error, and the generated record list will
+         * be empty.  This would happen if we tried to create a topic with incorrect
+         * parameters, for example.  Of course, partial errors are possible for batch
+         * operations.
+         *
+         * @return              A tuple containing a list of records, and an RPC result.
+         */
+        ControllerResult<T> generateRecordsAndResult() throws Exception;
+
+        /**
+         * Once we've passed the records to the Raft layer, we will invoke this function
+         * with the end offset at which those records were placed.  If there were no
+         * records to write, we'll just pass the last write offset.
+         */
+        default void processBatchEndOffset(long offset) {}
+    }
+
+    /**
+     * A controller event that modifies the controller state.
+     */
+    class ControllerWriteEvent<T> implements EventQueue.Event, DeferredEvent {
+        private final String name;
+        private final CompletableFuture<T> future;
+        private final ControllerWriteOperation<T> op;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+        private ControllerResultAndOffset<T> resultAndOffset;
+
+        ControllerWriteEvent(String name, ControllerWriteOperation<T> op) {
+            this.name = name;
+            this.future = new CompletableFuture<T>();
+            this.op = op;
+            this.resultAndOffset = null;
+        }
+
+        CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public void run() throws Exception {
+            long controllerEpoch = curClaimEpoch;
+            if (controllerEpoch == -1) {
+                throw newNotControllerException();
+            }
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            ControllerResult<T> result = op.generateRecordsAndResult();
+            if (result.records().isEmpty()) {
+                op.processBatchEndOffset(writeOffset);
+                // If the operation did not return any records, then it was actually just
+                //a read after all, and not a read + write.  However, this read was done
+                //from the latest in-memory state, which might contain uncommitted data.
+                Optional<Long> maybeOffset = purgatory.highestPendingOffset();
+                if (!maybeOffset.isPresent()) {
+                    // If the purgatory is empty, there are no pending operations and no
+                    // uncommitted state.  We can return immediately.
+                    this.resultAndOffset = new ControllerResultAndOffset<>(-1,
+                        new ArrayList<>(), result.response());
+                    log.debug("Completing read-only operation {} immediately because " +
+                        "the purgatory is empty.", this);
+                    complete(null);
+                    return;
+                }
+                // If there are operations in the purgatory, we want to wait for the latest
+                // one to complete before returning our result to the user.
+                this.resultAndOffset = new ControllerResultAndOffset<>(maybeOffset.get(),
+                    result.records(), result.response());
+                log.debug("Read-only operation {} will be completed when the log " +
+                    "reaches offset {}", this, resultAndOffset.offset());
+            } else {
+                // If the operation returned a batch of records, those records need to be
+                // written before we can return our result to the user.  Here, we hand off
+                // the batch of records to the metadata log manager.  They will be written
+                // out asynchronously.
+                long offset = logManager.scheduleWrite(controllerEpoch, result.records());
+                op.processBatchEndOffset(offset);
+                writeOffset = offset;
+                this.resultAndOffset = new ControllerResultAndOffset<>(offset,
+                    result.records(), result.response());
+                for (ApiMessageAndVersion message : result.records()) {
+                    replay(message.message());
+                }
+                snapshotRegistry.createSnapshot(offset);
+                log.debug("Read-write operation {} will be completed when the log " +
+                    "reaches offset {}.", this, resultAndOffset.offset());
+            }
+            purgatory.add(resultAndOffset.offset(), this);
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            complete(exception);
+        }
+
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                handleEventEnd(this.toString(), startProcessingTimeNs.get());
+                future.complete(resultAndOffset.response());
+            } else {
+                future.completeExceptionally(
+                    handleEventException(name, startProcessingTimeNs, exception));
+            }
+        }
+
+        @Override
+        public String toString() {
+            return name + "(" + System.identityHashCode(this) + ")";
+        }
+    }
+
+    private <T> CompletableFuture<T> appendWriteEvent(String name,
+                                                      long timeoutMs,
+                                                      ControllerWriteOperation<T> op) {
+        ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
+        queue.appendWithDeadline(time.nanoseconds() +
+            TimeUnit.NANOSECONDS.convert(timeoutMs, TimeUnit.MILLISECONDS), event);
+        return event.future();
+    }
+
+    private <T> CompletableFuture<T> appendWriteEvent(String name,
+                                                      ControllerWriteOperation<T> op) {
+        ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
+        queue.append(event);
+        return event.future();
+    }
+
+    class QuorumMetaLogListener implements MetaLogListener {
+        @Override
+        public void handleCommits(long offset, List<ApiMessage> messages) {
+            appendControlEvent("handleCommits[" + offset + "]", () -> {
+                if (curClaimEpoch == -1) {
+                    if (log.isDebugEnabled()) {
+                        if (log.isTraceEnabled()) {
+                            log.trace("Replaying commits from the active node up to " +
+                                "offset {}: {}.", offset, messages.stream().
+                                map(m -> m.toString()).collect(Collectors.joining(", ")));
+                        } else {
+                            log.debug("Replaying commits from the active node up to " +
+                                "offset {}.", offset);
+                        }
+                    }
+                    for (ApiMessage message : messages) {
+                        replay(message);

Review comment:
       We already replay the message when it's first appended to the log and here we replay the same message again after commit. This could temporarily revert the state. For example, the latest (uncommitted) config could be overwritten by a previously committed config. 

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##########
@@ -0,0 +1,712 @@
+/*
+ * 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.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.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.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+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.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.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;
+
+
+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(IsrChangeRecord record) {
+            return new PartitionControlInfo(replicas,
+                Replicas.toArray(record.isr()),
+                removingReplicas,
+                addingReplicas,
+                record.leader(),
+                record.leaderEpoch(),
+                record.partitionEpoch());
+        }
+
+        String diff(PartitionControlInfo prev) {
+            StringBuilder builder = new StringBuilder();
+            String prefix = "";
+            if (!Arrays.equals(replicas, prev.replicas)) {
+                builder.append("replicas=").append(Arrays.toString(replicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(isr, prev.isr)) {
+                builder.append(prefix).append("isr=").append(Arrays.toString(isr));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(removingReplicas, prev.removingReplicas)) {
+                builder.append(prefix).append("removingReplicas=").
+                    append(Arrays.toString(removingReplicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(addingReplicas, prev.addingReplicas)) {
+                builder.append(prefix).append("addingReplicas=").
+                    append(Arrays.toString(addingReplicas));
+                prefix = ", ";
+            }
+            if (leader != prev.leader) {
+                builder.append(prefix).append("leader=").append(leader);
+                prefix = ", ";
+            }
+            if (leaderEpoch != prev.leaderEpoch) {
+                builder.append(prefix).append("leaderEpoch=").append(leaderEpoch);
+            }
+            if (partitionEpoch != prev.partitionEpoch) {
+                builder.append(prefix).append("partitionEpoch=").append(partitionEpoch);
+            }
+            return builder.toString();
+        }
+
+        @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();
+        }
+
+        int chooseNewLeader(int[] newIsr, boolean unclean) {
+            for (int i = 0; i < replicas.length; i++) {
+                int replica = replicas[i];
+                if (Replicas.contains(newIsr, replica)) {
+                    return replica;
+                }
+            }
+            if (unclean && replicas.length > 0) {
+                return replicas[0];

Review comment:
       We need to choose at least a live replica.

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##########
@@ -0,0 +1,712 @@
+/*
+ * 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.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.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.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+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.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.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;
+
+
+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(IsrChangeRecord record) {
+            return new PartitionControlInfo(replicas,
+                Replicas.toArray(record.isr()),
+                removingReplicas,
+                addingReplicas,
+                record.leader(),
+                record.leaderEpoch(),
+                record.partitionEpoch());
+        }
+
+        String diff(PartitionControlInfo prev) {
+            StringBuilder builder = new StringBuilder();
+            String prefix = "";
+            if (!Arrays.equals(replicas, prev.replicas)) {
+                builder.append("replicas=").append(Arrays.toString(replicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(isr, prev.isr)) {
+                builder.append(prefix).append("isr=").append(Arrays.toString(isr));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(removingReplicas, prev.removingReplicas)) {
+                builder.append(prefix).append("removingReplicas=").
+                    append(Arrays.toString(removingReplicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(addingReplicas, prev.addingReplicas)) {
+                builder.append(prefix).append("addingReplicas=").
+                    append(Arrays.toString(addingReplicas));
+                prefix = ", ";
+            }
+            if (leader != prev.leader) {
+                builder.append(prefix).append("leader=").append(leader);
+                prefix = ", ";
+            }
+            if (leaderEpoch != prev.leaderEpoch) {
+                builder.append(prefix).append("leaderEpoch=").append(leaderEpoch);
+            }
+            if (partitionEpoch != prev.partitionEpoch) {
+                builder.append(prefix).append("partitionEpoch=").append(partitionEpoch);
+            }
+            return builder.toString();
+        }
+
+        @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();
+        }
+
+        int chooseNewLeader(int[] newIsr, boolean unclean) {
+            for (int i = 0; i < replicas.length; i++) {
+                int replica = replicas[i];
+                if (Replicas.contains(newIsr, replica)) {

Review comment:
       In the ZK based code, we also take live brokers into consideration when selecting a new leader.

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
##########
@@ -0,0 +1,367 @@
+/*
+ * 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.ConfigKey;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource.Type;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.metadata.ConfigRecord;
+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.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.APPEND;
+
+public class ConfigurationControlManager {
+    private final Logger log;
+    private final SnapshotRegistry snapshotRegistry;
+    private final Map<ConfigResource.Type, ConfigDef> configDefs;
+    private final TimelineHashMap<ConfigResource, TimelineHashMap<String, String>> configData;
+
+    ConfigurationControlManager(LogContext logContext,
+                                SnapshotRegistry snapshotRegistry,
+                                Map<ConfigResource.Type, ConfigDef> configDefs) {
+        this.log = logContext.logger(ConfigurationControlManager.class);
+        this.snapshotRegistry = snapshotRegistry;
+        this.configDefs = configDefs;
+        this.configData = new TimelineHashMap<>(snapshotRegistry, 0);
+    }
+
+    /**
+     * Determine the result of applying a batch of incremental configuration changes.  Note
+     * that this method does not change the contents of memory.  It just generates a
+     * result, that you can replay later if you wish using replay().
+     *
+     * Note that there can only be one result per ConfigResource.  So if you try to modify
+     * several keys and one modification fails, the whole ConfigKey fails and nothing gets
+     * changed.
+     *
+     * @param configChanges     Maps each resource to a map from config keys to
+     *                          operation data.
+     * @return                  The result.
+     */
+    ControllerResult<Map<ConfigResource, ApiError>> incrementalAlterConfigs(
+            Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges) {
+        List<ApiMessageAndVersion> outputRecords = new ArrayList<>();
+        Map<ConfigResource, ApiError> outputResults = new HashMap<>();
+        for (Entry<ConfigResource, Map<String, Entry<OpType, String>>> resourceEntry :
+                configChanges.entrySet()) {
+            incrementalAlterConfigResource(resourceEntry.getKey(),
+                resourceEntry.getValue(),
+                outputRecords,
+                outputResults);
+        }
+        return new ControllerResult<>(outputRecords, outputResults);
+    }
+
+    private void incrementalAlterConfigResource(ConfigResource configResource,
+                                                Map<String, Entry<OpType, String>> keysToOps,
+                                                List<ApiMessageAndVersion> outputRecords,
+                                                Map<ConfigResource, ApiError> outputResults) {
+        ApiError error = checkConfigResource(configResource);
+        if (error.isFailure()) {
+            outputResults.put(configResource, error);
+            return;
+        }
+        List<ApiMessageAndVersion> newRecords = new ArrayList<>();
+        for (Entry<String, Entry<OpType, String>> keysToOpsEntry : keysToOps.entrySet()) {
+            String key = keysToOpsEntry.getKey();
+            String currentValue = null;
+            TimelineHashMap<String, String> currentConfigs = configData.get(configResource);
+            if (currentConfigs != null) {
+                currentValue = currentConfigs.get(key);
+            }
+            String newValue = currentValue;
+            Entry<OpType, String> opTypeAndNewValue = keysToOpsEntry.getValue();
+            OpType opType = opTypeAndNewValue.getKey();
+            String opValue = opTypeAndNewValue.getValue();
+            switch (opType) {
+                case SET:
+                    newValue = opValue;
+                    break;
+                case DELETE:
+                    if (opValue != null) {
+                        outputResults.put(configResource, new ApiError(
+                            Errors.INVALID_REQUEST, "A DELETE op was given with a " +
+                            "non-null value."));
+                        return;
+                    }
+                    newValue = null;
+                    break;
+                case APPEND:
+                case SUBTRACT:
+                    if (!isSplittable(configResource.type(), key)) {
+                        outputResults.put(configResource, new ApiError(
+                            Errors.INVALID_CONFIG, "Can't " + opType + " to " +
+                            "key " + key + " because its type is not LIST."));
+                        return;
+                    }
+                    List<String> newValueParts = getParts(newValue, key, configResource);
+                    if (opType == APPEND) {
+                        if (!newValueParts.contains(opValue)) {
+                            newValueParts.add(opValue);
+                        }
+                        newValue = String.join(",", newValueParts);
+                    } else if (newValueParts.remove(opValue)) {
+                        newValue = String.join(",", newValueParts);
+                    }
+                    break;
+            }
+            if (!Objects.equals(currentValue, newValue)) {
+                newRecords.add(new ApiMessageAndVersion(new ConfigRecord().
+                    setResourceType(configResource.type().id()).
+                    setResourceName(configResource.name()).
+                    setName(key).
+                    setValue(newValue), (short) 0));
+            }
+        }
+        outputRecords.addAll(newRecords);
+        outputResults.put(configResource, ApiError.NONE);
+    }
+
+    /**
+     * Determine the result of applying a batch of legacy configuration changes.  Note
+     * that this method does not change the contents of memory.  It just generates a
+     * result, that you can replay later if you wish using replay().
+     *
+     * @param newConfigs        The new configurations to install for each resource.
+     *                          All existing configurations will be overwritten.
+     * @return                  The result.
+     */
+    ControllerResult<Map<ConfigResource, ApiError>> legacyAlterConfigs(
+        Map<ConfigResource, Map<String, String>> newConfigs) {
+        List<ApiMessageAndVersion> outputRecords = new ArrayList<>();
+        Map<ConfigResource, ApiError> outputResults = new HashMap<>();
+        for (Entry<ConfigResource, Map<String, String>> resourceEntry :
+            newConfigs.entrySet()) {
+            legacyAlterConfigResource(resourceEntry.getKey(),
+                resourceEntry.getValue(),
+                outputRecords,
+                outputResults);
+        }
+        return new ControllerResult<>(outputRecords, outputResults);
+    }
+
+    private void legacyAlterConfigResource(ConfigResource configResource,
+                                           Map<String, String> newConfigs,
+                                           List<ApiMessageAndVersion> outputRecords,
+                                           Map<ConfigResource, ApiError> outputResults) {
+        ApiError error = checkConfigResource(configResource);
+        if (error.isFailure()) {
+            outputResults.put(configResource, error);
+            return;
+        }
+        List<ApiMessageAndVersion> newRecords = new ArrayList<>();
+        Map<String, String> currentConfigs = configData.get(configResource);
+        if (currentConfigs == null) {
+            currentConfigs = Collections.emptyMap();
+        }
+        for (Entry<String, String> entry : newConfigs.entrySet()) {
+            String key = entry.getKey();
+            String newValue = entry.getValue();
+            String currentValue = currentConfigs.get(key);
+            if (!Objects.equals(newValue, currentValue)) {
+                newRecords.add(new ApiMessageAndVersion(new ConfigRecord().
+                    setResourceType(configResource.type().id()).
+                    setResourceName(configResource.name()).
+                    setName(key).
+                    setValue(newValue), (short) 0));
+            }
+        }
+        for (String key : currentConfigs.keySet()) {
+            if (!newConfigs.containsKey(key)) {
+                newRecords.add(new ApiMessageAndVersion(new ConfigRecord().
+                    setResourceType(configResource.type().id()).
+                    setResourceName(configResource.name()).
+                    setName(key).
+                    setValue(null), (short) 0));
+            }
+        }
+        outputRecords.addAll(newRecords);
+        outputResults.put(configResource, ApiError.NONE);
+    }
+
+    private List<String> getParts(String value, String key, ConfigResource configResource) {
+        if (value == null) {
+            value = getConfigValueDefault(configResource.type(), key);
+        }
+        List<String> parts = new ArrayList<>();
+        if (value == null) {
+            return parts;
+        }
+        String[] splitValues = value.split(",");
+        for (String splitValue : splitValues) {
+            if (!splitValue.isEmpty()) {
+                parts.add(splitValue);
+            }
+        }
+        return parts;
+    }
+
+    static ApiError checkConfigResource(ConfigResource configResource) {
+        switch (configResource.type()) {
+            case BROKER_LOGGER:
+                // We do not handle resources of type BROKER_LOGGER in

Review comment:
       Is that temporary?

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+    class ReadyBrokersFuture {
+        private final CompletableFuture<Void> future;
+        private final int minBrokers;
+
+        ReadyBrokersFuture(CompletableFuture<Void> future, int minBrokers) {
+            this.future = future;
+            this.minBrokers = minBrokers;
+        }
+
+        boolean check() {
+            int numUnfenced = 0;
+            for (BrokerRegistration registration : brokerRegistrations.values()) {
+                if (!registration.fenced()) {
+                    numUnfenced++;
+                }
+                if (numUnfenced >= minBrokers) {
+                    return true;
+                }
+            }
+            return false;
+        }
+    }
+
+    /**
+     * The SLF4J log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The SLF4J log object.
+     */
+    private final Logger log;
+
+    /**
+     * The Kafka clock object to use.
+     */
+    private final Time time;
+
+    /**
+     * How long sessions should last, in nanoseconds.
+     */
+    private final long sessionTimeoutNs;
+
+    /**
+     * The replica placement policy to use.
+     */
+    private final ReplicaPlacementPolicy placementPolicy;
+
+    /**
+     * Maps broker IDs to broker registrations.
+     */
+    private final TimelineHashMap<Integer, BrokerRegistration> brokerRegistrations;
+
+    /**
+     * The broker heartbeat manager, or null if this controller is on standby.
+     */
+    private BrokerHeartbeatManager heartbeatManager;
+
+    /**
+     * A future which is completed as soon as we have the given number of brokers
+     * ready.
+     */
+    private Optional<ReadyBrokersFuture> readyBrokersFuture;
+
+    ClusterControlManager(LogContext logContext,
+                          Time time,
+                          SnapshotRegistry snapshotRegistry,
+                          long sessionTimeoutNs,
+                          ReplicaPlacementPolicy placementPolicy) {
+        this.logContext = logContext;
+        this.log = logContext.logger(ClusterControlManager.class);
+        this.time = time;
+        this.sessionTimeoutNs = sessionTimeoutNs;
+        this.placementPolicy = placementPolicy;
+        this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.heartbeatManager = null;
+        this.readyBrokersFuture = Optional.empty();
+    }
+
+    /**
+     * Transition this ClusterControlManager to active.
+     */
+    public void activate() {
+        heartbeatManager = new BrokerHeartbeatManager(logContext, time, sessionTimeoutNs);
+        for (BrokerRegistration registration : brokerRegistrations.values()) {
+            heartbeatManager.touch(registration.id(), registration.fenced(), -1);
+        }
+    }
+
+    /**
+     * Transition this ClusterControlManager to standby.
+     */
+    public void deactivate() {
+        heartbeatManager = null;
+    }
+
+    // VisibleForTesting
+    Map<Integer, BrokerRegistration> brokerRegistrations() {
+        return brokerRegistrations;
+    }
+
+    /**
+     * Process an incoming broker registration request.
+     */
+    public ControllerResult<BrokerRegistrationReply> registerBroker(
+            BrokerRegistrationRequestData request, long brokerEpoch,
+            FeatureManager.FinalizedFeaturesAndEpoch finalizedFeaturesAndEpoch) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        int brokerId = request.brokerId();
+        BrokerRegistration existing = brokerRegistrations.get(brokerId);
+        if (existing != null) {
+            if (heartbeatManager.hasValidSession(brokerId)) {
+                if (!existing.incarnationId().equals(request.incarnationId())) {
+                    throw new DuplicateBrokerRegistrationException("Another broker is " +
+                        "registered with that broker id.");
+                }
+            } else {
+                if (!existing.incarnationId().equals(request.incarnationId())) {
+                    // Remove any existing session for the old broker incarnation.
+                    heartbeatManager.remove(brokerId);
+                    existing = null;
+                }
+            }
+        }
+
+        RegisterBrokerRecord record = new RegisterBrokerRecord().setBrokerId(brokerId).
+            setIncarnationId(request.incarnationId()).
+            setBrokerEpoch(brokerEpoch).
+            setRack(request.rack());
+        for (BrokerRegistrationRequestData.Listener listener : request.listeners()) {
+            record.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint().
+                setHost(listener.host()).
+                setName(listener.name()).
+                setPort(listener.port()).
+                setSecurityProtocol(listener.securityProtocol()));
+        }
+        for (BrokerRegistrationRequestData.Feature feature : request.features()) {
+            VersionRange supported = finalizedFeaturesAndEpoch.finalizedFeatures().
+                getOrDefault(feature.name(), VersionRange.ALL);
+            if (!supported.contains(new VersionRange(feature.minSupportedVersion(),
+                    feature.maxSupportedVersion()))) {
+                throw new UnsupportedVersionException("Unable to register because " +
+                    "the broker has an unsupported version of " + feature.name());
+            }
+            record.features().add(new RegisterBrokerRecord.BrokerFeature().
+                setName(feature.name()).
+                setMinSupportedVersion(feature.minSupportedVersion()).
+                setMaxSupportedVersion(feature.maxSupportedVersion()));
+        }
+
+        if (existing == null) {
+            heartbeatManager.touch(brokerId, true, -1);
+        } else {
+            heartbeatManager.touch(brokerId, existing.fenced(), -1);
+        }
+
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        records.add(new ApiMessageAndVersion(record, (short) 0));
+        return new ControllerResult<>(records, new BrokerRegistrationReply(brokerEpoch));
+    }
+
+    public ControllerResult<Void> decommissionBroker(int brokerId) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        BrokerRegistration existing = brokerRegistrations.get(brokerId);
+        if (existing == null) {
+            return new ControllerResult<>(new ArrayList<>(), null);
+        }
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        records.add(new ApiMessageAndVersion(
+            new UnregisterBrokerRecord().
+                setBrokerId(brokerId).
+                setBrokerEpoch(existing.epoch()),
+            (short) 0));
+        heartbeatManager.remove(brokerId);
+        return new ControllerResult<>(records, null);
+    }
+
+    public ControllerResult<BrokerHeartbeatReply> processBrokerHeartbeat(
+            BrokerHeartbeatRequestData request,
+            long lastCommittedOffset,
+            boolean movingLeadersForShutDown) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        int brokerId = request.brokerId();
+        BrokerRegistration registration = brokerRegistrations.get(brokerId);
+        if (registration == null) {

Review comment:
       We already did this check and the one below in the caller through `clusterControl.checkBrokerEpoch`.

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##########
@@ -0,0 +1,712 @@
+/*
+ * 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.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.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.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+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.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.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;
+
+
+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(IsrChangeRecord record) {
+            return new PartitionControlInfo(replicas,
+                Replicas.toArray(record.isr()),
+                removingReplicas,
+                addingReplicas,
+                record.leader(),
+                record.leaderEpoch(),
+                record.partitionEpoch());
+        }
+
+        String diff(PartitionControlInfo prev) {
+            StringBuilder builder = new StringBuilder();
+            String prefix = "";
+            if (!Arrays.equals(replicas, prev.replicas)) {
+                builder.append("replicas=").append(Arrays.toString(replicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(isr, prev.isr)) {
+                builder.append(prefix).append("isr=").append(Arrays.toString(isr));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(removingReplicas, prev.removingReplicas)) {
+                builder.append(prefix).append("removingReplicas=").
+                    append(Arrays.toString(removingReplicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(addingReplicas, prev.addingReplicas)) {
+                builder.append(prefix).append("addingReplicas=").
+                    append(Arrays.toString(addingReplicas));
+                prefix = ", ";
+            }
+            if (leader != prev.leader) {
+                builder.append(prefix).append("leader=").append(leader);
+                prefix = ", ";
+            }
+            if (leaderEpoch != prev.leaderEpoch) {
+                builder.append(prefix).append("leaderEpoch=").append(leaderEpoch);
+            }
+            if (partitionEpoch != prev.partitionEpoch) {
+                builder.append(prefix).append("partitionEpoch=").append(partitionEpoch);
+            }
+            return builder.toString();
+        }
+
+        @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();
+        }
+
+        int chooseNewLeader(int[] newIsr, boolean unclean) {
+            for (int i = 0; i < replicas.length; i++) {
+                int replica = replicas[i];
+                if (Replicas.contains(newIsr, replica)) {
+                    return replica;
+                }
+            }
+            if (unclean && replicas.length > 0) {
+                return replicas[0];
+            }
+            return -1;
+        }
+    }
+
+    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(IsrChangeRecord 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);
+        topicInfo.parts.put(record.partitionId(), newPartitionInfo);
+        brokersToIsrs.update(record.topicId(), record.partitionId(),
+            prevPartitionInfo.isr, newPartitionInfo.isr, prevPartitionInfo.leader,
+            newPartitionInfo.leader);
+        log.debug("Applied ISR change record: {}", record.toString());
+    }
+
+    ControllerResult<CreateTopicsResponseData>
+            createTopics(CreateTopicsRequestData request) {
+        Map<String, ApiError> topicErrors = new HashMap<>();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+
+        // Check the topic names.
+        validateNewTopicNames(topicErrors, request.topics());
+
+        // Identify topics that already exist and mark them with the appropriate error
+        request.topics().stream().filter(creatableTopic -> topicsByName.containsKey(creatableTopic.name()))
+                .forEach(t -> topicErrors.put(t.name(), new ApiError(Errors.TOPIC_ALREADY_EXISTS)));
+
+        // Verify that the configurations for the new topics are OK, and figure out what
+        // ConfigRecords should be created.
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges =
+            computeConfigChanges(topicErrors, request.topics());
+        ControllerResult<Map<ConfigResource, ApiError>> configResult =
+            configurationControl.incrementalAlterConfigs(configChanges);
+        for (Entry<ConfigResource, ApiError> entry : configResult.response().entrySet()) {
+            if (entry.getValue().isFailure()) {
+                topicErrors.put(entry.getKey().name(), entry.getValue());
+            }
+        }
+        records.addAll(configResult.records());
+
+        // Try to create whatever topics are needed.
+        Map<String, CreatableTopicResult> successes = new HashMap<>();
+        for (CreatableTopic topic : request.topics()) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            ApiError error = createTopic(topic, records, successes);
+            if (error.isFailure()) {
+                topicErrors.put(topic.name(), error);
+            }
+        }
+
+        // Create responses for all topics.
+        CreateTopicsResponseData data = new CreateTopicsResponseData();
+        StringBuilder resultsBuilder = new StringBuilder();
+        String resultsPrefix = "";
+        for (CreatableTopic topic : request.topics()) {
+            ApiError error = topicErrors.get(topic.name());
+            if (error != null) {
+                data.topics().add(new CreatableTopicResult().
+                    setName(topic.name()).
+                    setErrorCode(error.error().code()).
+                    setErrorMessage(error.message()));
+                resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                    append(error.error()).append(" (").append(error.message()).append(")");
+                resultsPrefix = ", ";
+                continue;
+            }
+            CreatableTopicResult result = successes.get(topic.name());
+            data.topics().add(result);
+            resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                append("SUCCESS");
+            resultsPrefix = ", ";
+        }
+        log.info("createTopics result(s): {}", resultsBuilder.toString());
+        return new ControllerResult<>(records, data);
+    }
+
+    private ApiError createTopic(CreatableTopic topic,
+                                 List<ApiMessageAndVersion> records,
+                                 Map<String, CreatableTopicResult> successes) {
+        Map<Integer, PartitionControlInfo> newParts = new HashMap<>();
+        if (!topic.assignments().isEmpty()) {
+            if (topic.replicationFactor() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but replication " +
+                    "factor was not set to -1.");
+            }
+            if (topic.numPartitions() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but numPartitions " +
+                        "was not set to -1.");
+            }
+            for (CreatableReplicaAssignment assignment : topic.assignments()) {
+                if (newParts.containsKey(assignment.partitionIndex())) {
+                    return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                        "Found multiple manual partition assignments for partition " +
+                            assignment.partitionIndex());
+                }
+                HashSet<Integer> brokerIds = new HashSet<>();
+                for (int brokerId : assignment.brokerIds()) {
+                    if (!brokerIds.add(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment specifies the same node " +
+                                "id more than once.");
+                    } else if (!clusterControl.unfenced(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment contains node " + brokerId +
+                                ", but that node is not usable.");
+                    }
+                }
+                int[] replicas = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    replicas[i] = assignment.brokerIds().get(i);
+                }
+                int[] isr = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    isr[i] = assignment.brokerIds().get(i);
+                }
+                newParts.put(assignment.partitionIndex(),
+                    new PartitionControlInfo(replicas, isr, null, null, isr[0], 0, 0));
+            }
+        } else if (topic.replicationFactor() < -1 || topic.replicationFactor() == 0) {
+            return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                "Replication factor was set to an invalid non-positive value.");
+        } else if (!topic.assignments().isEmpty()) {
+            return new ApiError(Errors.INVALID_REQUEST,
+                "Replication factor was not set to -1 but a manual partition " +
+                    "assignment was specified.");
+        } else if (topic.numPartitions() < -1 || topic.numPartitions() == 0) {
+            return new ApiError(Errors.INVALID_PARTITIONS,
+                "Number of partitions was set to an invalid non-positive value.");
+        } else {
+            int numPartitions = topic.numPartitions() == -1 ?
+                defaultNumPartitions : topic.numPartitions();
+            short replicationFactor = topic.replicationFactor() == -1 ?
+                defaultReplicationFactor : topic.replicationFactor();
+            try {
+                List<List<Integer>> replicas = clusterControl.
+                    placeReplicas(numPartitions, replicationFactor);
+                for (int partitionId = 0; partitionId < replicas.size(); partitionId++) {
+                    int[] r = Replicas.toArray(replicas.get(partitionId));
+                    newParts.put(partitionId,
+                        new PartitionControlInfo(r, r, null, null, r[0], 0, 0));
+                }
+            } catch (InvalidReplicationFactorException e) {
+                return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                    "Unable to replicate the partition " + replicationFactor +
+                        " times: " + e.getMessage());
+            }
+        }
+        Uuid topicId = new Uuid(random.nextLong(), random.nextLong());
+        successes.put(topic.name(), new CreatableTopicResult().
+            setName(topic.name()).
+            setTopicId(topicId).
+            setErrorCode((short) 0).
+            setErrorMessage(null).
+            setNumPartitions(newParts.size()).
+            setReplicationFactor((short) newParts.get(0).replicas.length));
+        records.add(new ApiMessageAndVersion(new TopicRecord().
+            setName(topic.name()).
+            setTopicId(topicId), (short) 0));
+        for (Entry<Integer, PartitionControlInfo> partEntry : newParts.entrySet()) {
+            int partitionIndex = partEntry.getKey();
+            PartitionControlInfo info = partEntry.getValue();
+            records.add(new ApiMessageAndVersion(new PartitionRecord().
+                setPartitionId(partitionIndex).
+                setTopicId(topicId).
+                setReplicas(Replicas.toList(info.replicas)).
+                setIsr(Replicas.toList(info.isr)).
+                setRemovingReplicas(null).
+                setAddingReplicas(null).
+                setLeader(info.leader).
+                setLeaderEpoch(info.leaderEpoch).
+                setPartitionEpoch(0), (short) 0));
+        }
+        return ApiError.NONE;
+    }
+
+    static void validateNewTopicNames(Map<String, ApiError> topicErrors,
+                                      CreatableTopicCollection topics) {
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            try {
+                Topic.validate(topic.name());
+            } catch (InvalidTopicException e) {
+                topicErrors.put(topic.name(),
+                    new ApiError(Errors.INVALID_TOPIC_EXCEPTION, e.getMessage()));
+            }
+        }
+    }
+
+    static Map<ConfigResource, Map<String, Entry<OpType, String>>>
+            computeConfigChanges(Map<String, ApiError> topicErrors,
+                                 CreatableTopicCollection topics) {
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges = new HashMap<>();
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            Map<String, Entry<OpType, String>> topicConfigs = new HashMap<>();
+            for (CreateTopicsRequestData.CreateableTopicConfig config : topic.configs()) {
+                topicConfigs.put(config.name(), new SimpleImmutableEntry<>(SET, config.value()));
+            }
+            if (!topicConfigs.isEmpty()) {
+                configChanges.put(new ConfigResource(TOPIC, topic.name()), topicConfigs);
+            }
+        }
+        return configChanges;
+    }
+
+    // VisibleForTesting
+    PartitionControlInfo getPartition(Uuid topicId, int partitionId) {
+        TopicControlInfo topic = topics.get(topicId);
+        if (topic == null) {
+            return null;
+        }
+        return topic.parts.get(partitionId);
+    }
+
+    // VisibleForTesting
+    BrokersToIsrs brokersToIsrs() {
+        return brokersToIsrs;
+    }
+
+    ControllerResult<AlterIsrResponseData> alterIsr(AlterIsrRequestData request) {
+        clusterControl.checkBrokerEpoch(request.brokerId(), request.brokerEpoch());
+        AlterIsrResponseData response = new AlterIsrResponseData();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        for (AlterIsrRequestData.TopicData topicData : request.topics()) {
+            AlterIsrResponseData.TopicData responseTopicData =
+                new AlterIsrResponseData.TopicData().setName(topicData.name());
+            response.topics().add(responseTopicData);
+            Uuid topicId = topicsByName.get(topicData.name());
+            if (topicId == null || !topics.containsKey(topicId)) {
+                for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()));
+                }
+                continue;
+            }
+            TopicControlInfo topic = topics.get(topicId);
+            for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) {
+                PartitionControlInfo partition = topic.parts.get(partitionData.partitionIndex());
+                if (partition == null) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()));
+                    continue;
+                }
+                if (partitionData.leaderEpoch() != partition.leaderEpoch) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.FENCED_LEADER_EPOCH.code()));
+                    continue;
+                }
+                if (partitionData.currentIsrVersion() != partition.partitionEpoch) {

Review comment:
       In the ZK case, we use the ZK version to do conditional updates. In Raft, could we associated each partitionState with the offset in the Raft log and use that as partitionEpoch for conditional updates? This way, we don't need to explicitly maintain a separate partitionEpoch field and the epoch is automatically bumped up for any change to the partition record, not just for leader and isr.

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+    class ReadyBrokersFuture {
+        private final CompletableFuture<Void> future;
+        private final int minBrokers;
+
+        ReadyBrokersFuture(CompletableFuture<Void> future, int minBrokers) {
+            this.future = future;
+            this.minBrokers = minBrokers;
+        }
+
+        boolean check() {
+            int numUnfenced = 0;
+            for (BrokerRegistration registration : brokerRegistrations.values()) {
+                if (!registration.fenced()) {
+                    numUnfenced++;
+                }
+                if (numUnfenced >= minBrokers) {
+                    return true;
+                }
+            }
+            return false;
+        }
+    }
+
+    /**
+     * The SLF4J log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The SLF4J log object.
+     */
+    private final Logger log;
+
+    /**
+     * The Kafka clock object to use.
+     */
+    private final Time time;
+
+    /**
+     * How long sessions should last, in nanoseconds.
+     */
+    private final long sessionTimeoutNs;
+
+    /**
+     * The replica placement policy to use.
+     */
+    private final ReplicaPlacementPolicy placementPolicy;
+
+    /**
+     * Maps broker IDs to broker registrations.
+     */
+    private final TimelineHashMap<Integer, BrokerRegistration> brokerRegistrations;
+
+    /**
+     * The broker heartbeat manager, or null if this controller is on standby.
+     */
+    private BrokerHeartbeatManager heartbeatManager;
+
+    /**
+     * A future which is completed as soon as we have the given number of brokers
+     * ready.
+     */
+    private Optional<ReadyBrokersFuture> readyBrokersFuture;
+
+    ClusterControlManager(LogContext logContext,
+                          Time time,
+                          SnapshotRegistry snapshotRegistry,
+                          long sessionTimeoutNs,
+                          ReplicaPlacementPolicy placementPolicy) {
+        this.logContext = logContext;
+        this.log = logContext.logger(ClusterControlManager.class);
+        this.time = time;
+        this.sessionTimeoutNs = sessionTimeoutNs;
+        this.placementPolicy = placementPolicy;
+        this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.heartbeatManager = null;
+        this.readyBrokersFuture = Optional.empty();
+    }
+
+    /**
+     * Transition this ClusterControlManager to active.
+     */
+    public void activate() {
+        heartbeatManager = new BrokerHeartbeatManager(logContext, time, sessionTimeoutNs);
+        for (BrokerRegistration registration : brokerRegistrations.values()) {
+            heartbeatManager.touch(registration.id(), registration.fenced(), -1);
+        }
+    }
+
+    /**
+     * Transition this ClusterControlManager to standby.
+     */
+    public void deactivate() {
+        heartbeatManager = null;
+    }
+
+    // VisibleForTesting
+    Map<Integer, BrokerRegistration> brokerRegistrations() {
+        return brokerRegistrations;
+    }
+
+    /**
+     * Process an incoming broker registration request.
+     */
+    public ControllerResult<BrokerRegistrationReply> registerBroker(
+            BrokerRegistrationRequestData request, long brokerEpoch,
+            FeatureManager.FinalizedFeaturesAndEpoch finalizedFeaturesAndEpoch) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        int brokerId = request.brokerId();
+        BrokerRegistration existing = brokerRegistrations.get(brokerId);
+        if (existing != null) {
+            if (heartbeatManager.hasValidSession(brokerId)) {
+                if (!existing.incarnationId().equals(request.incarnationId())) {
+                    throw new DuplicateBrokerRegistrationException("Another broker is " +
+                        "registered with that broker id.");
+                }
+            } else {
+                if (!existing.incarnationId().equals(request.incarnationId())) {
+                    // Remove any existing session for the old broker incarnation.
+                    heartbeatManager.remove(brokerId);
+                    existing = null;
+                }
+            }
+        }
+
+        RegisterBrokerRecord record = new RegisterBrokerRecord().setBrokerId(brokerId).
+            setIncarnationId(request.incarnationId()).
+            setBrokerEpoch(brokerEpoch).
+            setRack(request.rack());
+        for (BrokerRegistrationRequestData.Listener listener : request.listeners()) {
+            record.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint().
+                setHost(listener.host()).
+                setName(listener.name()).
+                setPort(listener.port()).
+                setSecurityProtocol(listener.securityProtocol()));
+        }
+        for (BrokerRegistrationRequestData.Feature feature : request.features()) {
+            VersionRange supported = finalizedFeaturesAndEpoch.finalizedFeatures().
+                getOrDefault(feature.name(), VersionRange.ALL);
+            if (!supported.contains(new VersionRange(feature.minSupportedVersion(),
+                    feature.maxSupportedVersion()))) {
+                throw new UnsupportedVersionException("Unable to register because " +
+                    "the broker has an unsupported version of " + feature.name());
+            }
+            record.features().add(new RegisterBrokerRecord.BrokerFeature().
+                setName(feature.name()).
+                setMinSupportedVersion(feature.minSupportedVersion()).
+                setMaxSupportedVersion(feature.maxSupportedVersion()));
+        }
+
+        if (existing == null) {
+            heartbeatManager.touch(brokerId, true, -1);
+        } else {
+            heartbeatManager.touch(brokerId, existing.fenced(), -1);
+        }
+
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        records.add(new ApiMessageAndVersion(record, (short) 0));
+        return new ControllerResult<>(records, new BrokerRegistrationReply(brokerEpoch));
+    }
+
+    public ControllerResult<Void> decommissionBroker(int brokerId) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        BrokerRegistration existing = brokerRegistrations.get(brokerId);
+        if (existing == null) {
+            return new ControllerResult<>(new ArrayList<>(), null);
+        }
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        records.add(new ApiMessageAndVersion(
+            new UnregisterBrokerRecord().
+                setBrokerId(brokerId).
+                setBrokerEpoch(existing.epoch()),
+            (short) 0));
+        heartbeatManager.remove(brokerId);
+        return new ControllerResult<>(records, null);
+    }
+
+    public ControllerResult<BrokerHeartbeatReply> processBrokerHeartbeat(
+            BrokerHeartbeatRequestData request,
+            long lastCommittedOffset,
+            boolean movingLeadersForShutDown) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        int brokerId = request.brokerId();
+        BrokerRegistration registration = brokerRegistrations.get(brokerId);
+        if (registration == null) {
+            throw new StaleBrokerEpochException("No registration found for broker " +
+                request.brokerId());
+        }
+        if (request.brokerEpoch() != registration.epoch()) {
+            throw new StaleBrokerEpochException("Expected broker epoch " +
+                request.brokerEpoch() + "; got epoch " + registration.epoch());
+        }
+        boolean isCaughtUp = request.currentMetadataOffset() >= lastCommittedOffset;
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        boolean isFenced = registration.fenced();
+        boolean shouldShutdown = false;
+        if (isFenced) {
+            if (request.wantShutDown()) {
+                // If the broker is fenced, and requests a shutdown, do it immediately.
+                isFenced = true;

Review comment:
       This seems unnecessary.

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##########
@@ -0,0 +1,712 @@
+/*
+ * 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.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.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.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+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.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.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;
+
+
+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(IsrChangeRecord record) {
+            return new PartitionControlInfo(replicas,
+                Replicas.toArray(record.isr()),
+                removingReplicas,
+                addingReplicas,
+                record.leader(),
+                record.leaderEpoch(),
+                record.partitionEpoch());
+        }
+
+        String diff(PartitionControlInfo prev) {
+            StringBuilder builder = new StringBuilder();
+            String prefix = "";
+            if (!Arrays.equals(replicas, prev.replicas)) {
+                builder.append("replicas=").append(Arrays.toString(replicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(isr, prev.isr)) {
+                builder.append(prefix).append("isr=").append(Arrays.toString(isr));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(removingReplicas, prev.removingReplicas)) {
+                builder.append(prefix).append("removingReplicas=").
+                    append(Arrays.toString(removingReplicas));
+                prefix = ", ";
+            }
+            if (!Arrays.equals(addingReplicas, prev.addingReplicas)) {
+                builder.append(prefix).append("addingReplicas=").
+                    append(Arrays.toString(addingReplicas));
+                prefix = ", ";
+            }
+            if (leader != prev.leader) {
+                builder.append(prefix).append("leader=").append(leader);
+                prefix = ", ";
+            }
+            if (leaderEpoch != prev.leaderEpoch) {
+                builder.append(prefix).append("leaderEpoch=").append(leaderEpoch);
+            }
+            if (partitionEpoch != prev.partitionEpoch) {
+                builder.append(prefix).append("partitionEpoch=").append(partitionEpoch);
+            }
+            return builder.toString();
+        }
+
+        @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();
+        }
+
+        int chooseNewLeader(int[] newIsr, boolean unclean) {
+            for (int i = 0; i < replicas.length; i++) {
+                int replica = replicas[i];
+                if (Replicas.contains(newIsr, replica)) {
+                    return replica;
+                }
+            }
+            if (unclean && replicas.length > 0) {
+                return replicas[0];
+            }
+            return -1;
+        }
+    }
+
+    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(IsrChangeRecord 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);
+        topicInfo.parts.put(record.partitionId(), newPartitionInfo);
+        brokersToIsrs.update(record.topicId(), record.partitionId(),
+            prevPartitionInfo.isr, newPartitionInfo.isr, prevPartitionInfo.leader,
+            newPartitionInfo.leader);
+        log.debug("Applied ISR change record: {}", record.toString());
+    }
+
+    ControllerResult<CreateTopicsResponseData>
+            createTopics(CreateTopicsRequestData request) {
+        Map<String, ApiError> topicErrors = new HashMap<>();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+
+        // Check the topic names.
+        validateNewTopicNames(topicErrors, request.topics());
+
+        // Identify topics that already exist and mark them with the appropriate error
+        request.topics().stream().filter(creatableTopic -> topicsByName.containsKey(creatableTopic.name()))
+                .forEach(t -> topicErrors.put(t.name(), new ApiError(Errors.TOPIC_ALREADY_EXISTS)));
+
+        // Verify that the configurations for the new topics are OK, and figure out what
+        // ConfigRecords should be created.
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges =
+            computeConfigChanges(topicErrors, request.topics());
+        ControllerResult<Map<ConfigResource, ApiError>> configResult =
+            configurationControl.incrementalAlterConfigs(configChanges);
+        for (Entry<ConfigResource, ApiError> entry : configResult.response().entrySet()) {
+            if (entry.getValue().isFailure()) {
+                topicErrors.put(entry.getKey().name(), entry.getValue());
+            }
+        }
+        records.addAll(configResult.records());
+
+        // Try to create whatever topics are needed.
+        Map<String, CreatableTopicResult> successes = new HashMap<>();
+        for (CreatableTopic topic : request.topics()) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            ApiError error = createTopic(topic, records, successes);
+            if (error.isFailure()) {
+                topicErrors.put(topic.name(), error);
+            }
+        }
+
+        // Create responses for all topics.
+        CreateTopicsResponseData data = new CreateTopicsResponseData();
+        StringBuilder resultsBuilder = new StringBuilder();
+        String resultsPrefix = "";
+        for (CreatableTopic topic : request.topics()) {
+            ApiError error = topicErrors.get(topic.name());
+            if (error != null) {
+                data.topics().add(new CreatableTopicResult().
+                    setName(topic.name()).
+                    setErrorCode(error.error().code()).
+                    setErrorMessage(error.message()));
+                resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                    append(error.error()).append(" (").append(error.message()).append(")");
+                resultsPrefix = ", ";
+                continue;
+            }
+            CreatableTopicResult result = successes.get(topic.name());
+            data.topics().add(result);
+            resultsBuilder.append(resultsPrefix).append(topic).append(": ").
+                append("SUCCESS");
+            resultsPrefix = ", ";
+        }
+        log.info("createTopics result(s): {}", resultsBuilder.toString());
+        return new ControllerResult<>(records, data);
+    }
+
+    private ApiError createTopic(CreatableTopic topic,
+                                 List<ApiMessageAndVersion> records,
+                                 Map<String, CreatableTopicResult> successes) {
+        Map<Integer, PartitionControlInfo> newParts = new HashMap<>();
+        if (!topic.assignments().isEmpty()) {
+            if (topic.replicationFactor() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but replication " +
+                    "factor was not set to -1.");
+            }
+            if (topic.numPartitions() != -1) {
+                return new ApiError(Errors.INVALID_REQUEST,
+                    "A manual partition assignment was specified, but numPartitions " +
+                        "was not set to -1.");
+            }
+            for (CreatableReplicaAssignment assignment : topic.assignments()) {
+                if (newParts.containsKey(assignment.partitionIndex())) {
+                    return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                        "Found multiple manual partition assignments for partition " +
+                            assignment.partitionIndex());
+                }
+                HashSet<Integer> brokerIds = new HashSet<>();
+                for (int brokerId : assignment.brokerIds()) {
+                    if (!brokerIds.add(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment specifies the same node " +
+                                "id more than once.");
+                    } else if (!clusterControl.unfenced(brokerId)) {
+                        return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT,
+                            "The manual partition assignment contains node " + brokerId +
+                                ", but that node is not usable.");
+                    }
+                }
+                int[] replicas = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    replicas[i] = assignment.brokerIds().get(i);
+                }
+                int[] isr = new int[assignment.brokerIds().size()];
+                for (int i = 0; i < replicas.length; i++) {
+                    isr[i] = assignment.brokerIds().get(i);
+                }
+                newParts.put(assignment.partitionIndex(),
+                    new PartitionControlInfo(replicas, isr, null, null, isr[0], 0, 0));
+            }
+        } else if (topic.replicationFactor() < -1 || topic.replicationFactor() == 0) {
+            return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                "Replication factor was set to an invalid non-positive value.");
+        } else if (!topic.assignments().isEmpty()) {
+            return new ApiError(Errors.INVALID_REQUEST,
+                "Replication factor was not set to -1 but a manual partition " +
+                    "assignment was specified.");
+        } else if (topic.numPartitions() < -1 || topic.numPartitions() == 0) {
+            return new ApiError(Errors.INVALID_PARTITIONS,
+                "Number of partitions was set to an invalid non-positive value.");
+        } else {
+            int numPartitions = topic.numPartitions() == -1 ?
+                defaultNumPartitions : topic.numPartitions();
+            short replicationFactor = topic.replicationFactor() == -1 ?
+                defaultReplicationFactor : topic.replicationFactor();
+            try {
+                List<List<Integer>> replicas = clusterControl.
+                    placeReplicas(numPartitions, replicationFactor);
+                for (int partitionId = 0; partitionId < replicas.size(); partitionId++) {
+                    int[] r = Replicas.toArray(replicas.get(partitionId));
+                    newParts.put(partitionId,
+                        new PartitionControlInfo(r, r, null, null, r[0], 0, 0));
+                }
+            } catch (InvalidReplicationFactorException e) {
+                return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
+                    "Unable to replicate the partition " + replicationFactor +
+                        " times: " + e.getMessage());
+            }
+        }
+        Uuid topicId = new Uuid(random.nextLong(), random.nextLong());
+        successes.put(topic.name(), new CreatableTopicResult().
+            setName(topic.name()).
+            setTopicId(topicId).
+            setErrorCode((short) 0).
+            setErrorMessage(null).
+            setNumPartitions(newParts.size()).
+            setReplicationFactor((short) newParts.get(0).replicas.length));
+        records.add(new ApiMessageAndVersion(new TopicRecord().
+            setName(topic.name()).
+            setTopicId(topicId), (short) 0));
+        for (Entry<Integer, PartitionControlInfo> partEntry : newParts.entrySet()) {
+            int partitionIndex = partEntry.getKey();
+            PartitionControlInfo info = partEntry.getValue();
+            records.add(new ApiMessageAndVersion(new PartitionRecord().
+                setPartitionId(partitionIndex).
+                setTopicId(topicId).
+                setReplicas(Replicas.toList(info.replicas)).
+                setIsr(Replicas.toList(info.isr)).
+                setRemovingReplicas(null).
+                setAddingReplicas(null).
+                setLeader(info.leader).
+                setLeaderEpoch(info.leaderEpoch).
+                setPartitionEpoch(0), (short) 0));
+        }
+        return ApiError.NONE;
+    }
+
+    static void validateNewTopicNames(Map<String, ApiError> topicErrors,
+                                      CreatableTopicCollection topics) {
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            try {
+                Topic.validate(topic.name());
+            } catch (InvalidTopicException e) {
+                topicErrors.put(topic.name(),
+                    new ApiError(Errors.INVALID_TOPIC_EXCEPTION, e.getMessage()));
+            }
+        }
+    }
+
+    static Map<ConfigResource, Map<String, Entry<OpType, String>>>
+            computeConfigChanges(Map<String, ApiError> topicErrors,
+                                 CreatableTopicCollection topics) {
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges = new HashMap<>();
+        for (CreatableTopic topic : topics) {
+            if (topicErrors.containsKey(topic.name())) continue;
+            Map<String, Entry<OpType, String>> topicConfigs = new HashMap<>();
+            for (CreateTopicsRequestData.CreateableTopicConfig config : topic.configs()) {
+                topicConfigs.put(config.name(), new SimpleImmutableEntry<>(SET, config.value()));
+            }
+            if (!topicConfigs.isEmpty()) {
+                configChanges.put(new ConfigResource(TOPIC, topic.name()), topicConfigs);
+            }
+        }
+        return configChanges;
+    }
+
+    // VisibleForTesting
+    PartitionControlInfo getPartition(Uuid topicId, int partitionId) {
+        TopicControlInfo topic = topics.get(topicId);
+        if (topic == null) {
+            return null;
+        }
+        return topic.parts.get(partitionId);
+    }
+
+    // VisibleForTesting
+    BrokersToIsrs brokersToIsrs() {
+        return brokersToIsrs;
+    }
+
+    ControllerResult<AlterIsrResponseData> alterIsr(AlterIsrRequestData request) {
+        clusterControl.checkBrokerEpoch(request.brokerId(), request.brokerEpoch());
+        AlterIsrResponseData response = new AlterIsrResponseData();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        for (AlterIsrRequestData.TopicData topicData : request.topics()) {
+            AlterIsrResponseData.TopicData responseTopicData =
+                new AlterIsrResponseData.TopicData().setName(topicData.name());
+            response.topics().add(responseTopicData);
+            Uuid topicId = topicsByName.get(topicData.name());
+            if (topicId == null || !topics.containsKey(topicId)) {
+                for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()));
+                }
+                continue;
+            }
+            TopicControlInfo topic = topics.get(topicId);
+            for (AlterIsrRequestData.PartitionData partitionData : topicData.partitions()) {
+                PartitionControlInfo partition = topic.parts.get(partitionData.partitionIndex());
+                if (partition == null) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()));
+                    continue;
+                }
+                if (partitionData.leaderEpoch() != partition.leaderEpoch) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.FENCED_LEADER_EPOCH.code()));
+                    continue;
+                }
+                if (partitionData.currentIsrVersion() != partition.partitionEpoch) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.INVALID_UPDATE_VERSION.code()));
+                    continue;
+                }
+                int[] newIsr = Replicas.toArray(partitionData.newIsr());
+                if (!Replicas.validateIsr(partition.replicas, newIsr)) {
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.INVALID_REQUEST.code()));
+                }
+                int newLeader, newLeaderEpoch;
+                if (Replicas.contains(newIsr, partition.leader)) {
+                    newLeader = partition.leader;
+                    newLeaderEpoch = partition.leaderEpoch;
+                } else {
+                    newLeader = partition.chooseNewLeader(newIsr, false);

Review comment:
       Currently, the follower never removes the leader out of ISR. So, perhaps we should just throw an exception if this is not the case.

##########
File path: metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##########
@@ -0,0 +1,920 @@
+/*
+ * 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() {
+        int latestController = logManager.leader().nodeId();
+        if (latestController < 0) {
+            return new NotControllerException("No controller appears to be active.");
+        } else {
+            return new NotControllerException(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX +
+                latestController);
+        }
+    }
+
+    public static int exceptionToApparentController(NotControllerException e) {
+        if (e.getMessage().startsWith(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX)) {
+            return Integer.parseInt(e.getMessage().substring(
+                ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX.length()));
+        } else {
+            return -1;
+        }
+    }
+
+    private void handleEventEnd(String name, long startProcessingTimeNs) {
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs;
+        log.debug("Processed {} in {} us", name,
+            TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS));
+    }
+
+    private Throwable handleEventException(String name,
+                                           Optional<Long> startProcessingTimeNs,
+                                           Throwable exception) {
+        if (!startProcessingTimeNs.isPresent()) {
+            log.debug("unable to start processing {} because of {}.", name,
+                exception.getClass().getSimpleName());
+            if (exception instanceof ApiException) {
+                return exception;
+            } else {
+                return new UnknownServerException(exception);
+            }
+        }
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs.get();
+        long deltaUs = TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS);
+        if (exception instanceof ApiException) {
+            log.debug("{}: failed with {} in {} us", name,
+                exception.getClass().getSimpleName(), deltaUs);
+            return exception;
+        }
+        log.warn("{}: failed with unknown server exception {} at epoch {} in {} us.  " +
+            "Reverting to last committed offset {}.",
+            this, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs,
+            lastCommittedOffset, exception);
+        renounce();
+        return new UnknownServerException(exception);
+    }
+
+    /**
+     * A controller event for handling internal state changes, such as Raft inputs.
+     */
+    class ControlEvent implements EventQueue.Event {
+        private final String name;
+        private final Runnable handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControlEvent(String name, Runnable handler) {
+            this.name = name;
+            this.handler = handler;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            log.debug("Executing {}.", this);
+            handler.run();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            handleEventException(name, startProcessingTimeNs, exception);
+        }
+
+        @Override
+        public String toString() {
+            return name;
+        }
+    }
+
+    private void appendControlEvent(String name, Runnable handler) {
+        ControlEvent event = new ControlEvent(name, handler);
+        queue.append(event);
+    }
+
+    /**
+     * A controller event that reads the committed internal state in order to expose it
+     * to an API.
+     */
+    class ControllerReadEvent<T> implements EventQueue.Event {
+        private final String name;
+        private final CompletableFuture<T> future;
+        private final Supplier<T> handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControllerReadEvent(String name, Supplier<T> handler) {
+            this.name = name;
+            this.future = new CompletableFuture<T>();
+            this.handler = handler;
+        }
+
+        CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            T value = handler.get();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+            future.complete(value);
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            future.completeExceptionally(
+                handleEventException(name, startProcessingTimeNs, exception));
+        }
+
+        @Override
+        public String toString() {
+            return name + "(" + System.identityHashCode(this) + ")";
+        }
+    }
+
+    // VisibleForTesting
+    ReplicationControlManager replicationControl() {
+        return replicationControl;
+    }
+
+    // VisibleForTesting
+    <T> CompletableFuture<T> appendReadEvent(String name, Supplier<T> handler) {
+        ControllerReadEvent<T> event = new ControllerReadEvent<T>(name, handler);
+        queue.append(event);
+        return event.future();
+    }
+
+    interface ControllerWriteOperation<T> {
+        /**
+         * Generate the metadata records needed to implement this controller write
+         * operation.  In general, this operation should not modify the "hard state" of
+         * the controller.  That modification will happen later on, when we replay the
+         * records generated by this function.
+         *
+         * There are cases where this function modifies the "soft state" of the
+         * controller.  Mainly, this happens when we process cluster heartbeats.
+         *
+         * This function also generates an RPC result.  In general, if the RPC resulted in
+         * an error, the RPC result will be an error, and the generated record list will
+         * be empty.  This would happen if we tried to create a topic with incorrect
+         * parameters, for example.  Of course, partial errors are possible for batch
+         * operations.
+         *
+         * @return              A tuple containing a list of records, and an RPC result.
+         */
+        ControllerResult<T> generateRecordsAndResult() throws Exception;
+
+        /**
+         * Once we've passed the records to the Raft layer, we will invoke this function
+         * with the end offset at which those records were placed.  If there were no
+         * records to write, we'll just pass the last write offset.
+         */
+        default void processBatchEndOffset(long offset) {}
+    }
+
+    /**
+     * A controller event that modifies the controller state.
+     */
+    class ControllerWriteEvent<T> implements EventQueue.Event, DeferredEvent {
+        private final String name;
+        private final CompletableFuture<T> future;
+        private final ControllerWriteOperation<T> op;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+        private ControllerResultAndOffset<T> resultAndOffset;
+
+        ControllerWriteEvent(String name, ControllerWriteOperation<T> op) {
+            this.name = name;
+            this.future = new CompletableFuture<T>();
+            this.op = op;
+            this.resultAndOffset = null;
+        }
+
+        CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public void run() throws Exception {
+            long controllerEpoch = curClaimEpoch;
+            if (controllerEpoch == -1) {
+                throw newNotControllerException();
+            }
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            ControllerResult<T> result = op.generateRecordsAndResult();
+            if (result.records().isEmpty()) {
+                op.processBatchEndOffset(writeOffset);
+                // If the operation did not return any records, then it was actually just
+                //a read after all, and not a read + write.  However, this read was done
+                //from the latest in-memory state, which might contain uncommitted data.
+                Optional<Long> maybeOffset = purgatory.highestPendingOffset();
+                if (!maybeOffset.isPresent()) {
+                    // If the purgatory is empty, there are no pending operations and no
+                    // uncommitted state.  We can return immediately.
+                    this.resultAndOffset = new ControllerResultAndOffset<>(-1,
+                        new ArrayList<>(), result.response());
+                    log.debug("Completing read-only operation {} immediately because " +
+                        "the purgatory is empty.", this);
+                    complete(null);
+                    return;
+                }
+                // If there are operations in the purgatory, we want to wait for the latest
+                // one to complete before returning our result to the user.
+                this.resultAndOffset = new ControllerResultAndOffset<>(maybeOffset.get(),
+                    result.records(), result.response());
+                log.debug("Read-only operation {} will be completed when the log " +
+                    "reaches offset {}", this, resultAndOffset.offset());
+            } else {
+                // If the operation returned a batch of records, those records need to be
+                // written before we can return our result to the user.  Here, we hand off
+                // the batch of records to the metadata log manager.  They will be written
+                // out asynchronously.
+                long offset = logManager.scheduleWrite(controllerEpoch, result.records());
+                op.processBatchEndOffset(offset);
+                writeOffset = offset;
+                this.resultAndOffset = new ControllerResultAndOffset<>(offset,
+                    result.records(), result.response());
+                for (ApiMessageAndVersion message : result.records()) {
+                    replay(message.message());

Review comment:
       Some of the replay (e.g. UNREGISTER_BROKER_RECORD) could throw exceptions. We probably need to turn the exception into an error response. Are we handling that already?

##########
File path: metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##########
@@ -0,0 +1,456 @@
+/*
+ * 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.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+    class ReadyBrokersFuture {
+        private final CompletableFuture<Void> future;
+        private final int minBrokers;
+
+        ReadyBrokersFuture(CompletableFuture<Void> future, int minBrokers) {
+            this.future = future;
+            this.minBrokers = minBrokers;
+        }
+
+        boolean check() {
+            int numUnfenced = 0;
+            for (BrokerRegistration registration : brokerRegistrations.values()) {
+                if (!registration.fenced()) {
+                    numUnfenced++;
+                }
+                if (numUnfenced >= minBrokers) {
+                    return true;
+                }
+            }
+            return false;
+        }
+    }
+
+    /**
+     * The SLF4J log context.
+     */
+    private final LogContext logContext;
+
+    /**
+     * The SLF4J log object.
+     */
+    private final Logger log;
+
+    /**
+     * The Kafka clock object to use.
+     */
+    private final Time time;
+
+    /**
+     * How long sessions should last, in nanoseconds.
+     */
+    private final long sessionTimeoutNs;
+
+    /**
+     * The replica placement policy to use.
+     */
+    private final ReplicaPlacementPolicy placementPolicy;
+
+    /**
+     * Maps broker IDs to broker registrations.
+     */
+    private final TimelineHashMap<Integer, BrokerRegistration> brokerRegistrations;
+
+    /**
+     * The broker heartbeat manager, or null if this controller is on standby.
+     */
+    private BrokerHeartbeatManager heartbeatManager;
+
+    /**
+     * A future which is completed as soon as we have the given number of brokers
+     * ready.
+     */
+    private Optional<ReadyBrokersFuture> readyBrokersFuture;
+
+    ClusterControlManager(LogContext logContext,
+                          Time time,
+                          SnapshotRegistry snapshotRegistry,
+                          long sessionTimeoutNs,
+                          ReplicaPlacementPolicy placementPolicy) {
+        this.logContext = logContext;
+        this.log = logContext.logger(ClusterControlManager.class);
+        this.time = time;
+        this.sessionTimeoutNs = sessionTimeoutNs;
+        this.placementPolicy = placementPolicy;
+        this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.heartbeatManager = null;
+        this.readyBrokersFuture = Optional.empty();
+    }
+
+    /**
+     * Transition this ClusterControlManager to active.
+     */
+    public void activate() {
+        heartbeatManager = new BrokerHeartbeatManager(logContext, time, sessionTimeoutNs);
+        for (BrokerRegistration registration : brokerRegistrations.values()) {
+            heartbeatManager.touch(registration.id(), registration.fenced(), -1);
+        }
+    }
+
+    /**
+     * Transition this ClusterControlManager to standby.
+     */
+    public void deactivate() {
+        heartbeatManager = null;
+    }
+
+    // VisibleForTesting
+    Map<Integer, BrokerRegistration> brokerRegistrations() {
+        return brokerRegistrations;
+    }
+
+    /**
+     * Process an incoming broker registration request.
+     */
+    public ControllerResult<BrokerRegistrationReply> registerBroker(
+            BrokerRegistrationRequestData request, long brokerEpoch,
+            FeatureManager.FinalizedFeaturesAndEpoch finalizedFeaturesAndEpoch) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        int brokerId = request.brokerId();
+        BrokerRegistration existing = brokerRegistrations.get(brokerId);
+        if (existing != null) {
+            if (heartbeatManager.hasValidSession(brokerId)) {
+                if (!existing.incarnationId().equals(request.incarnationId())) {
+                    throw new DuplicateBrokerRegistrationException("Another broker is " +
+                        "registered with that broker id.");
+                }
+            } else {
+                if (!existing.incarnationId().equals(request.incarnationId())) {
+                    // Remove any existing session for the old broker incarnation.
+                    heartbeatManager.remove(brokerId);
+                    existing = null;
+                }
+            }
+        }
+
+        RegisterBrokerRecord record = new RegisterBrokerRecord().setBrokerId(brokerId).
+            setIncarnationId(request.incarnationId()).
+            setBrokerEpoch(brokerEpoch).
+            setRack(request.rack());
+        for (BrokerRegistrationRequestData.Listener listener : request.listeners()) {
+            record.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint().
+                setHost(listener.host()).
+                setName(listener.name()).
+                setPort(listener.port()).
+                setSecurityProtocol(listener.securityProtocol()));
+        }
+        for (BrokerRegistrationRequestData.Feature feature : request.features()) {
+            VersionRange supported = finalizedFeaturesAndEpoch.finalizedFeatures().
+                getOrDefault(feature.name(), VersionRange.ALL);
+            if (!supported.contains(new VersionRange(feature.minSupportedVersion(),
+                    feature.maxSupportedVersion()))) {
+                throw new UnsupportedVersionException("Unable to register because " +
+                    "the broker has an unsupported version of " + feature.name());
+            }
+            record.features().add(new RegisterBrokerRecord.BrokerFeature().
+                setName(feature.name()).
+                setMinSupportedVersion(feature.minSupportedVersion()).
+                setMaxSupportedVersion(feature.maxSupportedVersion()));
+        }
+
+        if (existing == null) {
+            heartbeatManager.touch(brokerId, true, -1);
+        } else {
+            heartbeatManager.touch(brokerId, existing.fenced(), -1);
+        }
+
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        records.add(new ApiMessageAndVersion(record, (short) 0));
+        return new ControllerResult<>(records, new BrokerRegistrationReply(brokerEpoch));
+    }
+
+    public ControllerResult<Void> decommissionBroker(int brokerId) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        BrokerRegistration existing = brokerRegistrations.get(brokerId);
+        if (existing == null) {
+            return new ControllerResult<>(new ArrayList<>(), null);
+        }
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        records.add(new ApiMessageAndVersion(
+            new UnregisterBrokerRecord().
+                setBrokerId(brokerId).
+                setBrokerEpoch(existing.epoch()),
+            (short) 0));
+        heartbeatManager.remove(brokerId);
+        return new ControllerResult<>(records, null);
+    }
+
+    public ControllerResult<BrokerHeartbeatReply> processBrokerHeartbeat(
+            BrokerHeartbeatRequestData request,
+            long lastCommittedOffset,
+            boolean movingLeadersForShutDown) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        int brokerId = request.brokerId();
+        BrokerRegistration registration = brokerRegistrations.get(brokerId);
+        if (registration == null) {
+            throw new StaleBrokerEpochException("No registration found for broker " +
+                request.brokerId());
+        }
+        if (request.brokerEpoch() != registration.epoch()) {
+            throw new StaleBrokerEpochException("Expected broker epoch " +
+                request.brokerEpoch() + "; got epoch " + registration.epoch());
+        }
+        boolean isCaughtUp = request.currentMetadataOffset() >= lastCommittedOffset;
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        boolean isFenced = registration.fenced();
+        boolean shouldShutdown = false;
+        if (isFenced) {
+            if (request.wantShutDown()) {
+                // If the broker is fenced, and requests a shutdown, do it immediately.
+                isFenced = true;
+                shouldShutdown = true;
+            } else if (isCaughtUp && !request.wantFence()) {
+                records.add(new ApiMessageAndVersion(new UnfenceBrokerRecord().
+                    setId(brokerId).setEpoch(request.brokerEpoch()), (short) 0));
+                isFenced = false;
+                shouldShutdown = false;
+            }
+        } else {
+            if (request.wantFence()) {
+                records.add(new ApiMessageAndVersion(new FenceBrokerRecord().
+                    setId(brokerId).setEpoch(request.brokerEpoch()), (short) 0));
+                isFenced = true;
+                shouldShutdown = request.wantShutDown();
+            } else {
+                if (request.wantShutDown()) {
+                    heartbeatManager.beginBrokerShutDown(request.brokerId(), movingLeadersForShutDown);
+                }
+                if (heartbeatManager.shouldShutDown(request.brokerId())) {
+                    isFenced = true;
+                    shouldShutdown = true;
+                    records.add(new ApiMessageAndVersion(new FenceBrokerRecord().

Review comment:
       If we want to log the shutting down the broker, it seems it's more consistent if we always log it. Now, it seems we log it only when leaders need to be moved.

##########
File path: metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##########
@@ -0,0 +1,920 @@
+/*
+ * 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() {
+        int latestController = logManager.leader().nodeId();
+        if (latestController < 0) {
+            return new NotControllerException("No controller appears to be active.");
+        } else {
+            return new NotControllerException(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX +
+                latestController);
+        }
+    }
+
+    public static int exceptionToApparentController(NotControllerException e) {
+        if (e.getMessage().startsWith(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX)) {
+            return Integer.parseInt(e.getMessage().substring(
+                ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX.length()));
+        } else {
+            return -1;
+        }
+    }
+
+    private void handleEventEnd(String name, long startProcessingTimeNs) {
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs;
+        log.debug("Processed {} in {} us", name,
+            TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS));
+    }
+
+    private Throwable handleEventException(String name,
+                                           Optional<Long> startProcessingTimeNs,
+                                           Throwable exception) {
+        if (!startProcessingTimeNs.isPresent()) {
+            log.debug("unable to start processing {} because of {}.", name,
+                exception.getClass().getSimpleName());
+            if (exception instanceof ApiException) {
+                return exception;
+            } else {
+                return new UnknownServerException(exception);
+            }
+        }
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs.get();
+        long deltaUs = TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS);
+        if (exception instanceof ApiException) {
+            log.debug("{}: failed with {} in {} us", name,
+                exception.getClass().getSimpleName(), deltaUs);
+            return exception;
+        }
+        log.warn("{}: failed with unknown server exception {} at epoch {} in {} us.  " +
+            "Reverting to last committed offset {}.",
+            this, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs,
+            lastCommittedOffset, exception);
+        renounce();
+        return new UnknownServerException(exception);
+    }
+
+    /**
+     * A controller event for handling internal state changes, such as Raft inputs.
+     */
+    class ControlEvent implements EventQueue.Event {
+        private final String name;
+        private final Runnable handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControlEvent(String name, Runnable handler) {
+            this.name = name;
+            this.handler = handler;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            log.debug("Executing {}.", this);
+            handler.run();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            handleEventException(name, startProcessingTimeNs, exception);
+        }
+
+        @Override
+        public String toString() {
+            return name;
+        }
+    }
+
+    private void appendControlEvent(String name, Runnable handler) {
+        ControlEvent event = new ControlEvent(name, handler);
+        queue.append(event);
+    }
+
+    /**
+     * A controller event that reads the committed internal state in order to expose it
+     * to an API.
+     */
+    class ControllerReadEvent<T> implements EventQueue.Event {
+        private final String name;
+        private final CompletableFuture<T> future;
+        private final Supplier<T> handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControllerReadEvent(String name, Supplier<T> handler) {
+            this.name = name;
+            this.future = new CompletableFuture<T>();
+            this.handler = handler;
+        }
+
+        CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            T value = handler.get();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+            future.complete(value);
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            future.completeExceptionally(
+                handleEventException(name, startProcessingTimeNs, exception));
+        }
+
+        @Override
+        public String toString() {
+            return name + "(" + System.identityHashCode(this) + ")";
+        }
+    }
+
+    // VisibleForTesting
+    ReplicationControlManager replicationControl() {
+        return replicationControl;
+    }
+
+    // VisibleForTesting
+    <T> CompletableFuture<T> appendReadEvent(String name, Supplier<T> handler) {
+        ControllerReadEvent<T> event = new ControllerReadEvent<T>(name, handler);
+        queue.append(event);
+        return event.future();
+    }
+
+    interface ControllerWriteOperation<T> {
+        /**
+         * Generate the metadata records needed to implement this controller write
+         * operation.  In general, this operation should not modify the "hard state" of
+         * the controller.  That modification will happen later on, when we replay the
+         * records generated by this function.
+         *
+         * There are cases where this function modifies the "soft state" of the
+         * controller.  Mainly, this happens when we process cluster heartbeats.
+         *
+         * This function also generates an RPC result.  In general, if the RPC resulted in
+         * an error, the RPC result will be an error, and the generated record list will
+         * be empty.  This would happen if we tried to create a topic with incorrect
+         * parameters, for example.  Of course, partial errors are possible for batch
+         * operations.
+         *
+         * @return              A tuple containing a list of records, and an RPC result.
+         */
+        ControllerResult<T> generateRecordsAndResult() throws Exception;
+
+        /**
+         * Once we've passed the records to the Raft layer, we will invoke this function
+         * with the end offset at which those records were placed.  If there were no
+         * records to write, we'll just pass the last write offset.
+         */
+        default void processBatchEndOffset(long offset) {}
+    }
+
+    /**
+     * A controller event that modifies the controller state.
+     */
+    class ControllerWriteEvent<T> implements EventQueue.Event, DeferredEvent {
+        private final String name;
+        private final CompletableFuture<T> future;
+        private final ControllerWriteOperation<T> op;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+        private ControllerResultAndOffset<T> resultAndOffset;
+
+        ControllerWriteEvent(String name, ControllerWriteOperation<T> op) {
+            this.name = name;
+            this.future = new CompletableFuture<T>();
+            this.op = op;
+            this.resultAndOffset = null;
+        }
+
+        CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public void run() throws Exception {
+            long controllerEpoch = curClaimEpoch;
+            if (controllerEpoch == -1) {
+                throw newNotControllerException();
+            }
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            ControllerResult<T> result = op.generateRecordsAndResult();
+            if (result.records().isEmpty()) {
+                op.processBatchEndOffset(writeOffset);
+                // If the operation did not return any records, then it was actually just
+                //a read after all, and not a read + write.  However, this read was done
+                //from the latest in-memory state, which might contain uncommitted data.
+                Optional<Long> maybeOffset = purgatory.highestPendingOffset();
+                if (!maybeOffset.isPresent()) {
+                    // If the purgatory is empty, there are no pending operations and no
+                    // uncommitted state.  We can return immediately.
+                    this.resultAndOffset = new ControllerResultAndOffset<>(-1,
+                        new ArrayList<>(), result.response());
+                    log.debug("Completing read-only operation {} immediately because " +
+                        "the purgatory is empty.", this);
+                    complete(null);
+                    return;
+                }
+                // If there are operations in the purgatory, we want to wait for the latest
+                // one to complete before returning our result to the user.
+                this.resultAndOffset = new ControllerResultAndOffset<>(maybeOffset.get(),
+                    result.records(), result.response());
+                log.debug("Read-only operation {} will be completed when the log " +
+                    "reaches offset {}", this, resultAndOffset.offset());
+            } else {
+                // If the operation returned a batch of records, those records need to be
+                // written before we can return our result to the user.  Here, we hand off
+                // the batch of records to the metadata log manager.  They will be written
+                // out asynchronously.
+                long offset = logManager.scheduleWrite(controllerEpoch, result.records());
+                op.processBatchEndOffset(offset);
+                writeOffset = offset;
+                this.resultAndOffset = new ControllerResultAndOffset<>(offset,
+                    result.records(), result.response());
+                for (ApiMessageAndVersion message : result.records()) {
+                    replay(message.message());
+                }
+                snapshotRegistry.createSnapshot(offset);
+                log.debug("Read-write operation {} will be completed when the log " +
+                    "reaches offset {}.", this, resultAndOffset.offset());
+            }
+            purgatory.add(resultAndOffset.offset(), this);
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            complete(exception);
+        }
+
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                handleEventEnd(this.toString(), startProcessingTimeNs.get());
+                future.complete(resultAndOffset.response());
+            } else {
+                future.completeExceptionally(
+                    handleEventException(name, startProcessingTimeNs, exception));
+            }
+        }
+
+        @Override
+        public String toString() {
+            return name + "(" + System.identityHashCode(this) + ")";
+        }
+    }
+
+    private <T> CompletableFuture<T> appendWriteEvent(String name,
+                                                      long timeoutMs,
+                                                      ControllerWriteOperation<T> op) {
+        ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
+        queue.appendWithDeadline(time.nanoseconds() +
+            TimeUnit.NANOSECONDS.convert(timeoutMs, TimeUnit.MILLISECONDS), event);
+        return event.future();
+    }
+
+    private <T> CompletableFuture<T> appendWriteEvent(String name,
+                                                      ControllerWriteOperation<T> op) {
+        ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
+        queue.append(event);
+        return event.future();
+    }
+
+    class QuorumMetaLogListener implements MetaLogListener {
+        @Override
+        public void handleCommits(long offset, List<ApiMessage> messages) {
+            appendControlEvent("handleCommits[" + offset + "]", () -> {
+                if (curClaimEpoch == -1) {
+                    if (log.isDebugEnabled()) {
+                        if (log.isTraceEnabled()) {
+                            log.trace("Replaying commits from the active node up to " +
+                                "offset {}: {}.", offset, messages.stream().
+                                map(m -> m.toString()).collect(Collectors.joining(", ")));
+                        } else {
+                            log.debug("Replaying commits from the active node up to " +
+                                "offset {}.", offset);
+                        }
+                    }
+                    for (ApiMessage message : messages) {
+                        replay(message);
+                    }
+                } else {
+                    log.debug("Completing purgatory items up to offset {}.", offset);
+
+                    // Complete any events in the purgatory that were waiting for this offset.
+                    purgatory.completeUpTo(offset);
+
+                    // Delete all snapshots older than the offset.
+                    // TODO: add an exception here for when we're writing out a log snapshot
+                    snapshotRegistry.deleteSnapshotsUpTo(offset);
+                }
+                lastCommittedOffset = offset;
+            });
+        }
+
+        @Override
+        public void handleNewLeader(MetaLogLeader newLeader) {
+            if (newLeader.nodeId() == nodeId) {
+                final long newEpoch = newLeader.epoch();
+                appendControlEvent("handleClaim[" + newEpoch + "]", () -> {
+                    long curEpoch = curClaimEpoch;
+                    if (curEpoch != -1) {
+                        throw new RuntimeException("Tried to claim controller epoch " +
+                            newEpoch + ", but we never renounced controller epoch " +
+                            curEpoch);
+                    }
+                    log.info("Becoming active at controller epoch {}.", newEpoch);
+                    curClaimEpoch = newEpoch;
+                    writeOffset = lastCommittedOffset;
+                    clusterControl.activate();
+                });
+            }
+        }
+
+        @Override
+        public void handleRenounce(long oldEpoch) {
+            appendControlEvent("handleRenounce[" + oldEpoch + "]", () -> {
+                if (curClaimEpoch == oldEpoch) {
+                    log.info("Renouncing the leadership at oldEpoch {} due to a metadata " +
+                            "log event. Reverting to last committed offset {}.", curClaimEpoch,
+                        lastCommittedOffset);
+                    renounce();
+                }
+            });
+        }
+
+        @Override
+        public void beginShutdown() {
+            queue.beginShutdown("MetaLogManager.Listener");
+        }
+    }
+
+    private void renounce() {
+        curClaimEpoch = -1;
+        purgatory.failAll(newNotControllerException());
+        snapshotRegistry.revertToSnapshot(lastCommittedOffset);
+        snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset);
+        writeOffset = -1;
+        clusterControl.deactivate();
+        cancelMaybeFenceReplicas();
+    }
+
+    private <T> void scheduleDeferredWriteEvent(String name, long deadlineNs,
+                                                ControllerWriteOperation<T> op) {
+        ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
+        queue.scheduleDeferred(name, new EarliestDeadlineFunction(deadlineNs), event);
+        event.future.exceptionally(e -> {
+            if (e instanceof UnknownServerException && e.getCause() != null &&
+                    e.getCause() instanceof RejectedExecutionException) {
+                log.error("Cancelling write event {} because the event queue is closed.", name);
+                return null;
+            }
+            log.error("Unexpected exception while executing deferred write event {}. " +
+                "Rescheduling for a minute from now.", name, e);
+            scheduleDeferredWriteEvent(name,
+                deadlineNs + TimeUnit.NANOSECONDS.convert(1, TimeUnit.MINUTES), op);
+            return null;
+        });
+    }
+
+    static final String MAYBE_FENCE_REPLICAS = "maybeFenceReplicas";
+
+    class MaybeFenceReplicas implements ControllerWriteOperation<Void> {
+        @Override
+        public ControllerResult<Void> generateRecordsAndResult() {
+            ControllerResult<Set<Integer>> result =
+                clusterControl.maybeFenceLeastRecentlyContacted();
+            for (int brokerId : result.response()) {
+                replicationControl.removeFromIsr(brokerId, result.records());
+            }
+            rescheduleMaybeFenceReplicas();
+            return new ControllerResult<>(result.records(), null);
+        }
+    }
+
+    private void rescheduleMaybeFenceReplicas() {
+        long nextCheckTimeNs = clusterControl.nextCheckTimeNs();
+        if (nextCheckTimeNs == Long.MAX_VALUE) {
+            cancelMaybeFenceReplicas();
+        } else {
+            scheduleDeferredWriteEvent(MAYBE_FENCE_REPLICAS, nextCheckTimeNs,
+                new MaybeFenceReplicas());
+        }
+    }
+
+    private void cancelMaybeFenceReplicas() {
+        queue.cancelDeferred(MAYBE_FENCE_REPLICAS);
+    }
+
+    @SuppressWarnings("unchecked")
+    private void replay(ApiMessage message) {
+        try {
+            MetadataRecordType type = MetadataRecordType.fromId(message.apiKey());
+            switch (type) {
+                case REGISTER_BROKER_RECORD:
+                    clusterControl.replay((RegisterBrokerRecord) message);
+                    break;
+                case UNREGISTER_BROKER_RECORD:
+                    clusterControl.replay((UnregisterBrokerRecord) message);
+                    break;
+                case FENCE_BROKER_RECORD:
+                    clusterControl.replay((FenceBrokerRecord) message);
+                    break;
+                case UNFENCE_BROKER_RECORD:
+                    clusterControl.replay((UnfenceBrokerRecord) message);
+                    break;
+                case TOPIC_RECORD:
+                    replicationControl.replay((TopicRecord) message);
+                    break;
+                case PARTITION_RECORD:
+                    replicationControl.replay((PartitionRecord) message);
+                    break;
+                case CONFIG_RECORD:
+                    configurationControl.replay((ConfigRecord) message);
+                    break;
+                case QUOTA_RECORD:
+                    clientQuotaControlManager.replay((QuotaRecord) message);
+                    break;
+                case ISR_CHANGE_RECORD:
+                    replicationControl.replay((IsrChangeRecord) message);
+                    break;
+                default:
+                    throw new RuntimeException("Unhandled record type " + type);
+            }
+        } catch (Exception e) {
+            log.error("Error replaying record {}", message.toString(), e);
+        }
+    }
+
+    private final Logger log;
+
+    /**
+     * The ID of this controller node.
+     */
+    private final int nodeId;
+
+    /**
+     * The single-threaded queue that processes all of our events.
+     * It also processes timeouts.
+     */
+    private final KafkaEventQueue queue;
+
+    /**
+     * The Kafka clock object to use.
+     */
+    private final Time time;
+
+    /**
+     * A registry for snapshot data.  This must be accessed only by the event queue thread.
+     */
+    private final SnapshotRegistry snapshotRegistry;
+
+    /**
+     * The purgatory which holds deferred operations which are waiting for the metadata
+     * log's high water mark to advance.  This must be accessed only by the event queue thread.
+     */
+    private final ControllerPurgatory purgatory;
+
+    /**
+     * An object which stores the controller's dynamic configuration.
+     * This must be accessed only by the event queue thread.
+     */
+    private final ConfigurationControlManager configurationControl;
+
+    /**
+     * An object which stores the controller's dynamic client quotas.
+     * This must be accessed only by the event queue thread.
+     */
+    private final ClientQuotaControlManager clientQuotaControlManager;
+
+    /**
+     * An object which stores the controller's view of the cluster.
+     * This must be accessed only by the event queue thread.
+     */
+    private final ClusterControlManager clusterControl;
+
+    /**
+     * An object which stores the controller's view of the cluster features.
+     * This must be accessed only by the event queue thread.
+     */
+    private final FeatureControlManager featureControl;
+
+    /**
+     * An object which stores the controller's view of topics and partitions.
+     * This must be accessed only by the event queue thread.
+     */
+    private final ReplicationControlManager replicationControl;
+
+    /**
+     * The interface that we use to mutate the Raft log.
+     */
+    private final MetaLogManager logManager;
+
+    /**
+     * The interface that receives callbacks from the Raft log.  These callbacks are
+     * invoked from the Raft thread(s), not from the controller thread.
+     */
+    private final QuorumMetaLogListener metaLogListener;
+
+    /**
+     * If this controller is active, this is the non-negative controller epoch.
+     * Otherwise, this is -1.  This variable must be modified only from the controller
+     * thread, but it can be read from other threads.
+     */
+    private volatile long curClaimEpoch;

Review comment:
       I thought the raft leader epoch is an int since we store only int as leader epoch in the log?

##########
File path: metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##########
@@ -0,0 +1,920 @@
+/*
+ * 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() {
+        int latestController = logManager.leader().nodeId();
+        if (latestController < 0) {
+            return new NotControllerException("No controller appears to be active.");
+        } else {
+            return new NotControllerException(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX +
+                latestController);
+        }
+    }
+
+    public static int exceptionToApparentController(NotControllerException e) {
+        if (e.getMessage().startsWith(ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX)) {
+            return Integer.parseInt(e.getMessage().substring(
+                ACTIVE_CONTROLLER_EXCEPTION_TEXT_PREFIX.length()));
+        } else {
+            return -1;
+        }
+    }
+
+    private void handleEventEnd(String name, long startProcessingTimeNs) {
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs;
+        log.debug("Processed {} in {} us", name,
+            TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS));
+    }
+
+    private Throwable handleEventException(String name,
+                                           Optional<Long> startProcessingTimeNs,
+                                           Throwable exception) {
+        if (!startProcessingTimeNs.isPresent()) {
+            log.debug("unable to start processing {} because of {}.", name,
+                exception.getClass().getSimpleName());
+            if (exception instanceof ApiException) {
+                return exception;
+            } else {
+                return new UnknownServerException(exception);
+            }
+        }
+        long endProcessingTime = time.nanoseconds();
+        long deltaNs = endProcessingTime - startProcessingTimeNs.get();
+        long deltaUs = TimeUnit.MICROSECONDS.convert(deltaNs, TimeUnit.NANOSECONDS);
+        if (exception instanceof ApiException) {
+            log.debug("{}: failed with {} in {} us", name,
+                exception.getClass().getSimpleName(), deltaUs);
+            return exception;
+        }
+        log.warn("{}: failed with unknown server exception {} at epoch {} in {} us.  " +
+            "Reverting to last committed offset {}.",
+            this, exception.getClass().getSimpleName(), curClaimEpoch, deltaUs,
+            lastCommittedOffset, exception);
+        renounce();
+        return new UnknownServerException(exception);
+    }
+
+    /**
+     * A controller event for handling internal state changes, such as Raft inputs.
+     */
+    class ControlEvent implements EventQueue.Event {
+        private final String name;
+        private final Runnable handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControlEvent(String name, Runnable handler) {
+            this.name = name;
+            this.handler = handler;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            log.debug("Executing {}.", this);
+            handler.run();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            handleEventException(name, startProcessingTimeNs, exception);
+        }
+
+        @Override
+        public String toString() {
+            return name;
+        }
+    }
+
+    private void appendControlEvent(String name, Runnable handler) {
+        ControlEvent event = new ControlEvent(name, handler);
+        queue.append(event);
+    }
+
+    /**
+     * A controller event that reads the committed internal state in order to expose it
+     * to an API.
+     */
+    class ControllerReadEvent<T> implements EventQueue.Event {
+        private final String name;
+        private final CompletableFuture<T> future;
+        private final Supplier<T> handler;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControllerReadEvent(String name, Supplier<T> handler) {
+            this.name = name;
+            this.future = new CompletableFuture<T>();
+            this.handler = handler;
+        }
+
+        CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public void run() throws Exception {
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            T value = handler.get();
+            handleEventEnd(this.toString(), startProcessingTimeNs.get());
+            future.complete(value);
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            future.completeExceptionally(
+                handleEventException(name, startProcessingTimeNs, exception));
+        }
+
+        @Override
+        public String toString() {
+            return name + "(" + System.identityHashCode(this) + ")";
+        }
+    }
+
+    // VisibleForTesting
+    ReplicationControlManager replicationControl() {
+        return replicationControl;
+    }
+
+    // VisibleForTesting
+    <T> CompletableFuture<T> appendReadEvent(String name, Supplier<T> handler) {
+        ControllerReadEvent<T> event = new ControllerReadEvent<T>(name, handler);
+        queue.append(event);
+        return event.future();
+    }
+
+    interface ControllerWriteOperation<T> {
+        /**
+         * Generate the metadata records needed to implement this controller write
+         * operation.  In general, this operation should not modify the "hard state" of
+         * the controller.  That modification will happen later on, when we replay the
+         * records generated by this function.
+         *
+         * There are cases where this function modifies the "soft state" of the
+         * controller.  Mainly, this happens when we process cluster heartbeats.
+         *
+         * This function also generates an RPC result.  In general, if the RPC resulted in
+         * an error, the RPC result will be an error, and the generated record list will
+         * be empty.  This would happen if we tried to create a topic with incorrect
+         * parameters, for example.  Of course, partial errors are possible for batch
+         * operations.
+         *
+         * @return              A tuple containing a list of records, and an RPC result.
+         */
+        ControllerResult<T> generateRecordsAndResult() throws Exception;
+
+        /**
+         * Once we've passed the records to the Raft layer, we will invoke this function
+         * with the end offset at which those records were placed.  If there were no
+         * records to write, we'll just pass the last write offset.
+         */
+        default void processBatchEndOffset(long offset) {}
+    }
+
+    /**
+     * A controller event that modifies the controller state.
+     */
+    class ControllerWriteEvent<T> implements EventQueue.Event, DeferredEvent {
+        private final String name;
+        private final CompletableFuture<T> future;
+        private final ControllerWriteOperation<T> op;
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+        private ControllerResultAndOffset<T> resultAndOffset;
+
+        ControllerWriteEvent(String name, ControllerWriteOperation<T> op) {
+            this.name = name;
+            this.future = new CompletableFuture<T>();
+            this.op = op;
+            this.resultAndOffset = null;
+        }
+
+        CompletableFuture<T> future() {
+            return future;
+        }
+
+        @Override
+        public void run() throws Exception {
+            long controllerEpoch = curClaimEpoch;
+            if (controllerEpoch == -1) {
+                throw newNotControllerException();
+            }
+            startProcessingTimeNs = Optional.of(time.nanoseconds());
+            ControllerResult<T> result = op.generateRecordsAndResult();
+            if (result.records().isEmpty()) {
+                op.processBatchEndOffset(writeOffset);
+                // If the operation did not return any records, then it was actually just
+                //a read after all, and not a read + write.  However, this read was done
+                //from the latest in-memory state, which might contain uncommitted data.
+                Optional<Long> maybeOffset = purgatory.highestPendingOffset();
+                if (!maybeOffset.isPresent()) {
+                    // If the purgatory is empty, there are no pending operations and no
+                    // uncommitted state.  We can return immediately.
+                    this.resultAndOffset = new ControllerResultAndOffset<>(-1,
+                        new ArrayList<>(), result.response());
+                    log.debug("Completing read-only operation {} immediately because " +
+                        "the purgatory is empty.", this);
+                    complete(null);
+                    return;
+                }
+                // If there are operations in the purgatory, we want to wait for the latest
+                // one to complete before returning our result to the user.
+                this.resultAndOffset = new ControllerResultAndOffset<>(maybeOffset.get(),
+                    result.records(), result.response());
+                log.debug("Read-only operation {} will be completed when the log " +
+                    "reaches offset {}", this, resultAndOffset.offset());
+            } else {
+                // If the operation returned a batch of records, those records need to be
+                // written before we can return our result to the user.  Here, we hand off
+                // the batch of records to the metadata log manager.  They will be written
+                // out asynchronously.
+                long offset = logManager.scheduleWrite(controllerEpoch, result.records());
+                op.processBatchEndOffset(offset);
+                writeOffset = offset;
+                this.resultAndOffset = new ControllerResultAndOffset<>(offset,
+                    result.records(), result.response());
+                for (ApiMessageAndVersion message : result.records()) {
+                    replay(message.message());
+                }
+                snapshotRegistry.createSnapshot(offset);
+                log.debug("Read-write operation {} will be completed when the log " +
+                    "reaches offset {}.", this, resultAndOffset.offset());
+            }
+            purgatory.add(resultAndOffset.offset(), this);
+        }
+
+        @Override
+        public void handleException(Throwable exception) {
+            complete(exception);
+        }
+
+        @Override
+        public void complete(Throwable exception) {
+            if (exception == null) {
+                handleEventEnd(this.toString(), startProcessingTimeNs.get());
+                future.complete(resultAndOffset.response());
+            } else {
+                future.completeExceptionally(
+                    handleEventException(name, startProcessingTimeNs, exception));
+            }
+        }
+
+        @Override
+        public String toString() {
+            return name + "(" + System.identityHashCode(this) + ")";
+        }
+    }
+
+    private <T> CompletableFuture<T> appendWriteEvent(String name,
+                                                      long timeoutMs,
+                                                      ControllerWriteOperation<T> op) {
+        ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
+        queue.appendWithDeadline(time.nanoseconds() +
+            TimeUnit.NANOSECONDS.convert(timeoutMs, TimeUnit.MILLISECONDS), event);
+        return event.future();
+    }
+
+    private <T> CompletableFuture<T> appendWriteEvent(String name,
+                                                      ControllerWriteOperation<T> op) {
+        ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
+        queue.append(event);
+        return event.future();
+    }
+
+    class QuorumMetaLogListener implements MetaLogListener {
+        @Override
+        public void handleCommits(long offset, List<ApiMessage> messages) {
+            appendControlEvent("handleCommits[" + offset + "]", () -> {
+                if (curClaimEpoch == -1) {
+                    if (log.isDebugEnabled()) {
+                        if (log.isTraceEnabled()) {
+                            log.trace("Replaying commits from the active node up to " +
+                                "offset {}: {}.", offset, messages.stream().
+                                map(m -> m.toString()).collect(Collectors.joining(", ")));
+                        } else {
+                            log.debug("Replaying commits from the active node up to " +
+                                "offset {}.", offset);
+                        }
+                    }
+                    for (ApiMessage message : messages) {
+                        replay(message);
+                    }
+                } else {
+                    log.debug("Completing purgatory items up to offset {}.", offset);
+
+                    // Complete any events in the purgatory that were waiting for this offset.
+                    purgatory.completeUpTo(offset);
+
+                    // Delete all snapshots older than the offset.
+                    // TODO: add an exception here for when we're writing out a log snapshot
+                    snapshotRegistry.deleteSnapshotsUpTo(offset);
+                }
+                lastCommittedOffset = offset;
+            });
+        }
+
+        @Override
+        public void handleNewLeader(MetaLogLeader newLeader) {
+            if (newLeader.nodeId() == nodeId) {
+                final long newEpoch = newLeader.epoch();
+                appendControlEvent("handleClaim[" + newEpoch + "]", () -> {
+                    long curEpoch = curClaimEpoch;
+                    if (curEpoch != -1) {
+                        throw new RuntimeException("Tried to claim controller epoch " +
+                            newEpoch + ", but we never renounced controller epoch " +
+                            curEpoch);
+                    }
+                    log.info("Becoming active at controller epoch {}.", newEpoch);
+                    curClaimEpoch = newEpoch;
+                    writeOffset = lastCommittedOffset;
+                    clusterControl.activate();
+                });
+            }
+        }
+
+        @Override
+        public void handleRenounce(long oldEpoch) {
+            appendControlEvent("handleRenounce[" + oldEpoch + "]", () -> {
+                if (curClaimEpoch == oldEpoch) {
+                    log.info("Renouncing the leadership at oldEpoch {} due to a metadata " +
+                            "log event. Reverting to last committed offset {}.", curClaimEpoch,
+                        lastCommittedOffset);
+                    renounce();
+                }
+            });
+        }
+
+        @Override
+        public void beginShutdown() {
+            queue.beginShutdown("MetaLogManager.Listener");
+        }
+    }
+
+    private void renounce() {
+        curClaimEpoch = -1;
+        purgatory.failAll(newNotControllerException());
+        snapshotRegistry.revertToSnapshot(lastCommittedOffset);
+        snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset);
+        writeOffset = -1;
+        clusterControl.deactivate();
+        cancelMaybeFenceReplicas();
+    }
+
+    private <T> void scheduleDeferredWriteEvent(String name, long deadlineNs,
+                                                ControllerWriteOperation<T> op) {
+        ControllerWriteEvent<T> event = new ControllerWriteEvent<>(name, op);
+        queue.scheduleDeferred(name, new EarliestDeadlineFunction(deadlineNs), event);
+        event.future.exceptionally(e -> {
+            if (e instanceof UnknownServerException && e.getCause() != null &&
+                    e.getCause() instanceof RejectedExecutionException) {
+                log.error("Cancelling write event {} because the event queue is closed.", name);
+                return null;
+            }
+            log.error("Unexpected exception while executing deferred write event {}. " +
+                "Rescheduling for a minute from now.", name, e);
+            scheduleDeferredWriteEvent(name,
+                deadlineNs + TimeUnit.NANOSECONDS.convert(1, TimeUnit.MINUTES), op);
+            return null;
+        });
+    }
+
+    static final String MAYBE_FENCE_REPLICAS = "maybeFenceReplicas";
+
+    class MaybeFenceReplicas implements ControllerWriteOperation<Void> {
+        @Override
+        public ControllerResult<Void> generateRecordsAndResult() {
+            ControllerResult<Set<Integer>> result =
+                clusterControl.maybeFenceLeastRecentlyContacted();
+            for (int brokerId : result.response()) {
+                replicationControl.removeFromIsr(brokerId, result.records());
+            }
+            rescheduleMaybeFenceReplicas();
+            return new ControllerResult<>(result.records(), null);
+        }
+    }
+
+    private void rescheduleMaybeFenceReplicas() {
+        long nextCheckTimeNs = clusterControl.nextCheckTimeNs();
+        if (nextCheckTimeNs == Long.MAX_VALUE) {
+            cancelMaybeFenceReplicas();
+        } else {
+            scheduleDeferredWriteEvent(MAYBE_FENCE_REPLICAS, nextCheckTimeNs,
+                new MaybeFenceReplicas());
+        }
+    }
+
+    private void cancelMaybeFenceReplicas() {
+        queue.cancelDeferred(MAYBE_FENCE_REPLICAS);
+    }
+
+    @SuppressWarnings("unchecked")
+    private void replay(ApiMessage message) {
+        try {
+            MetadataRecordType type = MetadataRecordType.fromId(message.apiKey());
+            switch (type) {
+                case REGISTER_BROKER_RECORD:
+                    clusterControl.replay((RegisterBrokerRecord) message);
+                    break;
+                case UNREGISTER_BROKER_RECORD:
+                    clusterControl.replay((UnregisterBrokerRecord) message);
+                    break;
+                case FENCE_BROKER_RECORD:
+                    clusterControl.replay((FenceBrokerRecord) message);
+                    break;
+                case UNFENCE_BROKER_RECORD:
+                    clusterControl.replay((UnfenceBrokerRecord) message);
+                    break;
+                case TOPIC_RECORD:
+                    replicationControl.replay((TopicRecord) message);
+                    break;
+                case PARTITION_RECORD:
+                    replicationControl.replay((PartitionRecord) message);
+                    break;
+                case CONFIG_RECORD:
+                    configurationControl.replay((ConfigRecord) message);
+                    break;
+                case QUOTA_RECORD:
+                    clientQuotaControlManager.replay((QuotaRecord) message);
+                    break;
+                case ISR_CHANGE_RECORD:
+                    replicationControl.replay((IsrChangeRecord) message);
+                    break;
+                default:
+                    throw new RuntimeException("Unhandled record type " + type);
+            }
+        } catch (Exception e) {
+            log.error("Error replaying record {}", message.toString(), e);
+        }
+    }
+
+    private final Logger log;
+
+    /**
+     * The ID of this controller node.
+     */
+    private final int nodeId;
+
+    /**
+     * The single-threaded queue that processes all of our events.
+     * It also processes timeouts.
+     */
+    private final KafkaEventQueue queue;
+
+    /**
+     * The Kafka clock object to use.
+     */
+    private final Time time;
+
+    /**
+     * A registry for snapshot data.  This must be accessed only by the event queue thread.
+     */
+    private final SnapshotRegistry snapshotRegistry;
+
+    /**
+     * The purgatory which holds deferred operations which are waiting for the metadata
+     * log's high water mark to advance.  This must be accessed only by the event queue thread.
+     */
+    private final ControllerPurgatory purgatory;
+
+    /**
+     * An object which stores the controller's dynamic configuration.
+     * This must be accessed only by the event queue thread.
+     */
+    private final ConfigurationControlManager configurationControl;
+
+    /**
+     * An object which stores the controller's dynamic client quotas.
+     * This must be accessed only by the event queue thread.
+     */
+    private final ClientQuotaControlManager clientQuotaControlManager;
+
+    /**
+     * An object which stores the controller's view of the cluster.
+     * This must be accessed only by the event queue thread.
+     */
+    private final ClusterControlManager clusterControl;
+
+    /**
+     * An object which stores the controller's view of the cluster features.
+     * This must be accessed only by the event queue thread.
+     */
+    private final FeatureControlManager featureControl;
+
+    /**
+     * An object which stores the controller's view of topics and partitions.
+     * This must be accessed only by the event queue thread.
+     */
+    private final ReplicationControlManager replicationControl;
+
+    /**
+     * The interface that we use to mutate the Raft log.
+     */
+    private final MetaLogManager logManager;
+
+    /**
+     * The interface that receives callbacks from the Raft log.  These callbacks are
+     * invoked from the Raft thread(s), not from the controller thread.
+     */
+    private final QuorumMetaLogListener metaLogListener;
+
+    /**
+     * If this controller is active, this is the non-negative controller epoch.
+     * Otherwise, this is -1.  This variable must be modified only from the controller
+     * thread, but it can be read from other threads.
+     */
+    private volatile long curClaimEpoch;
+
+    /**
+     * The last offset we have committed, or -1 if we have not committed any offsets.
+     */
+    private long lastCommittedOffset;
+
+    /**
+     * If we have called scheduleWrite, this is the last offset we got back from it.
+     */
+    private long writeOffset;
+
+    private QuorumController(LogContext logContext,
+                             int nodeId,
+                             KafkaEventQueue queue,
+                             Time time,
+                             Map<ConfigResource.Type, ConfigDef> configDefs,
+                             MetaLogManager logManager,
+                             Map<String, VersionRange> supportedFeatures,
+                             short defaultReplicationFactor,
+                             int defaultNumPartitions,
+                             ReplicaPlacementPolicy replicaPlacementPolicy,
+                             long sessionTimeoutNs) throws Exception {
+        this.log = logContext.logger(QuorumController.class);
+        this.nodeId = nodeId;
+        this.queue = queue;
+        this.time = time;
+        this.snapshotRegistry = new SnapshotRegistry(logContext);
+        snapshotRegistry.createSnapshot(-1);
+        this.purgatory = new ControllerPurgatory();
+        this.configurationControl = new ConfigurationControlManager(logContext,
+            snapshotRegistry, configDefs);
+        this.clientQuotaControlManager = new ClientQuotaControlManager(snapshotRegistry);
+        this.clusterControl = new ClusterControlManager(logContext, time,
+            snapshotRegistry, sessionTimeoutNs, replicaPlacementPolicy);
+        this.featureControl = new FeatureControlManager(supportedFeatures, snapshotRegistry);
+        this.replicationControl = new ReplicationControlManager(snapshotRegistry,
+            logContext, new Random(), defaultReplicationFactor, defaultNumPartitions,
+            configurationControl, clusterControl);
+        this.logManager = logManager;
+        this.metaLogListener = new QuorumMetaLogListener();
+        this.curClaimEpoch = -1L;
+        this.lastCommittedOffset = -1L;
+        this.writeOffset = -1L;
+        this.logManager.register(metaLogListener);
+    }
+
+    @Override
+    public CompletableFuture<AlterIsrResponseData> alterIsr(AlterIsrRequestData request) {
+        return appendWriteEvent("alterIsr", () ->
+            replicationControl.alterIsr(request));
+    }
+
+    @Override
+    public CompletableFuture<CreateTopicsResponseData>
+            createTopics(CreateTopicsRequestData request) {
+        return appendWriteEvent("createTopics", () ->
+            replicationControl.createTopics(request));
+    }
+
+    @Override
+    public CompletableFuture<Void> decommissionBroker(int brokerId) {
+        return appendWriteEvent("decommissionBroker", () -> {
+            ControllerResult<Void> result = clusterControl.decommissionBroker(brokerId);
+            replicationControl.removeFromIsr(brokerId, result.records());
+            return result;
+        });
+    }
+
+    @Override
+    public CompletableFuture<Map<ConfigResource, ResultOrError<Map<String, String>>>>
+            describeConfigs(Map<ConfigResource, Collection<String>> resources) {
+        return appendReadEvent("describeConfigs", () ->
+            configurationControl.describeConfigs(lastCommittedOffset, resources));
+    }
+
+    @Override
+    public CompletableFuture<ElectLeadersResponseData>
+            electLeaders(ElectLeadersRequestData request) {
+        CompletableFuture<ElectLeadersResponseData> future = new CompletableFuture<>();
+        appendWriteEvent("electLeaders", request.timeoutMs(),
+            () -> replicationControl.electLeaders(request));
+        return future;
+    }
+
+    @Override
+    public CompletableFuture<FeatureManager.FinalizedFeaturesAndEpoch> finalizedFeatures() {
+        return appendReadEvent("getFinalizedFeatures",
+            () -> featureControl.finalizedFeaturesAndEpoch(lastCommittedOffset));
+    }
+
+    @Override
+    public CompletableFuture<Map<ConfigResource, ApiError>> incrementalAlterConfigs(
+        Map<ConfigResource, Map<String, Entry<OpType, String>>> configChanges,
+        boolean validateOnly) {
+        return appendWriteEvent("incrementalAlterConfigs", () -> {
+            ControllerResult<Map<ConfigResource, ApiError>> result =
+                configurationControl.incrementalAlterConfigs(configChanges);
+            if (validateOnly) {
+                return result.withoutRecords();
+            } else {
+                return result;
+            }
+        });
+    }
+
+    @Override
+    public CompletableFuture<Map<ConfigResource, ApiError>> legacyAlterConfigs(
+        Map<ConfigResource, Map<String, String>> newConfigs, boolean validateOnly) {
+        return appendWriteEvent("legacyAlterConfigs", () -> {
+            ControllerResult<Map<ConfigResource, ApiError>> result =
+                configurationControl.legacyAlterConfigs(newConfigs);
+            if (validateOnly) {
+                return result.withoutRecords();
+            } else {
+                return result;
+            }
+        });
+    }
+
+    class ProcessBrokerHeartbeatOperation
+            implements ControllerWriteOperation<BrokerHeartbeatReply> {
+        private final BrokerHeartbeatRequestData request;
+        private boolean updateShutdownOffset = false;
+
+        ProcessBrokerHeartbeatOperation(BrokerHeartbeatRequestData request) {
+            this.request = request;
+        }
+
+        @Override
+        public ControllerResult<BrokerHeartbeatReply> generateRecordsAndResult() {
+            clusterControl.checkBrokerEpoch(request.brokerId(), request.brokerEpoch());
+            List<ApiMessageAndVersion> records = new ArrayList<>();
+            boolean movingLeadersForShutDown = false;
+            if (request.wantShutDown()) {
+                records = replicationControl.removeLeaderships(request.brokerId());
+                movingLeadersForShutDown = !records.isEmpty();
+            }
+            ControllerResult<BrokerHeartbeatReply> result = clusterControl.
+                processBrokerHeartbeat(request, lastCommittedOffset,
+                    movingLeadersForShutDown);
+            if (movingLeadersForShutDown && !result.response().shouldShutDown()) {
+                updateShutdownOffset = true;
+            }
+            if (result.response().isFenced()) {
+                replicationControl.removeFromIsr(request.brokerId(), records);

Review comment:
       It seems that we if the request wants to shut down, we should always remove the shutting down broker from isr, just like moving the leader off the shutting down broker?




----------------------------------------------------------------
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:
users@infra.apache.org