kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cmcc...@apache.org
Subject [kafka] branch 2.8 updated: KAFKA-12276: Add the quorum controller code (#10070)
Date Sat, 20 Feb 2021 02:06:29 GMT
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 2.8
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.8 by this push:
     new 0110638  KAFKA-12276: Add the quorum controller code (#10070)
0110638 is described below

commit 0110638127651e0a2cf34246a44775469095d8e1
Author: Colin Patrick McCabe <cmccabe@confluent.io>
AuthorDate: Fri Feb 19 18:03:23 2021 -0800

    KAFKA-12276: Add the quorum controller code (#10070)
    
    The quorum controller stores metadata in the KIP-500 metadata log, not in Apache
    ZooKeeper. Each controller node is a voter in the metadata quorum. The leader of the
    quorum is the active controller, which processes write requests. The followers are standby
    controllers, which replay the operations written to the log. If the active controller goes away,
    a standby controller can take its place.
    
    Like the ZooKeeper-based controller, the quorum controller is based on an event queue
    backed by a single-threaded executor. However, unlike the ZK-based controller, the quorum
    controller can have multiple operations in flight-- it does not need to wait for one operation
    to be finished before starting another. Therefore, calls into the QuorumController return
    CompleteableFuture objects which are completed with either a result or an error when the
    operation is done. The QuorumController will also time out operations that have been
    sitting on the queue too long without being processed. In this case, the future is completed
    with a TimeoutException.
    
    The controller uses timeline data structures to store multiple "versions" of its in-memory
    state simultaneously. "Read operations" read only committed state, which is slightly older
    than the most up-to-date in-memory state. "Write operations" read and write the latest
    in-memory state. However, we can not return a successful result for a write operation until
    its state has been committed to the log. Therefore, if a client receives an RPC response, it
    knows that the requested operation has been performed, and can not be undone by a
    controller failover.
    
    Reviewers: Jun Rao <junrao@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
---
 build.gradle                                       |   1 +
 checkstyle/import-control.xml                      |   2 +
 checkstyle/suppressions.xml                        |   6 +
 .../errors/BrokerIdNotRegisteredException.java     |  29 +
 .../org/apache/kafka/common/protocol/Errors.java   |   6 +-
 .../kafka/common/requests/RequestResponseTest.java |   1 -
 .../src/main/scala/kafka/server/BrokerServer.scala |   3 +
 .../main/scala/kafka/server/ControllerApis.scala   |   2 +-
 .../main/scala/kafka/server/ControllerServer.scala |  25 +-
 .../server/metadata/BrokerMetadataListener.scala   |  10 +-
 .../kafka/server/metadata/MetadataPartitions.scala |  26 +-
 .../kafka/controller/BrokerControlState.java       |  46 +
 .../kafka/controller/BrokerControlStates.java      |  56 ++
 .../kafka/controller/BrokerHeartbeatManager.java   | 597 +++++++++++++
 .../org/apache/kafka/controller/BrokersToIsrs.java | 314 +++++++
 .../controller/ClientQuotaControlManager.java      | 275 ++++++
 .../kafka/controller/ClusterControlManager.java    | 346 ++++++++
 .../controller/ConfigurationControlManager.java    | 367 ++++++++
 .../org/apache/kafka/controller/Controller.java    |  17 +-
 .../apache/kafka/controller/ControllerMetrics.java |  29 +
 .../kafka/controller/ControllerPurgatory.java      | 108 +++
 .../{ResultOrError.java => ControllerResult.java}  |  65 +-
 ...OrError.java => ControllerResultAndOffset.java} |  63 +-
 .../org/apache/kafka/controller/DeferredEvent.java |  31 +
 .../kafka/controller/FeatureControlManager.java    | 136 +++
 .../apache/kafka/controller/QuorumController.java  | 941 +++++++++++++++++++++
 .../kafka/controller/QuorumControllerMetrics.java  |  70 ++
 .../kafka/controller/ReplicaPlacementPolicy.java   |  47 +
 .../java/org/apache/kafka/controller/Replicas.java | 180 ++++
 .../controller/ReplicationControlManager.java      | 908 ++++++++++++++++++++
 .../org/apache/kafka/controller/ResultOrError.java |   4 +-
 .../controller/SimpleReplicaPlacementPolicy.java   |  77 ++
 .../kafka/metadata/BrokerHeartbeatReply.java       |  15 +-
 .../org/apache/kafka/metadata/UsableBroker.java    |  61 ++
 .../common/metadata/AccessControlRecord.json       |   1 +
 ...hangeRecord.json => PartitionChangeRecord.json} |  17 +-
 .../resources/common/metadata/PartitionRecord.json |   2 +-
 .../controller/BrokerHeartbeatManagerTest.java     | 296 +++++++
 .../apache/kafka/controller/BrokersToIsrsTest.java | 109 +++
 .../controller/ClientQuotaControlManagerTest.java  | 238 ++++++
 .../controller/ClusterControlManagerTest.java      | 150 ++++
 .../ConfigurationControlManagerTest.java           | 203 +++++
 .../kafka/controller/ControllerPurgatoryTest.java  | 102 +++
 .../kafka/controller/ControllerTestUtils.java      |  51 ++
 .../controller/FeatureControlManagerTest.java      | 132 +++
 .../kafka/controller/MockControllerMetrics.java    |  47 +
 .../org/apache/kafka/controller/MockRandom.java    |  34 +
 .../kafka/controller/QuorumControllerTest.java     | 180 ++++
 .../kafka/controller/QuorumControllerTestEnv.java  |  88 ++
 .../org/apache/kafka/controller/ReplicasTest.java  |  96 +++
 .../controller/ReplicationControlManagerTest.java  | 204 +++++
 .../apache/kafka/controller/ResultOrErrorTest.java |  65 ++
 .../org/apache/kafka/metalog/LocalLogManager.java  |  43 +
 .../apache/kafka/metalog/LocalLogManagerTest.java  |   1 +
 .../apache/kafka/shell/MetadataNodeManager.java    |  20 +-
 55 files changed, 6822 insertions(+), 121 deletions(-)

diff --git a/build.gradle b/build.gradle
index d037488..bf92f96 100644
--- a/build.gradle
+++ b/build.gradle
@@ -1029,6 +1029,7 @@ project(':metadata') {
     compile project(':clients')
     compile libs.jacksonDatabind
     compile libs.jacksonJDK8Datatypes
+    compile libs.metrics
     compileOnly libs.log4j
     testCompile libs.junitJupiter
     testCompile libs.hamcrest
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 63ed7ab..d9d504d 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -203,8 +203,10 @@
   </subpackage>
 
   <subpackage name="controller">
+    <allow pkg="com.yammer.metrics"/>
     <allow pkg="org.apache.kafka.clients" />
     <allow pkg="org.apache.kafka.clients.admin" />
+    <allow pkg="org.apache.kafka.common.annotation" />
     <allow pkg="org.apache.kafka.common.config" />
     <allow pkg="org.apache.kafka.common.feature" />
     <allow pkg="org.apache.kafka.common.internals" />
diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index 1cfc630..46fb97b 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -267,6 +267,12 @@
               files="RequestResponseTest.java"/>
 
     <!-- metadata -->
+    <suppress checks="ClassDataAbstractionCoupling"
+              files="(ReplicationControlManager).java"/>
+    <suppress checks="ClassFanOutComplexity"
+              files="(QuorumController|ReplicationControlManager).java"/>
+    <suppress checks="CyclomaticComplexity"
+              files="(ReplicationControlManager).java"/>
     <suppress checks="NPathComplexity"
               files="KafkaEventQueue.java"/>
     <suppress checks="(NPathComplexity|ClassFanOutComplexity|CyclomaticComplexity|ClassDataAbstractionCoupling|LocalVariableName|MemberName|ParameterName|MethodLength|JavaNCSS|AvoidStarImport)"
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/BrokerIdNotRegisteredException.java b/clients/src/main/java/org/apache/kafka/common/errors/BrokerIdNotRegisteredException.java
new file mode 100644
index 0000000..cc8a47a
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/BrokerIdNotRegisteredException.java
@@ -0,0 +1,29 @@
+/*
+ * 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.common.errors;
+
+public class BrokerIdNotRegisteredException extends ApiException {
+
+    public BrokerIdNotRegisteredException(String message) {
+        super(message);
+    }
+
+    public BrokerIdNotRegisteredException(String message, Throwable throwable) {
+        super(message, throwable);
+    }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 5c2ca7d..ca246c1 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -18,6 +18,7 @@ package org.apache.kafka.common.protocol;
 
 import org.apache.kafka.common.InvalidRecordException;
 import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
 import org.apache.kafka.common.errors.BrokerNotAvailableException;
 import org.apache.kafka.common.errors.ClusterAuthorizationException;
 import org.apache.kafka.common.errors.ConcurrentTransactionsException;
@@ -357,8 +358,9 @@ public enum Errors {
         PositionOutOfRangeException::new),
     UNKNOWN_TOPIC_ID(100, "This server does not host this topic ID.", UnknownTopicIdException::new),
     DUPLICATE_BROKER_REGISTRATION(101, "This broker ID is already in use.", DuplicateBrokerRegistrationException::new),
-    INCONSISTENT_TOPIC_ID(102, "The log's topic ID did not match the topic ID in the request", InconsistentTopicIdException::new),
-    INCONSISTENT_CLUSTER_ID(103, "The clusterId in the request does not match that found on the server", InconsistentClusterIdException::new);
+    BROKER_ID_NOT_REGISTERED(102, "The given broker ID was not registered.", BrokerIdNotRegisteredException::new),
+    INCONSISTENT_TOPIC_ID(103, "The log's topic ID did not match the topic ID in the request", InconsistentTopicIdException::new),
+    INCONSISTENT_CLUSTER_ID(104, "The clusterId in the request does not match that found on the server", InconsistentClusterIdException::new);
 
     private static final Logger log = LoggerFactory.getLogger(Errors.class);
 
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index 4c70771..920a951 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -2663,7 +2663,6 @@ public class RequestResponseTest {
 
     private BrokerHeartbeatResponse createBrokerHeartbeatResponse() {
         BrokerHeartbeatResponseData data = new BrokerHeartbeatResponseData()
-                .setIsCaughtUp(true)
                 .setIsFenced(false)
                 .setShouldShutDown(false)
                 .setThrottleTimeMs(0);
diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index 9aae5e3..4c0aaab 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -399,6 +399,9 @@ class BrokerServer(
       info("shutting down")
 
       if (config.controlledShutdownEnable) {
+        // Shut down the broker metadata listener, so that we don't get added to any
+        // more ISRs.
+        brokerMetadataListener.beginShutdown()
         lifecycleManager.beginControlledShutdown()
         try {
           lifecycleManager.controlledShutdownFuture.get(5L, TimeUnit.MINUTES)
diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala
index 2386da5..abd8506 100644
--- a/core/src/main/scala/kafka/server/ControllerApis.scala
+++ b/core/src/main/scala/kafka/server/ControllerApis.scala
@@ -173,7 +173,7 @@ class ControllerApis(val requestChannel: RequestChannel,
           .setRack(node.rack))
       }
       metadataResponseData.setClusterId(metaProperties.clusterId.toString)
-      if (controller.curClaimEpoch() > 0) {
+      if (controller.isActive()) {
         metadataResponseData.setControllerId(config.nodeId)
       } else {
         metadataResponseData.setControllerId(MetadataResponse.NO_CONTROLLER_ID)
diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
index 625ce52..50fc6e2 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -1,4 +1,4 @@
-/**
+/*
  * 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.
@@ -17,24 +17,26 @@
 
 package kafka.server
 
-import java.util.concurrent.CompletableFuture
+import java.util.concurrent.{CompletableFuture, TimeUnit}
 import java.util
 import java.util.concurrent.locks.ReentrantLock
 
 import kafka.cluster.Broker.ServerInfo
+import kafka.log.LogConfig
 import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector}
 import kafka.network.SocketServer
 import kafka.raft.RaftManager
 import kafka.security.CredentialProvider
 import kafka.server.QuotaFactory.QuotaManagers
 import kafka.utils.{CoreUtils, Logging}
+import org.apache.kafka.common.config.ConfigResource
 import org.apache.kafka.common.message.ApiMessageType.ListenerType
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.security.scram.internals.ScramMechanism
 import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache
 import org.apache.kafka.common.utils.{LogContext, Time}
 import org.apache.kafka.common.{ClusterResource, Endpoint}
-import org.apache.kafka.controller.Controller
+import org.apache.kafka.controller.{Controller, QuorumController, QuorumControllerMetrics}
 import org.apache.kafka.metadata.{ApiMessageAndVersion, VersionRange}
 import org.apache.kafka.metalog.MetaLogManager
 import org.apache.kafka.raft.RaftConfig
@@ -138,7 +140,22 @@ class ControllerServer(
       socketServerFirstBoundPortFuture.complete(socketServer.boundPort(
         config.controllerListeners.head.listenerName))
 
-      controller = null
+      val configDefs = Map(ConfigResource.Type.BROKER -> KafkaConfig.configDef,
+        ConfigResource.Type.TOPIC -> LogConfig.configDefCopy).asJava
+      val threadNamePrefixAsString = threadNamePrefix.getOrElse("")
+      controller = new QuorumController.Builder(config.nodeId).
+        setTime(time).
+        setThreadNamePrefix(threadNamePrefixAsString).
+        setConfigDefs(configDefs).
+        setLogManager(metaLogManager).
+        setDefaultReplicationFactor(config.defaultReplicationFactor.toShort).
+        setDefaultNumPartitions(config.numPartitions.intValue()).
+        setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(),
+          TimeUnit.MILLISECONDS)).
+        setMetrics(new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry())).
+        build()
+
+
       quotaManagers = QuotaFactory.instantiate(config, metrics, time, threadNamePrefix.getOrElse(""))
       val controllerNodes =
         RaftConfig.quorumVoterStringsToNodes(controllerQuorumVotersFuture.get()).asScala
diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
index 6185a35..9c2bcca 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
@@ -152,8 +152,8 @@ class BrokerMetadataListener(brokerId: Int,
       case PARTITION_RECORD => handlePartitionRecord(imageBuilder,
         record.asInstanceOf[PartitionRecord])
       case CONFIG_RECORD => handleConfigRecord(record.asInstanceOf[ConfigRecord])
-      case ISR_CHANGE_RECORD => handleIsrChangeRecord(imageBuilder,
-        record.asInstanceOf[IsrChangeRecord])
+      case PARTITION_CHANGE_RECORD => handlePartitionChangeRecord(imageBuilder,
+        record.asInstanceOf[PartitionChangeRecord])
       case FENCE_BROKER_RECORD => handleFenceBrokerRecord(imageBuilder,
         record.asInstanceOf[FenceBrokerRecord])
       case UNFENCE_BROKER_RECORD => handleUnfenceBrokerRecord(imageBuilder,
@@ -203,9 +203,9 @@ class BrokerMetadataListener(brokerId: Int,
     configRepository.setConfig(resource, record.name(), record.value())
   }
 
-  def handleIsrChangeRecord(imageBuilder: MetadataImageBuilder,
-                            record: IsrChangeRecord): Unit = {
-    imageBuilder.partitionsBuilder().handleIsrChange(record)
+  def handlePartitionChangeRecord(imageBuilder: MetadataImageBuilder,
+                                  record: PartitionChangeRecord): Unit = {
+    imageBuilder.partitionsBuilder().handleChange(record)
   }
 
   def handleFenceBrokerRecord(imageBuilder: MetadataImageBuilder,
diff --git a/core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala b/core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala
index c3efac5..bd84e7a 100644
--- a/core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala
+++ b/core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala
@@ -23,13 +23,15 @@ import java.util.Collections
 import org.apache.kafka.common.message.LeaderAndIsrRequestData
 import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
 import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataPartitionState
-import org.apache.kafka.common.metadata.{IsrChangeRecord, PartitionRecord}
+import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord}
 import org.apache.kafka.common.{TopicPartition, Uuid}
 
 import scala.jdk.CollectionConverters._
 
 
 object MetadataPartition {
+  val NO_LEADER_CHANGE = -2
+
   def apply(name: String, record: PartitionRecord): MetadataPartition = {
     MetadataPartition(name,
       record.partitionId(),
@@ -83,13 +85,23 @@ case class MetadataPartition(topicName: String,
 
   def isReplicaFor(brokerId: Int): Boolean = replicas.contains(Integer.valueOf(brokerId))
 
-  def copyWithIsrChanges(record: IsrChangeRecord): MetadataPartition = {
+  def copyWithChanges(record: PartitionChangeRecord): MetadataPartition = {
+    val (newLeader, newLeaderEpoch) = if (record.leader() == MetadataPartition.NO_LEADER_CHANGE) {
+      (leaderId, leaderEpoch)
+    } else {
+      (record.leader(), leaderEpoch + 1)
+    }
+    val newIsr = if (record.isr() == null) {
+      isr
+    } else {
+      record.isr()
+    }
     MetadataPartition(topicName,
       partitionIndex,
-      record.leader(),
-      record.leaderEpoch(),
+      newLeader,
+      newLeaderEpoch,
       replicas,
-      record.isr(),
+      newIsr,
       offlineReplicas,
       addingReplicas,
       removingReplicas)
@@ -113,14 +125,14 @@ class MetadataPartitionsBuilder(val brokerId: Int,
     }
   }
 
-  def handleIsrChange(record: IsrChangeRecord): Unit = {
+  def handleChange(record: PartitionChangeRecord): Unit = {
     Option(newIdMap.get(record.topicId())) match {
       case None => throw new RuntimeException(s"Unable to locate topic with ID ${record.topicId()}")
       case Some(name) => Option(newNameMap.get(name)) match {
         case None => throw new RuntimeException(s"Unable to locate topic with name $name")
         case Some(partitionMap) => Option(partitionMap.get(record.partitionId())) match {
           case None => throw new RuntimeException(s"Unable to locate $name-${record.partitionId}")
-          case Some(partition) => set(partition.copyWithIsrChanges(record))
+          case Some(partition) => set(partition.copyWithChanges(record))
         }
       }
     }
diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokerControlState.java b/metadata/src/main/java/org/apache/kafka/controller/BrokerControlState.java
new file mode 100644
index 0000000..dfcf8ce
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/BrokerControlState.java
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+
+public enum BrokerControlState {
+    FENCED(true, false),
+    UNFENCED(false, false),
+    CONTROLLED_SHUTDOWN(false, false),
+    SHUTDOWN_NOW(true, true);
+
+    private final boolean fenced;
+    private final boolean shouldShutDown;
+
+    BrokerControlState(boolean fenced, boolean shouldShutDown) {
+        this.fenced = fenced;
+        this.shouldShutDown = shouldShutDown;
+    }
+
+    public boolean fenced() {
+        return fenced;
+    }
+
+    public boolean shouldShutDown() {
+        return shouldShutDown;
+    }
+
+    public boolean inControlledShutdown() {
+        return this == CONTROLLED_SHUTDOWN;
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokerControlStates.java b/metadata/src/main/java/org/apache/kafka/controller/BrokerControlStates.java
new file mode 100644
index 0000000..6605852
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/BrokerControlStates.java
@@ -0,0 +1,56 @@
+/*
+ * 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.Objects;
+
+
+class BrokerControlStates {
+    private final BrokerControlState current;
+    private final BrokerControlState next;
+
+    BrokerControlStates(BrokerControlState current, BrokerControlState next) {
+        this.current = current;
+        this.next = next;
+    }
+
+    BrokerControlState current() {
+        return current;
+    }
+
+    BrokerControlState next() {
+        return next;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(current, next);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof BrokerControlStates)) return false;
+        BrokerControlStates other = (BrokerControlStates) o;
+        return other.current == current && other.next == next;
+    }
+
+    @Override
+    public String toString() {
+        return "BrokerControlStates(current=" + current + ", next=" + next + ")";
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java b/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
new file mode 100644
index 0000000..4a41fb8
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
@@ -0,0 +1,597 @@
+/*
+ * 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.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+import static org.apache.kafka.controller.BrokerControlState.FENCED;
+import static org.apache.kafka.controller.BrokerControlState.CONTROLLED_SHUTDOWN;
+import static org.apache.kafka.controller.BrokerControlState.SHUTDOWN_NOW;
+import static org.apache.kafka.controller.BrokerControlState.UNFENCED;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state includes
+ * things like the last time each broker sent us a heartbeat, and whether the broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the active
+ * controller handles broker heartbeats.  Standby controllers will create a heartbeat
+ * manager as part of the process of activating.  This design minimizes the size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean that after
+ * a controller failover, we may take some extra time to fence brokers, since the new
+ * active controller does not know when the last heartbeats were received from each.
+ */
+public class BrokerHeartbeatManager {
+    static class BrokerHeartbeatState {
+        /**
+         * The broker ID.
+         */
+        private final int id;
+
+        /**
+         * The last time we received a heartbeat from this broker, in monotonic nanoseconds.
+         * When this field is updated, we also may have to update the broker's position in
+         * the unfenced list.
+         */
+        long lastContactNs;
+
+        /**
+         * The last metadata offset which this broker reported.  When this field is updated,
+         * we may also have to update the broker's position in the active set.
+         */
+        long metadataOffset;
+
+        /**
+         * The offset at which the broker should complete its controlled shutdown, or -1
+         * if the broker is not performing a controlled shutdown.  When this field is
+         * updated, we also have to update the broker's position in the shuttingDown set.
+         */
+        private long controlledShutDownOffset;
+
+        /**
+         * The previous entry in the unfenced list, or null if the broker is not in that list.
+         */
+        private BrokerHeartbeatState prev;
+
+        /**
+         * The next entry in the unfenced list, or null if the broker is not in that list.
+         */
+        private BrokerHeartbeatState next;
+
+        BrokerHeartbeatState(int id) {
+            this.id = id;
+            this.lastContactNs = 0;
+            this.prev = null;
+            this.next = null;
+            this.metadataOffset = -1;
+            this.controlledShutDownOffset = -1;
+        }
+
+        /**
+         * Returns the broker ID.
+         */
+        int id() {
+            return id;
+        }
+
+        /**
+         * Returns true only if the broker is fenced.
+         */
+        boolean fenced() {
+            return prev == null;
+        }
+
+        /**
+         * Returns true only if the broker is in controlled shutdown state.
+         */
+        boolean shuttingDown() {
+            return controlledShutDownOffset >= 0;
+        }
+    }
+
+    static class MetadataOffsetComparator implements Comparator<BrokerHeartbeatState> {
+        static final MetadataOffsetComparator INSTANCE = new MetadataOffsetComparator();
+
+        @Override
+        public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+            if (a.metadataOffset < b.metadataOffset) {
+                return -1;
+            } else if (a.metadataOffset > b.metadataOffset) {
+                return 1;
+            } else if (a.id < b.id) {
+                return -1;
+            } else if (a.id > b.id) {
+                return 1;
+            } else {
+                return 0;
+            }
+        }
+    }
+
+    static class BrokerHeartbeatStateList {
+        /**
+         * The head of the list of unfenced brokers.  The list is sorted in ascending order
+         * of last contact time.
+         */
+        private final BrokerHeartbeatState head;
+
+        BrokerHeartbeatStateList() {
+            this.head = new BrokerHeartbeatState(-1);
+            head.prev = head;
+            head.next = head;
+        }
+
+        /**
+         * Return the head of the list, or null if the list is empty.
+         */
+        BrokerHeartbeatState first() {
+            BrokerHeartbeatState result = head.next;
+            return result == head ? null : result;
+        }
+
+        /**
+         * Add the broker to the list. We start looking for a place to put it at the end
+         * of the list.
+         */
+        void add(BrokerHeartbeatState broker) {
+            BrokerHeartbeatState cur = head.prev;
+            while (true) {
+                if (cur == head || cur.lastContactNs <= broker.lastContactNs) {
+                    broker.next = cur.next;
+                    cur.next.prev = broker;
+                    broker.prev = cur;
+                    cur.next = broker;
+                    break;
+                }
+                cur = cur.prev;
+            }
+        }
+
+        /**
+         * Remove a broker from the list.
+         */
+        void remove(BrokerHeartbeatState broker) {
+            if (broker.next == null) {
+                throw new RuntimeException(broker + " is not in the  list.");
+            }
+            broker.prev.next = broker.next;
+            broker.next.prev = broker.prev;
+            broker.prev = null;
+            broker.next = null;
+        }
+
+        BrokerHeartbeatStateIterator iterator() {
+            return new BrokerHeartbeatStateIterator(head);
+        }
+    }
+
+    static class BrokerHeartbeatStateIterator implements Iterator<BrokerHeartbeatState> {
+        private final BrokerHeartbeatState head;
+        private BrokerHeartbeatState cur;
+
+        BrokerHeartbeatStateIterator(BrokerHeartbeatState head) {
+            this.head = head;
+            this.cur = head;
+        }
+
+        @Override
+        public boolean hasNext() {
+            return cur.next != head;
+        }
+
+        @Override
+        public BrokerHeartbeatState next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            BrokerHeartbeatState result = cur.next;
+            cur = cur.next;
+            return result;
+        }
+    }
+
+    private final Logger log;
+
+    /**
+     * The Kafka clock object to use.
+     */
+    private final Time time;
+
+    /**
+     * The broker session timeout in nanoseconds.
+     */
+    private final long sessionTimeoutNs;
+
+    /**
+     * Maps broker IDs to heartbeat states.
+     */
+    private final HashMap<Integer, BrokerHeartbeatState> brokers;
+
+    /**
+     * The list of unfenced brokers, sorted by last contact time.
+     */
+    private final BrokerHeartbeatStateList unfenced;
+
+    /**
+     * The set of active brokers.  A broker is active if it is unfenced, and not shutting
+     * down.
+     */
+    private final TreeSet<BrokerHeartbeatState> active;
+
+    BrokerHeartbeatManager(LogContext logContext,
+                           Time time,
+                           long sessionTimeoutNs) {
+        this.log = logContext.logger(BrokerHeartbeatManager.class);
+        this.time = time;
+        this.sessionTimeoutNs = sessionTimeoutNs;
+        this.brokers = new HashMap<>();
+        this.unfenced = new BrokerHeartbeatStateList();
+        this.active = new TreeSet<>(MetadataOffsetComparator.INSTANCE);
+    }
+
+    // VisibleForTesting
+    Time time() {
+        return time;
+    }
+
+    // VisibleForTesting
+    BrokerHeartbeatStateList unfenced() {
+        return unfenced;
+    }
+
+    /**
+     * Mark a broker as fenced.
+     *
+     * @param brokerId      The ID of the broker to mark as fenced.
+     */
+    void fence(int brokerId) {
+        BrokerHeartbeatState broker = brokers.get(brokerId);
+        if (broker != null) {
+            untrack(broker);
+        }
+    }
+
+    /**
+     * Remove a broker.
+     *
+     * @param brokerId      The ID of the broker to remove.
+     */
+    void remove(int brokerId) {
+        BrokerHeartbeatState broker = brokers.remove(brokerId);
+        if (broker != null) {
+            untrack(broker);
+        }
+    }
+
+    /**
+     * Stop tracking the broker in the unfenced list and active set, if it was tracked
+     * in either of these.
+     *
+     * @param broker        The broker state to stop tracking.
+     */
+    private void untrack(BrokerHeartbeatState broker) {
+        if (!broker.fenced()) {
+            unfenced.remove(broker);
+            if (!broker.shuttingDown()) {
+                active.remove(broker);
+            }
+        }
+    }
+
+    /**
+     * Check if the given broker has a valid session.
+     *
+     * @param brokerId      The broker ID to check.
+     *
+     * @return              True if the given broker has a valid session.
+     */
+    boolean hasValidSession(int brokerId) {
+        BrokerHeartbeatState broker = brokers.get(brokerId);
+        if (broker == null) return false;
+        return hasValidSession(broker);
+    }
+
+    /**
+     * Check if the given broker has a valid session.
+     *
+     * @param broker        The broker to check.
+     *
+     * @return              True if the given broker has a valid session.
+     */
+    private boolean hasValidSession(BrokerHeartbeatState broker) {
+        if (broker.fenced()) {
+            return false;
+        } else {
+            return broker.lastContactNs + sessionTimeoutNs >= time.nanoseconds();
+        }
+    }
+
+    /**
+     * Update broker state, including lastContactNs.
+     *
+     * @param brokerId          The broker ID.
+     * @param fenced            True only if the broker is currently fenced.
+     * @param metadataOffset    The latest metadata offset of the broker.
+     */
+    void touch(int brokerId, boolean fenced, long metadataOffset) {
+        BrokerHeartbeatState broker = brokers.get(brokerId);
+        if (broker == null) {
+            broker = new BrokerHeartbeatState(brokerId);
+            brokers.put(brokerId, broker);
+        } else {
+            // Remove the broker from the unfenced list and/or the active set. Its
+            // position in either of those data structures depends on values we are
+            // changing here. We will re-add it if necessary at the end of this function.
+            untrack(broker);
+        }
+        broker.lastContactNs = time.nanoseconds();
+        broker.metadataOffset = metadataOffset;
+        if (fenced) {
+            // If a broker is fenced, it leaves controlled shutdown.  On its next heartbeat,
+            // it will shut down immediately.
+            broker.controlledShutDownOffset = -1;
+        } else {
+            unfenced.add(broker);
+            if (!broker.shuttingDown()) {
+                active.add(broker);
+            }
+        }
+    }
+
+    long lowestActiveOffset() {
+        Iterator<BrokerHeartbeatState> iterator = active.iterator();
+        if (!iterator.hasNext()) {
+            return Long.MAX_VALUE;
+        }
+        BrokerHeartbeatState first = iterator.next();
+        return first.metadataOffset;
+    }
+
+    /**
+     * Mark a broker as being in the controlled shutdown state.
+     *
+     * @param brokerId                  The broker id.
+     * @param controlledShutDownOffset  The offset at which controlled shutdown will be complete.
+     */
+    void updateControlledShutdownOffset(int brokerId, long controlledShutDownOffset) {
+        BrokerHeartbeatState broker = brokers.get(brokerId);
+        if (broker == null) {
+            throw new RuntimeException("Unable to locate broker " + brokerId);
+        }
+        if (broker.fenced()) {
+            throw new RuntimeException("Fenced brokers cannot enter controlled shutdown.");
+        }
+        active.remove(broker);
+        broker.controlledShutDownOffset = controlledShutDownOffset;
+        log.debug("Updated the controlled shutdown offset for broker {} to {}.",
+            brokerId, controlledShutDownOffset);
+    }
+
+    /**
+     * Return the time in monotonic nanoseconds at which we should check if a broker
+     * session needs to be expired.
+     */
+    long nextCheckTimeNs() {
+        BrokerHeartbeatState broker = unfenced.first();
+        if (broker == null) {
+            return Long.MAX_VALUE;
+        } else {
+            return broker.lastContactNs + sessionTimeoutNs;
+        }
+    }
+
+    /**
+     * Find the stale brokers which haven't heartbeated in a long time, and which need to
+     * be fenced.
+     *
+     * @return      A list of node IDs.
+     */
+    List<Integer> findStaleBrokers() {
+        List<Integer> nodes = new ArrayList<>();
+        BrokerHeartbeatStateIterator iterator = unfenced.iterator();
+        while (iterator.hasNext()) {
+            BrokerHeartbeatState broker = iterator.next();
+            if (hasValidSession(broker)) {
+                break;
+            }
+            nodes.add(broker.id);
+        }
+        return nodes;
+    }
+
+    /**
+     * Place replicas on unfenced brokers.
+     *
+     * @param numPartitions     The number of partitions to place.
+     * @param numReplicas       The number of replicas for each partition.
+     * @param idToRack          A function mapping broker id to broker rack.
+     * @param policy            The replica placement policy to use.
+     *
+     * @return                  A list of replica lists.
+     *
+     * @throws InvalidReplicationFactorException    If too many replicas were requested.
+     */
+    List<List<Integer>> placeReplicas(int numPartitions, short numReplicas,
+                                      Function<Integer, Optional<String>> idToRack,
+                                      ReplicaPlacementPolicy policy) {
+        // TODO: support using fenced brokers here if necessary to get to the desired
+        // number of replicas. We probably need to add a fenced boolean in UsableBroker.
+        Iterator<UsableBroker> iterator = new UsableBrokerIterator(
+            unfenced.iterator(), idToRack);
+        return policy.createPlacement(numPartitions, numReplicas, iterator);
+    }
+
+    static class UsableBrokerIterator implements Iterator<UsableBroker> {
+        private final Iterator<BrokerHeartbeatState> iterator;
+        private final Function<Integer, Optional<String>> idToRack;
+        private UsableBroker next;
+
+        UsableBrokerIterator(Iterator<BrokerHeartbeatState> iterator,
+                             Function<Integer, Optional<String>> idToRack) {
+            this.iterator = iterator;
+            this.idToRack = idToRack;
+            this.next = null;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (next != null) {
+                return true;
+            }
+            BrokerHeartbeatState result;
+            do {
+                if (!iterator.hasNext()) {
+                    return false;
+                }
+                result = iterator.next();
+            } while (result.shuttingDown());
+            Optional<String> rack = idToRack.apply(result.id());
+            next = new UsableBroker(result.id(), rack);
+            return true;
+        }
+
+        @Override
+        public UsableBroker next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            UsableBroker result = next;
+            next = null;
+            return result;
+        }
+    }
+
+    BrokerControlState currentBrokerState(BrokerHeartbeatState broker) {
+        if (broker.shuttingDown()) {
+            return CONTROLLED_SHUTDOWN;
+        } else if (broker.fenced()) {
+            return FENCED;
+        } else {
+            return UNFENCED;
+        }
+    }
+
+    /**
+     * Calculate the next broker state for a broker that just sent a heartbeat request.
+     *
+     * @param brokerId              The broker id.
+     * @param request               The incoming heartbeat request.
+     * @param lastCommittedOffset   The last committed offset of the quorum controller.
+     * @param hasLeaderships        A callback which evaluates to true if the broker leads
+     *                              at least one partition.
+     *
+     * @return                      The current and next broker states.
+     */
+    BrokerControlStates calculateNextBrokerState(int brokerId,
+                                                 BrokerHeartbeatRequestData request,
+                                                 long lastCommittedOffset,
+                                                 Supplier<Boolean> hasLeaderships) {
+        BrokerHeartbeatState broker = brokers.getOrDefault(brokerId,
+            new BrokerHeartbeatState(brokerId));
+        BrokerControlState currentState = currentBrokerState(broker);
+        switch (currentState) {
+            case FENCED:
+                if (request.wantShutDown()) {
+                    log.info("Fenced broker {} has requested and been granted an immediate " +
+                        "shutdown.", brokerId);
+                    return new BrokerControlStates(currentState, SHUTDOWN_NOW);
+                } else if (!request.wantFence()) {
+                    if (request.currentMetadataOffset() >= lastCommittedOffset) {
+                        log.info("The request from broker {} to unfence has been granted " +
+                                "because it has caught up with the last committed metadata " +
+                                "offset {}.", brokerId, lastCommittedOffset);
+                        return new BrokerControlStates(currentState, UNFENCED);
+                    } else {
+                        if (log.isDebugEnabled()) {
+                            log.debug("The request from broker {} to unfence cannot yet " +
+                                "be granted because it has not caught up with the last " +
+                                "committed metadata offset {}. It is still at offset {}.",
+                                brokerId, lastCommittedOffset, request.currentMetadataOffset());
+                        }
+                        return new BrokerControlStates(currentState, FENCED);
+                    }
+                }
+                return new BrokerControlStates(currentState, FENCED);
+
+            case UNFENCED:
+                if (request.wantFence()) {
+                    if (request.wantShutDown()) {
+                        log.info("Unfenced broker {} has requested and been granted an " +
+                            "immediate shutdown.", brokerId);
+                        return new BrokerControlStates(currentState, SHUTDOWN_NOW);
+                    } else {
+                        log.info("Unfenced broker {} has requested and been granted " +
+                            "fencing", brokerId);
+                        return new BrokerControlStates(currentState, FENCED);
+                    }
+                } else if (request.wantShutDown()) {
+                    if (hasLeaderships.get()) {
+                        log.info("Unfenced broker {} has requested and been granted a " +
+                            "controlled shutdown.", brokerId);
+                        return new BrokerControlStates(currentState, CONTROLLED_SHUTDOWN);
+                    } else {
+                        log.info("Unfenced broker {} has requested and been granted an " +
+                            "immediate shutdown.", brokerId);
+                        return new BrokerControlStates(currentState, SHUTDOWN_NOW);
+                    }
+                }
+                return new BrokerControlStates(currentState, UNFENCED);
+
+            case CONTROLLED_SHUTDOWN:
+                if (hasLeaderships.get()) {
+                    log.debug("Broker {} is in controlled shutdown state, but can not " +
+                        "shut down because more leaders still need to be moved.", brokerId);
+                    return new BrokerControlStates(currentState, CONTROLLED_SHUTDOWN);
+                }
+                long lowestActiveOffset = lowestActiveOffset();
+                if (broker.controlledShutDownOffset <= lowestActiveOffset) {
+                    log.info("The request from broker {} to shut down has been granted " +
+                        "since the lowest active offset {} is now greater than the " +
+                        "broker's controlled shutdown offset {}.", brokerId,
+                        lowestActiveOffset, broker.controlledShutDownOffset);
+                    return new BrokerControlStates(currentState, SHUTDOWN_NOW);
+                }
+                log.debug("The request from broker {} to shut down can not yet be granted " +
+                    "because the lowest active offset {} is not greater than the broker's " +
+                    "shutdown offset {}.", brokerId, lowestActiveOffset,
+                    broker.controlledShutDownOffset);
+                return new BrokerControlStates(currentState, CONTROLLED_SHUTDOWN);
+
+            default:
+                return new BrokerControlStates(currentState, SHUTDOWN_NOW);
+        }
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java b/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java
new file mode 100644
index 0000000..6b219eb
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java
@@ -0,0 +1,314 @@
+/*
+ * 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.Uuid;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+
+import static org.apache.kafka.controller.ReplicationControlManager.NO_LEADER;
+
+
+/**
+ * Associates brokers with their in-sync partitions.
+ *
+ * This is useful when we need to remove a broker from all the ISRs, or move all leaders
+ * away from a broker.
+ *
+ * We also track all the partitions that currently have no leader.
+ *
+ * The core data structure is a map from broker IDs to topic maps.  Each topic map relates
+ * topic UUIDs to arrays of partition IDs.
+ *
+ * Each entry in the array has a high bit which indicates that the broker is the leader
+ * for the given partition, as well as 31 low bits which contain the partition id.  This
+ * works because partition IDs cannot be negative.
+ */
+public class BrokersToIsrs {
+    private final static int[] EMPTY = new int[0];
+
+    private final static int LEADER_FLAG = 0x8000_0000;
+
+    private final static int REPLICA_MASK = 0x7fff_ffff;
+
+    static class TopicPartition {
+        private final Uuid topicId;
+        private final int partitionId;
+
+        TopicPartition(Uuid topicId, int partitionId) {
+            this.topicId = topicId;
+            this.partitionId = partitionId;
+        }
+
+        public Uuid topicId() {
+            return topicId;
+        }
+
+        public int partitionId() {
+            return partitionId;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (!(o instanceof TopicPartition)) return false;
+            TopicPartition other = (TopicPartition) o;
+            return other.topicId.equals(topicId) && other.partitionId == partitionId;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(topicId, partitionId);
+        }
+
+        @Override
+        public String toString() {
+            return topicId + ":" + partitionId;
+        }
+    }
+
+    static class PartitionsOnReplicaIterator implements Iterator<TopicPartition> {
+        private final Iterator<Entry<Uuid, int[]>> iterator;
+        private final boolean leaderOnly;
+        private int offset = 0;
+        Uuid uuid = Uuid.ZERO_UUID;
+        int[] replicas = EMPTY;
+        private TopicPartition next = null;
+
+        PartitionsOnReplicaIterator(Map<Uuid, int[]> topicMap, boolean leaderOnly) {
+            this.iterator = topicMap.entrySet().iterator();
+            this.leaderOnly = leaderOnly;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (next != null) return true;
+            while (true) {
+                if (offset >= replicas.length) {
+                    if (!iterator.hasNext()) return false;
+                    offset = 0;
+                    Entry<Uuid, int[]> entry = iterator.next();
+                    uuid = entry.getKey();
+                    replicas = entry.getValue();
+                }
+                int replica = replicas[offset++];
+                if ((!leaderOnly) || (replica & LEADER_FLAG) != 0) {
+                    next = new TopicPartition(uuid, replica & REPLICA_MASK);
+                    return true;
+                }
+            }
+        }
+
+        @Override
+        public TopicPartition next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            TopicPartition result = next;
+            next = null;
+            return result;
+        }
+    }
+
+    private final SnapshotRegistry snapshotRegistry;
+
+    /**
+     * A map of broker IDs to the partitions that the broker is in the ISR for.
+     * Partitions with no isr members appear in this map under id NO_LEADER.
+     */
+    private final TimelineHashMap<Integer, TimelineHashMap<Uuid, int[]>> isrMembers;
+
+    BrokersToIsrs(SnapshotRegistry snapshotRegistry) {
+        this.snapshotRegistry = snapshotRegistry;
+        this.isrMembers = new TimelineHashMap<>(snapshotRegistry, 0);
+    }
+
+    /**
+     * Update our records of a partition's ISR.
+     *
+     * @param topicId       The topic ID of the partition.
+     * @param partitionId   The partition ID of the partition.
+     * @param prevIsr       The previous ISR, or null if the partition is new.
+     * @param nextIsr       The new ISR, or null if the partition is being removed.
+     * @param prevLeader    The previous leader, or NO_LEADER if the partition had no leader.
+     * @param nextLeader    The new leader, or NO_LEADER if the partition now has no leader.
+     */
+    void update(Uuid topicId, int partitionId, int[] prevIsr, int[] nextIsr,
+                int prevLeader, int nextLeader) {
+        int[] prev;
+        if (prevIsr == null) {
+            prev = EMPTY;
+        } else {
+            if (prevLeader == NO_LEADER) {
+                prev = Replicas.copyWith(prevIsr, NO_LEADER);
+            } else {
+                prev = Replicas.clone(prevIsr);
+            }
+            Arrays.sort(prev);
+        }
+        int[] next;
+        if (nextIsr == null) {
+            next = EMPTY;
+        } else {
+            if (nextLeader == NO_LEADER) {
+                next = Replicas.copyWith(nextIsr, NO_LEADER);
+            } else {
+                next = Replicas.clone(nextIsr);
+            }
+            Arrays.sort(next);
+        }
+        int i = 0, j = 0;
+        while (true) {
+            if (i == prev.length) {
+                if (j == next.length) {
+                    break;
+                }
+                int newReplica = next[j];
+                add(newReplica, topicId, partitionId, newReplica == nextLeader);
+                j++;
+            } else if (j == next.length) {
+                int prevReplica = prev[i];
+                remove(prevReplica, topicId, partitionId, prevReplica == prevLeader);
+                i++;
+            } else {
+                int prevReplica = prev[i];
+                int newReplica = next[j];
+                if (prevReplica < newReplica) {
+                    remove(prevReplica, topicId, partitionId, prevReplica == prevLeader);
+                    i++;
+                } else if (prevReplica > newReplica) {
+                    add(newReplica, topicId, partitionId, newReplica == nextLeader);
+                    j++;
+                } else {
+                    boolean wasLeader = prevReplica == prevLeader;
+                    boolean isLeader = prevReplica == nextLeader;
+                    if (wasLeader != isLeader) {
+                        change(prevReplica, topicId, partitionId, wasLeader, isLeader);
+                    }
+                    i++;
+                    j++;
+                }
+            }
+        }
+    }
+
+    private void add(int brokerId, Uuid topicId, int newPartition, boolean leader) {
+        if (leader) {
+            newPartition = newPartition | LEADER_FLAG;
+        }
+        TimelineHashMap<Uuid, int[]> topicMap = isrMembers.get(brokerId);
+        if (topicMap == null) {
+            topicMap = new TimelineHashMap<>(snapshotRegistry, 0);
+            isrMembers.put(brokerId, topicMap);
+        }
+        int[] partitions = topicMap.get(topicId);
+        int[] newPartitions;
+        if (partitions == null) {
+            newPartitions = new int[1];
+        } else {
+            newPartitions = new int[partitions.length + 1];
+            System.arraycopy(partitions, 0, newPartitions, 0, partitions.length);
+        }
+        newPartitions[newPartitions.length - 1] = newPartition;
+        topicMap.put(topicId, newPartitions);
+    }
+
+    private void change(int brokerId, Uuid topicId, int partition,
+                        boolean wasLeader, boolean isLeader) {
+        TimelineHashMap<Uuid, int[]> topicMap = isrMembers.get(brokerId);
+        if (topicMap == null) {
+            throw new RuntimeException("Broker " + brokerId + " has no isrMembers " +
+                "entry, so we can't change " + topicId + ":" + partition);
+        }
+        int[] partitions = topicMap.get(topicId);
+        if (partitions == null) {
+            throw new RuntimeException("Broker " + brokerId + " has no " +
+                "entry in isrMembers for topic " + topicId);
+        }
+        int[] newPartitions = new int[partitions.length];
+        int target = wasLeader ? partition | LEADER_FLAG : partition;
+        for (int i = 0; i < partitions.length; i++) {
+            int cur = partitions[i];
+            if (cur == target) {
+                newPartitions[i] = isLeader ? partition | LEADER_FLAG : partition;
+            } else {
+                newPartitions[i] = cur;
+            }
+        }
+        topicMap.put(topicId, newPartitions);
+    }
+
+    private void remove(int brokerId, Uuid topicId, int removedPartition, boolean leader) {
+        if (leader) {
+            removedPartition = removedPartition | LEADER_FLAG;
+        }
+        TimelineHashMap<Uuid, int[]> topicMap = isrMembers.get(brokerId);
+        if (topicMap == null) {
+            throw new RuntimeException("Broker " + brokerId + " has no isrMembers " +
+                "entry, so we can't remove " + topicId + ":" + removedPartition);
+        }
+        int[] partitions = topicMap.get(topicId);
+        if (partitions == null) {
+            throw new RuntimeException("Broker " + brokerId + " has no " +
+                "entry in isrMembers for topic " + topicId);
+        }
+        if (partitions.length == 1) {
+            if (partitions[0] != removedPartition) {
+                throw new RuntimeException("Broker " + brokerId + " has no " +
+                    "entry in isrMembers for " + topicId + ":" + removedPartition);
+            }
+            topicMap.remove(topicId);
+            if (topicMap.isEmpty()) {
+                isrMembers.remove(brokerId);
+            }
+        } else {
+            int[] newPartitions = new int[partitions.length - 1];
+            int j = 0;
+            for (int i = 0; i < partitions.length; i++) {
+                int partition = partitions[i];
+                if (partition != removedPartition) {
+                    newPartitions[j++] = partition;
+                }
+            }
+            topicMap.put(topicId, newPartitions);
+        }
+    }
+
+    PartitionsOnReplicaIterator iterator(int brokerId, boolean leadersOnly) {
+        Map<Uuid, int[]> topicMap = isrMembers.get(brokerId);
+        if (topicMap == null) {
+            topicMap = Collections.emptyMap();
+        }
+        return new PartitionsOnReplicaIterator(topicMap, leadersOnly);
+    }
+
+    PartitionsOnReplicaIterator noLeaderIterator() {
+        return iterator(NO_LEADER, true);
+    }
+
+    boolean hasLeaderships(int brokerId) {
+        return iterator(brokerId, true).hasNext();
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java
new file mode 100644
index 0000000..4aac9e4
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java
@@ -0,0 +1,275 @@
+/*
+ * 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.config.ConfigDef;
+import org.apache.kafka.common.config.internals.QuotaConfigs;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.protocol.Errors;
+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.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public class ClientQuotaControlManager {
+
+    private final SnapshotRegistry snapshotRegistry;
+
+    final TimelineHashMap<ClientQuotaEntity, Map<String, Double>> clientQuotaData;
+
+    ClientQuotaControlManager(SnapshotRegistry snapshotRegistry) {
+        this.snapshotRegistry = snapshotRegistry;
+        this.clientQuotaData = new TimelineHashMap<>(snapshotRegistry, 0);
+    }
+
+    /**
+     * Determine the result of applying a batch of client quota alteration.  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 quotaAlterations  List of client quota alterations to evaluate
+     * @return                  The result.
+     */
+    ControllerResult<Map<ClientQuotaEntity, ApiError>> alterClientQuotas(
+            Collection<ClientQuotaAlteration> quotaAlterations) {
+        List<ApiMessageAndVersion> outputRecords = new ArrayList<>();
+        Map<ClientQuotaEntity, ApiError> outputResults = new HashMap<>();
+
+        quotaAlterations.forEach(quotaAlteration -> {
+            // Note that the values in this map may be null
+            Map<String, Double> alterations = new HashMap<>(quotaAlteration.ops().size());
+            quotaAlteration.ops().forEach(op -> {
+                if (alterations.containsKey(op.key())) {
+                    outputResults.put(quotaAlteration.entity(), ApiError.fromThrowable(
+                            new InvalidRequestException("Duplicate quota key " + op.key() +
+                                " not updating quota for this entity " + quotaAlteration.entity())));
+                } else {
+                    alterations.put(op.key(), op.value());
+                }
+            });
+            if (outputResults.containsKey(quotaAlteration.entity())) {
+                outputResults.put(quotaAlteration.entity(), ApiError.fromThrowable(
+                        new InvalidRequestException("Ignoring duplicate entity " + quotaAlteration.entity())));
+            } else {
+                alterClientQuotaEntity(quotaAlteration.entity(), alterations, outputRecords, outputResults);
+            }
+        });
+
+        return new ControllerResult<>(outputRecords, outputResults);
+    }
+
+    /**
+     * Apply a quota record to the in-memory state.
+     *
+     * @param record    A QuotaRecord instance.
+     */
+    public void replay(QuotaRecord record) {
+        Map<String, String> entityMap = new HashMap<>(2);
+        record.entity().forEach(entityData -> entityMap.put(entityData.entityType(), entityData.entityName()));
+        ClientQuotaEntity entity = new ClientQuotaEntity(entityMap);
+        Map<String, Double> quotas = clientQuotaData.get(entity);
+        if (quotas == null) {
+            quotas = new TimelineHashMap<>(snapshotRegistry, 0);
+            clientQuotaData.put(entity, quotas);
+        }
+        if (record.remove()) {
+            quotas.remove(record.key());
+            if (quotas.size() == 0) {
+                clientQuotaData.remove(entity);
+            }
+        } else {
+            quotas.put(record.key(), record.value());
+        }
+    }
+
+    private void alterClientQuotaEntity(
+            ClientQuotaEntity entity,
+            Map<String, Double> newQuotaConfigs,
+            List<ApiMessageAndVersion> outputRecords,
+            Map<ClientQuotaEntity, ApiError> outputResults) {
+
+        // Check entity types and sanitize the names
+        Map<String, String> validatedEntityMap = new HashMap<>(3);
+        ApiError error = validateEntity(entity, validatedEntityMap);
+        if (error.isFailure()) {
+            outputResults.put(entity, error);
+            return;
+        }
+
+        // Check the combination of entity types and get the config keys
+        Map<String, ConfigDef.ConfigKey> configKeys = new HashMap<>(4);
+        error = configKeysForEntityType(validatedEntityMap, configKeys);
+        if (error.isFailure()) {
+            outputResults.put(entity, error);
+            return;
+        }
+
+        // Don't share objects between different records
+        Supplier<List<QuotaRecord.EntityData>> recordEntitySupplier = () ->
+                validatedEntityMap.entrySet().stream().map(mapEntry -> new QuotaRecord.EntityData()
+                        .setEntityType(mapEntry.getKey())
+                        .setEntityName(mapEntry.getValue()))
+                        .collect(Collectors.toList());
+
+        List<ApiMessageAndVersion> newRecords = new ArrayList<>(newQuotaConfigs.size());
+        Map<String, Double> currentQuotas = clientQuotaData.getOrDefault(entity, Collections.emptyMap());
+        newQuotaConfigs.forEach((key, newValue) -> {
+            if (newValue == null) {
+                if (currentQuotas.containsKey(key)) {
+                    // Null value indicates removal
+                    newRecords.add(new ApiMessageAndVersion(new QuotaRecord()
+                            .setEntity(recordEntitySupplier.get())
+                            .setKey(key)
+                            .setRemove(true), (short) 0));
+                }
+            } else {
+                ApiError validationError = validateQuotaKeyValue(configKeys, key, newValue);
+                if (validationError.isFailure()) {
+                    outputResults.put(entity, validationError);
+                } else {
+                    final Double currentValue = currentQuotas.get(key);
+                    if (!Objects.equals(currentValue, newValue)) {
+                        // Only record the new value if it has changed
+                        newRecords.add(new ApiMessageAndVersion(new QuotaRecord()
+                                .setEntity(recordEntitySupplier.get())
+                                .setKey(key)
+                                .setValue(newValue), (short) 0));
+                    }
+                }
+            }
+        });
+
+        outputRecords.addAll(newRecords);
+        outputResults.put(entity, ApiError.NONE);
+    }
+
+    private ApiError configKeysForEntityType(Map<String, String> entity, Map<String, ConfigDef.ConfigKey> output) {
+        // We only allow certain combinations of quota entity types. Which type is in use determines which config
+        // keys are valid
+        boolean hasUser = entity.containsKey(ClientQuotaEntity.USER);
+        boolean hasClientId = entity.containsKey(ClientQuotaEntity.CLIENT_ID);
+        boolean hasIp = entity.containsKey(ClientQuotaEntity.IP);
+
+        final Map<String, ConfigDef.ConfigKey> configKeys;
+        if (hasUser && hasClientId && !hasIp) {
+            configKeys = QuotaConfigs.userConfigs().configKeys();
+        } else if (hasUser && !hasClientId && !hasIp) {
+            configKeys = QuotaConfigs.userConfigs().configKeys();
+        } else if (!hasUser && hasClientId && !hasIp) {
+            configKeys = QuotaConfigs.clientConfigs().configKeys();
+        } else if (!hasUser && !hasClientId && hasIp) {
+            if (isValidIpEntity(entity.get(ClientQuotaEntity.IP))) {
+                configKeys = QuotaConfigs.ipConfigs().configKeys();
+            } else {
+                return new ApiError(Errors.INVALID_REQUEST, entity.get(ClientQuotaEntity.IP) + " is not a valid IP or resolvable host.");
+            }
+        } else {
+            return new ApiError(Errors.INVALID_REQUEST, "Invalid empty client quota entity");
+        }
+
+        output.putAll(configKeys);
+        return ApiError.NONE;
+    }
+
+    private ApiError validateQuotaKeyValue(Map<String, ConfigDef.ConfigKey> validKeys, String key, Double value) {
+        // TODO can this validation be shared with alter configs?
+        // Ensure we have an allowed quota key
+        ConfigDef.ConfigKey configKey = validKeys.get(key);
+        if (configKey == null) {
+            return new ApiError(Errors.INVALID_REQUEST, "Invalid configuration key " + key);
+        }
+
+        // Ensure the quota value is valid
+        switch (configKey.type()) {
+            case DOUBLE:
+                break;
+            case LONG:
+                Double epsilon = 1e-6;
+                Long longValue = Double.valueOf(value + epsilon).longValue();
+                if (Math.abs(longValue.doubleValue() - value) > epsilon) {
+                    return new ApiError(Errors.INVALID_REQUEST,
+                            "Configuration " + key + " must be a Long value");
+                }
+                break;
+            default:
+                return new ApiError(Errors.UNKNOWN_SERVER_ERROR,
+                        "Unexpected config type " + configKey.type() + " should be Long or Double");
+        }
+        return ApiError.NONE;
+    }
+
+    // TODO move this somewhere common?
+    private boolean isValidIpEntity(String ip) {
+        if (Objects.nonNull(ip)) {
+            try {
+                InetAddress.getByName(ip);
+                return true;
+            } catch (UnknownHostException e) {
+                return false;
+            }
+        } else {
+            return true;
+        }
+    }
+
+    private ApiError validateEntity(ClientQuotaEntity entity, Map<String, String> validatedEntityMap) {
+        // Given a quota entity (which is a mapping of entity type to entity name), validate it's types
+        if (entity.entries().isEmpty()) {
+            return new ApiError(Errors.INVALID_REQUEST, "Invalid empty client quota entity");
+        }
+
+        for (Map.Entry<String, String> entityEntry : entity.entries().entrySet()) {
+            String entityType = entityEntry.getKey();
+            String entityName = entityEntry.getValue();
+            if (validatedEntityMap.containsKey(entityType)) {
+                return new ApiError(Errors.INVALID_REQUEST, "Invalid empty client quota entity, duplicate entity entry " + entityType);
+            }
+            if (Objects.equals(entityType, ClientQuotaEntity.USER)) {
+                validatedEntityMap.put(ClientQuotaEntity.USER, entityName);
+            } else if (Objects.equals(entityType, ClientQuotaEntity.CLIENT_ID)) {
+                validatedEntityMap.put(ClientQuotaEntity.CLIENT_ID, entityName);
+            } else if (Objects.equals(entityType, ClientQuotaEntity.IP)) {
+                validatedEntityMap.put(ClientQuotaEntity.IP, entityName);
+            } else {
+                return new ApiError(Errors.INVALID_REQUEST, "Unhandled client quota entity type: " + entityType);
+            }
+
+            if (entityName != null && entityName.isEmpty()) {
+                return new ApiError(Errors.INVALID_REQUEST, "Empty " + entityType + " not supported");
+            }
+        }
+
+        return ApiError.NONE;
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
new file mode 100644
index 0000000..6e329c7
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
@@ -0,0 +1,346 @@
+/*
+ * 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.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.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureMapAndEpoch;
+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.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * The ClusterControlManager manages all the hard state associated with the Kafka cluster.
+ * Hard state is state which appears in the metadata log, such as broker registrations,
+ * brokers being fenced or unfenced, and broker feature versions.
+ */
+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;
+    }
+
+    Map<Integer, BrokerRegistration> brokerRegistrations() {
+        return brokerRegistrations;
+    }
+
+    /**
+     * Process an incoming broker registration request.
+     */
+    public ControllerResult<BrokerRegistrationReply> registerBroker(
+            BrokerRegistrationRequestData request,
+            long brokerEpoch,
+            FeatureMapAndEpoch finalizedFeatures) {
+        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()) {
+            Optional<VersionRange> finalized = finalizedFeatures.map().get(feature.name());
+            if (finalized.isPresent()) {
+                if (!finalized.get().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 void replay(RegisterBrokerRecord record) {
+        int brokerId = record.brokerId();
+        List<Endpoint> listeners = new ArrayList<>();
+        for (RegisterBrokerRecord.BrokerEndpoint endpoint : record.endPoints()) {
+            listeners.add(new Endpoint(endpoint.name(),
+                SecurityProtocol.forId(endpoint.securityProtocol()),
+                endpoint.host(), endpoint.port()));
+        }
+        Map<String, VersionRange> features = new HashMap<>();
+        for (RegisterBrokerRecord.BrokerFeature feature : record.features()) {
+            features.put(feature.name(), new VersionRange(
+                feature.minSupportedVersion(), feature.maxSupportedVersion()));
+        }
+        // Normally, all newly registered brokers start off in the fenced state.
+        // If this registration record is for a broker incarnation that was already
+        // registered, though, we preserve the existing fencing state.
+        boolean fenced = true;
+        BrokerRegistration prevRegistration = brokerRegistrations.get(brokerId);
+        if (prevRegistration != null &&
+                prevRegistration.incarnationId().equals(record.incarnationId())) {
+            fenced = prevRegistration.fenced();
+        }
+        // Update broker registrations.
+        brokerRegistrations.put(brokerId, new BrokerRegistration(brokerId,
+            record.brokerEpoch(), record.incarnationId(), listeners, features,
+            Optional.ofNullable(record.rack()), fenced));
+
+        if (prevRegistration == null) {
+            log.info("Registered new broker: {}", record);
+        } else if (prevRegistration.incarnationId().equals(record.incarnationId())) {
+            log.info("Re-registered broker incarnation: {}", record);
+        } else {
+            log.info("Re-registered broker id {}: {}", brokerId, record);
+        }
+    }
+
+    public void replay(UnregisterBrokerRecord record) {
+        int brokerId = record.brokerId();
+        BrokerRegistration registration = brokerRegistrations.get(brokerId);
+        if (registration == null) {
+            throw new RuntimeException(String.format("Unable to replay %s: no broker " +
+                "registration found for that id", record.toString()));
+        } else if (registration.epoch() !=  record.brokerEpoch()) {
+            throw new RuntimeException(String.format("Unable to replay %s: no broker " +
+                "registration with that epoch found", record.toString()));
+        } else {
+            brokerRegistrations.remove(brokerId);
+            log.info("Unregistered broker: {}", record);
+        }
+    }
+
+    public void replay(FenceBrokerRecord record) {
+        int brokerId = record.id();
+        BrokerRegistration registration = brokerRegistrations.get(brokerId);
+        if (registration == null) {
+            throw new RuntimeException(String.format("Unable to replay %s: no broker " +
+                "registration found for that id", record.toString()));
+        } else if (registration.epoch() !=  record.epoch()) {
+            throw new RuntimeException(String.format("Unable to replay %s: no broker " +
+                "registration with that epoch found", record.toString()));
+        } else {
+            brokerRegistrations.put(brokerId, registration.cloneWithFencing(true));
+            log.info("Fenced broker: {}", record);
+        }
+    }
+
+    public void replay(UnfenceBrokerRecord record) {
+        int brokerId = record.id();
+        BrokerRegistration registration = brokerRegistrations.get(brokerId);
+        if (registration == null) {
+            throw new RuntimeException(String.format("Unable to replay %s: no broker " +
+                "registration found for that id", record.toString()));
+        } else if (registration.epoch() !=  record.epoch()) {
+            throw new RuntimeException(String.format("Unable to replay %s: no broker " +
+                "registration with that epoch found", record.toString()));
+        } else {
+            brokerRegistrations.put(brokerId, registration.cloneWithFencing(false));
+            log.info("Unfenced broker: {}", record);
+        }
+        if (readyBrokersFuture.isPresent()) {
+            if (readyBrokersFuture.get().check()) {
+                readyBrokersFuture.get().future.complete(null);
+                readyBrokersFuture = Optional.empty();
+            }
+        }
+    }
+
+    public List<List<Integer>> placeReplicas(int numPartitions, short numReplicas) {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        return heartbeatManager.placeReplicas(numPartitions, numReplicas,
+            id -> brokerRegistrations.get(id).rack(), placementPolicy);
+    }
+
+    public boolean unfenced(int brokerId) {
+        BrokerRegistration registration = brokerRegistrations.get(brokerId);
+        if (registration == null) return false;
+        return !registration.fenced();
+    }
+
+    BrokerHeartbeatManager heartbeatManager() {
+        if (heartbeatManager == null) {
+            throw new RuntimeException("ClusterControlManager is not active.");
+        }
+        return heartbeatManager;
+    }
+
+    public void checkBrokerEpoch(int brokerId, long brokerEpoch) {
+        BrokerRegistration registration = brokerRegistrations.get(brokerId);
+        if (registration == null) {
+            throw new StaleBrokerEpochException("No broker registration found for " +
+                "broker id " + brokerId);
+        }
+        if (registration.epoch() != brokerEpoch) {
+            throw new StaleBrokerEpochException("Expected broker epoch " +
+                registration.epoch() + ", but got broker epoch " + brokerEpoch);
+        }
+    }
+
+    public void addReadyBrokersFuture(CompletableFuture<Void> future, int minBrokers) {
+        readyBrokersFuture = Optional.of(new ReadyBrokersFuture(future, minBrokers));
+        if (readyBrokersFuture.get().check()) {
+            readyBrokersFuture.get().future.complete(null);
+            readyBrokersFuture = Optional.empty();
+        }
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
new file mode 100644
index 0000000..4402b3a
--- /dev/null
+++ b/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
+                // ConfigurationControlManager, since they are not persisted to the
+                // metadata log.
+                //
+                // When using incrementalAlterConfigs, we handle changes to BROKER_LOGGER
+                // in ControllerApis.scala.  When using the legacy alterConfigs,
+                // BROKER_LOGGER is not supported at all.
+                return new ApiError(Errors.INVALID_REQUEST, "Unsupported " +
+                    "configuration resource type BROKER_LOGGER ");
+            case BROKER:
+                // Note: A Resource with type BROKER and an empty name represents a
+                // cluster configuration that applies to all brokers.
+                if (!configResource.name().isEmpty()) {
+                    try {
+                        int brokerId = Integer.parseInt(configResource.name());
+                        if (brokerId < 0) {
+                            return new ApiError(Errors.INVALID_REQUEST, "Illegal " +
+                                "negative broker ID in BROKER resource.");
+                        }
+                    } catch (NumberFormatException e) {
+                        return new ApiError(Errors.INVALID_REQUEST, "Illegal " +
+                            "non-integral BROKER resource type name.");
+                    }
+                }
+                return ApiError.NONE;
+            case TOPIC:
+                try {
+                    Topic.validate(configResource.name());
+                } catch (Exception e) {
+                    return new ApiError(Errors.INVALID_REQUEST, "Illegal topic name.");
+                }
+                return ApiError.NONE;
+            case UNKNOWN:
+                return new ApiError(Errors.INVALID_REQUEST, "Unsupported configuration " +
+                    "resource type UNKNOWN.");
+            default:
+                return new ApiError(Errors.INVALID_REQUEST, "Unsupported unexpected " +
+                    "resource type");
+        }
+    }
+
+    boolean isSplittable(ConfigResource.Type type, String key) {
+        ConfigDef configDef = configDefs.get(type);
+        if (configDef == null) {
+            return false;
+        }
+        ConfigKey configKey = configDef.configKeys().get(key);
+        if (configKey == null) {
+            return false;
+        }
+        return configKey.type == ConfigDef.Type.LIST;
+    }
+
+    String getConfigValueDefault(ConfigResource.Type type, String key) {
+        ConfigDef configDef = configDefs.get(type);
+        if (configDef == null) {
+            return null;
+        }
+        ConfigKey configKey = configDef.configKeys().get(key);
+        if (configKey == null || !configKey.hasDefault()) {
+            return null;
+        }
+        return ConfigDef.convertToString(configKey.defaultValue, configKey.type);
+    }
+
+    /**
+     * Apply a configuration record to the in-memory state.
+     *
+     * @param record            The ConfigRecord.
+     */
+    void replay(ConfigRecord record) {
+        Type type = Type.forId(record.resourceType());
+        ConfigResource configResource = new ConfigResource(type, record.resourceName());
+        TimelineHashMap<String, String> configs = configData.get(configResource);
+        if (configs == null) {
+            configs = new TimelineHashMap<>(snapshotRegistry, 0);
+            configData.put(configResource, configs);
+        }
+        if (record.value() == null) {
+            configs.remove(record.name());
+        } else {
+            configs.put(record.name(), record.value());
+        }
+        log.info("{}: set configuration {} to {}", configResource, record.name(), record.value());
+    }
+
+    // VisibleForTesting
+    Map<String, String> getConfigs(ConfigResource configResource) {
+        Map<String, String> map = configData.get(configResource);
+        if (map == null) {
+            return Collections.emptyMap();
+        } else {
+            return Collections.unmodifiableMap(new HashMap<>(map));
+        }
+    }
+
+    public Map<ConfigResource, ResultOrError<Map<String, String>>> describeConfigs(
+            long lastCommittedOffset, Map<ConfigResource, Collection<String>> resources) {
+        Map<ConfigResource, ResultOrError<Map<String, String>>> results = new HashMap<>();
+        for (Entry<ConfigResource, Collection<String>> resourceEntry : resources.entrySet()) {
+            ConfigResource resource = resourceEntry.getKey();
+            ApiError error = checkConfigResource(resource);
+            if (error.isFailure()) {
+                results.put(resource, new ResultOrError<>(error));
+                continue;
+            }
+            Map<String, String> foundConfigs = new HashMap<>();
+            TimelineHashMap<String, String> configs =
+                configData.get(resource, lastCommittedOffset);
+            if (configs != null) {
+                Collection<String> targetConfigs = resourceEntry.getValue();
+                if (targetConfigs.isEmpty()) {
+                    Iterator<Entry<String, String>> iter =
+                        configs.entrySet(lastCommittedOffset).iterator();
+                    while (iter.hasNext()) {
+                        Entry<String, String> entry = iter.next();
+                        foundConfigs.put(entry.getKey(), entry.getValue());
+                    }
+                } else {
+                    for (String key : targetConfigs) {
+                        String value = configs.get(key, lastCommittedOffset);
+                        if (value != null) {
+                            foundConfigs.put(key, value);
+                        }
+                    }
+                }
+            }
+            results.put(resource, new ResultOrError<>(foundConfigs));
+        }
+        return results;
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/Controller.java b/metadata/src/main/java/org/apache/kafka/controller/Controller.java
index 0f6a54b..1ce63e0 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java
@@ -41,7 +41,7 @@ import java.util.concurrent.CompletableFuture;
 
 public interface Controller extends AutoCloseable {
     /**
-     * Change the in-sync replica sets for some partitions.
+     * Change partition ISRs.
      *
      * @param request       The AlterIsrRequest data.
      *
@@ -103,7 +103,7 @@ public interface Controller extends AutoCloseable {
      * @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.
+     * @return              A future yielding a map from config resources to error results.
      */
     CompletableFuture<Map<ConfigResource, ApiError>> incrementalAlterConfigs(
         Map<ConfigResource, Map<String, Map.Entry<AlterConfigOp.OpType, String>>> configChanges,
@@ -115,7 +115,7 @@ public interface Controller extends AutoCloseable {
      * @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.
+     * @return              A future yielding a map from config resources to error results.
      */
     CompletableFuture<Map<ConfigResource, ApiError>> legacyAlterConfigs(
         Map<ConfigResource, Map<String, String>> newConfigs, boolean validateOnly);
@@ -125,7 +125,7 @@ public interface Controller extends AutoCloseable {
      *
      * @param request      The broker heartbeat request.
      *
-     * @return              A future yielding a heartbeat reply.
+     * @return             A future yielding the broker heartbeat reply.
      */
     CompletableFuture<BrokerHeartbeatReply> processBrokerHeartbeat(
         BrokerHeartbeatRequestData request);
@@ -135,7 +135,7 @@ public interface Controller extends AutoCloseable {
      *
      * @param request      The registration request.
      *
-     * @return              A future yielding a registration reply.
+     * @return             A future yielding the broker registration reply.
      */
     CompletableFuture<BrokerRegistrationReply> registerBroker(
         BrokerRegistrationRequestData request);
@@ -174,6 +174,13 @@ public interface Controller extends AutoCloseable {
     long curClaimEpoch();
 
     /**
+     * Returns true if this controller is currently active.
+     */
+    default boolean isActive() {
+        return curClaimEpoch() != -1;
+    }
+
+    /**
      * Blocks until we have shut down and freed all resources.
      */
     void close() throws InterruptedException;
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java
new file mode 100644
index 0000000..fd4f3be
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+
+public interface ControllerMetrics {
+    void setActive(boolean active);
+
+    boolean active();
+
+    void updateEventQueueTime(long durationMs);
+
+    void updateEventQueueProcessingTime(long durationMs);
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerPurgatory.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerPurgatory.java
new file mode 100644
index 0000000..ee6c1d1
--- /dev/null
+++ b/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 " + lastKey + ".  We should not add one with an offset of " +
+                    offset + " which " + "is lower than that.");
+            }
+        }
+        List<DeferredEvent> events = pending.get(offset);
+        if (events == null) {
+            events = new ArrayList<>();
+            pending.put(offset, events);
+        }
+        events.add(event);
+    }
+
+    /**
+     * Get the offset of the highest pending event, or empty if there are no pending
+     * events.
+     */
+    Optional<Long> highestPendingOffset() {
+        if (pending.isEmpty()) {
+            return Optional.empty();
+        } else {
+            return Optional.of(pending.lastKey());
+        }
+    }
+}
\ No newline at end of file
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java
similarity index 50%
copy from metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java
copy to metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java
index 82e2b49..4906c8b 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java
@@ -17,45 +17,34 @@
 
 package org.apache.kafka.controller;
 
-import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Objects;
+import java.util.stream.Collectors;
 
 
-class ResultOrError<T> {
-    private final ApiError error;
-    private final T result;
+class ControllerResult<T> {
+    private final List<ApiMessageAndVersion> records;
+    private final T response;
 
-    public ResultOrError(Errors error, String message) {
-        this(new ApiError(error, message));
+    public ControllerResult(T response) {
+        this(new ArrayList<>(), response);
     }
 
-    public ResultOrError(ApiError error) {
-        Objects.requireNonNull(error);
-        this.error = error;
-        this.result = null;
+    public ControllerResult(List<ApiMessageAndVersion> records, T response) {
+        Objects.requireNonNull(records);
+        this.records = records;
+        this.response = response;
     }
 
-    public ResultOrError(T result) {
-        this.error = null;
-        this.result = result;
+    public List<ApiMessageAndVersion> records() {
+        return records;
     }
 
-    public boolean isError() {
-        return error != null;
-    }
-
-    public boolean isResult() {
-        return error == null;
-    }
-
-    public ApiError error() {
-        return error;
-    }
-
-    public T result() {
-        return result;
+    public T response() {
+        return response;
     }
 
     @Override
@@ -63,22 +52,24 @@ class ResultOrError<T> {
         if (o == null || (!o.getClass().equals(getClass()))) {
             return false;
         }
-        ResultOrError other = (ResultOrError) o;
-        return error.equals(other.error) &&
-            Objects.equals(result, other.result);
+        ControllerResult other = (ControllerResult) o;
+        return records.equals(other.records) &&
+            Objects.equals(response, other.response);
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(error, result);
+        return Objects.hash(records, response);
     }
 
     @Override
     public String toString() {
-        if (error.isSuccess()) {
-            return "ResultOrError(" + result + ")";
-        } else {
-            return "ResultOrError(" + error + ")";
-        }
+        return "ControllerResult(records=" + String.join(",",
+            records.stream().map(r -> r.toString()).collect(Collectors.toList())) +
+            ", response=" + response + ")";
+    }
+
+    public ControllerResult<T> withoutRecords() {
+        return new ControllerResult<>(new ArrayList<>(), response);
     }
 }
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerResultAndOffset.java
similarity index 50%
copy from metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java
copy to metadata/src/main/java/org/apache/kafka/controller/ControllerResultAndOffset.java
index 82e2b49..5e483f7 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerResultAndOffset.java
@@ -17,45 +17,31 @@
 
 package org.apache.kafka.controller;
 
-import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Objects;
+import java.util.stream.Collectors;
 
 
-class ResultOrError<T> {
-    private final ApiError error;
-    private final T result;
+class ControllerResultAndOffset<T> extends ControllerResult<T> {
+    private final long offset;
 
-    public ResultOrError(Errors error, String message) {
-        this(new ApiError(error, message));
+    public ControllerResultAndOffset(T response) {
+        super(new ArrayList<>(), response);
+        this.offset = -1;
     }
 
-    public ResultOrError(ApiError error) {
-        Objects.requireNonNull(error);
-        this.error = error;
-        this.result = null;
+    public ControllerResultAndOffset(long offset,
+                              List<ApiMessageAndVersion> records,
+                              T response) {
+        super(records, response);
+        this.offset = offset;
     }
 
-    public ResultOrError(T result) {
-        this.error = null;
-        this.result = result;
-    }
-
-    public boolean isError() {
-        return error != null;
-    }
-
-    public boolean isResult() {
-        return error == null;
-    }
-
-    public ApiError error() {
-        return error;
-    }
-
-    public T result() {
-        return result;
+    public long offset() {
+        return offset;
     }
 
     @Override
@@ -63,22 +49,21 @@ class ResultOrError<T> {
         if (o == null || (!o.getClass().equals(getClass()))) {
             return false;
         }
-        ResultOrError other = (ResultOrError) o;
-        return error.equals(other.error) &&
-            Objects.equals(result, other.result);
+        ControllerResultAndOffset other = (ControllerResultAndOffset) o;
+        return records().equals(other.records()) &&
+            response().equals(other.response()) &&
+            offset == other.offset;
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(error, result);
+        return Objects.hash(records(), response(), offset);
     }
 
     @Override
     public String toString() {
-        if (error.isSuccess()) {
-            return "ResultOrError(" + result + ")";
-        } else {
-            return "ResultOrError(" + error + ")";
-        }
+        return "ControllerResultAndOffset(records=" + String.join(",",
+            records().stream().map(r -> r.toString()).collect(Collectors.toList())) +
+            ", response=" + response() + ", offset=" + offset + ")";
     }
 }
diff --git a/metadata/src/main/java/org/apache/kafka/controller/DeferredEvent.java b/metadata/src/main/java/org/apache/kafka/controller/DeferredEvent.java
new file mode 100644
index 0000000..e1606f3
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/DeferredEvent.java
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+/**
+ * Represents a deferred event in the controller purgatory.
+ */
+interface DeferredEvent {
+    /**
+     * Complete the event.
+     *
+     * @param exception         null if the event should be completed successfully; the
+     *                          error otherwise.
+     */
+    void complete(Throwable exception);
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
new file mode 100644
index 0000000..25ff3fd
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
@@ -0,0 +1,136 @@
+/*
+ * 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.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import org.apache.kafka.common.metadata.FeatureLevelRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.FeatureMap;
+import org.apache.kafka.metadata.FeatureMapAndEpoch;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.apache.kafka.timeline.TimelineHashSet;
+
+
+public class FeatureControlManager {
+    /**
+     * The features supported by this controller's software.
+     */
+    private final Map<String, VersionRange> supportedFeatures;
+
+    /**
+     * Maps feature names to finalized version ranges.
+     */
+    private final TimelineHashMap<String, VersionRange> finalizedVersions;
+
+    /**
+     * The latest feature epoch.
+     */
+    private final TimelineHashSet<Long> epoch;
+
+    FeatureControlManager(Map<String, VersionRange> supportedFeatures,
+                          SnapshotRegistry snapshotRegistry) {
+        this.supportedFeatures = supportedFeatures;
+        this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.epoch = new TimelineHashSet<>(snapshotRegistry, 0);
+    }
+
+    ControllerResult<Map<String, ApiError>> updateFeatures(
+            Map<String, VersionRange> updates, Set<String> downgradeables,
+            Map<Integer, Map<String, VersionRange>> brokerFeatures) {
+        TreeMap<String, ApiError> results = new TreeMap<>();
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        for (Entry<String, VersionRange> entry : updates.entrySet()) {
+            results.put(entry.getKey(), updateFeature(entry.getKey(), entry.getValue(),
+                downgradeables.contains(entry.getKey()), brokerFeatures, records));
+        }
+        return new ControllerResult<>(records, results);
+    }
+
+    private ApiError updateFeature(String featureName,
+                                   VersionRange newRange,
+                                   boolean downgradeable,
+                                   Map<Integer, Map<String, VersionRange>> brokerFeatures,
+                                   List<ApiMessageAndVersion> records) {
+        if (newRange.min() <= 0) {
+            return new ApiError(Errors.INVALID_UPDATE_VERSION,
+                "The lower value for the new range cannot be less than 1.");
+        }
+        if (newRange.max() <= 0) {
+            return new ApiError(Errors.INVALID_UPDATE_VERSION,
+                "The upper value for the new range cannot be less than 1.");
+        }
+        VersionRange localRange = supportedFeatures.get(featureName);
+        if (localRange == null || !localRange.contains(newRange)) {
+            return new ApiError(Errors.INVALID_UPDATE_VERSION,
+                "The controller does not support the given feature range.");
+        }
+        for (Entry<Integer, Map<String, VersionRange>> brokerEntry :
+            brokerFeatures.entrySet()) {
+            VersionRange brokerRange = brokerEntry.getValue().get(featureName);
+            if (brokerRange == null || !brokerRange.contains(newRange)) {
+                return new ApiError(Errors.INVALID_UPDATE_VERSION,
+                    "Broker " + brokerEntry.getKey() + " does not support the given " +
+                        "feature range.");
+            }
+        }
+        VersionRange currentRange = finalizedVersions.get(featureName);
+        if (currentRange != null && currentRange.max() > newRange.max()) {
+            if (!downgradeable) {
+                return new ApiError(Errors.INVALID_UPDATE_VERSION,
+                    "Can't downgrade the maximum version of this feature without " +
+                    "setting downgradable to true.");
+            }
+        }
+        records.add(new ApiMessageAndVersion(
+            new FeatureLevelRecord().setName(featureName).
+                setMinFeatureLevel(newRange.min()).setMaxFeatureLevel(newRange.max()),
+            (short) 0));
+        return ApiError.NONE;
+    }
+
+    FeatureMapAndEpoch finalizedFeatures(long lastCommittedOffset) {
+        Map<String, VersionRange> features = new HashMap<>();
+        for (Entry<String, VersionRange> entry : finalizedVersions.entrySet(lastCommittedOffset)) {
+            features.put(entry.getKey(), entry.getValue());
+        }
+        long currentEpoch = -1;
+        Iterator<Long> iterator = epoch.iterator(lastCommittedOffset);
+        if (iterator.hasNext()) {
+            currentEpoch = iterator.next();
+        }
+        return new FeatureMapAndEpoch(new FeatureMap(features), currentEpoch);
+    }
+
+    void replay(FeatureLevelRecord record, long offset) {
+        finalizedVersions.put(record.name(),
+            new VersionRange(record.minFeatureLevel(), record.maxFeatureLevel()));
+        epoch.clear();
+        epoch.add(offset);
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
new file mode 100644
index 0000000..1980975
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
@@ -0,0 +1,941 @@
+/*
+ * 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.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.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+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.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+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.FeatureMapAndEpoch;
+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 static java.util.concurrent.TimeUnit.MICROSECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+
+/**
+ * QuorumController implements the main logic of the KIP-500 controller.
+ *
+ * The node which is the leader of the metadata log becomes the active controller.  All
+ * other nodes remain in standby mode.  Standby controllers cannot create new metadata log
+ * entries.  They just replay the metadata log entries that the current active controller
+ * has created.
+ *
+ * The QuorumController is single-threaded.  A single event handler thread performs most
+ * operations.  This avoids the need for complex locking.
+ *
+ * The controller exposes an asynchronous, futures-based API to the world.  This reflects
+ * the fact that the controller may have several operations in progress at any given
+ * point.  The future associated with each operation will not be completed until the
+ * results of the operation have been made durable to the metadata log.
+ */
+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 = NANOSECONDS.convert(18, TimeUnit.SECONDS);
+        private ControllerMetrics controllerMetrics = null;
+
+        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 Builder setMetrics(ControllerMetrics controllerMetrics) {
+            this.controllerMetrics = controllerMetrics;
+            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));
+            }
+            if (controllerMetrics == null) {
+                controllerMetrics = (ControllerMetrics) Class.forName(
+                    "org.apache.kafka.controller.MockControllerMetrics").getConstructor().newInstance();
+            }
+            KafkaEventQueue queue = null;
+            try {
+                queue = new KafkaEventQueue(time, logContext, threadNamePrefix);
+                return new QuorumController(logContext, nodeId, queue, time, configDefs,
+                        logManager, supportedFeatures, defaultReplicationFactor,
+                        defaultNumPartitions, replicaPlacementPolicy, sessionTimeoutNs,
+                        controllerMetrics);
+            } 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,
+            MICROSECONDS.convert(deltaNs, NANOSECONDS));
+        controllerMetrics.updateEventQueueProcessingTime(NANOSECONDS.toMillis(deltaNs));
+    }
+
+    private Throwable handleEventException(String name,
+                                           Optional<Long> startProcessingTimeNs,
+                                           Throwable exception) {
+        if (!startProcessingTimeNs.isPresent()) {
+            log.info("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 = MICROSECONDS.convert(deltaNs, NANOSECONDS);
+        if (exception instanceof ApiException) {
+            log.info("{}: 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 long eventCreatedTimeNs = time.nanoseconds();
+        private Optional<Long> startProcessingTimeNs = Optional.empty();
+
+        ControlEvent(String name, Runnable handler) {
+            this.name = name;
+            this.handler = handler;
+        }
+
+        @Override
+        public void run() throws Exception {
+            long now = time.nanoseconds();
+            controllerMetrics.updateEventQueueTime(NANOSECONDS.toMillis(now - eventCreatedTimeNs));
+            startProcessingTimeNs = Optional.of(now);
+            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 long eventCreatedTimeNs = time.nanoseconds();
+        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 {
+            long now = time.nanoseconds();
+            controllerMetrics.updateEventQueueTime(NANOSECONDS.toMillis(now - eventCreatedTimeNs));
+            startProcessingTimeNs = Optional.of(now);
+            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 result containing a list of records, and the 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 long eventCreatedTimeNs = time.nanoseconds();
+        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 now = time.nanoseconds();
+            controllerMetrics.updateEventQueueTime(NANOSECONDS.toMillis(now - eventCreatedTimeNs));
+            long controllerEpoch = curClaimEpoch;
+            if (controllerEpoch == -1) {
+                throw newNotControllerException();
+            }
+            startProcessingTimeNs = Optional.of(now);
+            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.
+                    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.
+                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;
+                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() +
+            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 the controller is a standby, replay the records that were
+                    // created by the active controller.
+                    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 {
+                    // If the controller is active, the records were already replayed,
+                    // so we don't need to do it here.
+                    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;
+                    controllerMetrics.setActive(true);
+                    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;
+        controllerMetrics.setActive(false);
+        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 deferred write event {} because the event queue " +
+                    "is now closed.", name);
+                return null;
+            } else if (e instanceof NotControllerException) {
+                log.debug("Cancelling deferred write event {} because this controller " +
+                    "is no longer active.", name);
+                return null;
+            }
+            log.error("Unexpected exception while executing deferred write event {}. " +
+                "Rescheduling for a minute from now.", name, e);
+            scheduleDeferredWriteEvent(name,
+                deadlineNs + NANOSECONDS.convert(1, TimeUnit.MINUTES), op);
+            return null;
+        });
+    }
+
+    static final String MAYBE_FENCE_REPLICAS = "maybeFenceReplicas";
+
+    private void rescheduleMaybeFenceStaleBrokers() {
+        long nextCheckTimeNs = clusterControl.heartbeatManager().nextCheckTimeNs();
+        if (nextCheckTimeNs == Long.MAX_VALUE) {
+            cancelMaybeFenceReplicas();
+            return;
+        }
+        scheduleDeferredWriteEvent(MAYBE_FENCE_REPLICAS, nextCheckTimeNs, () -> {
+            ControllerResult<Void> result = replicationControl.maybeFenceStaleBrokers();
+            rescheduleMaybeFenceStaleBrokers();
+            return result;
+        });
+    }
+
+    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 PARTITION_CHANGE_RECORD:
+                    replicationControl.replay((PartitionChangeRecord) 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;
+
+    /**
+     * The controller metrics.
+     */
+    private final ControllerMetrics controllerMetrics;
+
+    /**
+     * 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,
+                             ControllerMetrics controllerMetrics) throws Exception {
+        this.log = logContext.logger(QuorumController.class);
+        this.nodeId = nodeId;
+        this.queue = queue;
+        this.time = time;
+        this.controllerMetrics = controllerMetrics;
+        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> unregisterBroker(int brokerId) {
+        return appendWriteEvent("unregisterBroker",
+            () -> replicationControl.unregisterBroker(brokerId));
+    }
+
+    @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) {
+        return appendWriteEvent("electLeaders", request.timeoutMs(),
+            () -> replicationControl.electLeaders(request));
+    }
+
+    @Override
+    public CompletableFuture<FeatureMapAndEpoch> finalizedFeatures() {
+        return appendReadEvent("getFinalizedFeatures",
+            () -> featureControl.finalizedFeatures(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;
+            }
+        });
+    }
+
+    @Override
+    public CompletableFuture<BrokerHeartbeatReply>
+            processBrokerHeartbeat(BrokerHeartbeatRequestData request) {
+        return appendWriteEvent("processBrokerHeartbeat",
+            new ControllerWriteOperation<BrokerHeartbeatReply>() {
+                private final int brokerId = request.brokerId();
+                private boolean inControlledShutdown = false;
+
+                @Override
+                public ControllerResult<BrokerHeartbeatReply> generateRecordsAndResult() {
+                    ControllerResult<BrokerHeartbeatReply> result = replicationControl.
+                        processBrokerHeartbeat(request, lastCommittedOffset);
+                    inControlledShutdown = result.response().inControlledShutdown();
+                    rescheduleMaybeFenceStaleBrokers();
+                    return result;
+                }
+
+                @Override
+                public void processBatchEndOffset(long offset) {
+                    if (inControlledShutdown) {
+                        clusterControl.heartbeatManager().
+                            updateControlledShutdownOffset(brokerId, offset);
+                    }
+                }
+            });
+    }
+
+    @Override
+    public CompletableFuture<BrokerRegistrationReply>
+            registerBroker(BrokerRegistrationRequestData request) {
+        return appendWriteEvent("registerBroker", () -> {
+            ControllerResult<BrokerRegistrationReply> result = clusterControl.
+                registerBroker(request, writeOffset + 1, featureControl.
+                    finalizedFeatures(Long.MAX_VALUE));
+            rescheduleMaybeFenceStaleBrokers();
+            return result;
+        });
+    }
+
+    @Override
+    public CompletableFuture<Map<ClientQuotaEntity, ApiError>> alterClientQuotas(
+            Collection<ClientQuotaAlteration> quotaAlterations, boolean validateOnly) {
+        return appendWriteEvent("alterClientQuotas", () -> {
+            ControllerResult<Map<ClientQuotaEntity, ApiError>> result =
+                clientQuotaControlManager.alterClientQuotas(quotaAlterations);
+            if (validateOnly) {
+                return result.withoutRecords();
+            } else {
+                return result;
+            }
+        });
+    }
+
+    @Override
+    public CompletableFuture<Void> waitForReadyBrokers(int minBrokers) {
+        final CompletableFuture<Void> future = new CompletableFuture<>();
+        appendControlEvent("waitForReadyBrokers", () -> {
+            clusterControl.addReadyBrokersFuture(future, minBrokers);
+        });
+        return future;
+    }
+
+    @Override
+    public void beginShutdown() {
+        queue.beginShutdown("QuorumController#beginShutdown");
+    }
+
+    public int nodeId() {
+        return nodeId;
+    }
+
+    @Override
+    public long curClaimEpoch() {
+        return curClaimEpoch;
+    }
+
+    @Override
+    public void close() throws InterruptedException {
+        queue.close();
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java
new file mode 100644
index 0000000..ad56faf
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java
@@ -0,0 +1,70 @@
+/*
+ * 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 com.yammer.metrics.core.Gauge;
+import com.yammer.metrics.core.Histogram;
+import com.yammer.metrics.core.MetricName;
+import com.yammer.metrics.core.MetricsRegistry;
+
+
+public final class QuorumControllerMetrics implements ControllerMetrics {
+    private final static MetricName ACTIVE_CONTROLLER_COUNT = new MetricName(
+        "kafka.controller", "KafkaController", "ActiveControllerCount", null);
+    private final static MetricName EVENT_QUEUE_TIME_MS = new MetricName(
+        "kafka.controller", "ControllerEventManager", "EventQueueTimeMs", null);
+    private final static MetricName EVENT_QUEUE_PROCESSING_TIME_MS = new MetricName(
+        "kafka.controller", "ControllerEventManager", "EventQueueProcessingTimeMs", null);
+
+    private volatile boolean active;
+    private final Gauge<Integer> activeControllerCount;
+    private final Histogram eventQueueTime;
+    private final Histogram eventQueueProcessingTime;
+
+    public QuorumControllerMetrics(MetricsRegistry registry) {
+        this.active = false;
+        this.activeControllerCount = registry.newGauge(ACTIVE_CONTROLLER_COUNT, new Gauge<Integer>() {
+            @Override
+            public Integer value() {
+                return active ? 1 : 0;
+            }
+        });
+        this.eventQueueTime = registry.newHistogram(EVENT_QUEUE_TIME_MS, true);
+        this.eventQueueProcessingTime = registry.newHistogram(EVENT_QUEUE_PROCESSING_TIME_MS, true);
+    }
+
+    @Override
+    public void setActive(boolean active) {
+        this.active = active;
+    }
+
+    @Override
+    public boolean active() {
+        return this.active;
+    }
+
+    @Override
+    public void updateEventQueueTime(long durationMs) {
+        eventQueueTime.update(durationMs);
+    }
+
+    @Override
+    public void updateEventQueueProcessingTime(long durationMs) {
+        eventQueueTime.update(durationMs);
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicaPlacementPolicy.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicaPlacementPolicy.java
new file mode 100644
index 0000000..44de85d
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicaPlacementPolicy.java
@@ -0,0 +1,47 @@
+/*
+ * 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.Iterator;
+import java.util.List;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.metadata.UsableBroker;
+
+
+/**
+ * The interface which a Kafka replica placement policy must implement.
+ */
+@InterfaceStability.Unstable
+interface ReplicaPlacementPolicy {
+    /**
+     * Create a new replica placement.
+     *
+     * @param numPartitions         The number of partitions to create placements for.
+     * @param numReplicas           The number of replicas to create for each partitions.
+     *                              Must be positive.
+     * @param iterator              An iterator that yields all the usable brokers.
+     *
+     * @return                      A list of replica lists.
+     *
+     * @throws InvalidReplicationFactorException    If too many replicas were requested.
+     */
+    List<List<Integer>> createPlacement(int numPartitions, short numReplicas,
+                                        Iterator<UsableBroker> iterator)
+        throws InvalidReplicationFactorException;
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/Replicas.java b/metadata/src/main/java/org/apache/kafka/controller/Replicas.java
new file mode 100644
index 0000000..104fffe
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/Replicas.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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+
+public class Replicas {
+    /**
+     * An empty replica array.
+     */
+    public final static int[] NONE = new int[0];
+
+    /**
+     * Convert an array of integers to a list of ints.
+     *
+     * @param array         The input array.
+     * @return              The output list.
+     */
+    public static List<Integer> toList(int[] array) {
+        if (array == null) return null;
+        ArrayList<Integer> list = new ArrayList<>(array.length);
+        for (int i = 0; i < array.length; i++) {
+            list.add(array[i]);
+        }
+        return list;
+    }
+
+    /**
+     * Convert a list of integers to an array of ints.
+     *
+     * @param list          The input list.
+     * @return              The output array.
+     */
+    public static int[] toArray(List<Integer> list) {
+        if (list == null) return null;
+        int[] array = new int[list.size()];
+        for (int i = 0; i < list.size(); i++) {
+            array[i] = list.get(i);
+        }
+        return array;
+    }
+
+    /**
+     * Copy an array of ints.
+     *
+     * @param array         The input array.
+     * @return              A copy of the array.
+     */
+    public static int[] clone(int[] array) {
+        int[] clone = new int[array.length];
+        System.arraycopy(array, 0, clone, 0, array.length);
+        return clone;
+    }
+
+    /**
+     * Check that a replica set is valid.
+     *
+     * @param replicas      The replica set.
+     * @return              True if none of the replicas are negative, and there are no
+     *                      duplicates.
+     */
+    public static boolean validate(int[] replicas) {
+        if (replicas.length == 0) return true;
+        int[] sortedReplicas = clone(replicas);
+        Arrays.sort(sortedReplicas);
+        int prev = sortedReplicas[0];
+        if (prev < 0) return false;
+        for (int i = 1; i < sortedReplicas.length; i++) {
+            int replica = sortedReplicas[i];
+            if (prev == replica) return false;
+            prev = replica;
+        }
+        return true;
+    }
+
+    /**
+     * Check that an isr set is valid.
+     *
+     * @param replicas      The replica set.
+     * @param isr           The in-sync replica set.
+     * @return              True if none of the in-sync replicas are negative, there are
+     *                      no duplicates, and all in-sync replicas are also replicas.
+     */
+    public static boolean validateIsr(int[] replicas, int[] isr) {
+        if (isr.length == 0) return true;
+        if (replicas.length == 0) return false;
+        int[] sortedReplicas = clone(replicas);
+        Arrays.sort(sortedReplicas);
+        int[] sortedIsr = clone(isr);
+        Arrays.sort(sortedIsr);
+        int j = 0;
+        if (sortedIsr[0] < 0) return false;
+        int prevIsr = -1;
+        for (int i = 0; i < sortedIsr.length; i++) {
+            int curIsr = sortedIsr[i];
+            if (prevIsr == curIsr) return false;
+            prevIsr = curIsr;
+            while (true) {
+                if (j == sortedReplicas.length) return false;
+                int curReplica = sortedReplicas[j++];
+                if (curReplica == curIsr) break;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * Returns true if an array of replicas contains a specific value.
+     *
+     * @param replicas      The replica array.
+     * @param value         The value to look for.
+     *
+     * @return              True only if the value is found in the array.
+     */
+    public static boolean contains(int[] replicas, int value) {
+        for (int i = 0; i < replicas.length; i++) {
+            if (replicas[i] == value) return true;
+        }
+        return false;
+    }
+
+    /**
+     * Copy a replica array without any occurrences of the given value.
+     *
+     * @param replicas      The replica array.
+     * @param value         The value to filter out.
+     *
+     * @return              A new array without the given value.
+     */
+    public static int[] copyWithout(int[] replicas, int value) {
+        int size = 0;
+        for (int i = 0; i < replicas.length; i++) {
+            if (replicas[i] != value) {
+                size++;
+            }
+        }
+        int[] result = new int[size];
+        int j = 0;
+        for (int i = 0; i < replicas.length; i++) {
+            int replica = replicas[i];
+            if (replica != value) {
+                result[j++] = replica;
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Copy a replica array with the given value.
+     *
+     * @param replicas      The replica array.
+     * @param value         The value to add.
+     *
+     * @return              A new array with the given value.
+     */
+    public static int[] copyWith(int[] replicas, int value) {
+        int[] newReplicas = new int[replicas.length + 1];
+        System.arraycopy(replicas, 0, newReplicas, 0, replicas.length);
+        newReplicas[newReplicas.length - 1] = value;
+        return newReplicas;
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
new file mode 100644
index 0000000..aa60f51
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
@@ -0,0 +1,908 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals with topics
+ * and partitions. It is responsible for managing the in-sync replica set and leader
+ * of each partition, as well as administrative tasks like creating or deleting topics.
+ */
+public class ReplicationControlManager {
+    /**
+     * A special value used to represent the leader for a partition with no leader. 
+     */
+    public static final int NO_LEADER = -1;
+
+    /**
+     * A special value used to represent a PartitionChangeRecord that does not change the
+     * partition leader.
+     */
+    public static final int NO_LEADER_CHANGE = -2;
+
+    static class TopicControlInfo {
+        private final Uuid id;
+        private final TimelineHashMap<Integer, PartitionControlInfo> parts;
+
+        TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+            this.id = id;
+            this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+        }
+    }
+
+    static class PartitionControlInfo {
+        private final int[] replicas;
+        private final int[] isr;
+        private final int[] removingReplicas;
+        private final int[] addingReplicas;
+        private final int leader;
+        private final int leaderEpoch;
+        private final int partitionEpoch;
+
+        PartitionControlInfo(PartitionRecord record) {
+            this(Replicas.toArray(record.replicas()),
+                Replicas.toArray(record.isr()),
+                Replicas.toArray(record.removingReplicas()),
+                Replicas.toArray(record.addingReplicas()),
+                record.leader(),
+                record.leaderEpoch(),
+                record.partitionEpoch());
+        }
+
+        PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+                             int[] addingReplicas, int leader, int leaderEpoch,
+                             int partitionEpoch) {
+            this.replicas = replicas;
+            this.isr = isr;
+            this.removingReplicas = removingReplicas;
+            this.addingReplicas = addingReplicas;
+            this.leader = leader;
+            this.leaderEpoch = leaderEpoch;
+            this.partitionEpoch = partitionEpoch;
+        }
+
+        PartitionControlInfo merge(PartitionChangeRecord record) {
+            int[] newIsr = (record.isr() == null) ? isr : Replicas.toArray(record.isr());
+            int newLeader;
+            int newLeaderEpoch;
+            if (record.leader() == NO_LEADER_CHANGE) {
+                newLeader = leader;
+                newLeaderEpoch = leaderEpoch;
+            } else {
+                newLeader = record.leader();
+                newLeaderEpoch = leaderEpoch + 1;
+            }
+            return new PartitionControlInfo(replicas,
+                newIsr,
+                removingReplicas,
+                addingReplicas,
+                newLeader,
+                newLeaderEpoch,
+                partitionEpoch + 1);
+        }
+
+        String diff(PartitionControlInfo prev) {
+            StringBuilder builder = new StringBuilder();
+            String prefix = "";
+            if (!Arrays.equals(replicas, prev.replicas)) {
+                builder.append(prefix).append("oldReplicas=").append(Arrays.toString(prev.replicas));
+                prefix = ", ";
+                builder.append(prefix).append("newReplicas=").append(Arrays.toString(replicas));
+            }
+            if (!Arrays.equals(isr, prev.isr)) {
+                builder.append(prefix).append("oldIsr=").append(Arrays.toString(prev.isr));
+                prefix = ", ";
+                builder.append(prefix).append("newIsr=").append(Arrays.toString(isr));
+            }
+            if (!Arrays.equals(removingReplicas, prev.removingReplicas)) {
+                builder.append(prefix).append("oldRemovingReplicas=").
+                    append(Arrays.toString(prev.removingReplicas));
+                prefix = ", ";
+                builder.append(prefix).append("newRemovingReplicas=").
+                    append(Arrays.toString(removingReplicas));
+            }
+            if (!Arrays.equals(addingReplicas, prev.addingReplicas)) {
+                builder.append(prefix).append("oldAddingReplicas=").
+                    append(Arrays.toString(prev.addingReplicas));
+                prefix = ", ";
+                builder.append(prefix).append("newAddingReplicas=").
+                    append(Arrays.toString(addingReplicas));
+            }
+            if (leader != prev.leader) {
+                builder.append(prefix).append("oldLeader=").append(prev.leader);
+                prefix = ", ";
+                builder.append(prefix).append("newLeader=").append(leader);
+            }
+            if (leaderEpoch != prev.leaderEpoch) {
+                builder.append(prefix).append("oldLeaderEpoch=").append(prev.leaderEpoch);
+                prefix = ", ";
+                builder.append(prefix).append("newLeaderEpoch=").append(leaderEpoch);
+            }
+            if (partitionEpoch != prev.partitionEpoch) {
+                builder.append(prefix).append("oldPartitionEpoch=").append(prev.partitionEpoch);
+                prefix = ", ";
+                builder.append(prefix).append("newPartitionEpoch=").append(partitionEpoch);
+            }
+            return builder.toString();
+        }
+
+        boolean hasLeader() {
+            return leader != NO_LEADER;
+        }
+
+        int preferredReplica() {
+            return replicas.length == 0 ? NO_LEADER : replicas[0];
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(replicas, isr, removingReplicas, addingReplicas, leader,
+                leaderEpoch, partitionEpoch);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (!(o instanceof PartitionControlInfo)) return false;
+            PartitionControlInfo other = (PartitionControlInfo) o;
+            return diff(other).isEmpty();
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder builder = new StringBuilder("PartitionControlInfo(");
+            builder.append("replicas=").append(Arrays.toString(replicas));
+            builder.append(", isr=").append(Arrays.toString(isr));
+            builder.append(", removingReplicas=").append(Arrays.toString(removingReplicas));
+            builder.append(", addingReplicas=").append(Arrays.toString(addingReplicas));
+            builder.append(", leader=").append(leader);
+            builder.append(", leaderEpoch=").append(leaderEpoch);
+            builder.append(", partitionEpoch=").append(partitionEpoch);
+            builder.append(")");
+            return builder.toString();
+        }
+    }
+
+    private final SnapshotRegistry snapshotRegistry;
+    private final Logger log;
+
+    /**
+     * The random number generator used by this object.
+     */
+    private final Random random;
+
+    /**
+     * The KIP-464 default replication factor that is used if a CreateTopics request does
+     * not specify one.
+     */
+    private final short defaultReplicationFactor;
+
+    /**
+     * The KIP-464 default number of partitions that is used if a CreateTopics request does
+     * not specify a number of partitions.
+     */
+    private final int defaultNumPartitions;
+
+    /**
+     * A reference to the controller's configuration control manager.
+     */
+    private final ConfigurationControlManager configurationControl;
+
+    /**
+     * A reference to the controller's cluster control manager.
+     */
+    final ClusterControlManager clusterControl;
+
+    /**
+     * Maps topic names to topic UUIDs.
+     */
+    private final TimelineHashMap<String, Uuid> topicsByName;
+
+    /**
+     * Maps topic UUIDs to structures containing topic information, including partitions.
+     */
+    private final TimelineHashMap<Uuid, TopicControlInfo> topics;
+
+    /**
+     * A map of broker IDs to the partitions that the broker is in the ISR for.
+     */
+    private final BrokersToIsrs brokersToIsrs;
+
+    ReplicationControlManager(SnapshotRegistry snapshotRegistry,
+                              LogContext logContext,
+                              Random random,
+                              short defaultReplicationFactor,
+                              int defaultNumPartitions,
+                              ConfigurationControlManager configurationControl,
+                              ClusterControlManager clusterControl) {
+        this.snapshotRegistry = snapshotRegistry;
+        this.log = logContext.logger(ReplicationControlManager.class);
+        this.random = random;
+        this.defaultReplicationFactor = defaultReplicationFactor;
+        this.defaultNumPartitions = defaultNumPartitions;
+        this.configurationControl = configurationControl;
+        this.clusterControl = clusterControl;
+        this.topicsByName = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.topics = new TimelineHashMap<>(snapshotRegistry, 0);
+        this.brokersToIsrs = new BrokersToIsrs(snapshotRegistry);
+    }
+
+    public void replay(TopicRecord record) {
+        topicsByName.put(record.name(), record.topicId());
+        topics.put(record.topicId(), new TopicControlInfo(snapshotRegistry, record.topicId()));
+        log.info("Created topic {} with ID {}.", record.name(), record.topicId());
+    }
+
+    public void replay(PartitionRecord record) {
+        TopicControlInfo topicInfo = topics.get(record.topicId());
+        if (topicInfo == null) {
+            throw new RuntimeException("Tried to create partition " + record.topicId() +
+                ":" + record.partitionId() + ", but no topic with that ID was found.");
+        }
+        PartitionControlInfo newPartInfo = new PartitionControlInfo(record);
+        PartitionControlInfo prevPartInfo = topicInfo.parts.get(record.partitionId());
+        if (prevPartInfo == null) {
+            log.info("Created partition {}:{} with {}.", record.topicId(),
+                record.partitionId(), newPartInfo.toString());
+            topicInfo.parts.put(record.partitionId(), newPartInfo);
+            brokersToIsrs.update(record.topicId(), record.partitionId(), null,
+                newPartInfo.isr, NO_LEADER, newPartInfo.leader);
+        } else {
+            String diff = newPartInfo.diff(prevPartInfo);
+            if (!diff.isEmpty()) {
+                log.info("Modified partition {}:{}: {}.", record.topicId(),
+                    record.partitionId(), diff);
+                topicInfo.parts.put(record.partitionId(), newPartInfo);
+                brokersToIsrs.update(record.topicId(), record.partitionId(),
+                    prevPartInfo.isr, newPartInfo.isr, prevPartInfo.leader,
+                    newPartInfo.leader);
+            }
+        }
+    }
+
+    public void replay(PartitionChangeRecord record) {
+        TopicControlInfo topicInfo = topics.get(record.topicId());
+        if (topicInfo == null) {
+            throw new RuntimeException("Tried to create partition " + record.topicId() +
+                ":" + record.partitionId() + ", but no topic with that ID was found.");
+        }
+        PartitionControlInfo prevPartitionInfo = topicInfo.parts.get(record.partitionId());
+        if (prevPartitionInfo == null) {
+            throw new RuntimeException("Tried to create partition " + record.topicId() +
+                ":" + record.partitionId() + ", but no partition with that id was found.");
+        }
+        PartitionControlInfo newPartitionInfo = prevPartitionInfo.merge(record);
+        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()));
+                    continue;
+                }
+                if (!Replicas.contains(newIsr, partition.leader)) {
+                    // An alterIsr request can't remove the current leader.
+                    responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData().
+                        setPartitionIndex(partitionData.partitionIndex()).
+                        setErrorCode(Errors.INVALID_REQUEST.code()));
+                    continue;
+                }
+                records.add(new ApiMessageAndVersion(new PartitionChangeRecord().
+                    setPartitionId(partitionData.partitionIndex()).
+                    setTopicId(topic.id).
+                    setIsr(partitionData.newIsr()), (short) 0));
+            }
+        }
+        return new ControllerResult<>(records, response);
+    }
+
+    /**
+     * Generate the appropriate records to handle a broker being fenced.
+     *
+     * First, we remove this broker from any non-singleton ISR. Then we generate a
+     * FenceBrokerRecord.
+     *
+     * @param brokerId      The broker id.
+     * @param records       The record list to append to.
+     */
+
+    void handleBrokerFenced(int brokerId, List<ApiMessageAndVersion> records) {
+        BrokerRegistration brokerRegistration = clusterControl.brokerRegistrations().get(brokerId);
+        if (brokerRegistration == null) {
+            throw new RuntimeException("Can't find broker registration for broker " + brokerId);
+        }
+        handleNodeDeactivated(brokerId, records);
+        records.add(new ApiMessageAndVersion(new FenceBrokerRecord().
+            setId(brokerId).setEpoch(brokerRegistration.epoch()), (short) 0));
+    }
+
+    /**
+     * Generate the appropriate records to handle a broker being unregistered.
+     *
+     * First, we remove this broker from any non-singleton ISR. Then we generate an
+     * UnregisterBrokerRecord.
+     *
+     * @param brokerId      The broker id.
+     * @param brokerEpoch   The broker epoch.
+     * @param records       The record list to append to.
+     */
+    void handleBrokerUnregistered(int brokerId, long brokerEpoch,
+                                  List<ApiMessageAndVersion> records) {
+        handleNodeDeactivated(brokerId, records);
+        records.add(new ApiMessageAndVersion(new UnregisterBrokerRecord().
+            setBrokerId(brokerId).setBrokerEpoch(brokerEpoch), (short) 0));
+    }
+
+    /**
+     * Handle a broker being deactivated. This means we remove it from any ISR that has
+     * more than one element. We do not remove the broker from ISRs where it is the only
+     * member since this would preclude clean leader election in the future.
+     * It is removed as the leader for all partitions it leads.
+     *
+     * @param brokerId              The broker id.
+     * @param records               The record list to append to.
+     */
+    void handleNodeDeactivated(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.");
+            }
+            PartitionChangeRecord record = new PartitionChangeRecord().
+                setPartitionId(topicPartition.partitionId()).
+                setTopicId(topic.id);
+            int[] newIsr = Replicas.copyWithout(partition.isr, brokerId);
+            if (newIsr.length == 0) {
+                // We don't want to shrink the ISR to size 0. So, leave the node in the ISR.
+                if (record.leader() != NO_LEADER) {
+                    // The partition is now leaderless, so set its leader to -1.
+                    record.setLeader(-1);
+                    records.add(new ApiMessageAndVersion(record, (short) 0));
+                }
+            } else {
+                record.setIsr(Replicas.toList(newIsr));
+                if (partition.leader == brokerId) {
+                    // The fenced node will no longer be the leader.
+                    int newLeader = bestLeader(partition.replicas, newIsr, false);
+                    record.setLeader(newLeader);
+                } else {
+                    // Bump the partition leader epoch.
+                    record.setLeader(partition.leader);
+                }
+                records.add(new ApiMessageAndVersion(record, (short) 0));
+            }
+        }
+    }
+
+    /**
+     * Generate the appropriate records to handle a broker becoming unfenced.
+     *
+     * First, we create an UnfenceBrokerRecord. Then, we check if if there are any
+     * partitions that don't currently have a leader that should be led by the newly
+     * unfenced broker.
+     *
+     * @param brokerId      The broker id.
+     * @param brokerEpoch   The broker epoch.
+     * @param records       The record list to append to.
+     */
+    void handleBrokerUnfenced(int brokerId, long brokerEpoch, List<ApiMessageAndVersion> records) {
+        records.add(new ApiMessageAndVersion(new UnfenceBrokerRecord().
+            setId(brokerId).setEpoch(brokerEpoch), (short) 0));
+        handleNodeActivated(brokerId, records);
+    }
+
+    /**
+     * Handle a broker being activated. This means we check if it can become the leader
+     * for any partition that currently has no leader (aka offline partition).
+     *
+     * @param brokerId      The broker id.
+     * @param records       The record list to append to.
+     */
+    void handleNodeActivated(int brokerId, List<ApiMessageAndVersion> records) {
+        Iterator<TopicPartition> iterator = brokersToIsrs.noLeaderIterator();
+        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.");
+            }
+            // TODO: if this partition is configured for unclean leader election,
+            // check the replica set rather than the ISR.
+            if (Replicas.contains(partition.isr, brokerId)) {
+                records.add(new ApiMessageAndVersion(new PartitionChangeRecord().
+                    setPartitionId(topicPartition.partitionId()).
+                    setTopicId(topic.id).
+                    setLeader(brokerId), (short) 0));
+            }
+        }
+    }
+
+    ControllerResult<ElectLeadersResponseData> electLeaders(ElectLeadersRequestData request) {
+        boolean unclean = electionIsUnclean(request.electionType());
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        ElectLeadersResponseData response = new ElectLeadersResponseData();
+        for (TopicPartitions topic : request.topicPartitions()) {
+            ReplicaElectionResult topicResults =
+                new ReplicaElectionResult().setTopic(topic.topic());
+            response.replicaElectionResults().add(topicResults);
+            for (int partitionId : topic.partitions()) {
+                ApiError error = electLeader(topic.topic(), partitionId, unclean, records);
+                topicResults.partitionResult().add(new PartitionResult().
+                    setPartitionId(partitionId).
+                    setErrorCode(error.error().code()).
+                    setErrorMessage(error.message()));
+            }
+        }
+        return new ControllerResult<>(records, response);
+    }
+
+    static boolean electionIsUnclean(byte electionType) {
+        ElectionType type;
+        try {
+            type = ElectionType.valueOf(electionType);
+        } catch (IllegalArgumentException e) {
+            throw new InvalidRequestException("Unknown election type " + (int) electionType);
+        }
+        return type == ElectionType.UNCLEAN;
+    }
+
+    ApiError electLeader(String topic, int partitionId, boolean unclean,
+                         List<ApiMessageAndVersion> records) {
+        Uuid topicId = topicsByName.get(topic);
+        if (topicId == null) {
+            return new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION,
+                "No such topic as " + topic);
+        }
+        TopicControlInfo topicInfo = topics.get(topicId);
+        if (topicInfo == null) {
+            return new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION,
+                "No such topic id as " + topicId);
+        }
+        PartitionControlInfo partitionInfo = topicInfo.parts.get(partitionId);
+        if (partitionInfo == null) {
+            return new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION,
+                "No such partition as " + topic + "-" + partitionId);
+        }
+        int newLeader = bestLeader(partitionInfo.replicas, partitionInfo.isr, unclean);
+        if (newLeader == NO_LEADER) {
+            // If we can't find any leader for the partition, return an error.
+            return new ApiError(Errors.LEADER_NOT_AVAILABLE,
+                "Unable to find any leader for the partition.");
+        }
+        if (newLeader == partitionInfo.leader) {
+            // If the new leader we picked is the same as the current leader, there is
+            // nothing to do.
+            return ApiError.NONE;
+        }
+        if (partitionInfo.hasLeader() && newLeader != partitionInfo.preferredReplica()) {
+            // It is not worth moving away from a valid leader to a new leader unless the
+            // new leader is the preferred replica.
+            return ApiError.NONE;
+        }
+        PartitionChangeRecord record = new PartitionChangeRecord().
+            setPartitionId(partitionId).
+            setTopicId(topicId);
+        if (unclean && !Replicas.contains(partitionInfo.isr, newLeader)) {
+            // If the election was unclean, we may have to forcibly add the replica to
+            // the ISR.  This can result in data loss!
+            record.setIsr(Collections.singletonList(newLeader));
+        }
+        record.setLeader(newLeader);
+        records.add(new ApiMessageAndVersion(record, (short) 0));
+        return ApiError.NONE;
+    }
+
+    ControllerResult<BrokerHeartbeatReply> processBrokerHeartbeat(
+                BrokerHeartbeatRequestData request, long lastCommittedOffset) {
+        int brokerId = request.brokerId();
+        long brokerEpoch = request.brokerEpoch();
+        clusterControl.checkBrokerEpoch(brokerId, brokerEpoch);
+        BrokerHeartbeatManager heartbeatManager = clusterControl.heartbeatManager();
+        BrokerControlStates states = heartbeatManager.calculateNextBrokerState(brokerId,
+            request, lastCommittedOffset, () -> brokersToIsrs.hasLeaderships(brokerId));
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        if (states.current() != states.next()) {
+            switch (states.next()) {
+                case FENCED:
+                    handleBrokerFenced(brokerId, records);
+                    break;
+                case UNFENCED:
+                    handleBrokerUnfenced(brokerId, brokerEpoch, records);
+                    break;
+                case CONTROLLED_SHUTDOWN:
+                    // Note: we always bump the leader epoch of each partition that the
+                    // shutting down broker is in here.  This prevents the broker from
+                    // getting re-added to the ISR later.
+                    handleNodeDeactivated(brokerId, records);
+                    break;
+                case SHUTDOWN_NOW:
+                    handleBrokerFenced(brokerId, records);
+                    break;
+            }
+        }
+        heartbeatManager.touch(brokerId,
+            states.next().fenced(),
+            request.currentMetadataOffset());
+        boolean isCaughtUp = request.currentMetadataOffset() >= lastCommittedOffset;
+        BrokerHeartbeatReply reply = new BrokerHeartbeatReply(isCaughtUp,
+                states.next().fenced(),
+                states.next().inControlledShutdown(),
+                states.next().shouldShutDown());
+        return new ControllerResult<>(records, reply);
+    }
+
+    int bestLeader(int[] replicas, int[] isr, boolean unclean) {
+        for (int i = 0; i < replicas.length; i++) {
+            int replica = replicas[i];
+            if (Replicas.contains(isr, replica)) {
+                return replica;
+            }
+        }
+        if (unclean) {
+            for (int i = 0; i < replicas.length; i++) {
+                int replica = replicas[i];
+                if (clusterControl.unfenced(replica)) {
+                    return replica;
+                }
+            }
+        }
+        return NO_LEADER;
+    }
+
+    public ControllerResult<Void> unregisterBroker(int brokerId) {
+        BrokerRegistration registration = clusterControl.brokerRegistrations().get(brokerId);
+        if (registration == null) {
+            throw new BrokerIdNotRegisteredException("Broker ID " + brokerId +
+                " is not currently registered");
+        }
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        handleBrokerUnregistered(brokerId, registration.epoch(), records);
+        return new ControllerResult<>(records, null);
+    }
+
+    ControllerResult<Void> maybeFenceStaleBrokers() {
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        BrokerHeartbeatManager heartbeatManager = clusterControl.heartbeatManager();
+        List<Integer> staleBrokers = heartbeatManager.findStaleBrokers();
+        for (int brokerId : staleBrokers) {
+            log.info("Fencing broker {} because its session has timed out.", brokerId);
+            handleBrokerFenced(brokerId, records);
+            heartbeatManager.fence(brokerId);
+        }
+        return new ControllerResult<>(records, null);
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java b/metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java
index 82e2b49..6d910e4 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ResultOrError.java
@@ -64,7 +64,7 @@ class ResultOrError<T> {
             return false;
         }
         ResultOrError other = (ResultOrError) o;
-        return error.equals(other.error) &&
+        return Objects.equals(error, other.error) &&
             Objects.equals(result, other.result);
     }
 
@@ -75,7 +75,7 @@ class ResultOrError<T> {
 
     @Override
     public String toString() {
-        if (error.isSuccess()) {
+        if (error == null) {
             return "ResultOrError(" + result + ")";
         } else {
             return "ResultOrError(" + error + ")";
diff --git a/metadata/src/main/java/org/apache/kafka/controller/SimpleReplicaPlacementPolicy.java b/metadata/src/main/java/org/apache/kafka/controller/SimpleReplicaPlacementPolicy.java
new file mode 100644
index 0000000..95e96cd
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/controller/SimpleReplicaPlacementPolicy.java
@@ -0,0 +1,77 @@
+/*
+ * 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.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.metadata.UsableBroker;
+
+
+/**
+ * A simple uniformly random placement policy.
+ *
+ * TODO: implement the current "striped" placement policy, plus rack aware placement
+ * policies, etc.
+ */
+public class SimpleReplicaPlacementPolicy implements ReplicaPlacementPolicy {
+    private final Random random;
+
+    public SimpleReplicaPlacementPolicy(Random random) {
+        this.random = random;
+    }
+
+    @Override
+    public List<List<Integer>> createPlacement(int numPartitions,
+                                               short numReplicas,
+                                               Iterator<UsableBroker> iterator) {
+        List<UsableBroker> usable = new ArrayList<>();
+        while (iterator.hasNext()) {
+            usable.add(iterator.next());
+        }
+        if (usable.size() < numReplicas) {
+            throw new InvalidReplicationFactorException("there are only " + usable.size() +
+                " usable brokers");
+        }
+        List<List<Integer>> results = new ArrayList<>();
+        for (int p = 0; p < numPartitions; p++) {
+            List<Integer> choices = new ArrayList<>();
+            // TODO: rack-awareness
+            List<Integer> indexes = new ArrayList<>();
+            int initialIndex = random.nextInt(usable.size());
+            for (int i = 0; i < numReplicas; i++) {
+                indexes.add((initialIndex + i) % usable.size());
+            }
+            indexes.sort(Integer::compareTo);
+            Iterator<UsableBroker> iter = usable.iterator();
+            for (int i = 0; choices.size() < indexes.size(); i++) {
+                int brokerId = iter.next().id();
+                if (indexes.get(choices.size()) == i) {
+                    choices.add(brokerId);
+                }
+            }
+            Collections.shuffle(choices, random);
+            results.add(choices);
+        }
+        return results;
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/BrokerHeartbeatReply.java b/metadata/src/main/java/org/apache/kafka/metadata/BrokerHeartbeatReply.java
index 5ab2a52..c936601 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/BrokerHeartbeatReply.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/BrokerHeartbeatReply.java
@@ -32,15 +32,22 @@ public class BrokerHeartbeatReply {
     private final boolean isFenced;
 
     /**
+     * True if the broker is currently in a controlled shutdown state.
+     */
+    private final boolean inControlledShutdown;
+
+    /**
      * True if the heartbeat reply should tell the broker that it should shut down.
      */
     private final boolean shouldShutDown;
 
     public BrokerHeartbeatReply(boolean isCaughtUp,
                                 boolean isFenced,
+                                boolean inControlledShutdown,
                                 boolean shouldShutDown) {
         this.isCaughtUp = isCaughtUp;
         this.isFenced = isFenced;
+        this.inControlledShutdown = inControlledShutdown;
         this.shouldShutDown = shouldShutDown;
     }
 
@@ -52,13 +59,17 @@ public class BrokerHeartbeatReply {
         return isFenced;
     }
 
+    public boolean inControlledShutdown() {
+        return inControlledShutdown;
+    }
+
     public boolean shouldShutDown() {
         return shouldShutDown;
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(isCaughtUp, isFenced, shouldShutDown);
+        return Objects.hash(isCaughtUp, isFenced, inControlledShutdown, shouldShutDown);
     }
 
     @Override
@@ -67,6 +78,7 @@ public class BrokerHeartbeatReply {
         BrokerHeartbeatReply other = (BrokerHeartbeatReply) o;
         return other.isCaughtUp == isCaughtUp &&
             other.isFenced == isFenced &&
+            other.inControlledShutdown == inControlledShutdown &&
             other.shouldShutDown == shouldShutDown;
     }
 
@@ -74,6 +86,7 @@ public class BrokerHeartbeatReply {
     public String toString() {
         return "BrokerHeartbeatReply(isCaughtUp=" + isCaughtUp +
             ", isFenced=" + isFenced +
+            ", inControlledShutdown=" + inControlledShutdown +
             ", shouldShutDown = " + shouldShutDown +
             ")";
     }
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/UsableBroker.java b/metadata/src/main/java/org/apache/kafka/metadata/UsableBroker.java
new file mode 100644
index 0000000..9a2db10
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/metadata/UsableBroker.java
@@ -0,0 +1,61 @@
+/*
+ * 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.metadata;
+
+import java.util.Objects;
+import java.util.Optional;
+
+
+/**
+ * A broker where a replica can be placed.
+ */
+public class UsableBroker {
+    private final int id;
+
+    private final Optional<String> rack;
+
+    public UsableBroker(int id, Optional<String> rack) {
+        this.id = id;
+        this.rack = rack;
+    }
+
+    public int id() {
+        return id;
+    }
+
+    public Optional<String> rack() {
+        return rack;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof UsableBroker)) return false;
+        UsableBroker other = (UsableBroker) o;
+        return other.id == id && other.rack.equals(rack);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(id, rack);
+    }
+
+    @Override
+    public String toString() {
+        return "UsableBroker(id=" + id + ", rack=" + rack + ")";
+    }
+}
diff --git a/metadata/src/main/resources/common/metadata/AccessControlRecord.json b/metadata/src/main/resources/common/metadata/AccessControlRecord.json
index 3acf469..deef33c 100644
--- a/metadata/src/main/resources/common/metadata/AccessControlRecord.json
+++ b/metadata/src/main/resources/common/metadata/AccessControlRecord.json
@@ -18,6 +18,7 @@
   "type": "metadata",
   "name": "AccessControlRecord",
   "validVersions": "0",
+  "flexibleVersions": "0+",
   "fields": [
     { "name": "ResourceType", "type": "int8", "versions": "0+",
       "about": "The resource type" },
diff --git a/metadata/src/main/resources/common/metadata/IsrChangeRecord.json b/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json
similarity index 63%
rename from metadata/src/main/resources/common/metadata/IsrChangeRecord.json
rename to metadata/src/main/resources/common/metadata/PartitionChangeRecord.json
index fd8d834..070e5c9 100644
--- a/metadata/src/main/resources/common/metadata/IsrChangeRecord.json
+++ b/metadata/src/main/resources/common/metadata/PartitionChangeRecord.json
@@ -16,20 +16,19 @@
 {
   "apiKey": 5,
   "type": "metadata",
-  "name": "IsrChangeRecord",
+  "name": "PartitionChangeRecord",
   "validVersions": "0",
+  "flexibleVersions": "0+",
   "fields": [
     { "name": "PartitionId", "type": "int32", "versions": "0+", "default": "-1",
       "about": "The partition id." },
     { "name": "TopicId", "type": "uuid", "versions": "0+",
       "about": "The unique ID of this topic." },
-    { "name": "Isr", "type":  "[]int32", "versions":  "0+",
-      "about": "The in-sync replicas of this partition" },
-    { "name": "Leader", "type": "int32", "versions": "0+", "default": "-1",
-      "about": "The lead replica, or -1 if there is no leader." },
-    { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "default": "-1",
-      "about": "An epoch that gets incremented each time we change the partition leader." },
-    { "name": "PartitionEpoch", "type": "int32", "versions": "0+", "default": "-1",
-      "about": "An epoch that gets incremented each time we change anything in the partition." }
+    { "name": "Isr", "type":  "[]int32", "default": "null",
+      "versions": "0+", "nullableVersions":  "0+", "taggedVersions": "0+", "tag": 0,
+      "about": "null if the ISR didn't change; the new in-sync replicas otherwise." },
+    { "name": "Leader", "type": "int32", "default": "-2",
+      "versions": "0+", "taggedVersions": "0+", "tag": 1,
+      "about": "-1 if there is now no leader; -2 if the leader didn't change; the new leader otherwise." }
   ]
 }
diff --git a/metadata/src/main/resources/common/metadata/PartitionRecord.json b/metadata/src/main/resources/common/metadata/PartitionRecord.json
index 5cc7d13..2a92c21 100644
--- a/metadata/src/main/resources/common/metadata/PartitionRecord.json
+++ b/metadata/src/main/resources/common/metadata/PartitionRecord.json
@@ -34,7 +34,7 @@
     { "name": "Leader", "type": "int32", "versions": "0+", "default": "-1",
       "about": "The lead replica, or -1 if there is no leader." },
     { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "default": "-1",
-      "about": "An epoch that gets incremented each time we change the partition leader." },
+      "about": "The epoch of the partition leader." },
     { "name": "PartitionEpoch", "type": "int32", "versions": "0+", "default": "-1",
       "about": "An epoch that gets incremented each time we change anything in the partition." }
   ]
diff --git a/metadata/src/test/java/org/apache/kafka/controller/BrokerHeartbeatManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/BrokerHeartbeatManagerTest.java
new file mode 100644
index 0000000..d70cc5c
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/BrokerHeartbeatManagerTest.java
@@ -0,0 +1,296 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.controller.BrokerHeartbeatManager.BrokerHeartbeatState;
+import org.apache.kafka.controller.BrokerHeartbeatManager.BrokerHeartbeatStateIterator;
+import org.apache.kafka.controller.BrokerHeartbeatManager.BrokerHeartbeatStateList;
+import org.apache.kafka.controller.BrokerHeartbeatManager.UsableBrokerIterator;
+import org.apache.kafka.metadata.UsableBroker;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.apache.kafka.controller.BrokerControlState.CONTROLLED_SHUTDOWN;
+import static org.apache.kafka.controller.BrokerControlState.FENCED;
+import static org.apache.kafka.controller.BrokerControlState.SHUTDOWN_NOW;
+import static org.apache.kafka.controller.BrokerControlState.UNFENCED;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+@Timeout(40)
+public class BrokerHeartbeatManagerTest {
+    private static BrokerHeartbeatManager newBrokerHeartbeatManager() {
+        LogContext logContext = new LogContext();
+        MockTime time = new MockTime(0, 1_000_000, 0);
+        return new BrokerHeartbeatManager(logContext, time, 10_000_000);
+    }
+
+    @Test
+    public void testHasValidSession() {
+        BrokerHeartbeatManager manager = newBrokerHeartbeatManager();
+        MockTime time = (MockTime)  manager.time();
+        assertFalse(manager.hasValidSession(0));
+        manager.touch(0, false, 0);
+        time.sleep(5);
+        manager.touch(1, false, 0);
+        manager.touch(2, false, 0);
+        assertTrue(manager.hasValidSession(0));
+        assertTrue(manager.hasValidSession(1));
+        assertTrue(manager.hasValidSession(2));
+        assertFalse(manager.hasValidSession(3));
+        time.sleep(6);
+        assertFalse(manager.hasValidSession(0));
+        assertTrue(manager.hasValidSession(1));
+        assertTrue(manager.hasValidSession(2));
+        assertFalse(manager.hasValidSession(3));
+        manager.remove(2);
+        assertFalse(manager.hasValidSession(2));
+        manager.remove(1);
+        assertFalse(manager.hasValidSession(1));
+    }
+
+    @Test
+    public void testFindStaleBrokers() {
+        BrokerHeartbeatManager manager = newBrokerHeartbeatManager();
+        MockTime time = (MockTime)  manager.time();
+        assertFalse(manager.hasValidSession(0));
+        manager.touch(0, false, 0);
+        time.sleep(5);
+        manager.touch(1, false, 0);
+        time.sleep(1);
+        manager.touch(2, false, 0);
+
+        Iterator<BrokerHeartbeatState> iter = manager.unfenced().iterator();
+        assertEquals(0, iter.next().id());
+        assertEquals(1, iter.next().id());
+        assertEquals(2, iter.next().id());
+        assertFalse(iter.hasNext());
+        assertEquals(Collections.emptyList(), manager.findStaleBrokers());
+
+        time.sleep(5);
+        assertEquals(Collections.singletonList(0), manager.findStaleBrokers());
+        manager.fence(0);
+        assertEquals(Collections.emptyList(), manager.findStaleBrokers());
+        iter = manager.unfenced().iterator();
+        assertEquals(1, iter.next().id());
+        assertEquals(2, iter.next().id());
+        assertFalse(iter.hasNext());
+
+        time.sleep(20);
+        assertEquals(Arrays.asList(1, 2), manager.findStaleBrokers());
+        manager.fence(1);
+        manager.fence(2);
+        assertEquals(Collections.emptyList(), manager.findStaleBrokers());
+        iter = manager.unfenced().iterator();
+        assertFalse(iter.hasNext());
+    }
+
+    @Test
+    public void testNextCheckTimeNs() {
+        BrokerHeartbeatManager manager = newBrokerHeartbeatManager();
+        MockTime time = (MockTime)  manager.time();
+        assertEquals(Long.MAX_VALUE, manager.nextCheckTimeNs());
+        manager.touch(0, false, 0);
+        time.sleep(2);
+        manager.touch(1, false, 0);
+        time.sleep(1);
+        manager.touch(2, false, 0);
+        time.sleep(1);
+        manager.touch(3, false, 0);
+        assertEquals(Collections.emptyList(), manager.findStaleBrokers());
+        assertEquals(10_000_000, manager.nextCheckTimeNs());
+        time.sleep(7);
+        assertEquals(10_000_000, manager.nextCheckTimeNs());
+        assertEquals(Collections.singletonList(0), manager.findStaleBrokers());
+        manager.fence(0);
+        assertEquals(12_000_000, manager.nextCheckTimeNs());
+        time.sleep(3);
+        assertEquals(Arrays.asList(1, 2), manager.findStaleBrokers());
+        manager.fence(1);
+        manager.fence(2);
+        assertEquals(14_000_000, manager.nextCheckTimeNs());
+    }
+
+    @Test
+    public void testMetadataOffsetComparator() {
+        TreeSet<BrokerHeartbeatState> set =
+            new TreeSet<>(BrokerHeartbeatManager.MetadataOffsetComparator.INSTANCE);
+        BrokerHeartbeatState broker1 = new BrokerHeartbeatState(1);
+        BrokerHeartbeatState broker2 = new BrokerHeartbeatState(2);
+        BrokerHeartbeatState broker3 = new BrokerHeartbeatState(3);
+        set.add(broker1);
+        set.add(broker2);
+        set.add(broker3);
+        Iterator<BrokerHeartbeatState> iterator = set.iterator();
+        assertEquals(broker1, iterator.next());
+        assertEquals(broker2, iterator.next());
+        assertEquals(broker3, iterator.next());
+        assertFalse(iterator.hasNext());
+        assertTrue(set.remove(broker1));
+        assertTrue(set.remove(broker2));
+        assertTrue(set.remove(broker3));
+        assertTrue(set.isEmpty());
+        broker1.metadataOffset = 800;
+        broker2.metadataOffset = 400;
+        broker3.metadataOffset = 100;
+        set.add(broker1);
+        set.add(broker2);
+        set.add(broker3);
+        iterator = set.iterator();
+        assertEquals(broker3, iterator.next());
+        assertEquals(broker2, iterator.next());
+        assertEquals(broker1, iterator.next());
+        assertFalse(iterator.hasNext());
+    }
+
+    private static Set<UsableBroker> usableBrokersToSet(BrokerHeartbeatManager manager) {
+        Set<UsableBroker> brokers = new HashSet<>();
+        for (Iterator<UsableBroker> iterator = new UsableBrokerIterator(
+            manager.unfenced().iterator(),
+            id -> id % 2 == 0 ? Optional.of("rack1") : Optional.of("rack2"));
+             iterator.hasNext(); ) {
+            brokers.add(iterator.next());
+        }
+        return brokers;
+    }
+
+    @Test
+    public void testUsableBrokerIterator() {
+        BrokerHeartbeatManager manager = newBrokerHeartbeatManager();
+        assertEquals(Collections.emptySet(), usableBrokersToSet(manager));
+        manager.touch(0, false, 100);
+        manager.touch(1, false, 100);
+        manager.touch(2, false, 98);
+        manager.touch(3, false, 100);
+        manager.touch(4, true, 100);
+        assertEquals(98L, manager.lowestActiveOffset());
+        Set<UsableBroker> expected = new HashSet<>();
+        expected.add(new UsableBroker(0, Optional.of("rack1")));
+        expected.add(new UsableBroker(1, Optional.of("rack2")));
+        expected.add(new UsableBroker(2, Optional.of("rack1")));
+        expected.add(new UsableBroker(3, Optional.of("rack2")));
+        assertEquals(expected, usableBrokersToSet(manager));
+        manager.updateControlledShutdownOffset(2, 0);
+        assertEquals(100L, manager.lowestActiveOffset());
+        assertThrows(RuntimeException.class,
+            () -> manager.updateControlledShutdownOffset(4, 0));
+        manager.touch(4, false, 100);
+        manager.updateControlledShutdownOffset(4, 0);
+        expected.remove(new UsableBroker(2, Optional.of("rack1")));
+        assertEquals(expected, usableBrokersToSet(manager));
+    }
+
+    @Test
+    public void testBrokerHeartbeatStateList() {
+        BrokerHeartbeatStateList list = new BrokerHeartbeatStateList();
+        assertEquals(null, list.first());
+        BrokerHeartbeatStateIterator iterator = list.iterator();
+        assertFalse(iterator.hasNext());
+        BrokerHeartbeatState broker0 = new BrokerHeartbeatState(0);
+        broker0.lastContactNs = 200;
+        BrokerHeartbeatState broker1 = new BrokerHeartbeatState(1);
+        broker1.lastContactNs = 100;
+        BrokerHeartbeatState broker2 = new BrokerHeartbeatState(2);
+        broker2.lastContactNs = 50;
+        BrokerHeartbeatState broker3 = new BrokerHeartbeatState(3);
+        broker3.lastContactNs = 150;
+        list.add(broker0);
+        list.add(broker1);
+        list.add(broker2);
+        list.add(broker3);
+        assertEquals(broker2, list.first());
+        iterator = list.iterator();
+        assertEquals(broker2, iterator.next());
+        assertEquals(broker1, iterator.next());
+        assertEquals(broker3, iterator.next());
+        assertEquals(broker0, iterator.next());
+        assertFalse(iterator.hasNext());
+        list.remove(broker1);
+        iterator = list.iterator();
+        assertEquals(broker2, iterator.next());
+        assertEquals(broker3, iterator.next());
+        assertEquals(broker0, iterator.next());
+        assertFalse(iterator.hasNext());
+    }
+
+    @Test
+    public void testCalculateNextBrokerState() {
+        BrokerHeartbeatManager manager = newBrokerHeartbeatManager();
+        manager.touch(0, true, 100);
+        manager.touch(1, false, 98);
+        manager.touch(2, false, 100);
+        manager.touch(3, false, 100);
+        manager.touch(4, true, 100);
+        manager.touch(5, false, 99);
+        manager.updateControlledShutdownOffset(5, 99);
+
+        assertEquals(98L, manager.lowestActiveOffset());
+
+        assertEquals(new BrokerControlStates(FENCED, SHUTDOWN_NOW),
+            manager.calculateNextBrokerState(0,
+                new BrokerHeartbeatRequestData().setWantShutDown(true), 100, () -> false));
+        assertEquals(new BrokerControlStates(FENCED, UNFENCED),
+            manager.calculateNextBrokerState(0,
+                new BrokerHeartbeatRequestData().setWantFence(false).
+                    setCurrentMetadataOffset(100), 100, () -> false));
+        assertEquals(new BrokerControlStates(FENCED, FENCED),
+            manager.calculateNextBrokerState(0,
+                new BrokerHeartbeatRequestData().setWantFence(false).
+                    setCurrentMetadataOffset(50), 100, () -> false));
+        assertEquals(new BrokerControlStates(FENCED, FENCED),
+            manager.calculateNextBrokerState(0,
+                new BrokerHeartbeatRequestData().setWantFence(true), 100, () -> false));
+
+        assertEquals(new BrokerControlStates(UNFENCED, CONTROLLED_SHUTDOWN),
+            manager.calculateNextBrokerState(1,
+                new BrokerHeartbeatRequestData().setWantShutDown(true), 100, () -> true));
+        assertEquals(new BrokerControlStates(UNFENCED, SHUTDOWN_NOW),
+            manager.calculateNextBrokerState(1,
+                new BrokerHeartbeatRequestData().setWantShutDown(true), 100, () -> false));
+        assertEquals(new BrokerControlStates(UNFENCED, UNFENCED),
+            manager.calculateNextBrokerState(1,
+                new BrokerHeartbeatRequestData().setWantFence(false), 100, () -> false));
+
+        assertEquals(new BrokerControlStates(CONTROLLED_SHUTDOWN, CONTROLLED_SHUTDOWN),
+            manager.calculateNextBrokerState(5,
+                new BrokerHeartbeatRequestData().setWantShutDown(true), 100, () -> true));
+        assertEquals(new BrokerControlStates(CONTROLLED_SHUTDOWN, CONTROLLED_SHUTDOWN),
+            manager.calculateNextBrokerState(5,
+                new BrokerHeartbeatRequestData().setWantShutDown(true), 100, () -> false));
+        manager.fence(1);
+        assertEquals(new BrokerControlStates(CONTROLLED_SHUTDOWN, SHUTDOWN_NOW),
+            manager.calculateNextBrokerState(5,
+                new BrokerHeartbeatRequestData().setWantShutDown(true), 100, () -> false));
+        assertEquals(new BrokerControlStates(CONTROLLED_SHUTDOWN, CONTROLLED_SHUTDOWN),
+            manager.calculateNextBrokerState(5,
+                new BrokerHeartbeatRequestData().setWantShutDown(true), 100, () -> true));
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java b/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java
new file mode 100644
index 0000000..6f124ad
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.Uuid;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.PartitionsOnReplicaIterator;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+
+@Timeout(40)
+public class BrokersToIsrsTest {
+    private static final Uuid[] UUIDS = new Uuid[] {
+        Uuid.fromString("z5XgH_fQSAK3-RYoF2ymgw"),
+        Uuid.fromString("U52uRe20RsGI0RvpcTx33Q")
+    };
+
+    private static Set<TopicPartition> toSet(TopicPartition... partitions) {
+        HashSet<TopicPartition> set = new HashSet<>();
+        for (TopicPartition partition : partitions) {
+            set.add(partition);
+        }
+        return set;
+    }
+
+    private static Set<TopicPartition> toSet(PartitionsOnReplicaIterator iterator) {
+        HashSet<TopicPartition> set = new HashSet<>();
+        while (iterator.hasNext()) {
+            set.add(iterator.next());
+        }
+        return set;
+    }
+
+    @Test
+    public void testIterator() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        BrokersToIsrs brokersToIsrs = new BrokersToIsrs(snapshotRegistry);
+        assertEquals(toSet(), toSet(brokersToIsrs.iterator(1, false)));
+        brokersToIsrs.update(UUIDS[0], 0, null, new int[] {1, 2, 3}, -1, 1);
+        brokersToIsrs.update(UUIDS[1], 1, null, new int[] {2, 3, 4}, -1, 4);
+        assertEquals(toSet(new TopicPartition(UUIDS[0], 0)),
+            toSet(brokersToIsrs.iterator(1, false)));
+        assertEquals(toSet(new TopicPartition(UUIDS[0], 0),
+                           new TopicPartition(UUIDS[1], 1)),
+            toSet(brokersToIsrs.iterator(2, false)));
+        assertEquals(toSet(new TopicPartition(UUIDS[1], 1)),
+            toSet(brokersToIsrs.iterator(4, false)));
+        assertEquals(toSet(), toSet(brokersToIsrs.iterator(5, false)));
+        brokersToIsrs.update(UUIDS[1], 2, null, new int[] {3, 2, 1}, -1, 3);
+        assertEquals(toSet(new TopicPartition(UUIDS[0], 0),
+                new TopicPartition(UUIDS[1], 1),
+                new TopicPartition(UUIDS[1], 2)),
+            toSet(brokersToIsrs.iterator(2, false)));
+    }
+
+    @Test
+    public void testLeadersOnlyIterator() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        BrokersToIsrs brokersToIsrs = new BrokersToIsrs(snapshotRegistry);
+        brokersToIsrs.update(UUIDS[0], 0, null, new int[]{1, 2, 3}, -1, 1);
+        brokersToIsrs.update(UUIDS[1], 1, null, new int[]{2, 3, 4}, -1, 4);
+        assertEquals(toSet(new TopicPartition(UUIDS[0], 0)),
+            toSet(brokersToIsrs.iterator(1, true)));
+        assertEquals(toSet(), toSet(brokersToIsrs.iterator(2, true)));
+        assertEquals(toSet(new TopicPartition(UUIDS[1], 1)),
+            toSet(brokersToIsrs.iterator(4, true)));
+        brokersToIsrs.update(UUIDS[0], 0, new int[]{1, 2, 3}, new int[]{1, 2, 3}, 1, 2);
+        assertEquals(toSet(), toSet(brokersToIsrs.iterator(1, true)));
+        assertEquals(toSet(new TopicPartition(UUIDS[0], 0)),
+            toSet(brokersToIsrs.iterator(2, true)));
+    }
+
+    @Test
+    public void testNoLeader() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        BrokersToIsrs brokersToIsrs = new BrokersToIsrs(snapshotRegistry);
+        brokersToIsrs.update(UUIDS[0], 2, null, new int[]{1, 2, 3}, -1, 3);
+        assertEquals(toSet(new TopicPartition(UUIDS[0], 2)),
+            toSet(brokersToIsrs.iterator(3, true)));
+        assertEquals(toSet(), toSet(brokersToIsrs.iterator(2, true)));
+        assertEquals(toSet(), toSet(brokersToIsrs.noLeaderIterator()));
+        brokersToIsrs.update(UUIDS[0], 2, new int[]{1, 2, 3}, new int[]{1, 2, 3}, 3, -1);
+        assertEquals(toSet(new TopicPartition(UUIDS[0], 2)),
+            toSet(brokersToIsrs.noLeaderIterator()));
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java
new file mode 100644
index 0000000..c4e8da8
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java
@@ -0,0 +1,238 @@
+/*
+ * 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.config.internals.QuotaConfigs;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.protocol.Errors;
+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.timeline.SnapshotRegistry;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Timeout(value = 40)
+public class ClientQuotaControlManagerTest {
+
+    @Test
+    public void testInvalidEntityTypes() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ClientQuotaControlManager manager = new ClientQuotaControlManager(snapshotRegistry);
+
+        // Unknown type "foo"
+        assertInvalidEntity(manager, entity("foo", "bar"));
+
+        // Null type
+        assertInvalidEntity(manager, entity(null, "null"));
+
+        // Valid + unknown combo
+        assertInvalidEntity(manager, entity(ClientQuotaEntity.USER, "user-1", "foo", "bar"));
+        assertInvalidEntity(manager, entity("foo", "bar", ClientQuotaEntity.IP, "1.2.3.4"));
+
+        // Invalid combinations
+        assertInvalidEntity(manager, entity(ClientQuotaEntity.USER, "user-1", ClientQuotaEntity.IP, "1.2.3.4"));
+        assertInvalidEntity(manager, entity(ClientQuotaEntity.CLIENT_ID, "user-1", ClientQuotaEntity.IP, "1.2.3.4"));
+
+        // Empty
+        assertInvalidEntity(manager, new ClientQuotaEntity(Collections.emptyMap()));
+    }
+
+    private void assertInvalidEntity(ClientQuotaControlManager manager, ClientQuotaEntity entity) {
+        List<ClientQuotaAlteration> alters = new ArrayList<>();
+        entityQuotaToAlterations(entity, quotas(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 10000.0), alters::add);
+        ControllerResult<Map<ClientQuotaEntity, ApiError>> result = manager.alterClientQuotas(alters);
+        assertEquals(Errors.INVALID_REQUEST, result.response().get(entity).error());
+        assertEquals(0, result.records().size());
+    }
+
+    @Test
+    public void testAlterAndRemove() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ClientQuotaControlManager manager = new ClientQuotaControlManager(snapshotRegistry);
+
+        ClientQuotaEntity userEntity = userEntity("user-1");
+        List<ClientQuotaAlteration> alters = new ArrayList<>();
+
+        // Add one quota
+        entityQuotaToAlterations(userEntity, quotas(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 10000.0), alters::add);
+        alterQuotas(alters, manager);
+        assertEquals(1, manager.clientQuotaData.get(userEntity).size());
+        assertEquals(10000.0, manager.clientQuotaData.get(userEntity).get(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG), 1e-6);
+
+        // Replace it and add another
+        alters.clear();
+        entityQuotaToAlterations(userEntity, quotas(
+            QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 10001.0,
+            QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, 20000.0
+        ), alters::add);
+        alterQuotas(alters, manager);
+        assertEquals(2, manager.clientQuotaData.get(userEntity).size());
+        assertEquals(10001.0, manager.clientQuotaData.get(userEntity).get(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG), 1e-6);
+        assertEquals(20000.0, manager.clientQuotaData.get(userEntity).get(QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG), 1e-6);
+
+        // Remove one of the quotas, the other remains
+        alters.clear();
+        entityQuotaToAlterations(userEntity, quotas(
+            QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, null
+        ), alters::add);
+        alterQuotas(alters, manager);
+        assertEquals(1, manager.clientQuotaData.get(userEntity).size());
+        assertEquals(20000.0, manager.clientQuotaData.get(userEntity).get(QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG), 1e-6);
+
+        // Remove non-existent quota, no change
+        alters.clear();
+        entityQuotaToAlterations(userEntity, quotas(
+                QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, null
+        ), alters::add);
+        alterQuotas(alters, manager);
+        assertEquals(1, manager.clientQuotaData.get(userEntity).size());
+        assertEquals(20000.0, manager.clientQuotaData.get(userEntity).get(QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG), 1e-6);
+
+        // All quotas removed, we should cleanup the map
+        alters.clear();
+        entityQuotaToAlterations(userEntity, quotas(
+                QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, null
+        ), alters::add);
+        alterQuotas(alters, manager);
+        assertFalse(manager.clientQuotaData.containsKey(userEntity));
+
+        // Remove non-existent quota, again no change
+        alters.clear();
+        entityQuotaToAlterations(userEntity, quotas(
+                QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, null
+        ), alters::add);
+        alterQuotas(alters, manager);
+        assertFalse(manager.clientQuotaData.containsKey(userEntity));
+
+        // Mixed update
+        alters.clear();
+        Map<String, Double> quotas = new HashMap<>(4);
+        quotas.put(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 99.0);
+        quotas.put(QuotaConfigs.CONTROLLER_MUTATION_RATE_OVERRIDE_CONFIG, null);
+        quotas.put(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 10002.0);
+        quotas.put(QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, 20001.0);
+
+        entityQuotaToAlterations(userEntity, quotas, alters::add);
+        alterQuotas(alters, manager);
+        assertEquals(3, manager.clientQuotaData.get(userEntity).size());
+        assertEquals(20001.0, manager.clientQuotaData.get(userEntity).get(QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG), 1e-6);
+        assertEquals(10002.0, manager.clientQuotaData.get(userEntity).get(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG), 1e-6);
+        assertEquals(99.0, manager.clientQuotaData.get(userEntity).get(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG), 1e-6);
+    }
+
+    @Test
+    public void testEntityTypes() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ClientQuotaControlManager manager = new ClientQuotaControlManager(snapshotRegistry);
+
+        Map<ClientQuotaEntity, Map<String, Double>> quotasToTest = new HashMap<>();
+        quotasToTest.put(userClientEntity("user-1", "client-id-1"),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 50.50));
+        quotasToTest.put(userClientEntity("user-2", "client-id-1"),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 51.51));
+        quotasToTest.put(userClientEntity("user-3", "client-id-2"),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 52.52));
+        quotasToTest.put(userClientEntity(null, "client-id-1"),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 53.53));
+        quotasToTest.put(userClientEntity("user-1", null),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 54.54));
+        quotasToTest.put(userClientEntity("user-3", null),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 55.55));
+        quotasToTest.put(userEntity("user-1"),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 56.56));
+        quotasToTest.put(userEntity("user-2"),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 57.57));
+        quotasToTest.put(userEntity("user-3"),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 58.58));
+        quotasToTest.put(userEntity(null),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 59.59));
+        quotasToTest.put(clientEntity("client-id-2"),
+                quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 60.60));
+
+
+        List<ClientQuotaAlteration> alters = new ArrayList<>();
+        quotasToTest.forEach((entity, quota) -> entityQuotaToAlterations(entity, quota, alters::add));
+        alterQuotas(alters, manager);
+    }
+
+    static void entityQuotaToAlterations(ClientQuotaEntity entity, Map<String, Double> quota,
+                                          Consumer<ClientQuotaAlteration> acceptor) {
+        Collection<ClientQuotaAlteration.Op> ops = quota.entrySet().stream()
+                .map(quotaEntry -> new ClientQuotaAlteration.Op(quotaEntry.getKey(), quotaEntry.getValue()))
+                .collect(Collectors.toList());
+        acceptor.accept(new ClientQuotaAlteration(entity, ops));
+    }
+
+    static void alterQuotas(List<ClientQuotaAlteration> alterations, ClientQuotaControlManager manager) {
+        ControllerResult<Map<ClientQuotaEntity, ApiError>> result = manager.alterClientQuotas(alterations);
+        assertTrue(result.response().values().stream().allMatch(ApiError::isSuccess));
+        result.records().forEach(apiMessageAndVersion -> manager.replay((QuotaRecord) apiMessageAndVersion.message()));
+    }
+
+    static Map<String, Double> quotas(String key, Double value) {
+        return Collections.singletonMap(key, value);
+    }
+
+    static Map<String, Double> quotas(String key1, Double value1, String key2, Double value2) {
+        Map<String, Double> quotas = new HashMap<>(2);
+        quotas.put(key1, value1);
+        quotas.put(key2, value2);
+        return quotas;
+    }
+
+    static ClientQuotaEntity entity(String type, String name) {
+        return new ClientQuotaEntity(Collections.singletonMap(type, name));
+    }
+
+    static ClientQuotaEntity entity(String type1, String name1, String type2, String name2) {
+        Map<String, String> entries = new HashMap<>(2);
+        entries.put(type1, name1);
+        entries.put(type2, name2);
+        return new ClientQuotaEntity(entries);
+    }
+
+    static ClientQuotaEntity userEntity(String user) {
+        return new ClientQuotaEntity(Collections.singletonMap(ClientQuotaEntity.USER, user));
+    }
+
+    static ClientQuotaEntity clientEntity(String clientId) {
+        return new ClientQuotaEntity(Collections.singletonMap(ClientQuotaEntity.CLIENT_ID, clientId));
+    }
+
+    static ClientQuotaEntity userClientEntity(String user, String clientId) {
+        Map<String, String> entries = new HashMap<>(2);
+        entries.put(ClientQuotaEntity.USER, user);
+        entries.put(ClientQuotaEntity.CLIENT_ID, clientId);
+        return new ClientQuotaEntity(entries);
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java
new file mode 100644
index 0000000..c410a68
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+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.MockTime;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+@Timeout(value = 40)
+public class ClusterControlManagerTest {
+    @Test
+    public void testReplay() {
+        MockTime time = new MockTime(0, 0, 0);
+
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ClusterControlManager clusterControl = new ClusterControlManager(
+            new LogContext(), time, snapshotRegistry, 1000,
+                new SimpleReplicaPlacementPolicy(new Random()));
+        clusterControl.activate();
+        assertFalse(clusterControl.unfenced(0));
+
+        RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().setBrokerEpoch(100).setBrokerId(1);
+        brokerRecord.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint().
+            setSecurityProtocol(SecurityProtocol.PLAINTEXT.id).
+            setPort((short) 9092).
+            setName("PLAINTEXT").
+            setHost("example.com"));
+        clusterControl.replay(brokerRecord);
+        clusterControl.checkBrokerEpoch(1, 100);
+        assertThrows(StaleBrokerEpochException.class,
+            () -> clusterControl.checkBrokerEpoch(1, 101));
+        assertThrows(StaleBrokerEpochException.class,
+            () -> clusterControl.checkBrokerEpoch(2, 100));
+        assertFalse(clusterControl.unfenced(0));
+        assertFalse(clusterControl.unfenced(1));
+
+        UnfenceBrokerRecord unfenceBrokerRecord =
+            new UnfenceBrokerRecord().setId(1).setEpoch(100);
+        clusterControl.replay(unfenceBrokerRecord);
+        assertFalse(clusterControl.unfenced(0));
+        assertTrue(clusterControl.unfenced(1));
+    }
+
+    @Test
+    public void testUnregister() throws Exception {
+        RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().
+            setBrokerId(1).
+            setBrokerEpoch(100).
+            setIncarnationId(Uuid.fromString("fPZv1VBsRFmnlRvmGcOW9w")).
+            setRack("arack");
+        brokerRecord.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint().
+            setSecurityProtocol(SecurityProtocol.PLAINTEXT.id).
+            setPort((short) 9092).
+            setName("PLAINTEXT").
+            setHost("example.com"));
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ClusterControlManager clusterControl = new ClusterControlManager(
+            new LogContext(), new MockTime(0, 0, 0), snapshotRegistry, 1000,
+            new SimpleReplicaPlacementPolicy(new Random()));
+        clusterControl.activate();
+        clusterControl.replay(brokerRecord);
+        assertEquals(new BrokerRegistration(1, 100,
+            Uuid.fromString("fPZv1VBsRFmnlRvmGcOW9w"), Collections.singletonMap("PLAINTEXT",
+            new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "example.com", 9092)),
+            Collections.emptyMap(), Optional.of("arack"), true),
+                clusterControl.brokerRegistrations().get(1));
+        UnregisterBrokerRecord unregisterRecord = new UnregisterBrokerRecord().
+            setBrokerId(1).
+            setBrokerEpoch(100);
+        clusterControl.replay(unregisterRecord);
+        assertFalse(clusterControl.brokerRegistrations().containsKey(1));
+    }
+
+    @ParameterizedTest
+    @ValueSource(ints = {3, 10})
+    public void testPlaceReplicas(int numUsableBrokers) throws Exception {
+        MockTime time = new MockTime(0, 0, 0);
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        MockRandom random = new MockRandom();
+        ClusterControlManager clusterControl = new ClusterControlManager(
+            new LogContext(), time, snapshotRegistry, 1000,
+            new SimpleReplicaPlacementPolicy(random));
+        clusterControl.activate();
+        for (int i = 0; i < numUsableBrokers; i++) {
+            RegisterBrokerRecord brokerRecord =
+                new RegisterBrokerRecord().setBrokerEpoch(100).setBrokerId(i);
+            brokerRecord.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint().
+                setSecurityProtocol(SecurityProtocol.PLAINTEXT.id).
+                setPort((short) 9092).
+                setName("PLAINTEXT").
+                setHost("example.com"));
+            clusterControl.replay(brokerRecord);
+            UnfenceBrokerRecord unfenceRecord =
+                new UnfenceBrokerRecord().setId(i).setEpoch(100);
+            clusterControl.replay(unfenceRecord);
+            clusterControl.heartbeatManager().touch(i, false, 0);
+        }
+        for (int i = 0; i < numUsableBrokers; i++) {
+            assertTrue(clusterControl.unfenced(i),
+                String.format("broker %d was not unfenced.", i));
+        }
+        for (int i = 0; i < 100; i++) {
+            List<List<Integer>> results = clusterControl.placeReplicas(1, (short) 3);
+            HashSet<Integer> seen = new HashSet<>();
+            for (Integer result : results.get(0)) {
+                assertTrue(result >= 0);
+                assertTrue(result < numUsableBrokers);
+                assertTrue(seen.add(result));
+            }
+        }
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java
new file mode 100644
index 0000000..49a5533
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java
@@ -0,0 +1,203 @@
+/*
+ * 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.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+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 java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.APPEND;
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.DELETE;
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.BROKER;
+import static org.apache.kafka.common.config.ConfigResource.Type.BROKER_LOGGER;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+import static org.apache.kafka.common.config.ConfigResource.Type.UNKNOWN;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+@Timeout(value = 40)
+public class ConfigurationControlManagerTest {
+
+    static final Map<ConfigResource.Type, ConfigDef> CONFIGS = new HashMap<>();
+
+    static {
+        CONFIGS.put(BROKER, new ConfigDef().
+            define("foo.bar", ConfigDef.Type.LIST, "1", ConfigDef.Importance.HIGH, "foo bar").
+            define("baz", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "baz").
+            define("quux", ConfigDef.Type.INT, ConfigDef.Importance.HIGH, "quux"));
+        CONFIGS.put(TOPIC, new ConfigDef().
+            define("abc", ConfigDef.Type.LIST, ConfigDef.Importance.HIGH, "abc").
+            define("def", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "def").
+            define("ghi", ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.HIGH, "ghi"));
+    }
+
+    static final ConfigResource BROKER0 = new ConfigResource(BROKER, "0");
+    static final ConfigResource MYTOPIC = new ConfigResource(TOPIC, "mytopic");
+
+    @SuppressWarnings("unchecked")
+    private static <A, B> Map<A, B> toMap(Entry... entries) {
+        Map<A, B> map = new HashMap<>();
+        for (Entry<A, B> entry : entries) {
+            map.put(entry.getKey(), entry.getValue());
+        }
+        return map;
+    }
+
+    static <A, B> Entry<A, B> entry(A a, B b) {
+        return new SimpleImmutableEntry<>(a, b);
+    }
+
+    @Test
+    public void testReplay() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ConfigurationControlManager manager =
+            new ConfigurationControlManager(new LogContext(), snapshotRegistry, CONFIGS);
+        assertEquals(Collections.emptyMap(), manager.getConfigs(BROKER0));
+        manager.replay(new ConfigRecord().
+            setResourceType(BROKER.id()).setResourceName("0").
+            setName("foo.bar").setValue("1,2"));
+        assertEquals(Collections.singletonMap("foo.bar", "1,2"),
+            manager.getConfigs(BROKER0));
+        manager.replay(new ConfigRecord().
+            setResourceType(BROKER.id()).setResourceName("0").
+            setName("foo.bar").setValue(null));
+        assertEquals(Collections.emptyMap(), manager.getConfigs(BROKER0));
+        manager.replay(new ConfigRecord().
+            setResourceType(TOPIC.id()).setResourceName("mytopic").
+            setName("abc").setValue("x,y,z"));
+        manager.replay(new ConfigRecord().
+            setResourceType(TOPIC.id()).setResourceName("mytopic").
+            setName("def").setValue("blah"));
+        assertEquals(toMap(entry("abc", "x,y,z"), entry("def", "blah")),
+            manager.getConfigs(MYTOPIC));
+    }
+
+    @Test
+    public void testCheckConfigResource() {
+        assertEquals(new ApiError(Errors.INVALID_REQUEST, "Unsupported " +
+            "configuration resource type BROKER_LOGGER ").toString(),
+            ConfigurationControlManager.checkConfigResource(
+                new ConfigResource(BROKER_LOGGER, "kafka.server.FetchContext")).toString());
+        assertEquals(new ApiError(Errors.INVALID_REQUEST, "Illegal topic name.").toString(),
+            ConfigurationControlManager.checkConfigResource(
+                new ConfigResource(TOPIC, "* @ invalid$")).toString());
+        assertEquals(new ApiError(Errors.INVALID_REQUEST, "Illegal topic name.").toString(),
+            ConfigurationControlManager.checkConfigResource(
+                new ConfigResource(TOPIC, "")).toString());
+        assertEquals(new ApiError(Errors.INVALID_REQUEST, "Illegal non-integral " +
+                "BROKER resource type name.").toString(),
+            ConfigurationControlManager.checkConfigResource(
+                new ConfigResource(BROKER, "bob")).toString());
+        assertEquals(new ApiError(Errors.NONE, null).toString(),
+            ConfigurationControlManager.checkConfigResource(
+                new ConfigResource(BROKER, "")).toString());
+        assertEquals(new ApiError(Errors.INVALID_REQUEST, "Unsupported configuration " +
+                "resource type UNKNOWN.").toString(),
+            ConfigurationControlManager.checkConfigResource(
+                new ConfigResource(UNKNOWN, "bob")).toString());
+    }
+
+    @Test
+    public void testIncrementalAlterConfigs() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ConfigurationControlManager manager =
+            new ConfigurationControlManager(new LogContext(), snapshotRegistry, CONFIGS);
+        assertEquals(new ControllerResult<Map<ConfigResource, ApiError>>(Collections.singletonList(
+            new ApiMessageAndVersion(new ConfigRecord().
+                setResourceType(TOPIC.id()).setResourceName("mytopic").
+                setName("abc").setValue("123"), (short) 0)),
+            toMap(entry(BROKER0, new ApiError(
+                Errors.INVALID_REQUEST, "A DELETE op was given with a non-null value.")),
+                entry(MYTOPIC, ApiError.NONE))),
+            manager.incrementalAlterConfigs(toMap(entry(BROKER0, toMap(
+                entry("foo.bar", entry(DELETE, "abc")),
+                entry("quux", entry(SET, "abc")))),
+            entry(MYTOPIC, toMap(
+                entry("abc", entry(APPEND, "123")))))));
+    }
+
+    @Test
+    public void testIsSplittable() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ConfigurationControlManager manager =
+            new ConfigurationControlManager(new LogContext(), snapshotRegistry, CONFIGS);
+        assertTrue(manager.isSplittable(BROKER, "foo.bar"));
+        assertFalse(manager.isSplittable(BROKER, "baz"));
+        assertFalse(manager.isSplittable(BROKER, "foo.baz.quux"));
+        assertFalse(manager.isSplittable(TOPIC, "baz"));
+        assertTrue(manager.isSplittable(TOPIC, "abc"));
+    }
+
+    @Test
+    public void testGetConfigValueDefault() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ConfigurationControlManager manager =
+            new ConfigurationControlManager(new LogContext(), snapshotRegistry, CONFIGS);
+        assertEquals("1", manager.getConfigValueDefault(BROKER, "foo.bar"));
+        assertEquals(null, manager.getConfigValueDefault(BROKER, "foo.baz.quux"));
+        assertEquals(null, manager.getConfigValueDefault(TOPIC, "abc"));
+        assertEquals("true", manager.getConfigValueDefault(TOPIC, "ghi"));
+    }
+
+    @Test
+    public void testLegacyAlterConfigs() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        ConfigurationControlManager manager =
+            new ConfigurationControlManager(new LogContext(), snapshotRegistry, CONFIGS);
+        List<ApiMessageAndVersion> expectedRecords1 = Arrays.asList(
+            new ApiMessageAndVersion(new ConfigRecord().
+                setResourceType(TOPIC.id()).setResourceName("mytopic").
+                setName("abc").setValue("456"), (short) 0),
+            new ApiMessageAndVersion(new ConfigRecord().
+                setResourceType(TOPIC.id()).setResourceName("mytopic").
+                setName("def").setValue("901"), (short) 0));
+        assertEquals(new ControllerResult<Map<ConfigResource, ApiError>>(
+                expectedRecords1,
+                toMap(entry(MYTOPIC, ApiError.NONE))),
+            manager.legacyAlterConfigs(toMap(entry(MYTOPIC, toMap(
+                entry("abc", "456"), entry("def", "901"))))));
+        for (ApiMessageAndVersion message : expectedRecords1) {
+            manager.replay((ConfigRecord) message.message());
+        }
+        assertEquals(new ControllerResult<Map<ConfigResource, ApiError>>(Arrays.asList(
+            new ApiMessageAndVersion(new ConfigRecord().
+                setResourceType(TOPIC.id()).setResourceName("mytopic").
+                setName("abc").setValue(null), (short) 0)),
+                toMap(entry(MYTOPIC, ApiError.NONE))),
+            manager.legacyAlterConfigs(toMap(entry(MYTOPIC, toMap(
+                entry("def", "901"))))));
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ControllerPurgatoryTest.java b/metadata/src/test/java/org/apache/kafka/controller/ControllerPurgatoryTest.java
new file mode 100644
index 0000000..57953e1
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/ControllerPurgatoryTest.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.controller;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Timeout(value = 40)
+public class ControllerPurgatoryTest {
+
+    static class SampleDeferredEvent implements DeferredEvent {
+        private final CompletableFuture<Void> future = new CompletableFuture<>();
+
+        @Override
+        public void complete(Throwable exception) {
+            if (exception != null) {
+                future.completeExceptionally(exception);
+            } else {
+                future.complete(null);
+            }
+        }
+
+        CompletableFuture<Void> future() {
+            return future;
+        }
+    }
+
+    @Test
+    public void testCompleteEvents() {
+        ControllerPurgatory purgatory = new ControllerPurgatory();
+        SampleDeferredEvent event1 = new SampleDeferredEvent();
+        SampleDeferredEvent event2 = new SampleDeferredEvent();
+        SampleDeferredEvent event3 = new SampleDeferredEvent();
+        purgatory.add(1, event1);
+        assertEquals(Optional.of(1L), purgatory.highestPendingOffset());
+        purgatory.add(1, event2);
+        assertEquals(Optional.of(1L), purgatory.highestPendingOffset());
+        purgatory.add(3, event3);
+        assertEquals(Optional.of(3L), purgatory.highestPendingOffset());
+        purgatory.completeUpTo(2);
+        assertTrue(event1.future.isDone());
+        assertTrue(event2.future.isDone());
+        assertFalse(event3.future.isDone());
+        purgatory.completeUpTo(4);
+        assertTrue(event3.future.isDone());
+        assertEquals(Optional.empty(), purgatory.highestPendingOffset());
+    }
+
+    @Test
+    public void testFailOnIncorrectOrdering() {
+        ControllerPurgatory purgatory = new ControllerPurgatory();
+        SampleDeferredEvent event1 = new SampleDeferredEvent();
+        SampleDeferredEvent event2 = new SampleDeferredEvent();
+        purgatory.add(2, event1);
+        assertThrows(RuntimeException.class, () -> purgatory.add(1, event2));
+    }
+
+    @Test
+    public void testFailEvents() {
+        ControllerPurgatory purgatory = new ControllerPurgatory();
+        SampleDeferredEvent event1 = new SampleDeferredEvent();
+        SampleDeferredEvent event2 = new SampleDeferredEvent();
+        SampleDeferredEvent event3 = new SampleDeferredEvent();
+        purgatory.add(1, event1);
+        purgatory.add(3, event2);
+        purgatory.add(3, event3);
+        purgatory.completeUpTo(2);
+        assertTrue(event1.future.isDone());
+        assertFalse(event2.future.isDone());
+        assertFalse(event3.future.isDone());
+        purgatory.failAll(new RuntimeException("failed"));
+        assertTrue(event2.future.isDone());
+        assertTrue(event3.future.isDone());
+        assertEquals(RuntimeException.class, assertThrows(ExecutionException.class,
+            () -> event2.future.get()).getCause().getClass());
+        assertEquals(RuntimeException.class, assertThrows(ExecutionException.class,
+            () -> event3.future.get()).getCause().getClass());
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java b/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java
new file mode 100644
index 0000000..746c7ef
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java
@@ -0,0 +1,51 @@
+/*
+ * 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.protocol.ApiMessage;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+
+import java.lang.reflect.Method;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+
+public class ControllerTestUtils {
+    public static void replayAll(Object target,
+                                 List<ApiMessageAndVersion> recordsAndVersions) throws Exception {
+        for (ApiMessageAndVersion recordAndVersion : recordsAndVersions) {
+            ApiMessage record = recordAndVersion.message();
+            try {
+                Method method = target.getClass().getMethod("replay", record.getClass());
+                method.invoke(target, record);
+            } catch (NoSuchMethodException e) {
+                // ignore
+            }
+        }
+    }
+
+    public static <T> Set<T> iteratorToSet(Iterator<T> iterator) {
+        HashSet<T> set = new HashSet<>();
+        while (iterator.hasNext()) {
+            set.add(iterator.next());
+        }
+        return set;
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
new file mode 100644
index 0000000..8687cc8
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import org.apache.kafka.common.metadata.FeatureLevelRecord;
+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.metadata.FeatureMap;
+import org.apache.kafka.metadata.FeatureMapAndEpoch;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+
+@Timeout(value = 40)
+public class FeatureControlManagerTest {
+    @SuppressWarnings("unchecked")
+    private static Map<String, VersionRange> rangeMap(Object... args) {
+        Map<String, VersionRange> result = new HashMap<>();
+        for (int i = 0; i < args.length; i += 3) {
+            String feature = (String) args[i];
+            Integer low = (Integer) args[i + 1];
+            Integer high = (Integer) args[i + 2];
+            result.put(feature, new VersionRange(low.shortValue(), high.shortValue()));
+        }
+        return result;
+    }
+
+    @Test
+    public void testUpdateFeatures() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        snapshotRegistry.createSnapshot(-1);
+        FeatureControlManager manager = new FeatureControlManager(
+            rangeMap("foo", 1, 2), snapshotRegistry);
+        assertEquals(new FeatureMapAndEpoch(new FeatureMap(Collections.emptyMap()), -1),
+            manager.finalizedFeatures(-1));
+        assertEquals(new ControllerResult<>(Collections.
+                singletonMap("foo", new ApiError(Errors.INVALID_UPDATE_VERSION,
+                    "The controller does not support the given feature range."))),
+            manager.updateFeatures(rangeMap("foo", 1, 3),
+                new HashSet<>(Arrays.asList("foo")),
+                Collections.emptyMap()));
+        ControllerResult<Map<String, ApiError>> result = manager.updateFeatures(
+            rangeMap("foo", 1, 2, "bar", 1, 1), Collections.emptySet(),
+                Collections.emptyMap());
+        Map<String, ApiError> expectedMap = new HashMap<>();
+        expectedMap.put("foo", ApiError.NONE);
+        expectedMap.put("bar", new ApiError(Errors.INVALID_UPDATE_VERSION,
+                "The controller does not support the given feature range."));
+        assertEquals(expectedMap, result.response());
+        List<ApiMessageAndVersion> expectedMessages = new ArrayList<>();
+        expectedMessages.add(new ApiMessageAndVersion(new FeatureLevelRecord().
+            setName("foo").setMinFeatureLevel((short) 1).setMaxFeatureLevel((short) 2),
+            (short) 0));
+        assertEquals(expectedMessages, result.records());
+    }
+
+    @Test
+    public void testReplay() {
+        FeatureLevelRecord record = new FeatureLevelRecord().
+            setName("foo").setMinFeatureLevel((short) 1).setMaxFeatureLevel((short) 2);
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        snapshotRegistry.createSnapshot(-1);
+        FeatureControlManager manager = new FeatureControlManager(
+            rangeMap("foo", 1, 2), snapshotRegistry);
+        manager.replay(record, 123);
+        snapshotRegistry.createSnapshot(123);
+        assertEquals(new FeatureMapAndEpoch(new FeatureMap(rangeMap("foo", 1, 2)), 123),
+            manager.finalizedFeatures(123));
+    }
+
+    @Test
+    public void testUpdateFeaturesErrorCases() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        FeatureControlManager manager = new FeatureControlManager(
+            rangeMap("foo", 1, 5, "bar", 1, 2), snapshotRegistry);
+        assertEquals(new ControllerResult<>(Collections.
+                singletonMap("foo", new ApiError(Errors.INVALID_UPDATE_VERSION,
+                    "Broker 5 does not support the given feature range."))),
+            manager.updateFeatures(rangeMap("foo", 1, 3),
+                new HashSet<>(Arrays.asList("foo")),
+                Collections.singletonMap(5, rangeMap())));
+
+        ControllerResult<Map<String, ApiError>> result = manager.updateFeatures(
+            rangeMap("foo", 1, 3), Collections.emptySet(), Collections.emptyMap());
+        assertEquals(Collections.singletonMap("foo", ApiError.NONE), result.response());
+        manager.replay((FeatureLevelRecord) result.records().get(0).message(), 3);
+        snapshotRegistry.createSnapshot(3);
+
+        assertEquals(new ControllerResult<>(Collections.
+                singletonMap("foo", new ApiError(Errors.INVALID_UPDATE_VERSION,
+                    "Can't downgrade the maximum version of this feature without " +
+                    "setting downgradable to true."))),
+            manager.updateFeatures(rangeMap("foo", 1, 2),
+                Collections.emptySet(), Collections.emptyMap()));
+
+        assertEquals(new ControllerResult<>(
+            Collections.singletonList(new ApiMessageAndVersion(new FeatureLevelRecord().
+                    setName("foo").setMinFeatureLevel((short) 1).setMaxFeatureLevel((short) 2),
+                    (short) 0)),
+                Collections.singletonMap("foo", ApiError.NONE)),
+            manager.updateFeatures(rangeMap("foo", 1, 2),
+                new HashSet<>(Collections.singletonList("foo")), Collections.emptyMap()));
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java b/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java
new file mode 100644
index 0000000..4e6523e
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java
@@ -0,0 +1,47 @@
+/*
+ * 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;
+
+
+public final class MockControllerMetrics implements ControllerMetrics {
+    private volatile boolean active;
+
+    public MockControllerMetrics() {
+        this.active = false;
+    }
+
+    @Override
+    public void setActive(boolean active) {
+        this.active = active;
+    }
+
+    @Override
+    public boolean active() {
+        return this.active;
+    }
+
+    @Override
+    public void updateEventQueueTime(long durationMs) {
+        // nothing to do
+    }
+
+    @Override
+    public void updateEventQueueProcessingTime(long durationMs) {
+        // nothing to do
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/MockRandom.java b/metadata/src/test/java/org/apache/kafka/controller/MockRandom.java
new file mode 100644
index 0000000..c42a158
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/MockRandom.java
@@ -0,0 +1,34 @@
+/*
+ * 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.Random;
+
+
+/**
+ * A subclass of Random with a fixed seed and generation algorithm.
+ */
+public class MockRandom extends Random {
+    private long state = 17;
+
+    @Override
+    protected int next(int bits) {
+        state = (state * 2862933555777941757L) + 3037000493L;
+        return (int) (state >>> (64 - bits));
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
new file mode 100644
index 0000000..16b52d1
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.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 java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.Listener;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+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.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metalog.LocalLogManagerTestEnv;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.controller.ConfigurationControlManagerTest.BROKER0;
+import static org.apache.kafka.controller.ConfigurationControlManagerTest.CONFIGS;
+import static org.apache.kafka.controller.ConfigurationControlManagerTest.entry;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+@Timeout(value = 40)
+public class QuorumControllerTest {
+    private static final Logger log =
+        LoggerFactory.getLogger(QuorumControllerTest.class);
+
+    /**
+     * Test creating a new QuorumController and closing it.
+     */
+    @Test
+    public void testCreateAndClose() throws Throwable {
+        try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1)) {
+            try (QuorumControllerTestEnv controlEnv =
+                     new QuorumControllerTestEnv(logEnv, __ -> { })) {
+            }
+        }
+    }
+
+    /**
+     * Test setting some configuration values and reading them back.
+     */
+    @Test
+    public void testConfigurationOperations() throws Throwable {
+        try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1)) {
+            try (QuorumControllerTestEnv controlEnv =
+                     new QuorumControllerTestEnv(logEnv, b -> b.setConfigDefs(CONFIGS))) {
+                testConfigurationOperations(controlEnv.activeController());
+            }
+        }
+    }
+
+    private void testConfigurationOperations(QuorumController controller) throws Throwable {
+        assertEquals(Collections.singletonMap(BROKER0, ApiError.NONE),
+            controller.incrementalAlterConfigs(Collections.singletonMap(
+                BROKER0, Collections.singletonMap("baz", entry(SET, "123"))), true).get());
+        assertEquals(Collections.singletonMap(BROKER0,
+            new ResultOrError<>(Collections.emptyMap())),
+            controller.describeConfigs(Collections.singletonMap(
+                BROKER0, Collections.emptyList())).get());
+        assertEquals(Collections.singletonMap(BROKER0, ApiError.NONE),
+            controller.incrementalAlterConfigs(Collections.singletonMap(
+                BROKER0, Collections.singletonMap("baz", entry(SET, "123"))), false).get());
+        assertEquals(Collections.singletonMap(BROKER0, new ResultOrError<>(Collections.
+                singletonMap("baz", "123"))),
+            controller.describeConfigs(Collections.singletonMap(
+                BROKER0, Collections.emptyList())).get());
+    }
+
+    /**
+     * Test that an incrementalAlterConfigs operation doesn't complete until the records
+     * can be written to the metadata log.
+     */
+    @Test
+    public void testDelayedConfigurationOperations() throws Throwable {
+        try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1)) {
+            try (QuorumControllerTestEnv controlEnv =
+                     new QuorumControllerTestEnv(logEnv, b -> b.setConfigDefs(CONFIGS))) {
+                testDelayedConfigurationOperations(logEnv, controlEnv.activeController());
+            }
+        }
+    }
+
+    private void testDelayedConfigurationOperations(LocalLogManagerTestEnv logEnv,
+                                                    QuorumController controller)
+                                                    throws Throwable {
+        logEnv.logManagers().forEach(m -> m.setMaxReadOffset(0L));
+        CompletableFuture<Map<ConfigResource, ApiError>> future1 =
+            controller.incrementalAlterConfigs(Collections.singletonMap(
+                BROKER0, Collections.singletonMap("baz", entry(SET, "123"))), false);
+        assertFalse(future1.isDone());
+        assertEquals(Collections.singletonMap(BROKER0,
+            new ResultOrError<>(Collections.emptyMap())),
+            controller.describeConfigs(Collections.singletonMap(
+                BROKER0, Collections.emptyList())).get());
+        logEnv.logManagers().forEach(m -> m.setMaxReadOffset(1L));
+        assertEquals(Collections.singletonMap(BROKER0, ApiError.NONE), future1.get());
+    }
+
+    @Test
+    public void testUnregisterBroker() throws Throwable {
+        try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1)) {
+            try (QuorumControllerTestEnv controlEnv =
+                     new QuorumControllerTestEnv(logEnv, b -> b.setConfigDefs(CONFIGS))) {
+                ListenerCollection listeners = new ListenerCollection();
+                listeners.add(new Listener().setName("PLAINTEXT").
+                    setHost("localhost").setPort(9092));
+                QuorumController active = controlEnv.activeController();
+                CompletableFuture<BrokerRegistrationReply> reply = active.registerBroker(
+                    new BrokerRegistrationRequestData().
+                        setBrokerId(0).
+                        setClusterId(Uuid.fromString("06B-K3N1TBCNYFgruEVP0Q")).
+                        setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")).
+                        setListeners(listeners));
+                assertEquals(0L, reply.get().epoch());
+                CreateTopicsRequestData createTopicsRequestData =
+                    new CreateTopicsRequestData().setTopics(
+                        new CreatableTopicCollection(Collections.singleton(
+                            new CreatableTopic().setName("foo").setNumPartitions(1).
+                                setReplicationFactor((short) 1)).iterator()));
+                // TODO: place on a fenced broker if we have no choice
+                assertEquals(Errors.INVALID_REPLICATION_FACTOR.code(), active.createTopics(
+                    createTopicsRequestData).get().topics().find("foo").errorCode());
+                assertEquals(new BrokerHeartbeatReply(true, false, false, false),
+                    active.processBrokerHeartbeat(new BrokerHeartbeatRequestData().
+                            setWantFence(false).setBrokerEpoch(0L).setBrokerId(0).
+                            setCurrentMetadataOffset(100000L)).get());
+                assertEquals(Errors.NONE.code(), active.createTopics(
+                    createTopicsRequestData).get().topics().find("foo").errorCode());
+                CompletableFuture<TopicPartition> topicPartitionFuture = active.appendReadEvent(
+                    "debugGetPartition", () -> {
+                        Iterator<TopicPartition> iterator = active.
+                            replicationControl().brokersToIsrs().iterator(0, true);
+                        assertTrue(iterator.hasNext());
+                        return iterator.next();
+                    });
+                assertEquals(0, topicPartitionFuture.get().partitionId());
+                active.unregisterBroker(0).get();
+                topicPartitionFuture = active.appendReadEvent(
+                    "debugGetPartition", () -> {
+                        Iterator<TopicPartition> iterator = active.
+                            replicationControl().brokersToIsrs().noLeaderIterator();
+                        assertTrue(iterator.hasNext());
+                        return iterator.next();
+                    });
+                assertEquals(0, topicPartitionFuture.get().partitionId());
+            }
+        }
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java
new file mode 100644
index 0000000..9927042
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java
@@ -0,0 +1,88 @@
+/*
+ * 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.metalog.LocalLogManagerTestEnv;
+import org.apache.kafka.test.TestUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+
+public class QuorumControllerTestEnv implements AutoCloseable {
+    private static final Logger log =
+        LoggerFactory.getLogger(QuorumControllerTestEnv.class);
+
+    private final List<QuorumController> controllers;
+
+    public QuorumControllerTestEnv(LocalLogManagerTestEnv logEnv,
+                                   Consumer<QuorumController.Builder> builderConsumer)
+                                   throws Exception {
+        int numControllers = logEnv.logManagers().size();
+        this.controllers = new ArrayList<>(numControllers);
+        try {
+            for (int i = 0; i < numControllers; i++) {
+                QuorumController.Builder builder = new QuorumController.Builder(i);
+                builder.setLogManager(logEnv.logManagers().get(i));
+                builderConsumer.accept(builder);
+                this.controllers.add(builder.build());
+            }
+        } catch (Exception e) {
+            close();
+            throw e;
+        }
+    }
+
+    QuorumController activeController() throws InterruptedException {
+        AtomicReference<QuorumController> value = new AtomicReference<>(null);
+        TestUtils.retryOnExceptionWithTimeout(3, 20000, () -> {
+            QuorumController activeController = null;
+            for (QuorumController controller : controllers) {
+                if (controller.isActive()) {
+                    if (activeController != null) {
+                        throw new RuntimeException("node " + activeController.nodeId() +
+                            " thinks it's the leader, but so does " + controller.nodeId());
+                    }
+                    activeController = controller;
+                }
+            }
+            if (activeController == null) {
+                throw new RuntimeException("No leader found.");
+            }
+            value.set(activeController);
+        });
+        return value.get();
+    }
+
+    public List<QuorumController> controllers() {
+        return controllers;
+    }
+
+    @Override
+    public void close() throws InterruptedException {
+        for (QuorumController controller : controllers) {
+            controller.beginShutdown();
+        }
+        for (QuorumController controller : controllers) {
+            controller.close();
+        }
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicasTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicasTest.java
new file mode 100644
index 0000000..6947c70
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicasTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.Arrays;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+@Timeout(40)
+public class ReplicasTest {
+    @Test
+    public void testToList() {
+        assertEquals(Arrays.asList(1, 2, 3, 4), Replicas.toList(new int[] {1, 2, 3, 4}));
+        assertEquals(Arrays.asList(), Replicas.toList(Replicas.NONE));
+        assertEquals(Arrays.asList(2), Replicas.toList(new int[] {2}));
+    }
+
+    @Test
+    public void testToArray() {
+        assertArrayEquals(new int[] {3, 2, 1}, Replicas.toArray(Arrays.asList(3, 2, 1)));
+        assertArrayEquals(new int[] {}, Replicas.toArray(Arrays.asList()));
+        assertArrayEquals(new int[] {2}, Replicas.toArray(Arrays.asList(2)));
+    }
+
+    @Test
+    public void testClone() {
+        assertArrayEquals(new int[]{3, 2, 1}, Replicas.clone(new int[]{3, 2, 1}));
+        assertArrayEquals(new int[]{}, Replicas.clone(new int[]{}));
+        assertArrayEquals(new int[]{2}, Replicas.clone(new int[]{2}));
+    }
+
+    @Test
+    public void testValidate() {
+        assertTrue(Replicas.validate(new int[] {}));
+        assertTrue(Replicas.validate(new int[] {3}));
+        assertTrue(Replicas.validate(new int[] {3, 1, 2, 6}));
+        assertFalse(Replicas.validate(new int[] {3, 3}));
+        assertFalse(Replicas.validate(new int[] {4, -1, 3}));
+        assertFalse(Replicas.validate(new int[] {-1}));
+        assertFalse(Replicas.validate(new int[] {3, 1, 2, 6, 1}));
+        assertTrue(Replicas.validate(new int[] {1, 100}));
+    }
+
+    @Test
+    public void testValidateIsr() {
+        assertTrue(Replicas.validateIsr(new int[] {}, new int[] {}));
+        assertTrue(Replicas.validateIsr(new int[] {1, 2, 3}, new int[] {}));
+        assertTrue(Replicas.validateIsr(new int[] {1, 2, 3}, new int[] {1, 2, 3}));
+        assertTrue(Replicas.validateIsr(new int[] {3, 1, 2}, new int[] {2, 1}));
+        assertFalse(Replicas.validateIsr(new int[] {3, 1, 2}, new int[] {4, 1}));
+        assertFalse(Replicas.validateIsr(new int[] {1, 2, 4}, new int[] {4, 4}));
+    }
+
+    @Test
+    public void testContains() {
+        assertTrue(Replicas.contains(new int[] {3, 0, 1}, 0));
+        assertFalse(Replicas.contains(new int[] {}, 0));
+        assertTrue(Replicas.contains(new int[] {1}, 1));
+    }
+
+    @Test
+    public void testCopyWithout() {
+        assertArrayEquals(new int[] {}, Replicas.copyWithout(new int[] {}, 0));
+        assertArrayEquals(new int[] {}, Replicas.copyWithout(new int[] {1}, 1));
+        assertArrayEquals(new int[] {1, 3}, Replicas.copyWithout(new int[] {1, 2, 3}, 2));
+        assertArrayEquals(new int[] {4, 1}, Replicas.copyWithout(new int[] {4, 2, 2, 1}, 2));
+    }
+
+    @Test
+    public void testCopyWith() {
+        assertArrayEquals(new int[] {-1}, Replicas.copyWith(new int[] {}, -1));
+        assertArrayEquals(new int[] {1, 2, 3, 4}, Replicas.copyWith(new int[] {1, 2, 3}, 4));
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
new file mode 100644
index 0000000..9cc4173
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+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.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.apache.kafka.common.protocol.Errors.INVALID_TOPIC_EXCEPTION;
+import static org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import static org.apache.kafka.controller.ReplicationControlManager.PartitionControlInfo;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+
+@Timeout(40)
+public class ReplicationControlManagerTest {
+    private static ReplicationControlManager newReplicationControlManager() {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
+        LogContext logContext = new LogContext();
+        MockTime time = new MockTime();
+        MockRandom random = new MockRandom();
+        ClusterControlManager clusterControl = new ClusterControlManager(
+            logContext, time, snapshotRegistry, 1000,
+            new SimpleReplicaPlacementPolicy(random));
+        clusterControl.activate();
+        ConfigurationControlManager configurationControl = new ConfigurationControlManager(
+            new LogContext(), snapshotRegistry, Collections.emptyMap());
+        return new ReplicationControlManager(snapshotRegistry,
+            new LogContext(),
+            random,
+            (short) 3,
+            1,
+            configurationControl,
+            clusterControl);
+    }
+
+    private static void registerBroker(int brokerId, ClusterControlManager clusterControl) {
+        RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().
+            setBrokerEpoch(100).setBrokerId(brokerId);
+        brokerRecord.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint().
+            setSecurityProtocol(SecurityProtocol.PLAINTEXT.id).
+            setPort((short) 9092 + brokerId).
+            setName("PLAINTEXT").
+            setHost("localhost"));
+        clusterControl.replay(brokerRecord);
+    }
+
+    private static void unfenceBroker(int brokerId,
+                                      ReplicationControlManager replicationControl) throws Exception {
+        ControllerResult<BrokerHeartbeatReply> result = replicationControl.
+            processBrokerHeartbeat(new BrokerHeartbeatRequestData().
+                setBrokerId(brokerId).setBrokerEpoch(100).setCurrentMetadataOffset(1).
+                setWantFence(false).setWantShutDown(false), 0);
+        assertEquals(new BrokerHeartbeatReply(true, false, false, false), result.response());
+        ControllerTestUtils.replayAll(replicationControl.clusterControl, result.records());
+    }
+
+    @Test
+    public void testCreateTopics() throws Exception {
+        ReplicationControlManager replicationControl = newReplicationControlManager();
+        CreateTopicsRequestData request = new CreateTopicsRequestData();
+        request.topics().add(new CreatableTopic().setName("foo").
+            setNumPartitions(-1).setReplicationFactor((short) -1));
+        ControllerResult<CreateTopicsResponseData> result =
+            replicationControl.createTopics(request);
+        CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData();
+        expectedResponse.topics().add(new CreatableTopicResult().setName("foo").
+            setErrorCode(Errors.INVALID_REPLICATION_FACTOR.code()).
+                setErrorMessage("Unable to replicate the partition 3 times: there are only 0 usable brokers"));
+        assertEquals(expectedResponse, result.response());
+
+        registerBroker(0, replicationControl.clusterControl);
+        unfenceBroker(0, replicationControl);
+        registerBroker(1, replicationControl.clusterControl);
+        unfenceBroker(1, replicationControl);
+        registerBroker(2, replicationControl.clusterControl);
+        unfenceBroker(2, replicationControl);
+        ControllerResult<CreateTopicsResponseData> result2 =
+            replicationControl.createTopics(request);
+        CreateTopicsResponseData expectedResponse2 = new CreateTopicsResponseData();
+        expectedResponse2.topics().add(new CreatableTopicResult().setName("foo").
+            setNumPartitions(1).setReplicationFactor((short) 3).
+            setErrorMessage(null).setErrorCode((short) 0).
+            setTopicId(result2.response().topics().find("foo").topicId()));
+        assertEquals(expectedResponse2, result2.response());
+        ControllerTestUtils.replayAll(replicationControl, result2.records());
+        assertEquals(new PartitionControlInfo(new int[] {2, 0, 1},
+            new int[] {2, 0, 1}, null, null, 2, 0, 0),
+            replicationControl.getPartition(
+                ((TopicRecord) result2.records().get(0).message()).topicId(), 0));
+        ControllerResult<CreateTopicsResponseData> result3 =
+                replicationControl.createTopics(request);
+        CreateTopicsResponseData expectedResponse3 = new CreateTopicsResponseData();
+        expectedResponse3.topics().add(new CreatableTopicResult().setName("foo").
+                setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code()).
+                setErrorMessage(Errors.TOPIC_ALREADY_EXISTS.exception().getMessage()));
+        assertEquals(expectedResponse3, result3.response());
+    }
+
+    @Test
+    public void testValidateNewTopicNames() {
+        Map<String, ApiError> topicErrors = new HashMap<>();
+        CreatableTopicCollection topics = new CreatableTopicCollection();
+        topics.add(new CreatableTopic().setName(""));
+        topics.add(new CreatableTopic().setName("woo"));
+        topics.add(new CreatableTopic().setName("."));
+        ReplicationControlManager.validateNewTopicNames(topicErrors, topics);
+        Map<String, ApiError> expectedTopicErrors = new HashMap<>();
+        expectedTopicErrors.put("", new ApiError(INVALID_TOPIC_EXCEPTION,
+            "Topic name is illegal, it can't be empty"));
+        expectedTopicErrors.put(".", new ApiError(INVALID_TOPIC_EXCEPTION,
+            "Topic name cannot be \".\" or \"..\""));
+        assertEquals(expectedTopicErrors, topicErrors);
+    }
+
+    private static CreatableTopicResult createTestTopic(
+            ReplicationControlManager replicationControl, String name,
+            int[][] replicas) throws Exception {
+        assertFalse(replicas.length == 0);
+        CreateTopicsRequestData request = new CreateTopicsRequestData();
+        CreatableTopic topic = new CreatableTopic().setName(name);
+        topic.setNumPartitions(-1).setReplicationFactor((short) -1);
+        for (int i = 0; i < replicas.length; i++) {
+            topic.assignments().add(new CreatableReplicaAssignment().
+                setPartitionIndex(i).setBrokerIds(Replicas.toList(replicas[i])));
+        }
+        request.topics().add(topic);
+        ControllerResult<CreateTopicsResponseData> result =
+            replicationControl.createTopics(request);
+        CreatableTopicResult topicResult = result.response().topics().find(name);
+        assertNotNull(topicResult);
+        assertEquals((short) 0, topicResult.errorCode());
+        assertEquals(replicas.length, topicResult.numPartitions());
+        assertEquals(replicas[0].length, topicResult.replicationFactor());
+        ControllerTestUtils.replayAll(replicationControl, result.records());
+        return topicResult;
+    }
+
+    @Test
+    public void testRemoveLeaderships() throws Exception {
+        ReplicationControlManager replicationControl = newReplicationControlManager();
+        for (int i = 0; i < 6; i++) {
+            registerBroker(i, replicationControl.clusterControl);
+            unfenceBroker(i, replicationControl);
+        }
+        CreatableTopicResult result = createTestTopic(replicationControl, "foo",
+            new int[][] {
+                new int[] {0, 1, 2},
+                new int[] {1, 2, 3},
+                new int[] {2, 3, 0},
+                new int[] {0, 2, 1}
+            });
+        Set<TopicPartition> expectedPartitions = new HashSet<>();
+        expectedPartitions.add(new TopicPartition(result.topicId(), 0));
+        expectedPartitions.add(new TopicPartition(result.topicId(), 3));
+        assertEquals(expectedPartitions, ControllerTestUtils.
+            iteratorToSet(replicationControl.brokersToIsrs().iterator(0, true)));
+        List<ApiMessageAndVersion> records = new ArrayList<>();
+        replicationControl.handleNodeDeactivated(0, records);
+        ControllerTestUtils.replayAll(replicationControl, records);
+        assertEquals(Collections.emptySet(), ControllerTestUtils.
+            iteratorToSet(replicationControl.brokersToIsrs().iterator(0, true)));
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ResultOrErrorTest.java b/metadata/src/test/java/org/apache/kafka/controller/ResultOrErrorTest.java
new file mode 100644
index 0000000..7d42b2e
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/controller/ResultOrErrorTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+@Timeout(value = 40)
+public class ResultOrErrorTest {
+    @Test
+    public void testError() {
+        ResultOrError<Integer> resultOrError =
+            new ResultOrError<>(Errors.INVALID_REQUEST, "missing foobar");
+        assertTrue(resultOrError.isError());
+        assertFalse(resultOrError.isResult());
+        assertEquals(null, resultOrError.result());
+        assertEquals(new ApiError(Errors.INVALID_REQUEST, "missing foobar"),
+            resultOrError.error());
+    }
+
+    @Test
+    public void testResult() {
+        ResultOrError<Integer> resultOrError = new ResultOrError<>(123);
+        assertFalse(resultOrError.isError());
+        assertTrue(resultOrError.isResult());
+        assertEquals(123, resultOrError.result());
+        assertEquals(null, resultOrError.error());
+    }
+
+    @Test
+    public void testEquals() {
+        ResultOrError<String> a = new ResultOrError<>(Errors.INVALID_REQUEST, "missing foobar");
+        ResultOrError<String> b = new ResultOrError<>("abcd");
+        assertFalse(a.equals(b));
+        assertFalse(b.equals(a));
+        assertTrue(a.equals(a));
+        assertTrue(b.equals(b));
+        ResultOrError<String> c = new ResultOrError<>(Errors.INVALID_REQUEST, "missing baz");
+        assertFalse(a.equals(c));
+        assertFalse(c.equals(a));
+        assertTrue(c.equals(c));
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
index ef85314..7b6cf06 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
@@ -39,6 +39,7 @@ 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.
  */
@@ -111,9 +112,26 @@ public final class LocalLogManager implements MetaLogManager, AutoCloseable {
 
     public static class SharedLogData {
         private final Logger log = LoggerFactory.getLogger(SharedLogData.class);
+
+        /**
+         * Maps node IDs to the matching log managers.
+         */
         private final HashMap<Integer, LocalLogManager> logManagers = new HashMap<>();
+
+        /**
+         * Maps offsets to record batches.
+         */
         private final TreeMap<Long, LocalBatch> batches = new TreeMap<>();
+
+        /**
+         * The current leader.
+         */
         private MetaLogLeader leader = new MetaLogLeader(-1, -1);
+
+        /**
+         * The start offset of the last batch that was created, or -1 if no batches have
+         * been created.
+         */
         private long prevOffset = -1;
 
         synchronized void registerLogManager(LocalLogManager logManager) {
@@ -197,20 +215,45 @@ public final class LocalLogManager implements MetaLogManager, AutoCloseable {
 
     private final Logger log;
 
+    /**
+     * The node ID of this local log manager. Each log manager must have a unique ID.
+     */
     private final int nodeId;
 
+    /**
+     * A reference to the in-memory state that unites all the log managers in use.
+     */
     private final SharedLogData shared;
 
+    /**
+     * The event queue used by this local log manager.
+     */
     private final EventQueue eventQueue;
 
+    /**
+     * Whether this LocalLogManager has been initialized.
+     */
     private boolean initialized = false;
 
+    /**
+     * Whether this LocalLogManager has been shut down.
+     */
     private boolean shutdown = false;
 
+    /**
+     * An offset that the log manager will not read beyond. This exists only for testing
+     * purposes.
+     */
     private long maxReadOffset = Long.MAX_VALUE;
 
+    /**
+     * The listener objects attached to this local log manager.
+     */
     private final List<MetaLogListenerData> listeners = new ArrayList<>();
 
+    /**
+     * The current leader, as seen by this log manager.
+     */
     private volatile MetaLogLeader leader = new MetaLogLeader(-1, -1);
 
     public LocalLogManager(LogContext logContext,
diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java
index 9dd6262..ac578fb 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java
@@ -34,6 +34,7 @@ import static org.apache.kafka.metalog.MockMetaLogManagerListener.LAST_COMMITTED
 import static org.apache.kafka.metalog.MockMetaLogManagerListener.SHUTDOWN;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
+
 @Timeout(value = 40)
 public class LocalLogManagerTest {
     private static final Logger log = LoggerFactory.getLogger(LocalLogManagerTest.class);
diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java
index 7910285..fafccfa 100644
--- a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java
+++ b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java
@@ -23,8 +23,8 @@ import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
 import org.apache.kafka.common.config.ConfigResource;
 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.PartitionChangeRecord;
 import org.apache.kafka.common.metadata.PartitionRecord;
 import org.apache.kafka.common.metadata.PartitionRecordJsonConverter;
 import org.apache.kafka.common.metadata.RegisterBrokerRecord;
@@ -56,6 +56,8 @@ import java.util.function.Consumer;
  * Maintains the in-memory metadata for the metadata tool.
  */
 public final class MetadataNodeManager implements AutoCloseable {
+    private static final int NO_LEADER_CHANGE = -2;
+
     private static final Logger log = LoggerFactory.getLogger(MetadataNodeManager.class);
 
     public static class Data {
@@ -259,17 +261,21 @@ public final class MetadataNodeManager implements AutoCloseable {
                 }
                 break;
             }
-            case ISR_CHANGE_RECORD: {
-                IsrChangeRecord record = (IsrChangeRecord) message;
+            case PARTITION_CHANGE_RECORD: {
+                PartitionChangeRecord record = (PartitionChangeRecord) message;
                 FileNode file = data.root.file("topicIds", record.topicId().toString(),
                     Integer.toString(record.partitionId()), "data");
                 JsonNode node = objectMapper.readTree(file.contents());
                 PartitionRecord partition = PartitionRecordJsonConverter.
                     read(node, PartitionRecord.HIGHEST_SUPPORTED_VERSION);
-                partition.setIsr(record.isr());
-                partition.setLeader(record.leader());
-                partition.setLeaderEpoch(record.leaderEpoch());
-                partition.setPartitionEpoch(record.partitionEpoch());
+                if (record.isr() != null) {
+                    partition.setIsr(record.isr());
+                }
+                if (record.leader() != NO_LEADER_CHANGE) {
+                    partition.setLeader(record.leader());
+                    partition.setLeaderEpoch(partition.leaderEpoch() + 1);
+                }
+                partition.setPartitionEpoch(partition.partitionEpoch() + 1);
                 file.setContents(PartitionRecordJsonConverter.write(partition,
                     PartitionRecord.HIGHEST_SUPPORTED_VERSION).toPrettyString());
                 break;


Mime
View raw message