You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by cm...@apache.org on 2022/12/16 21:05:19 UTC

[kafka] branch 3.4 updated (385c317df3e -> 59bad9e47f2)

This is an automated email from the ASF dual-hosted git repository.

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


    from 385c317df3e KAFKA-14496: Wrong Base64 encoder used by OIDC OAuthBearerLoginCallbackHandler (#13000)
     new e94a25173dc MINOR: Move dynamic config logic to DynamicConfigPublisher (#12958)
     new 4b03c8c4c3d KAFKA-14427 ZK client support for migrations (#12946)
     new e49132326e1 MINOR: Move MetadataQuorumCommand from `core` to `tools` (#12951)
     new 984c5eda7c0 KAFKA-14392: Fix overly long request timeouts in BrokerToControllerChannelManager (#12856)
     new 17e6bba20b4 MINOR: Introduce MetadataProvenance and ImageReWriter (#12964)
     new 793c36c5548 MINOR; Improve high watermark log messages (#12975)
     new 2de4451e27e MINOR: Pass snapshot ID directly in `RaftClient.createSnapshot` (#12981)
     new 1d528e395a9 KAFKA-14448 Let ZK brokers register with KRaft controller (#12965)
     new 47c5b001be1 MINOR: Change KRaft ZK controller registration algorithm (#12973)
     new aa532b44199 KAFKA-14446: API forwarding support from zkBrokers to the Controller (#12961)
     new 59bad9e47f2 MINOR: ControllerServer should use the new metadata loader and snapshot generator (#12983)

The 11 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 bin/kafka-metadata-quorum.sh                       |   2 +-
 bin/windows/kafka-metatada-quorum.bat              |   2 +-
 build.gradle                                       |   7 +-
 checkstyle/import-control.xml                      |  10 +-
 .../common/requests/AbstractControlRequest.java    |  10 +-
 .../common/requests/BrokerRegistrationRequest.java |   9 +
 .../kafka/common/requests/LeaderAndIsrRequest.java |  18 +-
 .../kafka/common/requests/StopReplicaRequest.java  |  18 +-
 .../common/requests/UpdateMetadataRequest.java     |  26 +-
 .../java/org/apache/kafka/common/utils/Utils.java  |  14 +-
 .../common/message/BrokerRegistrationRequest.json  |   4 +-
 .../common/message/BrokerRegistrationResponse.json |   2 +-
 .../resources/common/message/EnvelopeRequest.json  |   2 +-
 .../common/message/LeaderAndIsrRequest.json        |   4 +-
 .../common/message/StopReplicaRequest.json         |   4 +-
 .../common/message/UpdateMetadataRequest.json      |   4 +-
 .../common/requests/ApiVersionsResponseTest.java   |   1 -
 .../org/apache/kafka/common/utils/UtilsTest.java   |   7 +
 core/src/main/scala/kafka/Kafka.scala              |   8 +-
 .../scala/kafka/admin/MetadataQuorumCommand.scala  | 172 -------
 .../scala/kafka/common/InterBrokerSendThread.scala |   2 +-
 .../kafka/server/BrokerLifecycleManager.scala      |  20 +-
 .../src/main/scala/kafka/server/BrokerServer.scala |  30 +-
 .../server/BrokerToControllerChannelManager.scala  | 123 +++--
 .../main/scala/kafka/server/ControllerApis.scala   |   2 +-
 .../main/scala/kafka/server/ControllerServer.scala |  12 +-
 core/src/main/scala/kafka/server/KafkaApis.scala   |  18 +-
 core/src/main/scala/kafka/server/KafkaConfig.scala |  11 +-
 core/src/main/scala/kafka/server/KafkaServer.scala | 133 ++++-
 .../main/scala/kafka/server/MetadataCache.scala    |  18 +-
 .../src/main/scala/kafka/server/SharedServer.scala |  99 +++-
 .../main/scala/kafka/server/ZkAdminManager.scala   |  27 +-
 .../server/metadata/BrokerMetadataListener.scala   |  56 +-
 .../server/metadata/BrokerMetadataPublisher.scala  |  61 +--
 .../server/metadata/BrokerServerMetrics.scala      |  59 ++-
 .../server/metadata/DynamicConfigPublisher.scala   | 103 ++++
 .../kafka/server/metadata/KRaftMetadataCache.scala |  11 +-
 .../server/metadata/OffsetTrackingListener.scala   |  48 ++
 .../kafka/server/metadata/ZkMetadataCache.scala    |  58 ++-
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   | 376 ++++++++++++--
 core/src/main/scala/kafka/zk/ZkData.scala          |  24 +-
 .../main/scala/kafka/zk/ZkMigrationClient.scala    | 438 ++++++++++++++++
 core/src/test/java/kafka/test/ClusterInstance.java |   2 +
 core/src/test/java/kafka/test/MockController.java  |   5 -
 .../test/junit/RaftClusterInvocationContext.java   |   8 +
 .../test/junit/ZkClusterInvocationContext.java     |   8 +-
 .../kafka/api/IntegrationTestHarness.scala         |   4 +
 .../kafka/api/PlaintextAdminIntegrationTest.scala  |   2 +-
 .../server/KafkaServerKRaftRegistrationTest.scala  |  85 +++
 .../kafka/server/QuorumTestHarness.scala           |   4 +
 .../kafka/zk/ZkMigrationIntegrationTest.scala      | 105 ++++
 .../BrokerToControllerRequestThreadTest.scala      |  52 +-
 .../server/metadata/BrokerServerMetricsTest.scala  |  21 +-
 .../src/test/scala/kafka/utils/TestInfoUtils.scala |   8 +
 .../kafka/admin/MetadataQuorumCommandTest.scala    | 192 -------
 .../kafka/integration/KafkaServerTestHarness.scala |   3 +-
 .../unit/kafka/server/ApiVersionManagerTest.scala  |   6 +-
 .../kafka/server/BrokerLifecycleManagerTest.scala  |   9 +-
 .../server/BrokerRegistrationRequestTest.scala     | 202 ++++++++
 .../kafka/server/CreateTopicsRequestTest.scala     |  10 +-
 .../kafka/server/DeleteTopicsRequestTest.scala     |   5 +-
 .../kafka/server/DynamicConfigChangeTest.scala     |   4 +-
 .../unit/kafka/server/ForwardingManagerTest.scala  |  24 +-
 .../scala/unit/kafka/server/KafkaApisTest.scala    |   2 +-
 .../unit/kafka/server/MetadataCacheTest.scala      |  25 +-
 .../MockBrokerToControllerChannelManager.scala     |   2 +-
 .../server/ReplicaManagerConcurrencyTest.scala     |   8 +-
 .../unit/kafka/server/ReplicaManagerTest.scala     |   5 +-
 .../scala/unit/kafka/server/RequestQuotaTest.scala |   8 +-
 .../unit/kafka/server/ZkAdminManagerTest.scala     |  28 +-
 .../metadata/BrokerMetadataListenerTest.scala      |   8 +-
 .../metadata/BrokerMetadataPublisherTest.scala     |  45 +-
 .../metadata/BrokerMetadataSnapshotterTest.scala   |  18 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala    |   7 +-
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala    | 119 ++++-
 .../unit/kafka/zk/ZkMigrationClientTest.scala      | 362 +++++++++++++
 .../jmh/fetcher/ReplicaFetcherThreadBenchmark.java |   3 +-
 .../jmh/metadata/MetadataRequestBenchmark.java     |   3 +-
 .../apache/kafka/jmh/server/CheckpointBench.java   |   4 +-
 .../kafka/jmh/server/PartitionCreationBench.java   |   6 +-
 .../apache/kafka/controller/AclControlManager.java |  18 -
 .../controller/ClientQuotaControlManager.java      |  42 --
 .../kafka/controller/ClusterControlManager.java    |  87 +---
 .../controller/ConfigurationControlManager.java    |  36 --
 .../org/apache/kafka/controller/Controller.java    |   8 -
 .../kafka/controller/FeatureControlManager.java    |  44 --
 .../apache/kafka/controller/LogReplayTracker.java  |   3 +-
 .../kafka/controller/ProducerIdControlManager.java |  24 +-
 .../apache/kafka/controller/QuorumController.java  | 315 +-----------
 .../controller/ReplicationControlManager.java      |  34 --
 .../apache/kafka/controller/SnapshotGenerator.java | 132 -----
 .../org/apache/kafka/image/ClientQuotasImage.java  |   3 +-
 .../java/org/apache/kafka/image/MetadataDelta.java |  51 +-
 .../java/org/apache/kafka/image/MetadataImage.java |  26 +-
 .../org/apache/kafka/image/MetadataProvenance.java |  92 ++++
 .../apache/kafka/image/MetadataVersionChange.java  |  77 +++
 .../image/MetadataVersionChangeException.java      |  22 +-
 .../kafka/image/loader/LogDeltaManifest.java       | 106 ++++
 .../apache/kafka/image/loader/MetadataLoader.java  | 572 +++++++++++++++++++++
 .../kafka/image/loader/MetadataLoaderMetrics.java  |  34 +-
 .../kafka/image/loader/SnapshotManifest.java       |  77 +++
 .../kafka/image/publisher/MetadataPublisher.java   |  72 +++
 .../kafka/image/publisher/SnapshotEmitter.java     | 123 +++++
 .../kafka/image/publisher/SnapshotGenerator.java   | 275 ++++++++++
 .../{RecordListWriter.java => ImageReWriter.java}  |  34 +-
 .../kafka/image/writer/ImageWriterOptions.java     |  17 +-
 .../apache/kafka/metadata/BrokerRegistration.java  |  55 +-
 .../kafka/metadata/migration/MigrationClient.java  |  91 ++++
 .../migration/ZkMigrationLeadershipState.java      |  49 +-
 .../kafka/controller/AclControlManagerTest.java    |  24 +-
 .../controller/ClientQuotaControlManagerTest.java  |  59 ++-
 .../controller/ClusterControlManagerTest.java      |  27 +-
 .../ConfigurationControlManagerTest.java           |   8 -
 .../controller/FeatureControlManagerTest.java      |  38 +-
 .../controller/ProducerIdControlManagerTest.java   |  30 +-
 .../kafka/controller/QuorumControllerTest.java     | 401 ++-------------
 .../controller/ReplicationControlManagerTest.java  |  11 -
 .../kafka/controller/SnapshotGeneratorTest.java    | 112 ----
 .../org/apache/kafka/image/ImageDowngradeTest.java |   4 +-
 .../org/apache/kafka/image/MetadataImageTest.java  |  34 +-
 .../kafka/image/MetadataVersionChangeTest.java     |  63 +++
 .../kafka/image/loader/MetadataLoaderTest.java     | 517 +++++++++++++++++++
 .../kafka/image/publisher/SnapshotEmitterTest.java | 206 ++++++++
 .../image/publisher/SnapshotGeneratorTest.java     | 185 +++++++
 ...dListWriterTest.java => ImageReWriterTest.java} |  22 +-
 .../kafka/image/writer/ImageWriterOptionsTest.java |  52 ++
 .../kafka/metadata/BrokerRegistrationTest.java     |   4 +-
 .../org/apache/kafka/metadata/RecordTestUtils.java |  48 --
 .../org/apache/kafka/metalog/LocalLogManager.java  |  27 +-
 .../kafka/metalog/LocalLogManagerTestEnv.java      |   7 +
 .../java/org/apache/kafka/raft/FollowerState.java  |  69 ++-
 .../org/apache/kafka/raft/KafkaRaftClient.java     |   5 +-
 .../java/org/apache/kafka/raft/LeaderState.java    |  28 +-
 .../java/org/apache/kafka/raft/RaftClient.java     |  23 +-
 .../org/apache/kafka/raft/ReplicatedCounter.java   |   3 +-
 .../kafka/snapshot/RecordsSnapshotWriter.java      |  46 +-
 .../org/apache/kafka/snapshot/SnapshotWriter.java  |   2 +-
 .../kafka/raft/KafkaRaftClientSnapshotTest.java    |  48 +-
 .../kafka/snapshot/SnapshotWriterReaderTest.java   |   8 +-
 .../kafka/server/common/MetadataVersion.java       |  13 +-
 .../org/apache/kafka/server/util/ToolsUtils.java   |  19 +-
 .../kafka/server/common/MetadataVersionTest.java   |  10 +-
 tests/kafkatest/tests/core/snapshot_test.py        |   2 +-
 .../apache/kafka/tools/MetadataQuorumCommand.java  | 195 +++++++
 .../org/apache/kafka/tools/TerseException.java     |  15 +-
 .../apache/kafka/tools/TransactionsCommand.java    |  42 +-
 .../tools/MetadataQuorumCommandErrorTest.java      |  48 ++
 .../kafka/tools/MetadataQuorumCommandTest.java     | 161 ++++++
 .../org/apache/kafka/tools/ToolsTestUtils.java     |  51 ++
 .../kafka/tools/TransactionsCommandTest.java       |  12 +-
 150 files changed, 6250 insertions(+), 2413 deletions(-)
 delete mode 100644 core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala
 create mode 100644 core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala
 create mode 100644 core/src/main/scala/kafka/server/metadata/OffsetTrackingListener.scala
 create mode 100644 core/src/main/scala/kafka/zk/ZkMigrationClient.scala
 create mode 100644 core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala
 create mode 100644 core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
 delete mode 100644 core/src/test/scala/unit/kafka/admin/MetadataQuorumCommandTest.scala
 create mode 100644 core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala
 create mode 100644 core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala
 delete mode 100644 metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java
 create mode 100644 metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java
 create mode 100644 metadata/src/main/java/org/apache/kafka/image/MetadataVersionChange.java
 copy streams/src/main/java/org/apache/kafka/streams/errors/UnknownStateStoreException.java => metadata/src/main/java/org/apache/kafka/image/MetadataVersionChangeException.java (64%)
 create mode 100644 metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java
 create mode 100644 metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java
 copy generator/src/main/java/org/apache/kafka/message/MessageSpecType.java => metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoaderMetrics.java (57%)
 create mode 100644 metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java
 create mode 100644 metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java
 create mode 100644 metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java
 create mode 100644 metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java
 copy metadata/src/main/java/org/apache/kafka/image/writer/{RecordListWriter.java => ImageReWriter.java} (61%)
 create mode 100644 metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java
 delete mode 100644 metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java
 create mode 100644 metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java
 create mode 100644 metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java
 create mode 100644 metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java
 create mode 100644 metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java
 copy metadata/src/test/java/org/apache/kafka/image/writer/{RecordListWriterTest.java => ImageReWriterTest.java} (71%)
 create mode 100644 metadata/src/test/java/org/apache/kafka/image/writer/ImageWriterOptionsTest.java
 create mode 100644 tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java
 copy clients/src/main/java/org/apache/kafka/common/errors/GroupMaxSizeReachedException.java => tools/src/main/java/org/apache/kafka/tools/TerseException.java (67%)
 create mode 100644 tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandErrorTest.java
 create mode 100644 tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
 create mode 100644 tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java


[kafka] 04/11: KAFKA-14392: Fix overly long request timeouts in BrokerToControllerChannelManager (#12856)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 984c5eda7c0707577e26890b7b6018f86d78194e
Author: Ron Dagostino <rd...@confluent.io>
AuthorDate: Fri Dec 9 20:00:19 2022 -0500

    KAFKA-14392: Fix overly long request timeouts in BrokerToControllerChannelManager (#12856)
    
    In BrokerToControllerChannelManager, set the request timeout to the minimum of the retry timeout
    and the controller socket timeout. This fixes some cases where we were unintentionally setting an
    overly long request timeout.
    
    Also, the channel manager used by the BrokerLifecycleManager should set a retry timeout equal to
    half of the broker session timeout, rather than the entire broker session timeout, to allow for a
    retransmission if the initial attempt fails.
    
    These two fixes should address some cases where heartbeat broker requests were not being resent
    in a timely fashion after a network glitch.
    
    Reviewers: Colin P. McCabe <cm...@apache.org>, José Armando García Sancio <js...@apache.org>
---
 core/src/main/scala/kafka/server/BrokerServer.scala                   | 2 +-
 .../main/scala/kafka/server/BrokerToControllerChannelManager.scala    | 4 ++--
 core/src/main/scala/kafka/server/ControllerApis.scala                 | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index f55ceebffcc..a83da6e0258 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -338,7 +338,7 @@ class BrokerServer(
         config,
         "heartbeat",
         threadNamePrefix,
-        config.brokerSessionTimeoutMs.toLong
+        config.brokerSessionTimeoutMs / 2 // KAFKA-14392
       )
       lifecycleManager.start(
         () => metadataListener.highestMetadataOffset,
diff --git a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala
index 99e86722f2f..92754a793f5 100644
--- a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala
@@ -211,7 +211,7 @@ class BrokerToControllerChannelManagerImpl(
         50,
         Selectable.USE_DEFAULT_BUFFER_SIZE,
         Selectable.USE_DEFAULT_BUFFER_SIZE,
-        config.requestTimeoutMs,
+        Math.min(Int.MaxValue, Math.min(config.controllerSocketTimeoutMs, retryTimeoutMs)).toInt, // request timeout should not exceed the provided retry timeout
         config.connectionSetupTimeoutMs,
         config.connectionSetupTimeoutMaxMs,
         time,
@@ -283,7 +283,7 @@ class BrokerToControllerRequestThread(
   time: Time,
   threadName: String,
   retryTimeoutMs: Long
-) extends InterBrokerSendThread(threadName, networkClient, config.controllerSocketTimeoutMs, time, isInterruptible = false) {
+) extends InterBrokerSendThread(threadName, networkClient, Math.min(Int.MaxValue, Math.min(config.controllerSocketTimeoutMs, retryTimeoutMs)).toInt, time, isInterruptible = false) {
 
   private val requestQueue = new LinkedBlockingDeque[BrokerToControllerQueueItem]()
   private val activeController = new AtomicReference[Node](null)
diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala
index 657c2965533..1c3586263a7 100644
--- a/core/src/main/scala/kafka/server/ControllerApis.scala
+++ b/core/src/main/scala/kafka/server/ControllerApis.scala
@@ -576,7 +576,7 @@ class ControllerApis(val requestChannel: RequestChannel,
     val heartbeatRequest = request.body[BrokerHeartbeatRequest]
     authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
     val context = new ControllerRequestContext(request.context.header.data, request.context.principal,
-      requestTimeoutMsToDeadlineNs(time, config.brokerHeartbeatIntervalMs))
+      requestTimeoutMsToDeadlineNs(time, config.brokerHeartbeatIntervalMs / 2))
     controller.processBrokerHeartbeat(context, heartbeatRequest.data).handle[Unit] { (reply, e) =>
       def createResponseCallback(requestThrottleMs: Int,
                                  reply: BrokerHeartbeatReply,


[kafka] 06/11: MINOR; Improve high watermark log messages (#12975)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 793c36c55489ece8813e333bed082189da84a6f2
Author: José Armando García Sancio <js...@users.noreply.github.com>
AuthorDate: Mon Dec 12 16:32:16 2022 -0800

    MINOR; Improve high watermark log messages (#12975)
    
    While debugging KRaft and the metadata state machines it is helpful to always log the first time the replica discovers the high watermark. All other updates to the high watermark are logged at trace because they are more frequent and less useful.
    
    Reviewers: Luke Chen <sh...@gmail.com>
---
 .../java/org/apache/kafka/raft/FollowerState.java  | 69 +++++++++++++++++-----
 .../java/org/apache/kafka/raft/LeaderState.java    | 28 +++++++--
 2 files changed, 75 insertions(+), 22 deletions(-)

diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java
index e3a30479745..1eb9eb685d9 100644
--- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java
+++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java
@@ -105,26 +105,41 @@ public class FollowerState implements EpochState {
         fetchTimer.reset(timeoutMs);
     }
 
-    public boolean updateHighWatermark(OptionalLong highWatermark) {
-        if (!highWatermark.isPresent() && this.highWatermark.isPresent())
-            throw new IllegalArgumentException("Attempt to overwrite current high watermark " + this.highWatermark +
-                " with unknown value");
-
-        if (this.highWatermark.isPresent()) {
-            long previousHighWatermark = this.highWatermark.get().offset;
-            long updatedHighWatermark = highWatermark.getAsLong();
-
-            if (updatedHighWatermark < 0)
-                throw new IllegalArgumentException("Illegal negative high watermark update");
-            if (previousHighWatermark > updatedHighWatermark)
-                throw new IllegalArgumentException("Non-monotonic update of high watermark attempted");
-            if (previousHighWatermark == updatedHighWatermark)
+    public boolean updateHighWatermark(OptionalLong newHighWatermark) {
+        if (!newHighWatermark.isPresent() && highWatermark.isPresent()) {
+            throw new IllegalArgumentException(
+                String.format("Attempt to overwrite current high watermark %s with unknown value", highWatermark)
+            );
+        }
+
+        if (highWatermark.isPresent()) {
+            long previousHighWatermark = highWatermark.get().offset;
+            long updatedHighWatermark = newHighWatermark.getAsLong();
+
+            if (updatedHighWatermark < 0) {
+                throw new IllegalArgumentException(
+                    String.format("Illegal negative (%s) high watermark update", updatedHighWatermark)
+                );
+            } else if (previousHighWatermark > updatedHighWatermark) {
+                throw new IllegalArgumentException(
+                    String.format(
+                        "Non-monotonic update of high watermark from %s to %s",
+                        previousHighWatermark,
+                        updatedHighWatermark
+                    )
+                );
+            } else if (previousHighWatermark == updatedHighWatermark) {
                 return false;
+            }
         }
 
-        this.highWatermark = highWatermark.isPresent() ?
-            Optional.of(new LogOffsetMetadata(highWatermark.getAsLong())) :
+        Optional<LogOffsetMetadata> oldHighWatermark = highWatermark;
+        highWatermark = newHighWatermark.isPresent() ?
+            Optional.of(new LogOffsetMetadata(newHighWatermark.getAsLong())) :
             Optional.empty();
+
+        logHighWatermarkUpdate(oldHighWatermark, highWatermark);
+
         return true;
     }
 
@@ -169,4 +184,26 @@ public class FollowerState implements EpochState {
             fetchingSnapshot.get().close();
         }
     }
+
+    private void logHighWatermarkUpdate(
+        Optional<LogOffsetMetadata> oldHighWatermark,
+        Optional<LogOffsetMetadata> newHighWatermark
+    ) {
+        if (!oldHighWatermark.equals(newHighWatermark)) {
+            if (oldHighWatermark.isPresent()) {
+                log.trace(
+                    "High watermark set to {} from {} for epoch {}",
+                    newHighWatermark,
+                    oldHighWatermark.get(),
+                    epoch
+                );
+            } else {
+                log.info(
+                    "High watermark set to {} for the first time for epoch {}",
+                    newHighWatermark,
+                    epoch
+                );
+            }
+        }
+    }
 }
diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java
index 3aa7c0bfe75..ac0ef1260b9 100644
--- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java
+++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java
@@ -170,8 +170,10 @@ public class LeaderState<T> implements EpochState {
                     if (highWatermarkUpdateOffset > currentHighWatermarkMetadata.offset
                         || (highWatermarkUpdateOffset == currentHighWatermarkMetadata.offset &&
                             !highWatermarkUpdateMetadata.metadata.equals(currentHighWatermarkMetadata.metadata))) {
+                        Optional<LogOffsetMetadata> oldHighWatermark = highWatermark;
                         highWatermark = highWatermarkUpdateOpt;
                         logHighWatermarkUpdate(
+                            oldHighWatermark,
                             highWatermarkUpdateMetadata,
                             indexOfHw,
                             followersByDescendingFetchOffset
@@ -187,8 +189,10 @@ public class LeaderState<T> implements EpochState {
                         return false;
                     }
                 } else {
+                    Optional<LogOffsetMetadata> oldHighWatermark = highWatermark;
                     highWatermark = highWatermarkUpdateOpt;
                     logHighWatermarkUpdate(
+                        oldHighWatermark,
                         highWatermarkUpdateMetadata,
                         indexOfHw,
                         followersByDescendingFetchOffset
@@ -201,16 +205,28 @@ public class LeaderState<T> implements EpochState {
     }
 
     private void logHighWatermarkUpdate(
+        Optional<LogOffsetMetadata> oldHighWatermark,
         LogOffsetMetadata newHighWatermark,
         int indexOfHw,
         List<ReplicaState> followersByDescendingFetchOffset
     ) {
-        log.trace(
-            "High watermark set to {} based on indexOfHw {} and voters {}",
-            newHighWatermark,
-            indexOfHw,
-            followersByDescendingFetchOffset
-        );
+        if (oldHighWatermark.isPresent()) {
+            log.trace(
+                "High watermark set to {} from {} based on indexOfHw {} and voters {}",
+                newHighWatermark,
+                oldHighWatermark.get(),
+                indexOfHw,
+                followersByDescendingFetchOffset
+            );
+        } else {
+            log.info(
+                "High watermark set to {} for the first time for epoch {} based on indexOfHw {} and voters {}",
+                newHighWatermark,
+                epoch,
+                indexOfHw,
+                followersByDescendingFetchOffset
+            );
+        }
     }
 
     /**


[kafka] 02/11: KAFKA-14427 ZK client support for migrations (#12946)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 4b03c8c4c3dd5a0c32a11efabf9318883b0483ee
Author: David Arthur <mu...@gmail.com>
AuthorDate: Thu Dec 8 13:14:01 2022 -0500

    KAFKA-14427 ZK client support for migrations (#12946)
    
    This patch adds support for reading and writing ZooKeeper metadata during a KIP-866 migration.
    
    For reading metadata from ZK, methods from KafkaZkClient and ZkData are reused to ensure we are decoding the JSON consistently.
    
    For writing metadata, we use a new multi-op transaction that ensures only a single controller is writing to ZK. This is similar to the existing multi-op transaction that KafkaController uses, but it also includes a check on the new "/migration" ZNode. The transaction consists of three operations:
    
    * CheckOp on /controller_epoch
    * SetDataOp on /migration with zkVersion
    * CreateOp/SetDataOp/DeleteOp (the actual operation being applied)
    
    In the case of a batch of operations (such as topic creation), only the final MultiOp has a SetDataOp on /migration while the other requests use a CheckOp (similar to /controller_epoch).
    
    Reviewers: Colin Patrick McCabe <cm...@apache.org>, dengziming <de...@gmail.com>
---
 checkstyle/import-control.xml                      |   1 +
 .../main/scala/kafka/server/ZkAdminManager.scala   |  27 +-
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   | 333 ++++++++++++++--
 core/src/main/scala/kafka/zk/ZkData.scala          |   9 +-
 .../main/scala/kafka/zk/ZkMigrationClient.scala    | 441 +++++++++++++++++++++
 .../kafka/zk/ZkMigrationIntegrationTest.scala      | 105 +++++
 .../unit/kafka/server/ZkAdminManagerTest.scala     |  28 +-
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala    |  51 ++-
 .../unit/kafka/zk/ZkMigrationClientTest.scala      | 353 +++++++++++++++++
 .../org/apache/kafka/image/ClientQuotasImage.java  |   3 +-
 .../kafka/metadata/migration/MigrationClient.java  |  91 +++++
 .../migration/ZkMigrationLeadershipState.java      |  49 ++-
 12 files changed, 1423 insertions(+), 68 deletions(-)

diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index bd05521964e..7a62b671f84 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -227,6 +227,7 @@
     <allow pkg="org.apache.kafka.controller" />
     <allow pkg="org.apache.kafka.metadata" />
     <allow pkg="org.apache.kafka.metadata.authorizer" />
+    <allow pkg="org.apache.kafka.metadata.migration" />
     <allow pkg="org.apache.kafka.metalog" />
     <allow pkg="org.apache.kafka.queue" />
     <allow pkg="org.apache.kafka.raft" />
diff --git a/core/src/main/scala/kafka/server/ZkAdminManager.scala b/core/src/main/scala/kafka/server/ZkAdminManager.scala
index 634ce6b097c..1a22024d723 100644
--- a/core/src/main/scala/kafka/server/ZkAdminManager.scala
+++ b/core/src/main/scala/kafka/server/ZkAdminManager.scala
@@ -54,6 +54,19 @@ import org.apache.kafka.common.utils.Sanitizer
 import scala.collection.{Map, mutable, _}
 import scala.jdk.CollectionConverters._
 
+object ZkAdminManager {
+  def clientQuotaPropsToDoubleMap(props: Map[String, String]): Map[String, Double] = {
+    props.map { case (key, value) =>
+      val doubleValue = try value.toDouble catch {
+        case _: NumberFormatException =>
+          throw new IllegalStateException(s"Unexpected client quota configuration value: $key -> $value")
+      }
+      key -> doubleValue
+    }
+  }
+}
+
+
 class ZkAdminManager(val config: KafkaConfig,
                      val metrics: Metrics,
                      val metadataCache: MetadataCache,
@@ -636,16 +649,6 @@ class ZkAdminManager(val config: KafkaConfig,
 
   private def sanitized(name: Option[String]): String = name.map(n => sanitizeEntityName(n)).getOrElse("")
 
-  private def fromProps(props: Map[String, String]): Map[String, Double] = {
-    props.map { case (key, value) =>
-      val doubleValue = try value.toDouble catch {
-        case _: NumberFormatException =>
-          throw new IllegalStateException(s"Unexpected client quota configuration value: $key -> $value")
-      }
-      key -> doubleValue
-    }
-  }
-
   def handleDescribeClientQuotas(userComponent: Option[ClientQuotaFilterComponent],
     clientIdComponent: Option[ClientQuotaFilterComponent], strict: Boolean): Map[ClientQuotaEntity, Map[String, Double]] = {
 
@@ -706,7 +709,7 @@ class ZkAdminManager(val config: KafkaConfig,
     (userEntries ++ clientIdEntries ++ bothEntries).flatMap { case ((u, c), p) =>
       val quotaProps = p.asScala.filter { case (key, _) => QuotaConfigs.isClientOrUserQuotaConfig(key) }
       if (quotaProps.nonEmpty && matches(userComponent, u) && matches(clientIdComponent, c))
-        Some(userClientIdToEntity(u, c) -> fromProps(quotaProps))
+        Some(userClientIdToEntity(u, c) -> ZkAdminManager.clientQuotaPropsToDoubleMap(quotaProps))
       else
         None
     }.toMap
@@ -732,7 +735,7 @@ class ZkAdminManager(val config: KafkaConfig,
     ipEntries.flatMap { case (ip, props) =>
       val ipQuotaProps = props.asScala.filter { case (key, _) => DynamicConfig.Ip.names.contains(key) }
       if (ipQuotaProps.nonEmpty)
-        Some(ipToQuotaEntity(ip) -> fromProps(ipQuotaProps))
+        Some(ipToQuotaEntity(ip) -> ZkAdminManager.clientQuotaPropsToDoubleMap(ipQuotaProps))
       else
         None
     }
diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala
index 747673d37db..12f4bfb2c3e 100644
--- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala
+++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala
@@ -34,12 +34,13 @@ import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceT
 import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation}
 import org.apache.kafka.common.utils.{Time, Utils}
 import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
+import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
 import org.apache.zookeeper.KeeperException.{Code, NodeExistsException}
-import org.apache.zookeeper.OpResult.{CreateResult, ErrorResult, SetDataResult}
+import org.apache.zookeeper.OpResult.{CheckResult, CreateResult, ErrorResult, SetDataResult}
 import org.apache.zookeeper.client.ZKClientConfig
 import org.apache.zookeeper.common.ZKConfig
 import org.apache.zookeeper.data.{ACL, Stat}
-import org.apache.zookeeper.{CreateMode, KeeperException, ZooKeeper}
+import org.apache.zookeeper.{CreateMode, KeeperException, OpResult, ZooKeeper}
 
 import scala.collection.{Map, Seq, mutable}
 
@@ -156,6 +157,83 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
     tryCreateControllerZNodeAndIncrementEpoch()
   }
 
+  /**
+   * Registers a given KRaft controller in zookeeper as the active controller. Unlike the ZK equivalent of this method,
+   * this creates /controller as a persistent znode. This prevents ZK brokers from attempting to claim the controller
+   * leadership during a KRaft leadership failover.
+   *
+   * This method is called at the beginning of a KRaft migration and during subsequent KRaft leadership changes during
+   * the migration.
+   *
+   * To ensure that the KRaft controller epoch exceeds the current ZK controller epoch, this registration algorithm
+   * uses a conditional update on the /controller_epoch znode. If a new ZK controller is elected during this method,
+   * the conditional update on /controller_epoch fails which causes the whole multi-op transaction to fail.
+   *
+   * @param kraftControllerId ID of the KRaft controller node
+   * @param kraftControllerEpoch Epoch of the KRaft controller node
+   * @return An optional of the new zkVersion of /controller_epoch. None if we could not register the KRaft controller.
+   */
+  def tryRegisterKRaftControllerAsActiveController(kraftControllerId: Int, kraftControllerEpoch: Int): Option[Int] = {
+    val timestamp = time.milliseconds()
+    val curEpochOpt: Option[(Int, Int)] = getControllerEpoch.map(e => (e._1, e._2.getVersion))
+    val controllerOpt = getControllerId
+    val controllerEpochToStore = kraftControllerEpoch + 10000000 // TODO Remove this after KAFKA-14436
+    curEpochOpt match {
+      case None =>
+        throw new IllegalStateException(s"Cannot register KRaft controller $kraftControllerId as the active controller " +
+          s"since there is no ZK controller epoch present.")
+      case Some((curEpoch: Int, curEpochZk: Int)) =>
+        if (curEpoch >= controllerEpochToStore) {
+          // TODO KAFKA-14436 Need to ensure KRaft has a higher epoch an ZK
+          throw new IllegalStateException(s"Cannot register KRaft controller $kraftControllerId as the active controller " +
+            s"in ZK since its epoch ${controllerEpochToStore} is not higher than the current ZK epoch ${curEpoch}.")
+        }
+
+        val response = if (controllerOpt.isDefined) {
+          info(s"KRaft controller $kraftControllerId overwriting ${ControllerZNode.path} to become the active " +
+            s"controller with epoch $controllerEpochToStore. The previous controller was ${controllerOpt.get}.")
+          retryRequestUntilConnected(
+            MultiRequest(Seq(
+              SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(controllerEpochToStore), curEpochZk),
+              DeleteOp(ControllerZNode.path, ZkVersion.MatchAnyVersion),
+              CreateOp(ControllerZNode.path, ControllerZNode.encode(kraftControllerId, timestamp),
+                defaultAcls(ControllerZNode.path), CreateMode.PERSISTENT)))
+          )
+        } else {
+          info(s"KRaft controller $kraftControllerId creating ${ControllerZNode.path} to become the active " +
+            s"controller with epoch $controllerEpochToStore. There was no active controller.")
+          retryRequestUntilConnected(
+            MultiRequest(Seq(
+              SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(controllerEpochToStore), curEpochZk),
+              CreateOp(ControllerZNode.path, ControllerZNode.encode(kraftControllerId, timestamp),
+                defaultAcls(ControllerZNode.path), CreateMode.PERSISTENT)))
+          )
+        }
+
+        val failureSuffix = s"while trying to register KRaft controller $kraftControllerId with epoch " +
+          s"$controllerEpochToStore. KRaft controller was not registered."
+        response.resultCode match {
+          case Code.OK =>
+            info(s"Successfully registered KRaft controller $kraftControllerId with epoch $controllerEpochToStore")
+            // First op is always SetData on /controller_epoch
+            val setDataResult = response.zkOpResults(0).rawOpResult.asInstanceOf[SetDataResult]
+            Some(setDataResult.getStat.getVersion)
+          case Code.BADVERSION =>
+            info(s"The controller epoch changed $failureSuffix")
+            None
+          case Code.NONODE =>
+            info(s"The ephemeral node at ${ControllerZNode.path} went away $failureSuffix")
+            None
+          case Code.NODEEXISTS =>
+            info(s"The ephemeral node at ${ControllerZNode.path} was created by another controller $failureSuffix")
+            None
+          case code =>
+            error(s"ZooKeeper had an error $failureSuffix")
+            throw KeeperException.create(code)
+        }
+    }
+  }
+
   private def maybeCreateControllerEpochZNode(): (Int, Int) = {
     createControllerEpochRaw(KafkaController.InitialControllerEpoch).resultCode match {
       case Code.OK =>
@@ -340,6 +418,24 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
     }
   }
 
+  def getEntitiesConfigs(rootEntityType: String, sanitizedEntityNames: Set[String]): Map[String, Properties] = {
+    val getDataRequests: Seq[GetDataRequest] = sanitizedEntityNames.map { entityName =>
+      GetDataRequest(ConfigEntityZNode.path(rootEntityType, entityName), Some(entityName))
+    }.toSeq
+
+    val getDataResponses = retryRequestsUntilConnected(getDataRequests)
+    getDataResponses.map { response =>
+      val entityName = response.ctx.get.asInstanceOf[String]
+      response.resultCode match {
+        case Code.OK =>
+          entityName -> ConfigEntityZNode.decode(response.data)
+        case Code.NONODE =>
+          entityName -> new Properties()
+        case _ => throw response.resultException.get
+      }
+    }.toMap
+  }
+
   /**
    * Sets or creates the entity znode path with the given configs depending
    * on whether it already exists or not.
@@ -1554,6 +1650,36 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
     }
   }
 
+  def getOrCreateMigrationState(initialState: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    val getDataRequest = GetDataRequest(MigrationZNode.path)
+    val getDataResponse = retryRequestUntilConnected(getDataRequest)
+    getDataResponse.resultCode match {
+      case Code.OK =>
+        MigrationZNode.decode(getDataResponse.data, getDataResponse.stat.getVersion, getDataResponse.stat.getMtime)
+      case Code.NONODE =>
+        createInitialMigrationState(initialState)
+      case _ => throw getDataResponse.resultException.get
+    }
+  }
+
+  def createInitialMigrationState(initialState: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    val createRequest = CreateRequest(
+      MigrationZNode.path,
+      MigrationZNode.encode(initialState),
+      defaultAcls(MigrationZNode.path),
+      CreateMode.PERSISTENT)
+    val response = retryRequestUntilConnected(createRequest)
+    response.maybeThrow()
+    initialState.withMigrationZkVersion(0)
+  }
+
+  def updateMigrationState(migrationState: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    val req = SetDataRequest(MigrationZNode.path, MigrationZNode.encode(migrationState), migrationState.migrationZkVersion())
+    val resp = retryRequestUntilConnected(req)
+    resp.maybeThrow()
+    migrationState.withMigrationZkVersion(resp.stat.getVersion)
+  }
+
   /**
     * Return the ACLs of the node of the given path
     * @param path the given path for the node
@@ -1772,6 +1898,137 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
     }
   }
 
+  /**
+   * Safely performs a sequence of writes to ZooKeeper as part of a KRaft migration. For each request in {@code requests}, we
+   * wrap the operation in a multi-op transaction that includes a check op on /controller_epoch and /migration. This ensures
+   * that another KRaft controller or another ZK controller has unexpectedly taken leadership.
+   *
+   * In cases of KRaft failover during a migration, it is possible that a write is attempted before the old KRaft controller
+   * receives the new leader information. In this case, the check op on /migration acts as a guard against multiple writers.
+   *
+   * The multi-op for the last request in {@code requests} is used to update the /migration node with the latest migration
+   * state. This effectively checkpoints the progress of the migration in ZK relative to the metadata log.
+   *
+   * Each multi-op request is atomic. The overall sequence of multi-op requests is not atomic and we may fail during any
+   * of them. When the KRaft controller recovers the migration state, it will re-apply all of the writes needed to update
+   * the ZK state with the latest KRaft state. In the case of Create or Delete operations, these will fail if applied
+   * twice, so we need to ignore NodeExists and NoNode failures for those cases.
+   *
+   * @param requests  A sequence of ZK requests. Only Create, Delete, and SetData are supported.
+   * @param migrationState The current migration state. This is written out as part of the final multi-op request.
+   * @return  The new version of /migration ZNode and the sequence of responses for the given requests.
+   */
+  def retryMigrationRequestsUntilConnected[Req <: AsyncRequest](requests: Seq[Req],
+                                                                migrationState: ZkMigrationLeadershipState): (Int, Seq[Req#Response]) = {
+
+    if (requests.isEmpty) {
+      return (migrationState.migrationZkVersion(), Seq.empty)
+    }
+
+    def wrapMigrationRequest(request: Req, lastRequestInBatch: Boolean): MultiRequest = {
+      // Wrap a single request with the multi-op transactional request.
+      val checkOp = CheckOp(ControllerEpochZNode.path, migrationState.controllerZkVersion())
+      val migrationOp = if (lastRequestInBatch) {
+        SetDataOp(MigrationZNode.path, MigrationZNode.encode(migrationState), migrationState.migrationZkVersion())
+      } else {
+        CheckOp(MigrationZNode.path, migrationState.migrationZkVersion())
+      }
+
+      request match {
+        case CreateRequest(path, data, acl, createMode, ctx) =>
+          MultiRequest(Seq(checkOp, migrationOp, CreateOp(path, data, acl, createMode)), ctx)
+        case DeleteRequest(path, version, ctx) =>
+          MultiRequest(Seq(checkOp, migrationOp, DeleteOp(path, version)), ctx)
+        case SetDataRequest(path, data, version, ctx) =>
+          MultiRequest(Seq(checkOp, migrationOp, SetDataOp(path, data, version)), ctx)
+        case _ => throw new IllegalStateException(s"$request does not need controller epoch check")
+      }
+    }
+
+    def handleUnwrappedMigrationResult(migrationOp: ZkOp, migrationResult: OpResult): Int = {
+      // Handle just the operation that updated /migration ZNode
+      val (path: String, data: Option[Array[Byte]], version: Int) = migrationOp match {
+        case CheckOp(path, version) => (path, None, version)
+        case SetDataOp(path, data, version) => (path, Some(data), version)
+        case _ => throw new IllegalStateException("Unexpected result on /migration znode")
+      }
+
+      migrationResult match {
+        case _: CheckResult => version
+        case setDataResult: SetDataResult => setDataResult.getStat.getVersion
+        case errorResult: ErrorResult =>
+          if (path.equals(MigrationZNode.path)) {
+            val errorCode = Code.get(errorResult.getErr)
+            if (errorCode == Code.BADVERSION) {
+              data match {
+                case Some(value) =>
+                  val failedPayload = MigrationZNode.decode(value, version, -1)
+                  throw new RuntimeException(
+                    s"Conditional update on KRaft Migration ZNode failed. Expected zkVersion = ${version}. The failed " +
+                    s"write was: ${failedPayload}. This indicates that another KRaft controller is making writes to ZooKeeper.")
+                case None =>
+                  throw new RuntimeException(s"Check op on KRaft Migration ZNode failed. Expected zkVersion = ${version}. " +
+                    s"This indicates that another KRaft controller is making writes to ZooKeeper.")
+              }
+            } else if (errorCode == Code.OK) {
+              // This means the Check or SetData op would have been ok, but failed because of another operation in this multi-op
+              version
+            } else {
+              throw KeeperException.create(errorCode, path)
+            }
+          } else {
+            throw new RuntimeException(s"Got migration result for incorrect path $path")
+          }
+        case _ => throw new RuntimeException(
+          s"Expected either CheckResult, SetDataResult, or ErrorResult for migration op, but saw ${migrationResult}")
+      }
+    }
+
+    def unwrapMigrationResponse(response: AsyncResponse, lastRequestInBatch: Boolean): (AsyncResponse, Int) = {
+      response match {
+        case MultiResponse(resultCode, _, ctx, zkOpResults, responseMetadata) =>
+        zkOpResults match {
+          case Seq(ZkOpResult(checkOp: CheckOp, checkOpResult), ZkOpResult(migrationOp: CheckOp, migrationResult), zkOpResult) =>
+            // Matches all requests except or the last one (CheckOp on /migration)
+            if (lastRequestInBatch) {
+              throw new IllegalStateException("Should not see a Check operation on /migration in the last request.")
+            }
+            handleUnwrappedCheckOp(checkOp, checkOpResult)
+            val migrationVersion = handleUnwrappedMigrationResult(migrationOp, migrationResult)
+            (handleUnwrappedZkOp(zkOpResult, resultCode, ctx, responseMetadata), migrationVersion)
+          case Seq(ZkOpResult(checkOp: CheckOp, checkOpResult), ZkOpResult(migrationOp: SetDataOp, migrationResult), zkOpResult) =>
+            // Matches the last request in a batch (SetDataOp on /migration)
+            if (!lastRequestInBatch) {
+              throw new IllegalStateException("Should only see a SetData operation on /migration in the last request.")
+            }
+            handleUnwrappedCheckOp(checkOp, checkOpResult)
+            val migrationVersion = handleUnwrappedMigrationResult(migrationOp, migrationResult)
+            (handleUnwrappedZkOp(zkOpResult, resultCode, ctx, responseMetadata), migrationVersion)
+          case null => throw KeeperException.create(resultCode)
+          case _ => throw new IllegalStateException(
+            s"Cannot unwrap $response because it does not contain the expected operations for a migration operation.")
+        }
+        case _ => throw new IllegalStateException(s"Cannot unwrap $response because it is not a MultiResponse")
+      }
+    }
+
+    migrationState.controllerZkVersion() match {
+      case ZkVersion.MatchAnyVersion => throw new IllegalArgumentException(
+        s"Expected a controller epoch zkVersion when making migration writes, not -1.")
+      case version if version >= 0 =>
+        logger.trace(s"Performing ${requests.size} migration update(s) with migrationState=$migrationState")
+        val wrappedRequests = requests.map(req => wrapMigrationRequest(req, req == requests.last))
+        val results = retryRequestsUntilConnected(wrappedRequests)
+        val unwrappedResults = results.map(resp => unwrapMigrationResponse(resp, resp == results.last))
+        val migrationZkVersion = unwrappedResults.last._2
+        // Return the new version of /migration and the sequence of responses to the original requests
+        (migrationZkVersion, unwrappedResults.map(_._1.asInstanceOf[Req#Response]))
+      case invalidVersion =>
+        throw new IllegalArgumentException(
+          s"Expected controller epoch zkVersion $invalidVersion should be non-negative or equal to ${ZkVersion.MatchAnyVersion}")
+    }
+  }
+
   private def retryRequestsUntilConnected[Req <: AsyncRequest](requests: Seq[Req]): Seq[Req#Response] = {
     val remainingRequests = new mutable.ArrayBuffer(requests.size) ++= requests
     val responses = new mutable.ArrayBuffer[Req#Response]
@@ -1997,6 +2254,45 @@ object KafkaZkClient {
       }
   }
 
+  private def handleUnwrappedCheckOp(checkOp: CheckOp, checkOpResult: OpResult): Unit = {
+    checkOpResult match {
+      case errorResult: ErrorResult =>
+        if (checkOp.path.equals(ControllerEpochZNode.path)) {
+          val errorCode = Code.get(errorResult.getErr)
+          if (errorCode == Code.BADVERSION)
+          // Throw ControllerMovedException when the zkVersionCheck is performed on the controller epoch znode and the check fails
+            throw new ControllerMovedException(s"Controller epoch zkVersion check fails. Expected zkVersion = ${checkOp.version}")
+          else if (errorCode != Code.OK)
+            throw KeeperException.create(errorCode, checkOp.path)
+        }
+      case _ =>
+    }
+  }
+
+  private def handleUnwrappedZkOp(zkOpResult: ZkOpResult,
+                                  resultCode: Code,
+                                  ctx: Option[Any],
+                                  responseMetadata: ResponseMetadata): AsyncResponse = {
+    val rawOpResult = zkOpResult.rawOpResult
+    zkOpResult.zkOp match {
+      case createOp: CreateOp =>
+        val name = rawOpResult match {
+          case c: CreateResult => c.getPath
+          case _ => null
+        }
+        CreateResponse(resultCode, createOp.path, ctx, name, responseMetadata)
+      case deleteOp: DeleteOp =>
+        DeleteResponse(resultCode, deleteOp.path, ctx, responseMetadata)
+      case setDataOp: SetDataOp =>
+        val stat = rawOpResult match {
+          case s: SetDataResult => s.getStat
+          case _ => null
+        }
+        SetDataResponse(resultCode, setDataOp.path, ctx, stat, responseMetadata)
+      case zkOp => throw new IllegalStateException(s"Unexpected zkOp: $zkOp")
+    }
+  }
+
   // A helper function to transform a MultiResponse with the check on
   // controller epoch znode zkVersion back into a regular response.
   // ControllerMovedException will be thrown if the controller epoch
@@ -2006,37 +2302,10 @@ object KafkaZkClient {
     response match {
       case MultiResponse(resultCode, _, ctx, zkOpResults, responseMetadata) =>
         zkOpResults match {
+          // In normal ZK writes, we just have a MultiOp with a CheckOp and the actual operation we're performing
           case Seq(ZkOpResult(checkOp: CheckOp, checkOpResult), zkOpResult) =>
-            checkOpResult match {
-              case errorResult: ErrorResult =>
-                if (checkOp.path.equals(ControllerEpochZNode.path)) {
-                  val errorCode = Code.get(errorResult.getErr)
-                  if (errorCode == Code.BADVERSION)
-                  // Throw ControllerMovedException when the zkVersionCheck is performed on the controller epoch znode and the check fails
-                    throw new ControllerMovedException(s"Controller epoch zkVersion check fails. Expected zkVersion = ${checkOp.version}")
-                  else if (errorCode != Code.OK)
-                    throw KeeperException.create(errorCode, checkOp.path)
-                }
-              case _ =>
-            }
-            val rawOpResult = zkOpResult.rawOpResult
-            zkOpResult.zkOp match {
-              case createOp: CreateOp =>
-                val name = rawOpResult match {
-                  case c: CreateResult => c.getPath
-                  case _ => null
-                }
-                CreateResponse(resultCode, createOp.path, ctx, name, responseMetadata)
-              case deleteOp: DeleteOp =>
-                DeleteResponse(resultCode, deleteOp.path, ctx, responseMetadata)
-              case setDataOp: SetDataOp =>
-                val stat = rawOpResult match {
-                  case s: SetDataResult => s.getStat
-                  case _ => null
-                }
-                SetDataResponse(resultCode, setDataOp.path, ctx, stat, responseMetadata)
-              case zkOp => throw new IllegalStateException(s"Unexpected zkOp: $zkOp")
-            }
+            handleUnwrappedCheckOp(checkOp, checkOpResult)
+            handleUnwrappedZkOp(zkOpResult, resultCode, ctx, responseMetadata)
           case null => throw KeeperException.create(resultCode)
           case _ => throw new IllegalStateException(s"Cannot unwrap $response because the first zookeeper op is not check op in original MultiRequest")
         }
diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala
index 6bd3b19d7dc..84b767d4d3b 100644
--- a/core/src/main/scala/kafka/zk/ZkData.scala
+++ b/core/src/main/scala/kafka/zk/ZkData.scala
@@ -1048,7 +1048,14 @@ object MigrationZNode {
       val controllerEpoch = js("kraft_controller_epoch").to[Int]
       val metadataOffset = js("kraft_metadata_offset").to[Long]
       val metadataEpoch = js("kraft_metadata_epoch").to[Int]
-      Some(new ZkMigrationLeadershipState(controllerId, controllerEpoch, metadataOffset, metadataEpoch, modifyTimeMs, zkVersion, -2))
+      Some(new ZkMigrationLeadershipState(
+        controllerId,
+        controllerEpoch,
+        metadataOffset,
+        metadataEpoch,
+        modifyTimeMs,
+        zkVersion,
+        ZkVersion.UnknownVersion))
     }.getOrElse(throw new KafkaException(s"Failed to parse the migration json $jsonDataAsString"))
   }
 }
diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
new file mode 100644
index 00000000000..77f46b9c794
--- /dev/null
+++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
@@ -0,0 +1,441 @@
+/**
+ * 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 kafka.zk
+
+import kafka.api.LeaderAndIsr
+import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
+import kafka.server.{ConfigEntityName, ConfigType, ZkAdminManager}
+import kafka.utils.Logging
+import kafka.zk.TopicZNode.TopicIdReplicaAssignment
+import kafka.zookeeper._
+import org.apache.kafka.common.config.ConfigResource
+import org.apache.kafka.common.errors.ControllerMovedException
+import org.apache.kafka.common.metadata.ClientQuotaRecord.EntityData
+import org.apache.kafka.common.metadata._
+import org.apache.kafka.common.quota.ClientQuotaEntity
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.metadata.{LeaderRecoveryState, PartitionRegistration}
+import org.apache.kafka.metadata.migration.{MigrationClient, ZkMigrationLeadershipState}
+import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion, ProducerIdsBlock}
+import org.apache.zookeeper.KeeperException.Code
+import org.apache.zookeeper.{CreateMode, KeeperException}
+
+import java.util
+import java.util.Properties
+import java.util.function.Consumer
+import scala.collection.Seq
+import scala.jdk.CollectionConverters._
+
+
+class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Logging {
+
+  override def getOrCreateMigrationRecoveryState(initialState: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    zkClient.createTopLevelPaths()
+    zkClient.getOrCreateMigrationState(initialState)
+  }
+
+  override def setMigrationRecoveryState(state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    zkClient.updateMigrationState(state)
+  }
+
+  override def claimControllerLeadership(state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    val epochZkVersionOpt = zkClient.tryRegisterKRaftControllerAsActiveController(
+      state.kraftControllerId(), state.kraftControllerEpoch())
+    if (epochZkVersionOpt.isDefined) {
+      state.withControllerZkVersion(epochZkVersionOpt.get)
+    } else {
+      state.withControllerZkVersion(-1)
+    }
+  }
+
+  override def releaseControllerLeadership(state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    try {
+      zkClient.deleteController(state.controllerZkVersion())
+      state.withControllerZkVersion(-1)
+    } catch {
+      case _: ControllerMovedException =>
+        // If the controller moved, no need to release
+        state.withControllerZkVersion(-1)
+      case t: Throwable =>
+        throw new RuntimeException("Could not release controller leadership due to underlying error", t)
+    }
+  }
+
+  def migrateTopics(metadataVersion: MetadataVersion,
+                    recordConsumer: Consumer[util.List[ApiMessageAndVersion]],
+                    brokerIdConsumer: Consumer[Integer]): Unit = {
+    val topics = zkClient.getAllTopicsInCluster()
+    val topicConfigs = zkClient.getEntitiesConfigs(ConfigType.Topic, topics)
+    val replicaAssignmentAndTopicIds = zkClient.getReplicaAssignmentAndTopicIdForTopics(topics)
+    replicaAssignmentAndTopicIds.foreach { case TopicIdReplicaAssignment(topic, topicIdOpt, partitionAssignments) =>
+      val partitions = partitionAssignments.keys.toSeq
+      val leaderIsrAndControllerEpochs = zkClient.getTopicPartitionStates(partitions)
+      val topicBatch = new util.ArrayList[ApiMessageAndVersion]()
+      topicBatch.add(new ApiMessageAndVersion(new TopicRecord()
+        .setName(topic)
+        .setTopicId(topicIdOpt.get), TopicRecord.HIGHEST_SUPPORTED_VERSION))
+
+      partitionAssignments.foreach { case (topicPartition, replicaAssignment) =>
+        replicaAssignment.replicas.foreach(brokerIdConsumer.accept(_))
+        replicaAssignment.addingReplicas.foreach(brokerIdConsumer.accept(_))
+        val replicaList = replicaAssignment.replicas.map(Integer.valueOf).asJava
+        val record = new PartitionRecord()
+          .setTopicId(topicIdOpt.get)
+          .setPartitionId(topicPartition.partition)
+          .setReplicas(replicaList)
+          .setAddingReplicas(replicaAssignment.addingReplicas.map(Integer.valueOf).asJava)
+          .setRemovingReplicas(replicaAssignment.removingReplicas.map(Integer.valueOf).asJava)
+        leaderIsrAndControllerEpochs.get(topicPartition) match {
+          case Some(leaderIsrAndEpoch) => record
+              .setIsr(leaderIsrAndEpoch.leaderAndIsr.isr.map(Integer.valueOf).asJava)
+              .setLeader(leaderIsrAndEpoch.leaderAndIsr.leader)
+              .setLeaderEpoch(leaderIsrAndEpoch.leaderAndIsr.leaderEpoch)
+              .setPartitionEpoch(leaderIsrAndEpoch.leaderAndIsr.partitionEpoch)
+              .setLeaderRecoveryState(leaderIsrAndEpoch.leaderAndIsr.leaderRecoveryState.value())
+          case None =>
+            warn(s"Could not find partition state in ZK for $topicPartition. Initializing this partition " +
+              s"with ISR={$replicaList} and leaderEpoch=0.")
+            record
+              .setIsr(replicaList)
+              .setLeader(replicaList.get(0))
+              .setLeaderEpoch(0)
+              .setPartitionEpoch(0)
+              .setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value())
+        }
+        topicBatch.add(new ApiMessageAndVersion(record, PartitionRecord.HIGHEST_SUPPORTED_VERSION))
+      }
+
+      val props = topicConfigs(topic)
+      props.forEach { case (key: Object, value: Object) =>
+        topicBatch.add(new ApiMessageAndVersion(new ConfigRecord()
+          .setResourceType(ConfigResource.Type.TOPIC.id)
+          .setResourceName(topic)
+          .setName(key.toString)
+          .setValue(value.toString), ConfigRecord.HIGHEST_SUPPORTED_VERSION))
+      }
+      recordConsumer.accept(topicBatch)
+    }
+  }
+
+  def migrateBrokerConfigs(metadataVersion: MetadataVersion,
+                           recordConsumer: Consumer[util.List[ApiMessageAndVersion]]): Unit = {
+    val brokerEntities = zkClient.getAllEntitiesWithConfig(ConfigType.Broker)
+    val batch = new util.ArrayList[ApiMessageAndVersion]()
+    zkClient.getEntitiesConfigs(ConfigType.Broker, brokerEntities.toSet).foreach { case (broker, props) =>
+      val brokerResource = if (broker == ConfigEntityName.Default) {
+        ""
+      } else {
+        broker
+      }
+      props.forEach { case (key: Object, value: Object) =>
+        batch.add(new ApiMessageAndVersion(new ConfigRecord()
+          .setResourceType(ConfigResource.Type.BROKER.id)
+          .setResourceName(brokerResource)
+          .setName(key.toString)
+          .setValue(value.toString), ConfigRecord.HIGHEST_SUPPORTED_VERSION))
+      }
+    }
+    if (!batch.isEmpty) {
+      recordConsumer.accept(batch)
+    }
+  }
+
+  def migrateClientQuotas(metadataVersion: MetadataVersion,
+                          recordConsumer: Consumer[util.List[ApiMessageAndVersion]]): Unit = {
+    val adminZkClient = new AdminZkClient(zkClient)
+
+    def migrateEntityType(entityType: String): Unit = {
+      adminZkClient.fetchAllEntityConfigs(entityType).foreach { case (name, props) =>
+        val entity = new EntityData().setEntityType(entityType).setEntityName(name)
+        val batch = new util.ArrayList[ApiMessageAndVersion]()
+        ZkAdminManager.clientQuotaPropsToDoubleMap(props.asScala).foreach { case (key: String, value: Double) =>
+          batch.add(new ApiMessageAndVersion(new ClientQuotaRecord()
+            .setEntity(List(entity).asJava)
+            .setKey(key)
+            .setValue(value), ClientQuotaRecord.HIGHEST_SUPPORTED_VERSION))
+        }
+        recordConsumer.accept(batch)
+      }
+    }
+
+    migrateEntityType(ConfigType.User)
+    migrateEntityType(ConfigType.Client)
+    adminZkClient.fetchAllChildEntityConfigs(ConfigType.User, ConfigType.Client).foreach { case (name, props) =>
+      // Taken from ZkAdminManager
+      val components = name.split("/")
+      if (components.size != 3 || components(1) != "clients")
+        throw new IllegalArgumentException(s"Unexpected config path: ${name}")
+      val entity = List(
+        new EntityData().setEntityType(ConfigType.User).setEntityName(components(0)),
+        new EntityData().setEntityType(ConfigType.Client).setEntityName(components(2))
+      )
+
+      val batch = new util.ArrayList[ApiMessageAndVersion]()
+      ZkAdminManager.clientQuotaPropsToDoubleMap(props.asScala).foreach { case (key: String, value: Double) =>
+        batch.add(new ApiMessageAndVersion(new ClientQuotaRecord()
+          .setEntity(entity.asJava)
+          .setKey(key)
+          .setValue(value), ClientQuotaRecord.HIGHEST_SUPPORTED_VERSION))
+      }
+      recordConsumer.accept(batch)
+    }
+
+    migrateEntityType(ConfigType.Ip)
+  }
+
+  def migrateProducerId(metadataVersion: MetadataVersion,
+                        recordConsumer: Consumer[util.List[ApiMessageAndVersion]]): Unit = {
+    val (dataOpt, _) = zkClient.getDataAndVersion(ProducerIdBlockZNode.path)
+    dataOpt match {
+      case Some(data) =>
+        val producerIdBlock = ProducerIdBlockZNode.parseProducerIdBlockData(data)
+        recordConsumer.accept(List(new ApiMessageAndVersion(new ProducerIdsRecord()
+          .setBrokerEpoch(-1)
+          .setBrokerId(producerIdBlock.assignedBrokerId)
+          .setNextProducerId(producerIdBlock.firstProducerId), ProducerIdsRecord.HIGHEST_SUPPORTED_VERSION)).asJava)
+      case None => // Nothing to migrate
+    }
+  }
+
+  override def readAllMetadata(batchConsumer: Consumer[util.List[ApiMessageAndVersion]],
+                               brokerIdConsumer: Consumer[Integer]): Unit = {
+    migrateTopics(MetadataVersion.latest(), batchConsumer, brokerIdConsumer)
+    migrateBrokerConfigs(MetadataVersion.latest(), batchConsumer)
+    migrateClientQuotas(MetadataVersion.latest(), batchConsumer)
+    migrateProducerId(MetadataVersion.latest(), batchConsumer)
+  }
+
+  override def readBrokerIds(): util.Set[Integer] = {
+    zkClient.getSortedBrokerList.map(Integer.valueOf).toSet.asJava
+  }
+
+  override def readBrokerIdsFromTopicAssignments(): util.Set[Integer] = {
+    val topics = zkClient.getAllTopicsInCluster()
+    val replicaAssignmentAndTopicIds = zkClient.getReplicaAssignmentAndTopicIdForTopics(topics)
+    val brokersWithAssignments = new util.HashSet[Integer]()
+    replicaAssignmentAndTopicIds.foreach { case TopicIdReplicaAssignment(_, _, assignments) =>
+      assignments.values.foreach { assignment =>
+        assignment.replicas.foreach { brokerId => brokersWithAssignments.add(brokerId) }
+      }
+    }
+    brokersWithAssignments
+  }
+
+  override def createTopic(topicName: String,
+                           topicId: Uuid,
+                           partitions: util.Map[Integer, PartitionRegistration],
+                           state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    val assignments = partitions.asScala.map { case (partitionId, partition) =>
+      new TopicPartition(topicName, partitionId) ->
+        ReplicaAssignment(partition.replicas, partition.addingReplicas, partition.removingReplicas)
+    }
+
+    val createTopicZNode = {
+      val path = TopicZNode.path(topicName)
+      CreateRequest(
+        path,
+        TopicZNode.encode(Some(topicId), assignments),
+        zkClient.defaultAcls(path),
+        CreateMode.PERSISTENT)
+    }
+    val createPartitionsZNode = {
+      val path = TopicPartitionsZNode.path(topicName)
+      CreateRequest(
+        path,
+        null,
+        zkClient.defaultAcls(path),
+        CreateMode.PERSISTENT)
+    }
+
+    val createPartitionZNodeReqs = partitions.asScala.flatMap { case (partitionId, partition) =>
+      val topicPartition = new TopicPartition(topicName, partitionId)
+      Seq(
+        createTopicPartition(topicPartition),
+        createTopicPartitionState(topicPartition, partition, state.kraftControllerEpoch())
+      )
+    }
+
+    val requests = Seq(createTopicZNode, createPartitionsZNode) ++ createPartitionZNodeReqs
+    val (migrationZkVersion, _) = zkClient.retryMigrationRequestsUntilConnected(requests, state)
+    state.withMigrationZkVersion(migrationZkVersion)
+  }
+
+  private def createTopicPartition(topicPartition: TopicPartition): CreateRequest = {
+    val path = TopicPartitionZNode.path(topicPartition)
+    CreateRequest(path, null, zkClient.defaultAcls(path), CreateMode.PERSISTENT, Some(topicPartition))
+  }
+
+  private def partitionStatePathAndData(topicPartition: TopicPartition,
+                                        partitionRegistration: PartitionRegistration,
+                                        controllerEpoch: Int): (String, Array[Byte]) = {
+    val path = TopicPartitionStateZNode.path(topicPartition)
+    val data = TopicPartitionStateZNode.encode(LeaderIsrAndControllerEpoch(new LeaderAndIsr(
+      partitionRegistration.leader,
+      partitionRegistration.leaderEpoch,
+      partitionRegistration.isr.toList,
+      partitionRegistration.leaderRecoveryState,
+      partitionRegistration.partitionEpoch), controllerEpoch))
+    (path, data)
+  }
+
+  private def createTopicPartitionState(topicPartition: TopicPartition,
+                                        partitionRegistration: PartitionRegistration,
+                                        controllerEpoch: Int): CreateRequest = {
+    val (path, data) = partitionStatePathAndData(topicPartition, partitionRegistration, controllerEpoch)
+    CreateRequest(path, data, zkClient.defaultAcls(path), CreateMode.PERSISTENT, Some(topicPartition))
+  }
+
+  private def updateTopicPartitionState(topicPartition: TopicPartition,
+                                        partitionRegistration: PartitionRegistration,
+                                        controllerEpoch: Int): SetDataRequest = {
+    val (path, data) = partitionStatePathAndData(topicPartition, partitionRegistration, controllerEpoch)
+    SetDataRequest(path, data, ZkVersion.MatchAnyVersion, Some(topicPartition))
+  }
+
+  override def updateTopicPartitions(topicPartitions: util.Map[String, util.Map[Integer, PartitionRegistration]],
+                                     state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    val requests = topicPartitions.asScala.flatMap { case (topicName, partitionRegistrations) =>
+      partitionRegistrations.asScala.flatMap { case (partitionId, partitionRegistration) =>
+        val topicPartition = new TopicPartition(topicName, partitionId)
+        Seq(updateTopicPartitionState(topicPartition, partitionRegistration, state.kraftControllerEpoch()))
+      }
+    }
+    if (requests.isEmpty) {
+      state
+    } else {
+      val (migrationZkVersion, _) = zkClient.retryMigrationRequestsUntilConnected(requests.toSeq, state)
+      state.withMigrationZkVersion(migrationZkVersion)
+    }
+  }
+
+  // Try to update an entity config and the migration state. If NoNode is encountered, it probably means we
+  // need to recursively create the parent ZNode. In this case, return None.
+  def tryWriteEntityConfig(entityType: String,
+                           path: String,
+                           props: Properties,
+                           create: Boolean,
+                           state: ZkMigrationLeadershipState): Option[ZkMigrationLeadershipState] = {
+    val configData = ConfigEntityZNode.encode(props)
+
+    val requests = if (create) {
+      Seq(CreateRequest(ConfigEntityZNode.path(entityType, path), configData, zkClient.defaultAcls(path), CreateMode.PERSISTENT))
+    } else {
+      Seq(SetDataRequest(ConfigEntityZNode.path(entityType, path), configData, ZkVersion.MatchAnyVersion))
+    }
+    val (migrationZkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(requests, state)
+    if (!create && responses.head.resultCode.equals(Code.NONODE)) {
+      // Not fatal. Just means we need to Create this node instead of SetData
+      None
+    } else if (responses.head.resultCode.equals(Code.OK)) {
+      Some(state.withMigrationZkVersion(migrationZkVersion))
+    } else {
+      throw KeeperException.create(responses.head.resultCode, path)
+    }
+  }
+
+  def writeClientQuotas(entity: ClientQuotaEntity,
+                        quotas: util.Map[String, Double],
+                        state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    val entityMap = entity.entries().asScala
+    val hasUser = entityMap.contains(ConfigType.User)
+    val hasClient = entityMap.contains(ConfigType.Client)
+    val hasIp = entityMap.contains(ConfigType.Ip)
+    val props = new Properties()
+    // We store client quota values as strings in the ZK JSON
+    quotas.forEach { case (key, value) => props.put(key, value.toString) }
+    val (configType, path) = if (hasUser && !hasClient) {
+      (Some(ConfigType.User), Some(entityMap(ConfigType.User)))
+    } else if (hasUser && hasClient) {
+      (Some(ConfigType.User), Some(s"${entityMap(ConfigType.User)}/clients/${entityMap(ConfigType.Client)}"))
+    } else if (hasClient) {
+      (Some(ConfigType.Client), Some(entityMap(ConfigType.Client)))
+    } else if (hasIp) {
+      (Some(ConfigType.Ip), Some(entityMap(ConfigType.Ip)))
+    } else {
+      (None, None)
+    }
+
+    if (path.isEmpty) {
+      error(s"Skipping unknown client quota entity $entity")
+      return state
+    }
+
+    // Try to write the client quota configs once with create=false, and again with create=true if the first operation fails
+    tryWriteEntityConfig(configType.get, path.get, props, create=false, state) match {
+      case Some(newState) =>
+        newState
+      case None =>
+        // If we didn't update the migration state, we failed to write the client quota. Try again
+        // after recursively create its parent znodes
+        val createPath = if (hasUser && hasClient) {
+          s"${ConfigEntityTypeZNode.path(configType.get)}/${entityMap(ConfigType.User)}/clients"
+        } else {
+          ConfigEntityTypeZNode.path(configType.get)
+        }
+        zkClient.createRecursive(createPath, throwIfPathExists=false)
+        debug(s"Recursively creating ZNode $createPath and attempting to write $entity quotas a second time.")
+
+        tryWriteEntityConfig(configType.get, path.get, props, create=true, state) match {
+          case Some(newStateSecondTry) => newStateSecondTry
+          case None => throw new RuntimeException(
+            s"Could not write client quotas for $entity on second attempt when using Create instead of SetData")
+        }
+    }
+  }
+
+  def writeProducerId(nextProducerId: Long, state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    val newProducerIdBlockData = ProducerIdBlockZNode.generateProducerIdBlockJson(
+      new ProducerIdsBlock(-1, nextProducerId, ProducerIdsBlock.PRODUCER_ID_BLOCK_SIZE))
+
+    val request = SetDataRequest(ProducerIdBlockZNode.path, newProducerIdBlockData, ZkVersion.MatchAnyVersion)
+    val (migrationZkVersion, _) = zkClient.retryMigrationRequestsUntilConnected(Seq(request), state)
+    state.withMigrationZkVersion(migrationZkVersion)
+  }
+
+  def writeConfigs(resource: ConfigResource,
+                   configs: util.Map[String, String],
+                   state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
+    val configType = resource.`type`() match {
+      case ConfigResource.Type.BROKER => Some(ConfigType.Broker)
+      case ConfigResource.Type.TOPIC => Some(ConfigType.Topic)
+      case _ => None
+    }
+
+    val configName = resource.name()
+    if (configType.isDefined) {
+      val props = new Properties()
+      configs.forEach { case (key, value) => props.put(key, value) }
+      tryWriteEntityConfig(configType.get, configName, props, create=false, state) match {
+        case Some(newState) =>
+          newState
+        case None =>
+          val createPath = ConfigEntityTypeZNode.path(configType.get)
+          debug(s"Recursively creating ZNode $createPath and attempting to write $resource configs a second time.")
+          zkClient.createRecursive(createPath, throwIfPathExists=false)
+
+          tryWriteEntityConfig(configType.get, configName, props, create=true, state) match {
+            case Some(newStateSecondTry) => newStateSecondTry
+            case None => throw new RuntimeException(
+              s"Could not write ${configType.get} configs on second attempt when using Create instead of SetData.")
+          }
+      }
+    } else {
+      debug(s"Not updating ZK for $resource since it is not a Broker or Topic entity.")
+      state
+    }
+  }
+}
diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
new file mode 100644
index 00000000000..54b1156ccb1
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
@@ -0,0 +1,105 @@
+/**
+ * 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 kafka.zk
+
+import kafka.test.ClusterInstance
+import kafka.test.annotation.{ClusterTest, Type}
+import kafka.test.junit.ClusterTestExtensions
+import kafka.test.junit.ZkClusterInvocationContext.ZkClusterInstance
+import org.apache.kafka.clients.admin.NewTopic
+import org.apache.kafka.common.config.TopicConfig
+import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity}
+import org.apache.kafka.image.{MetadataDelta, MetadataImage}
+import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
+import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull}
+import org.junit.jupiter.api.extension.ExtendWith
+
+import java.util
+import java.util.concurrent.TimeUnit
+import scala.jdk.CollectionConverters._
+
+@ExtendWith(value = Array(classOf[ClusterTestExtensions]))
+class ZkMigrationIntegrationTest {
+
+  class MetadataDeltaVerifier {
+    val metadataDelta = new MetadataDelta(MetadataImage.EMPTY)
+    var offset = 0
+    def accept(batch: java.util.List[ApiMessageAndVersion]): Unit = {
+      batch.forEach(message => {
+        metadataDelta.replay(offset, 0, message.message())
+        offset += 1
+      })
+    }
+
+    def verify(verifier: MetadataImage => Unit): Unit = {
+      val image = metadataDelta.apply()
+      verifier.apply(image)
+    }
+  }
+
+  @ClusterTest(brokers = 3, clusterType = Type.ZK, metadataVersion = MetadataVersion.IBP_3_4_IV0)
+  def testMigrate(clusterInstance: ClusterInstance): Unit = {
+    val admin = clusterInstance.createAdminClient()
+    val newTopics = new util.ArrayList[NewTopic]()
+    newTopics.add(new NewTopic("test-topic-1", 2, 3.toShort)
+      .configs(Map(TopicConfig.SEGMENT_BYTES_CONFIG -> "102400", TopicConfig.SEGMENT_MS_CONFIG -> "300000").asJava))
+    newTopics.add(new NewTopic("test-topic-2", 1, 3.toShort))
+    newTopics.add(new NewTopic("test-topic-3", 10, 3.toShort))
+    val createTopicResult = admin.createTopics(newTopics)
+    createTopicResult.all().get(60, TimeUnit.SECONDS)
+
+    val quotas = new util.ArrayList[ClientQuotaAlteration]()
+    quotas.add(new ClientQuotaAlteration(
+      new ClientQuotaEntity(Map("user" -> "user1").asJava),
+      List(new ClientQuotaAlteration.Op("consumer_byte_rate", 1000.0)).asJava))
+    quotas.add(new ClientQuotaAlteration(
+      new ClientQuotaEntity(Map("user" -> "user1", "client-id" -> "clientA").asJava),
+      List(new ClientQuotaAlteration.Op("consumer_byte_rate", 800.0), new ClientQuotaAlteration.Op("producer_byte_rate", 100.0)).asJava))
+    quotas.add(new ClientQuotaAlteration(
+      new ClientQuotaEntity(Map("ip" -> "8.8.8.8").asJava),
+      List(new ClientQuotaAlteration.Op("connection_creation_rate", 10.0)).asJava))
+    admin.alterClientQuotas(quotas)
+
+    val zkClient = clusterInstance.asInstanceOf[ZkClusterInstance].getUnderlying().zkClient
+    val migrationClient = new ZkMigrationClient(zkClient)
+    var migrationState = migrationClient.getOrCreateMigrationRecoveryState(ZkMigrationLeadershipState.EMPTY)
+    migrationState = migrationState.withNewKRaftController(3000, 42)
+    migrationState = migrationClient.claimControllerLeadership(migrationState)
+
+    val brokers = new java.util.HashSet[Integer]()
+    val verifier = new MetadataDeltaVerifier()
+    migrationClient.readAllMetadata(batch => verifier.accept(batch), brokerId => brokers.add(brokerId))
+    assertEquals(Seq(0, 1, 2), brokers.asScala.toSeq)
+
+    verifier.verify { image =>
+      assertNotNull(image.topics().getTopic("test-topic-1"))
+      assertEquals(2, image.topics().getTopic("test-topic-1").partitions().size())
+
+      assertNotNull(image.topics().getTopic("test-topic-2"))
+      assertEquals(1, image.topics().getTopic("test-topic-2").partitions().size())
+
+      assertNotNull(image.topics().getTopic("test-topic-3"))
+      assertEquals(10, image.topics().getTopic("test-topic-3").partitions().size())
+
+      val clientQuotas = image.clientQuotas().entities()
+      assertEquals(3, clientQuotas.size())
+    }
+
+    migrationState = migrationClient.releaseControllerLeadership(migrationState)
+  }
+}
diff --git a/core/src/test/scala/unit/kafka/server/ZkAdminManagerTest.scala b/core/src/test/scala/unit/kafka/server/ZkAdminManagerTest.scala
index 2057fda2c52..4fe64622d79 100644
--- a/core/src/test/scala/unit/kafka/server/ZkAdminManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ZkAdminManagerTest.scala
@@ -18,7 +18,6 @@
 package kafka.server
 
 import java.util.Properties
-
 import kafka.server.metadata.ZkConfigRepository
 import kafka.utils.TestUtils
 import kafka.zk.{AdminZkClient, KafkaZkClient}
@@ -28,10 +27,7 @@ import org.apache.kafka.common.message.DescribeConfigsRequestData
 import org.apache.kafka.common.message.DescribeConfigsResponseData
 import org.apache.kafka.common.protocol.Errors
 import org.junit.jupiter.api.{AfterEach, Test}
-import org.junit.jupiter.api.Assertions.assertEquals
-import org.junit.jupiter.api.Assertions.assertFalse
-import org.junit.jupiter.api.Assertions.assertNotNull
-import org.junit.jupiter.api.Assertions.assertNotEquals
+import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertThrows}
 import org.mockito.Mockito.{mock, when}
 
 import scala.jdk.CollectionConverters._
@@ -54,6 +50,28 @@ class ZkAdminManagerTest {
     new ConfigHelper(metadataCache, KafkaConfig.fromProps(props), new ZkConfigRepository(new AdminZkClient(zkClient)))
   }
 
+  @Test
+  def testClientQuotasToProps(): Unit = {
+    val emptyProps = ZkAdminManager.clientQuotaPropsToDoubleMap(Map.empty)
+    assertEquals(0, emptyProps.size)
+
+    val oneProp = ZkAdminManager.clientQuotaPropsToDoubleMap(Map("foo" -> "1234"))
+    assertEquals(1, oneProp.size)
+    assertEquals(1234.0, oneProp("foo"))
+
+    // This is probably not desired, but kept for compatability with existing usages
+    val emptyKey = ZkAdminManager.clientQuotaPropsToDoubleMap(Map("" -> "-42.1"))
+    assertEquals(1, emptyKey.size)
+    assertEquals(-42.1, emptyKey(""))
+
+    val manyProps = ZkAdminManager.clientQuotaPropsToDoubleMap(Map("foo" -> "1234", "bar" -> "0", "spam" -> "-1234.56"))
+    assertEquals(3, manyProps.size)
+
+    assertThrows(classOf[NullPointerException], () => ZkAdminManager.clientQuotaPropsToDoubleMap(Map("foo" -> null)))
+    assertThrows(classOf[IllegalStateException], () => ZkAdminManager.clientQuotaPropsToDoubleMap(Map("foo" -> "bar")))
+    assertThrows(classOf[IllegalStateException], () => ZkAdminManager.clientQuotaPropsToDoubleMap(Map("foo" -> "")))
+  }
+
   @Test
   def testDescribeConfigsWithNullConfigurationKeys(): Unit = {
     when(zkClient.getEntityConfigs(ConfigType.Topic, topic)).thenReturn(TestUtils.createBrokerConfig(brokerId, "zk"))
diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala
index 7b7ddfbc56f..11eae3386f8 100644
--- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala
@@ -19,7 +19,6 @@ package kafka.zk
 import java.nio.charset.StandardCharsets.UTF_8
 import java.util.concurrent.{CountDownLatch, TimeUnit}
 import java.util.{Collections, Properties}
-
 import kafka.api.LeaderAndIsr
 import kafka.cluster.{Broker, EndPoint}
 import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
@@ -43,9 +42,10 @@ import org.apache.kafka.common.security.token.delegation.TokenInformation
 import org.apache.kafka.common.utils.{SecurityUtils, Time}
 import org.apache.kafka.common.{TopicPartition, Uuid}
 import org.apache.kafka.metadata.LeaderRecoveryState
+import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
 import org.apache.kafka.server.common.MetadataVersion
 import org.apache.zookeeper.KeeperException.{Code, NoAuthException, NoNodeException, NodeExistsException}
-import org.apache.zookeeper.ZooDefs
+import org.apache.zookeeper.{CreateMode, ZooDefs}
 import org.apache.zookeeper.client.ZKClientConfig
 import org.apache.zookeeper.common.ZKConfig
 import org.apache.zookeeper.data.Stat
@@ -1373,6 +1373,53 @@ class KafkaZkClientTest extends QuorumTestHarness {
     } finally System.clearProperty(ZKConfig.JUTE_MAXBUFFER)
   }
 
+  @Test
+  def testFailToUpdateMigrationZNode(): Unit = {
+    val (_, stat) = zkClient.getControllerEpoch.get
+    var migrationState = new ZkMigrationLeadershipState(3000, 42, 100, 42, Time.SYSTEM.milliseconds(), -1, stat.getVersion)
+    migrationState = zkClient.getOrCreateMigrationState(migrationState)
+    assertEquals(0, migrationState.migrationZkVersion())
+
+    // A batch of migration writes to make. The last one will fail causing the migration znode to not be updated
+    val requests_bad = Seq(
+      CreateRequest("/foo", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT),
+      CreateRequest("/foo/bar", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT),
+      CreateRequest("/foo/bar/spam", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT),
+      CreateRequest("/foo", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT),
+    )
+
+    migrationState = migrationState.withControllerZkVersion(stat.getVersion)
+    zkClient.retryMigrationRequestsUntilConnected(requests_bad, migrationState) match {
+      case (zkVersion: Int, requests: Seq[AsyncRequest#Response]) =>
+        assertEquals(0, zkVersion)
+        assert(requests.take(3).forall(resp => resp.resultCode.equals(Code.OK)))
+        assertEquals(Code.NODEEXISTS, requests.last.resultCode)
+      case _ => fail()
+    }
+
+    // Check state again
+    val loadedState = zkClient.getOrCreateMigrationState(ZkMigrationLeadershipState.EMPTY)
+    assertEquals(0, loadedState.migrationZkVersion())
+
+    // Resend the same requests, with the last one succeeding this time. This will result in NODEEXISTS, but
+    // should still update the migration state
+    val requests_good = Seq(
+      CreateRequest("/foo", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT),
+      CreateRequest("/foo/bar", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT),
+      CreateRequest("/foo/bar/spam", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT),
+      CreateRequest("/foo/bar/eggs", Array(), zkClient.defaultAcls("/foo"), CreateMode.PERSISTENT),
+    )
+
+    migrationState = migrationState.withControllerZkVersion(stat.getVersion)
+    zkClient.retryMigrationRequestsUntilConnected(requests_good, migrationState) match {
+      case (zkVersion: Int, requests: Seq[AsyncRequest#Response]) =>
+        assertEquals(1, zkVersion)
+        assert(requests.take(3).forall(resp => resp.resultCode.equals(Code.NODEEXISTS)))
+        assertEquals(Code.OK, requests.last.resultCode)
+      case _ => fail()
+    }
+  }
+
   class ExpiredKafkaZkClient private (zooKeeperClient: ZooKeeperClient, isSecure: Boolean, time: Time)
     extends KafkaZkClient(zooKeeperClient, isSecure, time) {
     // Overwriting this method from the parent class to force the client to re-register the Broker.
diff --git a/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala
new file mode 100644
index 00000000000..7fae24f650e
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala
@@ -0,0 +1,353 @@
+/**
+ * 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 kafka.zk
+
+import kafka.api.LeaderAndIsr
+import kafka.controller.LeaderIsrAndControllerEpoch
+import kafka.coordinator.transaction.ProducerIdManager
+import kafka.server.{ConfigType, QuorumTestHarness, ZkAdminManager}
+import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
+import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.config.internals.QuotaConfigs
+import org.apache.kafka.common.errors.ControllerMovedException
+import org.apache.kafka.common.metadata.{ConfigRecord, MetadataRecordType, ProducerIdsRecord}
+import org.apache.kafka.common.quota.ClientQuotaEntity
+import org.apache.kafka.common.utils.Time
+import org.apache.kafka.metadata.{LeaderRecoveryState, PartitionRegistration}
+import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
+import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue, fail}
+import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
+
+import java.util.Properties
+import scala.collection.Map
+import scala.jdk.CollectionConverters._
+
+/**
+ * ZooKeeper integration tests that verify the interoperability of KafkaZkClient and ZkMigrationClient.
+ */
+class ZkMigrationClientTest extends QuorumTestHarness {
+
+  private var migrationClient: ZkMigrationClient = _
+
+  private var migrationState: ZkMigrationLeadershipState = _
+
+  @BeforeEach
+  override def setUp(testInfo: TestInfo): Unit = {
+    super.setUp(testInfo)
+    zkClient.createControllerEpochRaw(1)
+
+    migrationClient = new ZkMigrationClient(zkClient)
+    migrationState = initialMigrationState
+    migrationState = migrationClient.getOrCreateMigrationRecoveryState(migrationState)
+   }
+
+  private def initialMigrationState: ZkMigrationLeadershipState = {
+    val (_, stat) = zkClient.getControllerEpoch.get
+    new ZkMigrationLeadershipState(3000, 42, 100, 42, Time.SYSTEM.milliseconds(), -1, stat.getVersion)
+  }
+
+  @Test
+  def testMigrateEmptyZk(): Unit = {
+    val brokers = new java.util.ArrayList[Integer]()
+    val batches = new java.util.ArrayList[java.util.List[ApiMessageAndVersion]]()
+
+    migrationClient.readAllMetadata(batch => batches.add(batch), brokerId => brokers.add(brokerId))
+    assertEquals(0, brokers.size())
+    assertEquals(0, batches.size())
+  }
+
+  @Test
+  def testEmptyWrite(): Unit = {
+    val (zkVersion, responses) = zkClient.retryMigrationRequestsUntilConnected(Seq(), migrationState)
+    assertEquals(migrationState.migrationZkVersion(), zkVersion)
+    assertTrue(responses.isEmpty)
+  }
+
+  @Test
+  def testUpdateExistingPartitions(): Unit = {
+    // Create a topic and partition state in ZK like KafkaController would
+    val assignment = Map(
+      new TopicPartition("test", 0) -> List(0, 1, 2),
+      new TopicPartition("test", 1) -> List(1, 2, 3)
+    )
+    zkClient.createTopicAssignment("test", Some(Uuid.randomUuid()), assignment)
+
+    val leaderAndIsrs = Map(
+      new TopicPartition("test", 0) -> LeaderIsrAndControllerEpoch(
+        new LeaderAndIsr(0, 5, List(0, 1, 2), LeaderRecoveryState.RECOVERED, -1), 1),
+      new TopicPartition("test", 1) -> LeaderIsrAndControllerEpoch(
+        new LeaderAndIsr(1, 5, List(1, 2, 3), LeaderRecoveryState.RECOVERED, -1), 1)
+    )
+    zkClient.createTopicPartitionStatesRaw(leaderAndIsrs, 0)
+
+    // Now verify that we can update it with migration client
+    assertEquals(0, migrationState.migrationZkVersion())
+
+    val partitions = Map(
+      0 -> new PartitionRegistration(Array(0, 1, 2), Array(1, 2), Array(), Array(), 1, LeaderRecoveryState.RECOVERED, 6, -1),
+      1 -> new PartitionRegistration(Array(1, 2, 3), Array(3), Array(), Array(), 3, LeaderRecoveryState.RECOVERED, 7, -1)
+    ).map { case (k, v) => Integer.valueOf(k) -> v }.asJava
+    migrationState = migrationClient.updateTopicPartitions(Map("test" -> partitions).asJava, migrationState)
+    assertEquals(1, migrationState.migrationZkVersion())
+
+    // Read back with Zk client
+    val partition0 = zkClient.getTopicPartitionState(new TopicPartition("test", 0)).get.leaderAndIsr
+    assertEquals(1, partition0.leader)
+    assertEquals(6, partition0.leaderEpoch)
+    assertEquals(List(1, 2), partition0.isr)
+
+    val partition1 = zkClient.getTopicPartitionState(new TopicPartition("test", 1)).get.leaderAndIsr
+    assertEquals(3, partition1.leader)
+    assertEquals(7, partition1.leaderEpoch)
+    assertEquals(List(3), partition1.isr)
+  }
+
+  @Test
+  def testCreateNewPartitions(): Unit = {
+    assertEquals(0, migrationState.migrationZkVersion())
+
+    val partitions = Map(
+      0 -> new PartitionRegistration(Array(0, 1, 2), Array(0, 1, 2), Array(), Array(), 0, LeaderRecoveryState.RECOVERED, 0, -1),
+      1 -> new PartitionRegistration(Array(1, 2, 3), Array(1, 2, 3), Array(), Array(), 1, LeaderRecoveryState.RECOVERED, 0, -1)
+    ).map { case (k, v) => Integer.valueOf(k) -> v }.asJava
+    migrationState = migrationClient.createTopic("test", Uuid.randomUuid(), partitions, migrationState)
+    assertEquals(1, migrationState.migrationZkVersion())
+
+    // Read back with Zk client
+    val partition0 = zkClient.getTopicPartitionState(new TopicPartition("test", 0)).get.leaderAndIsr
+    assertEquals(0, partition0.leader)
+    assertEquals(0, partition0.leaderEpoch)
+    assertEquals(List(0, 1, 2), partition0.isr)
+
+    val partition1 = zkClient.getTopicPartitionState(new TopicPartition("test", 1)).get.leaderAndIsr
+    assertEquals(1, partition1.leader)
+    assertEquals(0, partition1.leaderEpoch)
+    assertEquals(List(1, 2, 3), partition1.isr)
+  }
+
+  // Write Client Quotas using ZkMigrationClient and read them back using AdminZkClient
+  private def writeClientQuotaAndVerify(migrationClient: ZkMigrationClient,
+                                        adminZkClient: AdminZkClient,
+                                        migrationState: ZkMigrationLeadershipState,
+                                        entity: Map[String, String],
+                                        quotas: Map[String, Double],
+                                        zkEntityType: String,
+                                        zkEntityName: String): ZkMigrationLeadershipState = {
+    val nextMigrationState = migrationClient.writeClientQuotas(
+      new ClientQuotaEntity(entity.asJava),
+      quotas.asJava,
+      migrationState)
+    val newProps = ZkAdminManager.clientQuotaPropsToDoubleMap(
+      adminZkClient.fetchEntityConfig(zkEntityType, zkEntityName).asScala)
+    assertEquals(quotas, newProps)
+    nextMigrationState
+  }
+
+
+  @Test
+  def testWriteExistingClientQuotas(): Unit = {
+    val props = new Properties()
+    props.put(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, "100000")
+    adminZkClient.changeConfigs(ConfigType.User, "user1", props)
+    adminZkClient.changeConfigs(ConfigType.User, "user1/clients/clientA", props)
+
+    assertEquals(0, migrationState.migrationZkVersion())
+    migrationState = writeClientQuotaAndVerify(migrationClient, adminZkClient, migrationState,
+      Map(ConfigType.User -> "user1"),
+      Map(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG -> 20000.0),
+      ConfigType.User, "user1")
+    assertEquals(1, migrationState.migrationZkVersion())
+
+    migrationState = writeClientQuotaAndVerify(migrationClient, adminZkClient, migrationState,
+      Map(ConfigType.User -> "user1"),
+      Map(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG -> 10000.0),
+      ConfigType.User, "user1")
+    assertEquals(2, migrationState.migrationZkVersion())
+
+    migrationState = writeClientQuotaAndVerify(migrationClient, adminZkClient, migrationState,
+      Map(ConfigType.User -> "user1"),
+      Map.empty,
+      ConfigType.User, "user1")
+    assertEquals(3, migrationState.migrationZkVersion())
+
+    migrationState = writeClientQuotaAndVerify(migrationClient, adminZkClient, migrationState,
+      Map(ConfigType.User -> "user1"),
+      Map(QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG -> 100.0),
+      ConfigType.User, "user1")
+    assertEquals(4, migrationState.migrationZkVersion())
+  }
+
+  @Test
+  def testWriteNewClientQuotas(): Unit = {
+    assertEquals(0, migrationState.migrationZkVersion())
+    migrationState = writeClientQuotaAndVerify(migrationClient, adminZkClient, migrationState,
+      Map(ConfigType.User -> "user2"),
+      Map(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG -> 20000.0, QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG -> 100.0),
+      ConfigType.User, "user2")
+
+    assertEquals(1, migrationState.migrationZkVersion())
+
+    migrationState = writeClientQuotaAndVerify(migrationClient, adminZkClient, migrationState,
+      Map(ConfigType.User -> "user2", ConfigType.Client -> "clientA"),
+      Map(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG -> 10000.0, QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG -> 200.0),
+      ConfigType.User, "user2/clients/clientA")
+
+    assertEquals(2, migrationState.migrationZkVersion())
+  }
+
+  @Test
+  def testClaimAbsentController(): Unit = {
+    assertEquals(0, migrationState.migrationZkVersion())
+    migrationState = migrationClient.claimControllerLeadership(migrationState)
+    assertEquals(1, migrationState.controllerZkVersion())
+  }
+
+  @Test
+  def testExistingKRaftControllerClaim(): Unit = {
+    assertEquals(0, migrationState.migrationZkVersion())
+    migrationState = migrationClient.claimControllerLeadership(migrationState)
+    assertEquals(1, migrationState.controllerZkVersion())
+
+    // We don't require a KRaft controller to release the controller in ZK before another KRaft controller
+    // can claim it. This is because KRaft leadership comes from Raft and we are just synchronizing it to ZK.
+    var otherNodeState = new ZkMigrationLeadershipState(3001, 43, 100, 42, Time.SYSTEM.milliseconds(), -1, -1)
+    otherNodeState = migrationClient.claimControllerLeadership(otherNodeState)
+    assertEquals(2, otherNodeState.controllerZkVersion())
+    assertEquals(3001, otherNodeState.kraftControllerId())
+    assertEquals(43, otherNodeState.kraftControllerEpoch())
+  }
+
+  @Test
+  def testNonIncreasingKRaftEpoch(): Unit = {
+    assertEquals(0, migrationState.migrationZkVersion())
+
+    migrationState = migrationClient.claimControllerLeadership(migrationState)
+    assertEquals(1, migrationState.controllerZkVersion())
+
+    migrationState = migrationState.withNewKRaftController(3000, 40)
+    val t1 = assertThrows(classOf[IllegalStateException], () => migrationClient.claimControllerLeadership(migrationState))
+    assertEquals("Cannot register KRaft controller 3000 as the active controller in ZK since its epoch 10000040 is not higher than the current ZK epoch 10000042.", t1.getMessage)
+
+    migrationState = migrationState.withNewKRaftController(3000, 42)
+    val t2 = assertThrows(classOf[IllegalStateException], () => migrationClient.claimControllerLeadership(migrationState))
+    assertEquals("Cannot register KRaft controller 3000 as the active controller in ZK since its epoch 10000042 is not higher than the current ZK epoch 10000042.", t2.getMessage)
+  }
+
+  @Test
+  def testClaimAndReleaseExistingController(): Unit = {
+    assertEquals(0, migrationState.migrationZkVersion())
+
+    val (epoch, zkVersion) = zkClient.registerControllerAndIncrementControllerEpoch(100)
+    assertEquals(epoch, 2)
+    assertEquals(zkVersion, 1)
+
+    migrationState = migrationClient.claimControllerLeadership(migrationState)
+    assertEquals(2, migrationState.controllerZkVersion())
+    zkClient.getControllerEpoch match {
+      case Some((kraftEpoch, stat)) =>
+        assertEquals(10000042, kraftEpoch)
+        assertEquals(2, stat.getVersion)
+      case None => fail()
+    }
+    assertEquals(3000, zkClient.getControllerId.get)
+    assertThrows(classOf[ControllerMovedException], () => zkClient.registerControllerAndIncrementControllerEpoch(100))
+
+    migrationState = migrationClient.releaseControllerLeadership(migrationState)
+    val (epoch1, zkVersion1) = zkClient.registerControllerAndIncrementControllerEpoch(100)
+    assertEquals(epoch1, 10000043)
+    assertEquals(zkVersion1, 3)
+  }
+
+  @Test
+  def testReadAndWriteProducerId(): Unit = {
+    def generateNextProducerIdWithZkAndRead(): Long = {
+      // Generate a producer ID in ZK
+      val manager = ProducerIdManager.zk(1, zkClient)
+      manager.generateProducerId()
+
+      val records = new java.util.ArrayList[java.util.List[ApiMessageAndVersion]]()
+      migrationClient.migrateProducerId(MetadataVersion.latest(), batch => records.add(batch))
+      assertEquals(1, records.size())
+      assertEquals(1, records.get(0).size())
+
+      val record = records.get(0).get(0).message().asInstanceOf[ProducerIdsRecord]
+      record.nextProducerId()
+    }
+
+    // Initialize with ZK ProducerIdManager
+    assertEquals(0, generateNextProducerIdWithZkAndRead())
+
+    // Update next producer ID via migration client
+    migrationState = migrationClient.writeProducerId(6000, migrationState)
+    assertEquals(1, migrationState.migrationZkVersion())
+
+    // Switch back to ZK, it should provision the next block
+    assertEquals(7000, generateNextProducerIdWithZkAndRead())
+  }
+
+  @Test
+  def testMigrateTopicConfigs(): Unit = {
+    val props = new Properties()
+    props.put(TopicConfig.FLUSH_MS_CONFIG, "60000")
+    props.put(TopicConfig.RETENTION_MS_CONFIG, "300000")
+    adminZkClient.createTopicWithAssignment("test", props, Map(0 -> Seq(0, 1, 2), 1 -> Seq(1, 2, 0), 2 -> Seq(2, 0, 1)), usesTopicId = true)
+
+    val brokers = new java.util.ArrayList[Integer]()
+    val batches = new java.util.ArrayList[java.util.List[ApiMessageAndVersion]]()
+    migrationClient.migrateTopics(MetadataVersion.latest(), batch => batches.add(batch), brokerId => brokers.add(brokerId))
+    assertEquals(1, batches.size())
+    val configs = batches.get(0)
+      .asScala
+      .map {_.message()}
+      .filter(message => MetadataRecordType.fromId(message.apiKey()).equals(MetadataRecordType.CONFIG_RECORD))
+      .map {_.asInstanceOf[ConfigRecord]}
+      .toSeq
+    assertEquals(2, configs.size)
+    assertEquals(TopicConfig.FLUSH_MS_CONFIG, configs.head.name())
+    assertEquals("60000", configs.head.value())
+    assertEquals(TopicConfig.RETENTION_MS_CONFIG, configs.last.name())
+    assertEquals("300000", configs.last.value())
+  }
+
+  @Test
+  def testWriteNewTopicConfigs(): Unit = {
+    migrationState = migrationClient.writeConfigs(new ConfigResource(ConfigResource.Type.TOPIC, "test"),
+      java.util.Collections.singletonMap(TopicConfig.SEGMENT_MS_CONFIG, "100000"), migrationState)
+    assertEquals(1, migrationState.migrationZkVersion())
+
+    val newProps = zkClient.getEntityConfigs(ConfigType.Topic, "test")
+    assertEquals(1, newProps.size())
+    assertEquals("100000", newProps.getProperty(TopicConfig.SEGMENT_MS_CONFIG))
+  }
+
+  @Test
+  def testWriteExistingTopicConfigs(): Unit = {
+    val props = new Properties()
+    props.put(TopicConfig.FLUSH_MS_CONFIG, "60000")
+    props.put(TopicConfig.RETENTION_MS_CONFIG, "300000")
+    zkClient.setOrCreateEntityConfigs(ConfigType.Topic, "test", props)
+
+    migrationState = migrationClient.writeConfigs(new ConfigResource(ConfigResource.Type.TOPIC, "test"),
+      java.util.Collections.singletonMap(TopicConfig.SEGMENT_MS_CONFIG, "100000"), migrationState)
+    assertEquals(1, migrationState.migrationZkVersion())
+
+    val newProps = zkClient.getEntityConfigs(ConfigType.Topic, "test")
+    assertEquals(1, newProps.size())
+    assertEquals("100000", newProps.getProperty(TopicConfig.SEGMENT_MS_CONFIG))
+  }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java b/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java
index bbdc1bf9779..3eb3e3c4e95 100644
--- a/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java
+++ b/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java
@@ -62,7 +62,8 @@ public final class ClientQuotasImage {
         return entities.isEmpty();
     }
 
-    Map<ClientQuotaEntity, ClientQuotaImage> entities() {
+    // Visible for testing
+    public Map<ClientQuotaEntity, ClientQuotaImage> entities() {
         return entities;
     }
 
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java
new file mode 100644
index 00000000000..5eccbc70625
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/MigrationClient.java
@@ -0,0 +1,91 @@
+/*
+ * 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.migration;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.metadata.PartitionRegistration;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+
+/**
+ * Methods for interacting with ZooKeeper during a KIP-866 migration. The migration leadership state is stored under
+ * a ZNode /migration. All write operations to ZK during the migration are performed as a multi-op transaction which
+ * also updates the state of /migration.
+ */
+public interface MigrationClient {
+
+    /**
+     * Read or initialize the ZK migration leader state in ZK. If the ZNode is absent, the given {@code initialState}
+     * will be written and subsequently returned with the zkVersion of the node. If the ZNode is present, it will be
+     * read and returned.
+     * @param initialState  An initial, emtpy, state to write to ZooKeeper for the migration state.
+     * @return  The existing migration state, or the initial state given.
+     */
+    ZkMigrationLeadershipState getOrCreateMigrationRecoveryState(ZkMigrationLeadershipState initialState);
+
+    /**
+     * Overwrite the migration state in ZK. This is done as a conditional update using
+     * {@link ZkMigrationLeadershipState#migrationZkVersion()}. If the conditional update fails, an exception is thrown.
+     * @param state The migration state to persist
+     * @return  The persisted migration state or an exception.
+     */
+    ZkMigrationLeadershipState setMigrationRecoveryState(ZkMigrationLeadershipState state);
+
+    /**
+     * Attempt to claim controller leadership of the cluster in ZooKeeper. This involves overwriting the /controller
+     * and /controller_epoch ZNodes. The epoch given by {@code state} must be greater than the current epoch in ZooKeeper.
+     *
+     *
+     * @param state The current migration leadership state
+     * @return      An updated migration leadership state including the version of /controller_epoch ZNode, if the
+     *              leadership claim was successful. Otherwise, return the previous state unmodified.
+     */
+    ZkMigrationLeadershipState claimControllerLeadership(ZkMigrationLeadershipState state);
+
+    /**
+     * Release an existing claim on the cluster leadership in ZooKeeper. This involves deleting the /controller ZNode
+     * so that another controller can claim leadership.
+     *
+     * @param state The current migration leadership state.
+     * @return      An updated migration leadership state with controllerZkVersion = 1, or raise an exception if ZooKeeper
+     *
+     *
+     */
+    ZkMigrationLeadershipState releaseControllerLeadership(ZkMigrationLeadershipState state);
+
+    ZkMigrationLeadershipState createTopic(
+        String topicName,
+        Uuid topicId,
+        Map<Integer, PartitionRegistration> topicPartitions,
+        ZkMigrationLeadershipState state
+    );
+
+    ZkMigrationLeadershipState updateTopicPartitions(
+        Map<String, Map<Integer, PartitionRegistration>> topicPartitions,
+        ZkMigrationLeadershipState state
+    );
+
+    void readAllMetadata(Consumer<List<ApiMessageAndVersion>> batchConsumer, Consumer<Integer> brokerIdConsumer);
+
+    Set<Integer> readBrokerIds();
+
+    Set<Integer> readBrokerIdsFromTopicAssignments();
+}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java
index c54cf7d0e4f..b6217ee80c2 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/ZkMigrationLeadershipState.java
@@ -53,16 +53,22 @@ public class ZkMigrationLeadershipState {
         this.controllerZkVersion = controllerZkVersion;
     }
 
-    public ZkMigrationLeadershipState withZkVersion(int zkVersion) {
+    public ZkMigrationLeadershipState withMigrationZkVersion(int zkVersion) {
         return new ZkMigrationLeadershipState(
-                this.kraftControllerId, this.kraftControllerEpoch, this.kraftMetadataOffset,
-                this.kraftMetadataEpoch, this.lastUpdatedTimeMs, zkVersion, this.controllerZkVersion);
+            this.kraftControllerId, this.kraftControllerEpoch, this.kraftMetadataOffset,
+            this.kraftMetadataEpoch, this.lastUpdatedTimeMs, zkVersion, this.controllerZkVersion);
+    }
+
+    public ZkMigrationLeadershipState withControllerZkVersion(int zkVersion) {
+        return new ZkMigrationLeadershipState(
+            this.kraftControllerId, this.kraftControllerEpoch, this.kraftMetadataOffset,
+            this.kraftMetadataEpoch, this.lastUpdatedTimeMs, this.migrationZkVersion, zkVersion);
     }
 
     public ZkMigrationLeadershipState withNewKRaftController(int controllerId, int controllerEpoch) {
         return new ZkMigrationLeadershipState(
-                controllerId, controllerEpoch, this.kraftMetadataOffset,
-                this.kraftMetadataEpoch, this.lastUpdatedTimeMs, this.migrationZkVersion, this.controllerZkVersion);
+            controllerId, controllerEpoch, this.kraftMetadataOffset,
+            this.kraftMetadataEpoch, this.lastUpdatedTimeMs, this.migrationZkVersion, this.controllerZkVersion);
     }
 
     public int kraftControllerId() {
@@ -100,14 +106,14 @@ public class ZkMigrationLeadershipState {
     @Override
     public String toString() {
         return "ZkMigrationLeadershipState{" +
-                "kraftControllerId=" + kraftControllerId +
-                ", kraftControllerEpoch=" + kraftControllerEpoch +
-                ", kraftMetadataOffset=" + kraftMetadataOffset +
-                ", kraftMetadataEpoch=" + kraftMetadataEpoch +
-                ", lastUpdatedTimeMs=" + lastUpdatedTimeMs +
-                ", migrationZkVersion=" + migrationZkVersion +
-                ", controllerZkVersion=" + controllerZkVersion +
-                '}';
+            "kraftControllerId=" + kraftControllerId +
+            ", kraftControllerEpoch=" + kraftControllerEpoch +
+            ", kraftMetadataOffset=" + kraftMetadataOffset +
+            ", kraftMetadataEpoch=" + kraftMetadataEpoch +
+            ", lastUpdatedTimeMs=" + lastUpdatedTimeMs +
+            ", migrationZkVersion=" + migrationZkVersion +
+            ", controllerZkVersion=" + controllerZkVersion +
+            '}';
     }
 
     @Override
@@ -115,11 +121,24 @@ public class ZkMigrationLeadershipState {
         if (this == o) return true;
         if (o == null || getClass() != o.getClass()) return false;
         ZkMigrationLeadershipState that = (ZkMigrationLeadershipState) o;
-        return kraftControllerId == that.kraftControllerId && kraftControllerEpoch == that.kraftControllerEpoch && kraftMetadataOffset == that.kraftMetadataOffset && kraftMetadataEpoch == that.kraftMetadataEpoch && lastUpdatedTimeMs == that.lastUpdatedTimeMs && migrationZkVersion == that.migrationZkVersion && controllerZkVersion == that.controllerZkVersion;
+        return kraftControllerId == that.kraftControllerId
+            && kraftControllerEpoch == that.kraftControllerEpoch
+            && kraftMetadataOffset == that.kraftMetadataOffset
+            && kraftMetadataEpoch == that.kraftMetadataEpoch
+            && lastUpdatedTimeMs == that.lastUpdatedTimeMs
+            && migrationZkVersion == that.migrationZkVersion
+            && controllerZkVersion == that.controllerZkVersion;
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(kraftControllerId, kraftControllerEpoch, kraftMetadataOffset, kraftMetadataEpoch, lastUpdatedTimeMs, migrationZkVersion, controllerZkVersion);
+        return Objects.hash(
+            kraftControllerId,
+            kraftControllerEpoch,
+            kraftMetadataOffset,
+            kraftMetadataEpoch,
+            lastUpdatedTimeMs,
+            migrationZkVersion,
+            controllerZkVersion);
     }
 }


[kafka] 05/11: MINOR: Introduce MetadataProvenance and ImageReWriter (#12964)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 17e6bba20b47835e832eb59a150af49a68d1fff0
Author: Colin Patrick McCabe <cm...@apache.org>
AuthorDate: Mon Dec 12 09:52:06 2022 -0800

    MINOR: Introduce MetadataProvenance and ImageReWriter (#12964)
    
    Introduce MetadataProvenance to encapsulate the three-tuple of (offset, epoch, timestamp) that is
    associated with each MetadataImage, as well as each on-disk snapshot. Also introduce a builder
    for MetadataDelta.
    
    Remove offset and epoch tracking from MetadataDelta. We do not really need to know this information
    until we are creating the final MetadataImage object. Therefore, this bookkeeping should be done by
    the metadata loading code, not inside the delta code, like the other bookkeeping. This simplifies a
    lot of tests, as well as simplifying RecordTestUtils.  It also makes more sense for snapshots, where
    the offset and epoch are the same for every record.
    
    Add ImageReWriter, an ImageWriter that applies records to a MetadataDelta. This is useful when you
    need to create a MetadataDelta object that holds the contents of a MetadataImage. This will be
    used in the new image loader code (coming soon).
    
    Add ImageWriterOptionsTest to test ImageWriterOptions.
    
    Reviewers: David Arthur <mu...@gmail.com>
---
 .../server/metadata/BrokerMetadataListener.scala   | 19 +++--
 .../kafka/zk/ZkMigrationIntegrationTest.scala      |  6 +-
 .../unit/kafka/server/MetadataCacheTest.scala      | 25 +++---
 .../server/ReplicaManagerConcurrencyTest.scala     |  8 +-
 .../unit/kafka/server/ReplicaManagerTest.scala     |  5 +-
 .../metadata/BrokerMetadataSnapshotterTest.scala   | 18 +++--
 .../java/org/apache/kafka/image/MetadataDelta.java | 51 +++++-------
 .../java/org/apache/kafka/image/MetadataImage.java | 26 ++++--
 .../org/apache/kafka/image/MetadataProvenance.java | 92 ++++++++++++++++++++++
 .../apache/kafka/image/MetadataVersionChange.java  | 77 ++++++++++++++++++
 .../image/MetadataVersionChangeException.java      | 37 +++++++++
 .../apache/kafka/image/writer/ImageReWriter.java   | 59 ++++++++++++++
 .../kafka/image/writer/ImageWriterOptions.java     | 17 +++-
 .../org/apache/kafka/image/ImageDowngradeTest.java |  4 +-
 .../org/apache/kafka/image/MetadataImageTest.java  | 34 ++++----
 .../kafka/image/MetadataVersionChangeTest.java     | 63 +++++++++++++++
 .../kafka/image/writer/ImageReWriterTest.java      | 63 +++++++++++++++
 .../kafka/image/writer/ImageWriterOptionsTest.java | 52 ++++++++++++
 .../org/apache/kafka/metadata/RecordTestUtils.java | 48 -----------
 19 files changed, 558 insertions(+), 146 deletions(-)

diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
index 064f89e5917..21fc126691f 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
@@ -22,13 +22,14 @@ import java.util.concurrent.{CompletableFuture, TimeUnit}
 import kafka.metrics.KafkaMetricsGroup
 import org.apache.kafka.common.utils.{LogContext, Time}
 import org.apache.kafka.image.writer.{ImageWriterOptions, RecordListWriter}
-import org.apache.kafka.image.{MetadataDelta, MetadataImage}
+import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance}
 import org.apache.kafka.metadata.util.SnapshotReason
 import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
 import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient}
 import org.apache.kafka.server.common.ApiMessageAndVersion
 import org.apache.kafka.server.fault.FaultHandler
 import org.apache.kafka.snapshot.SnapshotReader
+
 import scala.compat.java8.OptionConverters._
 
 
@@ -79,11 +80,19 @@ class BrokerMetadataListener(
    */
   @volatile var _highestOffset = -1L
 
+  /**
+   * The highest metadata epoch that we've seen.  Written only from the event queue thread.
+   */
+  private var _highestEpoch = -1
+
   /**
    * The highest metadata log time that we've seen. Written only from the event queue thread.
    */
   private var _highestTimestamp = -1L
 
+  private def provenance(): MetadataProvenance =
+    new MetadataProvenance(_highestOffset, _highestEpoch, _highestTimestamp)
+
   /**
    * The current broker metadata image. Accessed only from the event queue thread.
    */
@@ -181,7 +190,7 @@ class BrokerMetadataListener(
     snapshotter.foreach { snapshotter =>
       if (metadataFaultOccurred.get()) {
         trace("Not starting metadata snapshot since we previously had an error")
-      } else if (snapshotter.maybeStartSnapshot(_highestTimestamp, _delta.apply(), reason)) {
+      } else if (snapshotter.maybeStartSnapshot(_highestTimestamp, _delta.apply(provenance()), reason)) {
         _bytesSinceLastSnapshot = 0L
       }
     }
@@ -259,7 +268,7 @@ class BrokerMetadataListener(
     while (iterator.hasNext) {
       val batch = iterator.next()
 
-      val epoch = lastCommittedEpoch.getOrElse(batch.epoch())
+      _highestEpoch = lastCommittedEpoch.getOrElse(batch.epoch())
       _highestTimestamp = lastAppendTimestamp.getOrElse(batch.appendTimestamp())
 
       var index = 0
@@ -270,7 +279,7 @@ class BrokerMetadataListener(
         }
         _highestOffset = lastCommittedOffset.getOrElse(batch.baseOffset() + index)
         try {
-          delta.replay(highestMetadataOffset, epoch, messageAndVersion.message())
+          delta.replay(messageAndVersion.message())
         } catch {
           case e: Throwable => snapshotName match {
             case None => metadataLoadingFaultHandler.handleFault(
@@ -341,7 +350,7 @@ class BrokerMetadataListener(
   private def publish(publisher: MetadataPublisher): Unit = {
     val delta = _delta
     try {
-      _image = _delta.apply()
+      _image = _delta.apply(provenance())
     } catch {
       case t: Throwable =>
         // If we cannot apply the delta, this publish event will throw and we will not publish a new image.
diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
index 54b1156ccb1..d9b2e286956 100644
--- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
@@ -23,7 +23,7 @@ import kafka.test.junit.ZkClusterInvocationContext.ZkClusterInstance
 import org.apache.kafka.clients.admin.NewTopic
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity}
-import org.apache.kafka.image.{MetadataDelta, MetadataImage}
+import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance}
 import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
 import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
 import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull}
@@ -41,13 +41,13 @@ class ZkMigrationIntegrationTest {
     var offset = 0
     def accept(batch: java.util.List[ApiMessageAndVersion]): Unit = {
       batch.forEach(message => {
-        metadataDelta.replay(offset, 0, message.message())
+        metadataDelta.replay(message.message())
         offset += 1
       })
     }
 
     def verify(verifier: MetadataImage => Unit): Unit = {
-      val image = metadataDelta.apply()
+      val image = metadataDelta.apply(new MetadataProvenance(offset, 0, 0))
       verifier.apply(image)
     }
   }
diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
index d92c76f7118..7dadd5bf759 100644
--- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
+++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
@@ -32,9 +32,8 @@ import org.apache.kafka.common.requests.UpdateMetadataRequest
 import org.apache.kafka.common.security.auth.SecurityProtocol
 import org.apache.kafka.common.metadata.{BrokerRegistrationChangeRecord, PartitionRecord, RegisterBrokerRecord, RemoveTopicRecord, TopicRecord}
 import org.apache.kafka.common.metadata.RegisterBrokerRecord.{BrokerEndpoint, BrokerEndpointCollection}
-import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage}
+import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage, MetadataProvenance}
 import org.apache.kafka.server.common.MetadataVersion
-import org.apache.kafka.raft.{OffsetAndEpoch => RaftOffsetAndEpoch}
 
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.params.ParameterizedTest
@@ -65,7 +64,7 @@ object MetadataCacheTest {
         // contains no brokers, but which contains the previous partitions.
         val image = c.currentImage()
         val partialImage = new MetadataImage(
-          new RaftOffsetAndEpoch(100, 10),
+          new MetadataProvenance(100L, 10, 1000L),
           image.features(),
           ClusterImage.EMPTY,
           image.topics(),
@@ -73,7 +72,7 @@ object MetadataCacheTest {
           image.clientQuotas(),
           image.producerIds(),
           image.acls())
-        val delta = new MetadataDelta(partialImage)
+        val delta = new MetadataDelta.Builder().setImage(partialImage).build()
 
         def toRecord(broker: UpdateMetadataBroker): RegisterBrokerRecord = {
           val endpoints = new BrokerEndpointCollection()
@@ -100,7 +99,7 @@ object MetadataCacheTest {
             setFenced(fenced)
         }
         request.liveBrokers().iterator().asScala.foreach { brokerInfo =>
-          delta.replay(100, 10, toRecord(brokerInfo))
+          delta.replay(toRecord(brokerInfo))
         }
 
         def toRecords(topic: UpdateMetadataTopicState): Seq[ApiMessage] = {
@@ -125,9 +124,9 @@ object MetadataCacheTest {
           results
         }
         request.topicStates().forEach { topic =>
-          toRecords(topic).foreach(delta.replay(100, 10, _))
+          toRecords(topic).foreach(delta.replay)
         }
-        c.setImage(delta.apply())
+        c.setImage(delta.apply(new MetadataProvenance(100L, 10, 1000L)))
       }
       case _ => throw new RuntimeException("Unsupported cache type")
     }
@@ -646,12 +645,12 @@ class MetadataCacheTest {
   def testIsBrokerFenced(): Unit = {
     val metadataCache = MetadataCache.kRaftMetadataCache(0)
 
-    val delta = new MetadataDelta(MetadataImage.EMPTY)
+    val delta = new MetadataDelta.Builder().build()
     delta.replay(new RegisterBrokerRecord()
       .setBrokerId(0)
       .setFenced(false))
 
-    metadataCache.setImage(delta.apply())
+    metadataCache.setImage(delta.apply(MetadataProvenance.EMPTY))
 
     assertFalse(metadataCache.isBrokerFenced(0))
 
@@ -659,7 +658,7 @@ class MetadataCacheTest {
       .setBrokerId(0)
       .setFenced(1.toByte))
 
-    metadataCache.setImage(delta.apply())
+    metadataCache.setImage(delta.apply(MetadataProvenance.EMPTY))
 
     assertTrue(metadataCache.isBrokerFenced(0))
   }
@@ -668,12 +667,12 @@ class MetadataCacheTest {
   def testIsBrokerInControlledShutdown(): Unit = {
     val metadataCache = MetadataCache.kRaftMetadataCache(0)
 
-    val delta = new MetadataDelta(MetadataImage.EMPTY)
+    val delta = new MetadataDelta.Builder().build()
     delta.replay(new RegisterBrokerRecord()
       .setBrokerId(0)
       .setInControlledShutdown(false))
 
-    metadataCache.setImage(delta.apply())
+    metadataCache.setImage(delta.apply(MetadataProvenance.EMPTY))
 
     assertFalse(metadataCache.isBrokerShuttingDown(0))
 
@@ -681,7 +680,7 @@ class MetadataCacheTest {
       .setBrokerId(0)
       .setInControlledShutdown(1.toByte))
 
-    metadataCache.setImage(delta.apply())
+    metadataCache.setImage(delta.apply(MetadataProvenance.EMPTY))
 
     assertTrue(metadataCache.isBrokerShuttingDown(0))
   }
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala
index 651451afad4..559485ff4dd 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala
@@ -349,7 +349,7 @@ class ReplicaManagerConcurrencyTest {
     override def doWork(): Unit = {
       channel.poll() match {
         case InitializeEvent =>
-          val delta = new MetadataDelta(latestImage)
+          val delta = new MetadataDelta.Builder().setImage(latestImage).build()
           brokerIds.foreach { brokerId =>
             delta.replay(new RegisterBrokerRecord()
               .setBrokerId(brokerId)
@@ -357,14 +357,14 @@ class ReplicaManagerConcurrencyTest {
             )
           }
           topic.initialize(delta)
-          latestImage = delta.apply()
+          latestImage = delta.apply(latestImage.provenance())
           metadataCache.setImage(latestImage)
           replicaManager.applyDelta(delta.topicsDelta, latestImage)
 
         case AlterIsrEvent(future, topicPartition, leaderAndIsr) =>
-          val delta = new MetadataDelta(latestImage)
+          val delta = new MetadataDelta.Builder().setImage(latestImage).build()
           val updatedLeaderAndIsr = topic.alterIsr(topicPartition, leaderAndIsr, delta)
-          latestImage = delta.apply()
+          latestImage = delta.apply(latestImage.provenance())
           future.complete(updatedLeaderAndIsr)
           replicaManager.applyDelta(delta.topicsDelta, latestImage)
 
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 3d28f885b27..3273824ffa4 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -54,10 +54,9 @@ import org.apache.kafka.common.requests._
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.common.utils.{LogContext, Time, Utils}
 import org.apache.kafka.common.{IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid}
-import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, ConfigurationsImage, FeaturesImage, MetadataImage, ProducerIdsImage, TopicsDelta, TopicsImage}
+import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, ConfigurationsImage, FeaturesImage, MetadataImage, MetadataProvenance, ProducerIdsImage, TopicsDelta, TopicsImage}
 import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
 import org.apache.kafka.metadata.LeaderRecoveryState
-import org.apache.kafka.raft.{OffsetAndEpoch => RaftOffsetAndEpoch}
 import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
@@ -4111,7 +4110,7 @@ class ReplicaManagerTest {
 
   private def imageFromTopics(topicsImage: TopicsImage): MetadataImage = {
     new MetadataImage(
-      new RaftOffsetAndEpoch(100, 10),
+      new MetadataProvenance(100L, 10, 1000L),
       FeaturesImage.EMPTY,
       ClusterImageTest.IMAGE1,
       topicsImage,
diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala
index 6849a0923b3..de3c71b04e8 100644
--- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala
+++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataSnapshotterTest.scala
@@ -24,7 +24,7 @@ import org.apache.kafka.common.memory.MemoryPool
 import org.apache.kafka.common.protocol.ByteBufferAccessor
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords}
 import org.apache.kafka.common.utils.Time
-import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataImageTest}
+import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataImageTest, MetadataProvenance}
 import org.apache.kafka.metadata.MetadataRecordSerde
 import org.apache.kafka.metadata.util.SnapshotReason
 import org.apache.kafka.queue.EventQueue
@@ -55,7 +55,7 @@ class BrokerMetadataSnapshotterTest {
       RecordsSnapshotWriter.createWithHeader(
         () => {
           Optional.of(
-            new MockRawSnapshotWriter(offsetAndEpoch, consumeSnapshotBuffer(committedOffset, committedEpoch))
+            new MockRawSnapshotWriter(offsetAndEpoch, consumeSnapshotBuffer(committedOffset, committedEpoch, lastContainedLogTime))
           )
         },
         1024,
@@ -67,7 +67,11 @@ class BrokerMetadataSnapshotterTest {
       ).asScala
     }
 
-    def consumeSnapshotBuffer(committedOffset: Long, committedEpoch: Int)(buffer: ByteBuffer): Unit = {
+    def consumeSnapshotBuffer(
+      committedOffset: Long,
+      committedEpoch: Int,
+      lastContainedLogTime: Long
+    )(buffer: ByteBuffer): Unit = {
       val delta = new MetadataDelta(MetadataImage.EMPTY)
       val memoryRecords = MemoryRecords.readableRecords(buffer)
       val batchIterator = memoryRecords.batchIterator()
@@ -78,11 +82,11 @@ class BrokerMetadataSnapshotterTest {
             val recordBuffer = record.value().duplicate()
             val messageAndVersion = MetadataRecordSerde.INSTANCE.read(
               new ByteBufferAccessor(recordBuffer), recordBuffer.remaining())
-            delta.replay(committedOffset, committedEpoch, messageAndVersion.message())
+            delta.replay(messageAndVersion.message())
           })
         }
       }
-      image.complete(delta.apply())
+      image.complete(delta.apply(new MetadataProvenance(committedOffset, committedEpoch, lastContainedLogTime)))
     }
   }
 
@@ -101,8 +105,8 @@ class BrokerMetadataSnapshotterTest {
       val reasons = Set(SnapshotReason.UNKNOWN)
 
       snapshotter.eventQueue.append(blockingEvent)
-      assertTrue(snapshotter.maybeStartSnapshot(10000L, MetadataImageTest.IMAGE1, reasons))
-      assertFalse(snapshotter.maybeStartSnapshot(11000L, MetadataImageTest.IMAGE2, reasons))
+      assertTrue(snapshotter.maybeStartSnapshot(2000L, MetadataImageTest.IMAGE1, reasons))
+      assertFalse(snapshotter.maybeStartSnapshot(4000L, MetadataImageTest.IMAGE2, reasons))
       blockingEvent.latch.countDown()
       assertEquals(MetadataImageTest.IMAGE1, writerBuilder.image.get())
     } finally {
diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java b/metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java
index a82216769d0..3d5ee9821ae 100644
--- a/metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java
+++ b/metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java
@@ -34,26 +34,29 @@ 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.raft.OffsetAndEpoch;
-import org.apache.kafka.server.common.ApiMessageAndVersion;
 import org.apache.kafka.server.common.MetadataVersion;
 
-import java.util.Iterator;
-import java.util.List;
 import java.util.Optional;
 
 
 /**
  * A change to the broker metadata image.
- *
- * This class is thread-safe.
  */
 public final class MetadataDelta {
-    private final MetadataImage image;
+    public static class Builder {
+        private MetadataImage image = MetadataImage.EMPTY;
+
+        public Builder setImage(MetadataImage image) {
+            this.image = image;
+            return this;
+        }
 
-    private long highestOffset;
+        public MetadataDelta build() {
+            return new MetadataDelta(image);
+        }
+    }
 
-    private int highestEpoch;
+    private final MetadataImage image;
 
     private FeaturesDelta featuresDelta = null;
 
@@ -71,8 +74,6 @@ public final class MetadataDelta {
 
     public MetadataDelta(MetadataImage image) {
         this.image = image;
-        this.highestOffset = image.highestOffsetAndEpoch().offset();
-        this.highestEpoch = image.highestOffsetAndEpoch().epoch();
     }
 
     public MetadataImage image() {
@@ -152,19 +153,7 @@ public final class MetadataDelta {
         }
     }
 
-    public void read(long highestOffset, int highestEpoch, Iterator<List<ApiMessageAndVersion>> reader) {
-        while (reader.hasNext()) {
-            List<ApiMessageAndVersion> batch = reader.next();
-            for (ApiMessageAndVersion messageAndVersion : batch) {
-                replay(highestOffset, highestEpoch, messageAndVersion.message());
-            }
-        }
-    }
-
-    public void replay(long offset, int epoch, ApiMessage record) {
-        highestOffset = offset;
-        highestEpoch = epoch;
-
+    public void replay(ApiMessage record) {
         MetadataRecordType type = MetadataRecordType.fromId(record.apiKey());
         switch (type) {
             case REGISTER_BROKER_RECORD:
@@ -223,13 +212,11 @@ public final class MetadataDelta {
     }
 
     public void replay(RegisterBrokerRecord record) {
-        if (clusterDelta == null) clusterDelta = new ClusterDelta(image.cluster());
-        clusterDelta.replay(record);
+        getOrCreateClusterDelta().replay(record);
     }
 
     public void replay(UnregisterBrokerRecord record) {
-        if (clusterDelta == null) clusterDelta = new ClusterDelta(image.cluster());
-        clusterDelta.replay(record);
+        getOrCreateClusterDelta().replay(record);
     }
 
     public void replay(TopicRecord record) {
@@ -308,7 +295,7 @@ public final class MetadataDelta {
         getOrCreateAclsDelta().finishSnapshot();
     }
 
-    public MetadataImage apply() {
+    public MetadataImage apply(MetadataProvenance provenance) {
         FeaturesImage newFeatures;
         if (featuresDelta == null) {
             newFeatures = image.features();
@@ -352,7 +339,7 @@ public final class MetadataDelta {
             newAcls = aclsDelta.apply();
         }
         return new MetadataImage(
-            new OffsetAndEpoch(highestOffset, highestEpoch),
+            provenance,
             newFeatures,
             newCluster,
             newTopics,
@@ -366,9 +353,7 @@ public final class MetadataDelta {
     @Override
     public String toString() {
         return "MetadataDelta(" +
-            "highestOffset=" + highestOffset +
-            ", highestEpoch=" + highestEpoch +
-            ", featuresDelta=" + featuresDelta +
+            "featuresDelta=" + featuresDelta +
             ", clusterDelta=" + clusterDelta +
             ", topicsDelta=" + topicsDelta +
             ", configsDelta=" + configsDelta +
diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java
index 18c836ee3bd..36db5dab1de 100644
--- a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java
+++ b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java
@@ -31,7 +31,7 @@ import java.util.Objects;
  */
 public final class MetadataImage {
     public final static MetadataImage EMPTY = new MetadataImage(
-        new OffsetAndEpoch(0, 0),
+        MetadataProvenance.EMPTY,
         FeaturesImage.EMPTY,
         ClusterImage.EMPTY,
         TopicsImage.EMPTY,
@@ -40,7 +40,7 @@ public final class MetadataImage {
         ProducerIdsImage.EMPTY,
         AclsImage.EMPTY);
 
-    private final OffsetAndEpoch highestOffsetAndEpoch;
+    private final MetadataProvenance provenance;
 
     private final FeaturesImage features;
 
@@ -57,7 +57,7 @@ public final class MetadataImage {
     private final AclsImage acls;
 
     public MetadataImage(
-        OffsetAndEpoch highestOffsetAndEpoch,
+        MetadataProvenance provenance,
         FeaturesImage features,
         ClusterImage cluster,
         TopicsImage topics,
@@ -66,7 +66,7 @@ public final class MetadataImage {
         ProducerIdsImage producerIds,
         AclsImage acls
     ) {
-        this.highestOffsetAndEpoch = highestOffsetAndEpoch;
+        this.provenance = provenance;
         this.features = features;
         this.cluster = cluster;
         this.topics = topics;
@@ -86,8 +86,16 @@ public final class MetadataImage {
             acls.isEmpty();
     }
 
+    public MetadataProvenance provenance() {
+        return provenance;
+    }
+
     public OffsetAndEpoch highestOffsetAndEpoch() {
-        return highestOffsetAndEpoch;
+        return new OffsetAndEpoch(provenance.offset(), provenance.epoch());
+    }
+
+    public long offset() {
+        return provenance.offset();
     }
 
     public FeaturesImage features() {
@@ -135,7 +143,7 @@ public final class MetadataImage {
     public boolean equals(Object o) {
         if (o == null || !o.getClass().equals(this.getClass())) return false;
         MetadataImage other = (MetadataImage) o;
-        return highestOffsetAndEpoch.equals(other.highestOffsetAndEpoch) &&
+        return provenance.equals(other.provenance) &&
             features.equals(other.features) &&
             cluster.equals(other.cluster) &&
             topics.equals(other.topics) &&
@@ -147,7 +155,8 @@ public final class MetadataImage {
 
     @Override
     public int hashCode() {
-        return Objects.hash(highestOffsetAndEpoch,
+        return Objects.hash(
+            provenance,
             features,
             cluster,
             topics,
@@ -159,7 +168,8 @@ public final class MetadataImage {
 
     @Override
     public String toString() {
-        return "MetadataImage(highestOffsetAndEpoch=" + highestOffsetAndEpoch +
+        return "MetadataImage(" +
+            "provenance=" + provenance +
             ", features=" + features +
             ", cluster=" + cluster +
             ", topics=" + topics +
diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java b/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java
new file mode 100644
index 00000000000..3e65d3cfb80
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java
@@ -0,0 +1,92 @@
+/*
+ * 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.image;
+
+import org.apache.kafka.raft.OffsetAndEpoch;
+
+import java.util.Objects;
+
+
+/**
+ * Information about the source of a metadata image.
+ */
+public final class MetadataProvenance {
+    public static final MetadataProvenance EMPTY = new MetadataProvenance(-1L, -1, -1L);
+
+    private final long offset;
+    private final int epoch;
+    private final long lastContainedLogTimeMs;
+
+    public MetadataProvenance(
+        long offset,
+        int epoch,
+        long lastContainedLogTimeMs
+    ) {
+        this.offset = offset;
+        this.epoch = epoch;
+        this.lastContainedLogTimeMs = lastContainedLogTimeMs;
+    }
+
+    public OffsetAndEpoch offsetAndEpoch() {
+        return new OffsetAndEpoch(offset, epoch);
+    }
+
+    public long offset() {
+        return offset;
+    }
+
+    public int epoch() {
+        return epoch;
+    }
+
+    public long lastContainedLogTimeMs() {
+        return lastContainedLogTimeMs;
+    }
+
+    /**
+     * Returns the name that a snapshot with this provenance would have.
+     */
+    public String snapshotName() {
+        return String.format("snapshot %020d-%010d", offset, epoch);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null || !o.getClass().equals(this.getClass())) return false;
+        MetadataProvenance other = (MetadataProvenance) o;
+        return offset == other.offset &&
+            epoch == other.epoch &&
+            lastContainedLogTimeMs == other.lastContainedLogTimeMs;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(offset,
+            epoch,
+            lastContainedLogTimeMs);
+    }
+
+    @Override
+    public String toString() {
+        return "MetadataProvenance(" +
+            "offset=" + offset +
+            ", epoch=" + epoch +
+            ", lastContainedLogTimeMs=" + lastContainedLogTimeMs +
+            ")";
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChange.java b/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChange.java
new file mode 100644
index 00000000000..d9b43a306e5
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChange.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.image;
+
+import org.apache.kafka.server.common.MetadataVersion;
+
+import java.util.Objects;
+
+
+/**
+ * A change in the MetadataVersion.
+ */
+public final class MetadataVersionChange {
+    private final MetadataVersion oldVersion;
+    private final MetadataVersion newVersion;
+
+    public MetadataVersionChange(
+            MetadataVersion oldVersion,
+            MetadataVersion newVersion
+    ) {
+        this.oldVersion = Objects.requireNonNull(oldVersion);
+        this.newVersion = Objects.requireNonNull(newVersion);
+    }
+
+    public MetadataVersion oldVersion() {
+        return oldVersion;
+    }
+
+    public MetadataVersion newVersion() {
+        return newVersion;
+    }
+
+    public boolean isUpgrade() {
+        return oldVersion.isLessThan(newVersion);
+    }
+
+    public boolean isDowngrade() {
+        return newVersion.isLessThan(oldVersion);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null || !o.getClass().equals(this.getClass())) return false;
+        MetadataVersionChange other = (MetadataVersionChange) o;
+        return oldVersion.equals(other.oldVersion) &&
+                newVersion.equals(other.newVersion);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(oldVersion,
+                newVersion);
+    }
+
+    @Override
+    public String toString() {
+        return "MetadataVersionChange(" +
+                "oldVersion=" + oldVersion +
+                ", newVersion=" + newVersion +
+                ")";
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChangeException.java b/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChangeException.java
new file mode 100644
index 00000000000..a4c931f77b7
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/MetadataVersionChangeException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.image;
+
+
+/**
+ * Indicates that the metadata version has changed.
+ */
+public final class MetadataVersionChangeException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+    private final MetadataVersionChange change;
+
+    public MetadataVersionChangeException(MetadataVersionChange change) {
+        super("The metadata version is changing from " + change.oldVersion() + " to " +
+                change.newVersion());
+        this.change = change;
+    }
+
+    public MetadataVersionChange change() {
+        return change;
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/writer/ImageReWriter.java b/metadata/src/main/java/org/apache/kafka/image/writer/ImageReWriter.java
new file mode 100644
index 00000000000..42a0aaa93a1
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/writer/ImageReWriter.java
@@ -0,0 +1,59 @@
+/*
+ * 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.image.writer;
+
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+
+
+/**
+ * ImageReWriter writes a metadata image out to another metadata image.
+ *
+ * There are a few reasons why you might want to do this. One is to obtain a MetadataDelta
+ * object which contains everything in the image. Another is to translate an image from
+ * one metadata version to another.
+ */
+public class ImageReWriter implements ImageWriter {
+    private final MetadataDelta delta;
+    private boolean closed = false;
+    private MetadataImage image = null;
+
+    public ImageReWriter(MetadataDelta delta) {
+        this.delta = delta;
+    }
+
+    @Override
+    public void write(ApiMessageAndVersion record) {
+        if (closed) throw new ImageWriterClosedException();
+        delta.replay(record.message());
+    }
+
+    @Override
+    public void close(boolean complete) {
+        if (closed) return;
+        closed = true;
+        if (complete) {
+            image = delta.apply(delta.image().provenance());
+        }
+    }
+
+    public MetadataImage image() {
+        return image;
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/writer/ImageWriterOptions.java b/metadata/src/main/java/org/apache/kafka/image/writer/ImageWriterOptions.java
index 635a6c77c40..0d4550932bc 100644
--- a/metadata/src/main/java/org/apache/kafka/image/writer/ImageWriterOptions.java
+++ b/metadata/src/main/java/org/apache/kafka/image/writer/ImageWriterOptions.java
@@ -17,6 +17,7 @@
 
 package org.apache.kafka.image.writer;
 
+import org.apache.kafka.image.MetadataImage;
 import org.apache.kafka.server.common.MetadataVersion;
 
 import java.util.function.Consumer;
@@ -27,11 +28,19 @@ import java.util.function.Consumer;
  */
 public final class ImageWriterOptions {
     public static class Builder {
-        private MetadataVersion metadataVersion = MetadataVersion.latest();
+        private MetadataVersion metadataVersion;
         private Consumer<UnwritableMetadataException> lossHandler = e -> {
             throw e;
         };
 
+        public Builder() {
+            this.metadataVersion = MetadataVersion.latest();
+        }
+
+        public Builder(MetadataImage image) {
+            this.metadataVersion = image.features().metadataVersion();
+        }
+
         public Builder setMetadataVersion(MetadataVersion metadataVersion) {
             if (metadataVersion.isLessThan(MetadataVersion.MINIMUM_BOOTSTRAP_VERSION)) {
                 // When writing an image, all versions less than 3.3-IV0 are treated as 3.0-IV1.
@@ -43,12 +52,16 @@ public final class ImageWriterOptions {
             return this;
         }
 
-        // Package-private for testing
+        // Visible for testing
         public Builder setRawMetadataVersion(MetadataVersion metadataVersion) {
             this.metadataVersion = metadataVersion;
             return this;
         }
 
+        public MetadataVersion metadataVersion() {
+            return metadataVersion;
+        }
+
         public Builder setLossHandler(Consumer<UnwritableMetadataException> lossHandler) {
             this.lossHandler = lossHandler;
             return this;
diff --git a/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java b/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java
index 5a7c2e6c0b6..86eca51b4c1 100644
--- a/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java
+++ b/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java
@@ -133,9 +133,9 @@ public class ImageDowngradeTest {
         List<ApiMessageAndVersion> expectedOutputs
     ) {
         MockLossConsumer lossConsumer = new MockLossConsumer(metadataVersion);
-        MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY);
+        MetadataDelta delta = new MetadataDelta.Builder().build();
         RecordTestUtils.replayAll(delta, inputs);
-        MetadataImage image = delta.apply();
+        MetadataImage image = delta.apply(MetadataProvenance.EMPTY);
         RecordListWriter writer = new RecordListWriter();
         image.write(writer, new ImageWriterOptions.Builder().
                 setRawMetadataVersion(metadataVersion).
diff --git a/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java b/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java
index 290214369cc..be21a87bd69 100644
--- a/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java
+++ b/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java
@@ -20,7 +20,6 @@ package org.apache.kafka.image;
 import org.apache.kafka.image.writer.ImageWriterOptions;
 import org.apache.kafka.image.writer.RecordListWriter;
 import org.apache.kafka.metadata.RecordTestUtils;
-import org.apache.kafka.raft.OffsetAndEpoch;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Timeout;
 
@@ -37,7 +36,7 @@ public class MetadataImageTest {
 
     static {
         IMAGE1 = new MetadataImage(
-            new OffsetAndEpoch(100, 4),
+            new MetadataProvenance(100, 4, 2000),
             FeaturesImageTest.IMAGE1,
             ClusterImageTest.IMAGE1,
             TopicsImageTest.IMAGE1,
@@ -46,17 +45,19 @@ public class MetadataImageTest {
             ProducerIdsImageTest.IMAGE1,
             AclsImageTest.IMAGE1);
 
-        DELTA1 = new MetadataDelta(IMAGE1);
-        RecordTestUtils.replayAll(DELTA1, 200, 5, FeaturesImageTest.DELTA1_RECORDS);
-        RecordTestUtils.replayAll(DELTA1, 200, 5, ClusterImageTest.DELTA1_RECORDS);
-        RecordTestUtils.replayAll(DELTA1, 200, 5, TopicsImageTest.DELTA1_RECORDS);
-        RecordTestUtils.replayAll(DELTA1, 200, 5, ConfigurationsImageTest.DELTA1_RECORDS);
-        RecordTestUtils.replayAll(DELTA1, 200, 5, ClientQuotasImageTest.DELTA1_RECORDS);
-        RecordTestUtils.replayAll(DELTA1, 200, 5, ProducerIdsImageTest.DELTA1_RECORDS);
-        RecordTestUtils.replayAll(DELTA1, 200, 5, AclsImageTest.DELTA1_RECORDS);
+        DELTA1 = new MetadataDelta.Builder().
+                setImage(IMAGE1).
+                build();
+        RecordTestUtils.replayAll(DELTA1, FeaturesImageTest.DELTA1_RECORDS);
+        RecordTestUtils.replayAll(DELTA1, ClusterImageTest.DELTA1_RECORDS);
+        RecordTestUtils.replayAll(DELTA1, TopicsImageTest.DELTA1_RECORDS);
+        RecordTestUtils.replayAll(DELTA1, ConfigurationsImageTest.DELTA1_RECORDS);
+        RecordTestUtils.replayAll(DELTA1, ClientQuotasImageTest.DELTA1_RECORDS);
+        RecordTestUtils.replayAll(DELTA1, ProducerIdsImageTest.DELTA1_RECORDS);
+        RecordTestUtils.replayAll(DELTA1, AclsImageTest.DELTA1_RECORDS);
 
         IMAGE2 = new MetadataImage(
-            new OffsetAndEpoch(200, 5),
+            new MetadataProvenance(200, 5, 4000),
             FeaturesImageTest.IMAGE2,
             ClusterImageTest.IMAGE2,
             TopicsImageTest.IMAGE2,
@@ -78,7 +79,7 @@ public class MetadataImageTest {
 
     @Test
     public void testApplyDelta1() throws Throwable {
-        assertEquals(IMAGE2, DELTA1.apply());
+        assertEquals(IMAGE2, DELTA1.apply(IMAGE2.provenance()));
     }
 
     @Test
@@ -88,13 +89,10 @@ public class MetadataImageTest {
 
     private void testToImageAndBack(MetadataImage image) throws Throwable {
         RecordListWriter writer = new RecordListWriter();
-        image.write(writer, new ImageWriterOptions.Builder().build());
+        image.write(writer, new ImageWriterOptions.Builder(image).build());
         MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY);
-        RecordTestUtils.replayAll(delta,
-                image.highestOffsetAndEpoch().offset(),
-                image.highestOffsetAndEpoch().epoch(),
-                writer.records());
-        MetadataImage nextImage = delta.apply();
+        RecordTestUtils.replayAll(delta, writer.records());
+        MetadataImage nextImage = delta.apply(image.provenance());
         assertEquals(image, nextImage);
     }
 }
diff --git a/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java b/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java
new file mode 100644
index 00000000000..bef49adc37d
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/image/MetadataVersionChangeTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.image;
+
+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.server.common.MetadataVersion.IBP_3_0_IV1;
+import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV0;
+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 MetadataVersionChangeTest {
+    private static final Logger log = LoggerFactory.getLogger(MetadataVersionChangeTest.class);
+
+    private final static MetadataVersionChange CHANGE_3_0_IV1_TO_3_3_IV0 =
+        new MetadataVersionChange(IBP_3_0_IV1, IBP_3_3_IV0);
+
+    private final static MetadataVersionChange CHANGE_3_3_IV0_TO_3_0_IV1 =
+        new MetadataVersionChange(IBP_3_3_IV0, IBP_3_0_IV1);
+
+    @Test
+    public void testIsUpgrade() throws Throwable {
+        assertTrue(CHANGE_3_0_IV1_TO_3_3_IV0.isUpgrade());
+        assertFalse(CHANGE_3_3_IV0_TO_3_0_IV1.isUpgrade());
+    }
+
+    @Test
+    public void testIsDowngrade() throws Throwable {
+        assertFalse(CHANGE_3_0_IV1_TO_3_3_IV0.isDowngrade());
+        assertTrue(CHANGE_3_3_IV0_TO_3_0_IV1.isDowngrade());
+    }
+
+    @Test
+    public void testMetadataVersionChangeExceptionToString() throws Throwable {
+        assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata " +
+            "version is changing from 3.0-IV1 to 3.3-IV0",
+                new MetadataVersionChangeException(CHANGE_3_0_IV1_TO_3_3_IV0).toString());
+        assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata " +
+            "version is changing from 3.3-IV0 to 3.0-IV1",
+                new MetadataVersionChangeException(CHANGE_3_3_IV0_TO_3_0_IV1).toString());
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/image/writer/ImageReWriterTest.java b/metadata/src/test/java/org/apache/kafka/image/writer/ImageReWriterTest.java
new file mode 100644
index 00000000000..640924fe076
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/image/writer/ImageReWriterTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.image.writer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.image.MetadataDelta;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import static org.apache.kafka.metadata.RecordTestUtils.testRecord;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+
+@Timeout(value = 40)
+public class ImageReWriterTest {
+    @Test
+    public void testWrite() {
+        MetadataDelta delta = new MetadataDelta.Builder().build();
+        ImageReWriter writer = new ImageReWriter(delta);
+        writer.write(testRecord(0));
+        writer.write(testRecord(1));
+        writer.close(true);
+        assertEquals(2, delta.getOrCreateTopicsDelta().changedTopics().size());
+        assertEquals(2, writer.image().topics().topicsById().size());
+    }
+
+    @Test
+    public void testCloseWithoutFreeze() {
+        MetadataDelta delta = new MetadataDelta.Builder().build();
+        ImageReWriter writer = new ImageReWriter(delta);
+        writer.close();
+        assertNull(writer.image());
+    }
+
+    @Test
+    public void testWriteAfterClose() {
+        MetadataDelta delta = new MetadataDelta.Builder().build();
+        ImageReWriter writer = new ImageReWriter(delta);
+        writer.close(true);
+        assertThrows(ImageWriterClosedException.class, () ->
+                writer.write(0, new TopicRecord().
+                        setName("foo").
+                        setTopicId(Uuid.fromString("3B134hrsQgKtz8Sp6QBIfg"))));
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/image/writer/ImageWriterOptionsTest.java b/metadata/src/test/java/org/apache/kafka/image/writer/ImageWriterOptionsTest.java
new file mode 100644
index 00000000000..07ddd4ff8cb
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/image/writer/ImageWriterOptionsTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.image.writer;
+
+import org.apache.kafka.server.common.MetadataVersion;
+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.assertThrows;
+
+
+@Timeout(value = 40)
+public class ImageWriterOptionsTest {
+    @Test
+    public void testDefaultLossHandler() {
+        ImageWriterOptions options = new ImageWriterOptions.Builder().build();
+        assertEquals("stuff", assertThrows(UnwritableMetadataException.class,
+                () -> options.handleLoss("stuff")).loss());
+    }
+
+    @Test
+    public void testSetMetadataVersion() {
+        for (int i = MetadataVersion.MINIMUM_KRAFT_VERSION.ordinal();
+                 i < MetadataVersion.VERSIONS.length;
+                 i++) {
+            MetadataVersion version = MetadataVersion.VERSIONS[i];
+            ImageWriterOptions.Builder options = new ImageWriterOptions.Builder().
+                    setMetadataVersion(version);
+            if (i < MetadataVersion.MINIMUM_BOOTSTRAP_VERSION.ordinal()) {
+                assertEquals(MetadataVersion.MINIMUM_KRAFT_VERSION, options.metadataVersion());
+            } else {
+                assertEquals(version, options.metadataVersion());
+            }
+        }
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java
index 7fd2a29d5b1..04da77b0fc3 100644
--- a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java
+++ b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java
@@ -23,7 +23,6 @@ import org.apache.kafka.common.protocol.ApiMessage;
 import org.apache.kafka.common.protocol.Message;
 import org.apache.kafka.common.protocol.ObjectSerializationCache;
 import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
-import org.apache.kafka.image.MetadataDelta;
 import org.apache.kafka.raft.Batch;
 import org.apache.kafka.raft.BatchReader;
 import org.apache.kafka.raft.internals.MemoryBatchReader;
@@ -57,14 +56,6 @@ public class RecordTestUtils {
      */
     public static void replayAll(Object target,
                                  List<ApiMessageAndVersion> recordsAndVersions) {
-        if (target instanceof MetadataDelta) {
-            MetadataDelta delta = (MetadataDelta) target;
-            replayAll(delta,
-                    delta.image().highestOffsetAndEpoch().offset(),
-                    delta.image().highestOffsetAndEpoch().epoch(),
-                    recordsAndVersions);
-            return;
-        }
         for (ApiMessageAndVersion recordAndVersion : recordsAndVersions) {
             ApiMessage record = recordAndVersion.message();
             try {
@@ -96,26 +87,6 @@ public class RecordTestUtils {
         }
     }
 
-    /**
-     * Replay a list of records to the metadata delta.
-     *
-     * @param delta the metadata delta on which to replay the records
-     * @param highestOffset highest offset from the list of records
-     * @param highestEpoch highest epoch from the list of records
-     * @param recordsAndVersions list of records
-     */
-    public static void replayAll(
-        MetadataDelta delta,
-        long highestOffset,
-        int highestEpoch,
-        List<ApiMessageAndVersion> recordsAndVersions
-    ) {
-        for (ApiMessageAndVersion recordAndVersion : recordsAndVersions) {
-            ApiMessage record = recordAndVersion.message();
-            delta.replay(highestOffset, highestEpoch, record);
-        }
-    }
-
     /**
      * Replay a list of record batches.
      *
@@ -129,25 +100,6 @@ public class RecordTestUtils {
         }
     }
 
-    /**
-     * Replay a list of record batches to the metadata delta.
-     *
-     * @param delta the metadata delta on which to replay the records
-     * @param highestOffset highest offset from the list of record batches
-     * @param highestEpoch highest epoch from the list of record batches
-     * @param batches list of batches of records
-     */
-    public static void replayAllBatches(
-        MetadataDelta delta,
-        long highestOffset,
-        int highestEpoch,
-        List<List<ApiMessageAndVersion>> batches
-    ) {
-        for (List<ApiMessageAndVersion> batch : batches) {
-            replayAll(delta, highestOffset, highestEpoch, batch);
-        }
-    }
-
     /**
      * Materialize the output of an iterator into a set.
      *


[kafka] 01/11: MINOR: Move dynamic config logic to DynamicConfigPublisher (#12958)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e94a25173dc469da650f4d620ab312a1363242a2
Author: Colin Patrick McCabe <cm...@apache.org>
AuthorDate: Wed Dec 7 10:43:34 2022 -0800

    MINOR: Move dynamic config logic to DynamicConfigPublisher (#12958)
    
    Split out the logic for applying dynamic configurations to a KafkaConfig object from
    BrokerMetadataPublisher into a new class, DynamicConfigPublisher. This will allow the
    ControllerServer to also run this code, in a follow-up change.
    
    Create separate KafkaConfig objects in BrokerServer versus ControllerServer. This is necessary
    because the controller will apply configuration changes as soon as its raft client catches up to
    the high water mark, whereas the broker will wait for the active controller to acknowledge it has
    caught up in a heartbeat response. So when running in combined mode, we want two separate
    KafkaConfig objects that are changed at different times.
    
    Minor changes: improve the error message when catching up broker metadata fails. Fix incorrect
    indentation in checkstyle/import-control.xml. Invoke AppInfoParser.unregisterAppInfo from
    SharedServer.stop so that it happens only when both the controller and broker have shut down.
    
    Reviewers: David Arthur <mu...@gmail.com>
---
 checkstyle/import-control.xml                      |   2 +-
 .../src/main/scala/kafka/server/BrokerServer.scala |  24 +++--
 .../main/scala/kafka/server/ControllerServer.scala |   6 +-
 .../src/main/scala/kafka/server/SharedServer.scala |  22 +++--
 .../server/metadata/BrokerMetadataPublisher.scala  |  61 +-----------
 .../server/metadata/DynamicConfigPublisher.scala   | 103 +++++++++++++++++++++
 .../metadata/BrokerMetadataPublisherTest.scala     |  45 +++------
 7 files changed, 150 insertions(+), 113 deletions(-)

diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 0609ee7fd6c..bd05521964e 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -306,7 +306,7 @@
   </subpackage>
 
   <subpackage name="queue">
-      <allow pkg="org.apache.kafka.test" />
+    <allow pkg="org.apache.kafka.test" />
   </subpackage>
 
   <subpackage name="clients">
diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index d6b4fa92c3a..f55ceebffcc 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -30,7 +30,7 @@ import kafka.network.{DataPlaneAcceptor, SocketServer}
 import kafka.raft.KafkaRaftManager
 import kafka.security.CredentialProvider
 import kafka.server.KafkaRaftServer.ControllerRole
-import kafka.server.metadata.{BrokerMetadataListener, BrokerMetadataPublisher, BrokerMetadataSnapshotter, ClientQuotaMetadataManager, KRaftMetadataCache, SnapshotWriterBuilder}
+import kafka.server.metadata.{BrokerMetadataListener, BrokerMetadataPublisher, BrokerMetadataSnapshotter, ClientQuotaMetadataManager, DynamicConfigPublisher, KRaftMetadataCache, SnapshotWriterBuilder}
 import kafka.utils.{CoreUtils, KafkaScheduler}
 import org.apache.kafka.common.feature.SupportedVersionRange
 import org.apache.kafka.common.message.ApiMessageType.ListenerType
@@ -39,7 +39,7 @@ import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.security.auth.SecurityProtocol
 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.{AppInfoParser, LogContext, Time, Utils}
+import org.apache.kafka.common.utils.{LogContext, Time, Utils}
 import org.apache.kafka.common.{ClusterResource, Endpoint}
 import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
 import org.apache.kafka.metadata.{BrokerState, VersionRange}
@@ -71,7 +71,7 @@ class BrokerServer(
   val initialOfflineDirs: Seq[String],
 ) extends KafkaBroker {
   val threadNamePrefix = sharedServer.threadNamePrefix
-  val config = sharedServer.config
+  val config = sharedServer.brokerConfig
   val time = sharedServer.time
   def metrics = sharedServer.metrics
 
@@ -420,8 +420,13 @@ class BrokerServer(
         config.numIoThreads, s"${DataPlaneAcceptor.MetricPrefix}RequestHandlerAvgIdlePercent",
         DataPlaneAcceptor.ThreadPrefix)
 
-      // Block until we've caught up with the latest metadata from the controller quorum.
-      lifecycleManager.initialCatchUpFuture.get()
+      info("Waiting for broker metadata to catch up.")
+      try {
+        lifecycleManager.initialCatchUpFuture.get()
+      } catch {
+        case t: Throwable => throw new RuntimeException("Received a fatal error while " +
+          "waiting for the broker to catch up with the current cluster metadata.", t)
+      }
 
       // Apply the metadata log changes that we've accumulated.
       metadataPublisher = new BrokerMetadataPublisher(config,
@@ -431,7 +436,11 @@ class BrokerServer(
         groupCoordinator,
         transactionCoordinator,
         clientQuotaMetadataManager,
-        dynamicConfigHandlers.toMap,
+        new DynamicConfigPublisher(
+          config,
+          sharedServer.metadataPublishingFaultHandler,
+          dynamicConfigHandlers.toMap,
+        "broker"),
         authorizer,
         sharedServer.initialBrokerMetadataLoadFaultHandler,
         sharedServer.metadataPublishingFaultHandler)
@@ -567,9 +576,8 @@ class BrokerServer(
       isShuttingDown.set(false)
 
       CoreUtils.swallow(lifecycleManager.close(), this)
+      CoreUtils.swallow(config.dynamicConfig.clear(), this)
       sharedServer.stopForBroker()
-
-      CoreUtils.swallow(AppInfoParser.unregisterAppInfo(MetricsPrefix, config.nodeId.toString, metrics), this)
       info("shut down completed")
     } catch {
       case e: Throwable =>
diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
index f73088b30f0..2bd518cde2a 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -59,14 +59,12 @@ class ControllerServer(
 
   import kafka.server.Server._
 
-  val config = sharedServer.config
+  val config = sharedServer.controllerConfig
   val time = sharedServer.time
   def metrics = sharedServer.metrics
   val threadNamePrefix = sharedServer.threadNamePrefix.getOrElse("")
   def raftManager: KafkaRaftManager[ApiMessageAndVersion] = sharedServer.raftManager
 
-  config.dynamicConfig.initialize(zkClientOpt = None)
-
   val lock = new ReentrantLock()
   val awaitShutdownCond = lock.newCondition()
   var status: ProcessStatus = SHUTDOWN
@@ -109,6 +107,7 @@ class ControllerServer(
     if (!maybeChangeStatus(SHUTDOWN, STARTING)) return
     try {
       info("Starting controller")
+      config.dynamicConfig.initialize(zkClientOpt = None)
 
       maybeChangeStatus(STARTING, STARTED)
       this.logIdent = new LogContext(s"[ControllerServer id=${config.nodeId}] ").logPrefix()
@@ -284,6 +283,7 @@ class ControllerServer(
       createTopicPolicy.foreach(policy => CoreUtils.swallow(policy.close(), this))
       alterConfigPolicy.foreach(policy => CoreUtils.swallow(policy.close(), this))
       socketServerFirstBoundPortFuture.completeExceptionally(new RuntimeException("shutting down"))
+      CoreUtils.swallow(config.dynamicConfig.clear(), this)
       sharedServer.stopForController()
     } catch {
       case e: Throwable =>
diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala
index a420c9afa38..8b647e7464f 100644
--- a/core/src/main/scala/kafka/server/SharedServer.scala
+++ b/core/src/main/scala/kafka/server/SharedServer.scala
@@ -19,10 +19,11 @@ package kafka.server
 
 import kafka.raft.KafkaRaftManager
 import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole}
+import kafka.server.Server.MetricsPrefix
 import kafka.server.metadata.BrokerServerMetrics
 import kafka.utils.{CoreUtils, Logging}
 import org.apache.kafka.common.metrics.Metrics
-import org.apache.kafka.common.utils.{LogContext, Time}
+import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time}
 import org.apache.kafka.controller.QuorumControllerMetrics
 import org.apache.kafka.metadata.MetadataRecordSerde
 import org.apache.kafka.raft.RaftConfig.AddressSpec
@@ -77,7 +78,7 @@ class StandardFaultHandlerFactory extends FaultHandlerFactory {
  * make debugging easier and reduce the chance of resource leaks.
  */
 class SharedServer(
-  val config: KafkaConfig,
+  private val sharedServerConfig: KafkaConfig,
   val metaProps: MetaProperties,
   val time: Time,
   private val _metrics: Metrics,
@@ -85,11 +86,13 @@ class SharedServer(
   val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]],
   val faultHandlerFactory: FaultHandlerFactory
 ) extends Logging {
-  private val logContext: LogContext = new LogContext(s"[SharedServer id=${config.nodeId}] ")
+  private val logContext: LogContext = new LogContext(s"[SharedServer id=${sharedServerConfig.nodeId}] ")
   this.logIdent = logContext.logPrefix
   private var started = false
   private var usedByBroker: Boolean = false
   private var usedByController: Boolean = false
+  val brokerConfig = new KafkaConfig(sharedServerConfig.props, false, None)
+  val controllerConfig = new KafkaConfig(sharedServerConfig.props, false, None)
   @volatile var metrics: Metrics = _metrics
   @volatile var raftManager: KafkaRaftManager[ApiMessageAndVersion] = _
   @volatile var brokerMetrics: BrokerServerMetrics = _
@@ -143,7 +146,7 @@ class SharedServer(
    * The fault handler to use when metadata loading fails.
    */
   def metadataLoaderFaultHandler: FaultHandler = faultHandlerFactory.build("metadata loading",
-    fatal = config.processRoles.contains(ControllerRole),
+    fatal = sharedServerConfig.processRoles.contains(ControllerRole),
     action = () => SharedServer.this.synchronized {
       if (brokerMetrics != null) brokerMetrics.metadataLoadErrorCount.getAndIncrement()
       if (controllerMetrics != null) controllerMetrics.incrementMetadataErrorCount()
@@ -188,17 +191,17 @@ class SharedServer(
           // This is only done in tests.
           metrics = new Metrics()
         }
-        config.dynamicConfig.initialize(zkClientOpt = None)
+        sharedServerConfig.dynamicConfig.initialize(zkClientOpt = None)
 
-        if (config.processRoles.contains(BrokerRole)) {
+        if (sharedServerConfig.processRoles.contains(BrokerRole)) {
           brokerMetrics = BrokerServerMetrics(metrics)
         }
-        if (config.processRoles.contains(ControllerRole)) {
+        if (sharedServerConfig.processRoles.contains(ControllerRole)) {
           controllerMetrics = new QuorumControllerMetrics(KafkaYammerMetrics.defaultRegistry(), time)
         }
         raftManager = new KafkaRaftManager[ApiMessageAndVersion](
           metaProps,
-          config,
+          sharedServerConfig,
           new MetadataRecordSerde,
           KafkaRaftServer.MetadataPartition,
           KafkaRaftServer.MetadataTopicId,
@@ -248,8 +251,7 @@ class SharedServer(
         CoreUtils.swallow(metrics.close(), this)
         metrics = null
       }
-      // Clear all reconfigurable instances stored in DynamicBrokerConfig
-      CoreUtils.swallow(config.dynamicConfig.clear(), this)
+      CoreUtils.swallow(AppInfoParser.unregisterAppInfo(MetricsPrefix, sharedServerConfig.nodeId.toString, metrics), this)
       started = false
     }
   }
diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
index 0192bb4afcf..933a6bf8924 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
@@ -22,11 +22,9 @@ import java.util.concurrent.atomic.AtomicLong
 import kafka.coordinator.group.GroupCoordinator
 import kafka.coordinator.transaction.TransactionCoordinator
 import kafka.log.{LogManager, UnifiedLog}
-import kafka.server.ConfigAdminManager.toLoggableProps
-import kafka.server.{ConfigEntityName, ConfigHandler, ConfigType, KafkaConfig, ReplicaManager, RequestLocal}
+import kafka.server.{KafkaConfig, ReplicaManager, RequestLocal}
 import kafka.utils.Logging
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, TOPIC}
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta, TopicsImage}
 import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
@@ -103,7 +101,7 @@ class BrokerMetadataPublisher(
   groupCoordinator: GroupCoordinator,
   txnCoordinator: TransactionCoordinator,
   clientQuotaMetadataManager: ClientQuotaMetadataManager,
-  dynamicConfigHandlers: Map[String, ConfigHandler],
+  var dynamicConfigPublisher: DynamicConfigPublisher,
   private val _authorizer: Option[Authorizer],
   fatalFaultHandler: FaultHandler,
   metadataPublishingFaultHandler: FaultHandler
@@ -211,61 +209,10 @@ class BrokerMetadataPublisher(
       }
 
       // Apply configuration deltas.
-      Option(delta.configsDelta()).foreach { configsDelta =>
-        configsDelta.changes().keySet().forEach { resource =>
-          val props = newImage.configs().configProperties(resource)
-          resource.`type`() match {
-            case TOPIC =>
-              try {
-                // Apply changes to a topic's dynamic configuration.
-                info(s"Updating topic ${resource.name()} with new configuration : " +
-                  toLoggableProps(resource, props).mkString(","))
-                dynamicConfigHandlers(ConfigType.Topic).
-                  processConfigChanges(resource.name(), props)
-              } catch {
-                case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating topic " +
-                  s"${resource.name()} with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +
-                  s"in ${deltaName}", t)
-              }
-            case BROKER =>
-              if (resource.name().isEmpty) {
-                try {
-                  // Apply changes to "cluster configs" (also known as default BROKER configs).
-                  // These are stored in KRaft with an empty name field.
-                  info("Updating cluster configuration : " +
-                    toLoggableProps(resource, props).mkString(","))
-                  dynamicConfigHandlers(ConfigType.Broker).
-                    processConfigChanges(ConfigEntityName.Default, props)
-                } catch {
-                  case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating " +
-                    s"cluster with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +
-                    s"in ${deltaName}", t)
-                }
-              } else if (resource.name() == brokerId.toString) {
-                try {
-                  // Apply changes to this broker's dynamic configuration.
-                  info(s"Updating broker $brokerId with new configuration : " +
-                    toLoggableProps(resource, props).mkString(","))
-                  dynamicConfigHandlers(ConfigType.Broker).
-                    processConfigChanges(resource.name(), props)
-                  // When applying a per broker config (not a cluster config), we also
-                  // reload any associated file. For example, if the ssl.keystore is still
-                  // set to /tmp/foo, we still want to reload /tmp/foo in case its contents
-                  // have changed. This doesn't apply to topic configs or cluster configs.
-                  reloadUpdatedFilesWithoutConfigChange(props)
-                } catch {
-                  case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating " +
-                    s"broker with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +
-                    s"in ${deltaName}", t)
-                }
-              }
-            case _ => // nothing to do
-          }
-        }
-      }
+      dynamicConfigPublisher.publish(delta, newImage)
 
+      // Apply client quotas delta.
       try {
-        // Apply client quotas delta.
         Option(delta.clientQuotasDelta()).foreach { clientQuotasDelta =>
           clientQuotaMetadataManager.update(clientQuotasDelta)
         }
diff --git a/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala b/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala
new file mode 100644
index 00000000000..12ff51d4039
--- /dev/null
+++ b/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala
@@ -0,0 +1,103 @@
+/**
+ * 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 kafka.server.metadata
+
+import java.util.Properties
+import kafka.server.ConfigAdminManager.toLoggableProps
+import kafka.server.{ConfigEntityName, ConfigHandler, ConfigType, KafkaConfig}
+import kafka.utils.Logging
+import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, TOPIC}
+import org.apache.kafka.image.{MetadataDelta, MetadataImage}
+import org.apache.kafka.server.fault.FaultHandler
+
+
+class DynamicConfigPublisher(
+  conf: KafkaConfig,
+  faultHandler: FaultHandler,
+  dynamicConfigHandlers: Map[String, ConfigHandler],
+  nodeType: String
+) extends Logging {
+  logIdent = s"[DynamicConfigPublisher nodeType=${nodeType} id=${conf.nodeId}] "
+
+  def publish(delta: MetadataDelta, newImage: MetadataImage): Unit = {
+    val deltaName = s"MetadataDelta up to ${newImage.highestOffsetAndEpoch().offset}"
+    try {
+      // Apply configuration deltas.
+      Option(delta.configsDelta()).foreach { configsDelta =>
+        configsDelta.changes().keySet().forEach { resource =>
+          val props = newImage.configs().configProperties(resource)
+          resource.`type`() match {
+            case TOPIC =>
+              dynamicConfigHandlers.get(ConfigType.Topic).foreach(topicConfigHandler =>
+                try {
+                  // Apply changes to a topic's dynamic configuration.
+                  info(s"Updating topic ${resource.name()} with new configuration : " +
+                    toLoggableProps(resource, props).mkString(","))
+                  topicConfigHandler.processConfigChanges(resource.name(), props)
+                } catch {
+                  case t: Throwable => faultHandler.handleFault("Error updating topic " +
+                    s"${resource.name()} with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +
+                    s"in ${deltaName}", t)
+                }
+              )
+            case BROKER =>
+              dynamicConfigHandlers.get(ConfigType.Broker).foreach(nodeConfigHandler =>
+                if (resource.name().isEmpty) {
+                  try {
+                    // Apply changes to "cluster configs" (also known as default BROKER configs).
+                    // These are stored in KRaft with an empty name field.
+                    info("Updating cluster configuration : " +
+                      toLoggableProps(resource, props).mkString(","))
+                    nodeConfigHandler.processConfigChanges(ConfigEntityName.Default, props)
+                  } catch {
+                    case t: Throwable => faultHandler.handleFault("Error updating " +
+                      s"cluster with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +
+                      s"in ${deltaName}", t)
+                  }
+                } else if (resource.name() == conf.nodeId.toString) {
+                  try {
+                    // Apply changes to this node's dynamic configuration.
+                    info(s"Updating node ${conf.nodeId} with new configuration : " +
+                      toLoggableProps(resource, props).mkString(","))
+                    nodeConfigHandler.processConfigChanges(resource.name(), props)
+                    // When applying a per node config (not a cluster config), we also
+                    // reload any associated file. For example, if the ssl.keystore is still
+                    // set to /tmp/foo, we still want to reload /tmp/foo in case its contents
+                    // have changed. This doesn't apply to topic configs or cluster configs.
+                    reloadUpdatedFilesWithoutConfigChange(props)
+                  } catch {
+                    case t: Throwable => faultHandler.handleFault("Error updating " +
+                      s"node with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +
+                      s"in ${deltaName}", t)
+                  }
+                }
+              )
+            case _ => // nothing to do
+          }
+        }
+      }
+    } catch {
+      case t: Throwable => faultHandler.handleFault("Uncaught exception while " +
+        s"publishing dynamic configuration changes from ${deltaName}", t)
+    }
+  }
+
+  def reloadUpdatedFilesWithoutConfigChange(props: Properties): Unit = {
+    conf.dynamicConfig.reloadUpdatedFilesWithoutConfigChange(props)
+  }
+}
diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala
index 8874a235a52..317da428888 100644
--- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala
+++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala
@@ -17,14 +17,11 @@
 
 package kafka.server.metadata
 
-import kafka.coordinator.group.GroupCoordinator
-import kafka.coordinator.transaction.TransactionCoordinator
-
 import java.util.Collections.{singleton, singletonList, singletonMap}
 import java.util.Properties
 import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
-import kafka.log.{LogManager, UnifiedLog}
-import kafka.server.{BrokerServer, KafkaConfig, ReplicaManager}
+import kafka.log.UnifiedLog
+import kafka.server.{BrokerServer, KafkaConfig}
 import kafka.testkit.{KafkaClusterTestKit, TestKitNodes}
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET
@@ -36,7 +33,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid}
 import org.apache.kafka.image.{MetadataImageTest, TopicImage, TopicsImage}
 import org.apache.kafka.metadata.LeaderRecoveryState
 import org.apache.kafka.metadata.PartitionRegistration
-import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler}
+import org.apache.kafka.server.fault.FaultHandler
 import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue}
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
 import org.mockito.ArgumentMatchers.any
@@ -179,29 +176,15 @@ class BrokerMetadataPublisherTest {
     new TopicsImage(idsMap.asJava, namesMap.asJava)
   }
 
-  private def newMockPublisher(
+  private def newMockDynamicConfigPublisher(
     broker: BrokerServer,
-    logManager: LogManager,
-    replicaManager: ReplicaManager,
-    groupCoordinator: GroupCoordinator,
-    txnCoordinator: TransactionCoordinator,
-    errorHandler: FaultHandler = new MockFaultHandler("publisher")
-  ): BrokerMetadataPublisher = {
-    val mockLogManager = Mockito.mock(classOf[LogManager])
-    Mockito.when(mockLogManager.allLogs).thenReturn(Iterable.empty[UnifiedLog])
-    Mockito.spy(new BrokerMetadataPublisher(
+    errorHandler: FaultHandler
+  ): DynamicConfigPublisher = {
+    Mockito.spy(new DynamicConfigPublisher(
       conf = broker.config,
-      metadataCache = broker.metadataCache,
-      logManager,
-      replicaManager,
-      groupCoordinator,
-      txnCoordinator,
-      clientQuotaMetadataManager = broker.clientQuotaMetadataManager,
+      faultHandler = errorHandler,
       dynamicConfigHandlers = broker.dynamicConfigHandlers.toMap,
-      _authorizer = Option.empty,
-      errorHandler,
-      errorHandler
-    ))
+      nodeType = "broker"))
   }
 
   @Test
@@ -215,20 +198,14 @@ class BrokerMetadataPublisherTest {
       cluster.startup()
       cluster.waitForReadyBrokers()
       val broker = cluster.brokers().values().iterator().next()
-      val mockLogManager = Mockito.mock(classOf[LogManager])
-      Mockito.when(mockLogManager.allLogs).thenReturn(Iterable.empty[UnifiedLog])
-      val mockReplicaManager = Mockito.mock(classOf[ReplicaManager])
-      val mockGroupCoordinator = Mockito.mock(classOf[GroupCoordinator])
-      val mockTxnCoordinator = Mockito.mock(classOf[TransactionCoordinator])
-
-      val publisher = newMockPublisher(broker, mockLogManager, mockReplicaManager, mockGroupCoordinator, mockTxnCoordinator)
+      val publisher = newMockDynamicConfigPublisher(broker, cluster.nonFatalFaultHandler())
 
       val numTimesReloadCalled = new AtomicInteger(0)
       Mockito.when(publisher.reloadUpdatedFilesWithoutConfigChange(any[Properties]())).
         thenAnswer(new Answer[Unit]() {
           override def answer(invocation: InvocationOnMock): Unit = numTimesReloadCalled.addAndGet(1)
         })
-      broker.metadataListener.alterPublisher(publisher).get()
+      broker.metadataPublisher.dynamicConfigPublisher = publisher
       val admin = Admin.create(cluster.clientProperties())
       try {
         assertEquals(0, numTimesReloadCalled.get())


[kafka] 09/11: MINOR: Change KRaft ZK controller registration algorithm (#12973)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 47c5b001be1deea5a4542e52baf0738def36b430
Author: David Arthur <mu...@gmail.com>
AuthorDate: Wed Dec 14 08:37:06 2022 -0500

    MINOR: Change KRaft ZK controller registration algorithm (#12973)
    
    Increment the value in "/controller_epoch" when registering a KRaft controller as the active controller. Use the "kraftControllerEpoch" stored under "/controller" to ensure we are registering a newer KRaft controller.
    
    Reviewers: Colin P. McCabe <cm...@apache.org>
---
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   | 102 +++++++++++++--------
 core/src/main/scala/kafka/zk/ZkData.scala          |  15 ++-
 .../main/scala/kafka/zk/ZkMigrationClient.scala    |   9 +-
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala    |  68 +++++++++++++-
 .../unit/kafka/zk/ZkMigrationClientTest.scala      |  29 ++++--
 5 files changed, 166 insertions(+), 57 deletions(-)

diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala
index 115446572e1..361c74c5a7c 100644
--- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala
+++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala
@@ -45,6 +45,10 @@ import org.apache.zookeeper.{CreateMode, KeeperException, OpResult, ZooKeeper}
 
 import scala.collection.{Map, Seq, mutable}
 
+sealed trait KRaftRegistrationResult
+case class FailedRegistrationResult() extends KRaftRegistrationResult
+case class SuccessfulRegistrationResult(zkControllerEpoch: Int, controllerEpochZkVersion: Int) extends KRaftRegistrationResult
+
 /**
  * Provides higher level Kafka-specific operations on top of the pipelined [[kafka.zookeeper.ZooKeeperClient]].
  *
@@ -167,67 +171,76 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
    * the migration.
    *
    * To ensure that the KRaft controller epoch exceeds the current ZK controller epoch, this registration algorithm
-   * uses a conditional update on the /controller_epoch znode. If a new ZK controller is elected during this method,
-   * the conditional update on /controller_epoch fails which causes the whole multi-op transaction to fail.
+   * uses a conditional update on the /controller and /controller_epoch znodes.
+   *
+   * If a new controller is registered concurrently with this registration, one of the two will fail the CAS
+   * operation on /controller_epoch. For KRaft, we have an extra guard against the registered KRaft epoch going
+   * backwards. If a KRaft controller had previously registered, an additional CAS operation is done on the /controller
+   * ZNode to ensure that the KRaft epoch being registered is newer.
    *
    * @param kraftControllerId ID of the KRaft controller node
    * @param kraftControllerEpoch Epoch of the KRaft controller node
-   * @return An optional of the new zkVersion of /controller_epoch. None if we could not register the KRaft controller.
+   * @return A result object containing the written ZK controller epoch and version, or nothing.
    */
-  def tryRegisterKRaftControllerAsActiveController(kraftControllerId: Int, kraftControllerEpoch: Int): Option[Int] = {
+  def tryRegisterKRaftControllerAsActiveController(kraftControllerId: Int, kraftControllerEpoch: Int): KRaftRegistrationResult = {
     val timestamp = time.milliseconds()
     val curEpochOpt: Option[(Int, Int)] = getControllerEpoch.map(e => (e._1, e._2.getVersion))
-    val controllerOpt = getControllerId
-    val controllerEpochToStore = kraftControllerEpoch + 10000000 // TODO Remove this after KAFKA-14436
+    val controllerOpt = getControllerRegistration
+
+    // If we have a KRaft epoch registered in /controller, and it is not _older_ than the requested epoch, throw an error.
+    controllerOpt.flatMap(_.kraftEpoch).foreach { kraftEpochInZk =>
+      if (kraftEpochInZk >= kraftControllerEpoch) {
+        throw new ControllerMovedException(s"Cannot register KRaft controller $kraftControllerId with epoch $kraftControllerEpoch " +
+          s"as the current controller register in ZK has the same or newer epoch $kraftEpochInZk.")
+      }
+    }
+
     curEpochOpt match {
       case None =>
         throw new IllegalStateException(s"Cannot register KRaft controller $kraftControllerId as the active controller " +
           s"since there is no ZK controller epoch present.")
       case Some((curEpoch: Int, curEpochZk: Int)) =>
-        if (curEpoch >= controllerEpochToStore) {
-          // TODO KAFKA-14436 Need to ensure KRaft has a higher epoch an ZK
-          throw new IllegalStateException(s"Cannot register KRaft controller $kraftControllerId as the active controller " +
-            s"in ZK since its epoch ${controllerEpochToStore} is not higher than the current ZK epoch ${curEpoch}.")
-        }
-
-        val response = if (controllerOpt.isDefined) {
-          info(s"KRaft controller $kraftControllerId overwriting ${ControllerZNode.path} to become the active " +
-            s"controller with epoch $controllerEpochToStore. The previous controller was ${controllerOpt.get}.")
-          retryRequestUntilConnected(
-            MultiRequest(Seq(
-              SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(controllerEpochToStore), curEpochZk),
-              DeleteOp(ControllerZNode.path, ZkVersion.MatchAnyVersion),
-              CreateOp(ControllerZNode.path, ControllerZNode.encode(kraftControllerId, timestamp),
-                defaultAcls(ControllerZNode.path), CreateMode.PERSISTENT)))
-          )
-        } else {
-          info(s"KRaft controller $kraftControllerId creating ${ControllerZNode.path} to become the active " +
-            s"controller with epoch $controllerEpochToStore. There was no active controller.")
-          retryRequestUntilConnected(
-            MultiRequest(Seq(
-              SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(controllerEpochToStore), curEpochZk),
-              CreateOp(ControllerZNode.path, ControllerZNode.encode(kraftControllerId, timestamp),
-                defaultAcls(ControllerZNode.path), CreateMode.PERSISTENT)))
-          )
+        val newControllerEpoch = curEpoch + 1
+
+        val response = controllerOpt match {
+          case Some(controller) =>
+            info(s"KRaft controller $kraftControllerId overwriting ${ControllerZNode.path} to become the active " +
+              s"controller with ZK epoch $newControllerEpoch. The previous controller was ${controller.broker}.")
+            retryRequestUntilConnected(
+              MultiRequest(Seq(
+                SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(newControllerEpoch), curEpochZk),
+                DeleteOp(ControllerZNode.path, controller.zkVersion),
+                CreateOp(ControllerZNode.path, ControllerZNode.encode(kraftControllerId, timestamp, kraftControllerEpoch),
+                  defaultAcls(ControllerZNode.path), CreateMode.PERSISTENT)))
+            )
+          case None =>
+            info(s"KRaft controller $kraftControllerId creating ${ControllerZNode.path} to become the active " +
+              s"controller with ZK epoch $newControllerEpoch. There was no active controller.")
+            retryRequestUntilConnected(
+              MultiRequest(Seq(
+                SetDataOp(ControllerEpochZNode.path, ControllerEpochZNode.encode(newControllerEpoch), curEpochZk),
+                CreateOp(ControllerZNode.path, ControllerZNode.encode(kraftControllerId, timestamp, kraftControllerEpoch),
+                  defaultAcls(ControllerZNode.path), CreateMode.PERSISTENT)))
+            )
         }
 
-        val failureSuffix = s"while trying to register KRaft controller $kraftControllerId with epoch " +
-          s"$controllerEpochToStore. KRaft controller was not registered."
+        val failureSuffix = s"while trying to register KRaft controller $kraftControllerId with ZK epoch " +
+          s"$newControllerEpoch. KRaft controller was not registered."
         response.resultCode match {
           case Code.OK =>
-            info(s"Successfully registered KRaft controller $kraftControllerId with epoch $controllerEpochToStore")
+            info(s"Successfully registered KRaft controller $kraftControllerId with ZK epoch $newControllerEpoch")
             // First op is always SetData on /controller_epoch
             val setDataResult = response.zkOpResults(0).rawOpResult.asInstanceOf[SetDataResult]
-            Some(setDataResult.getStat.getVersion)
+            SuccessfulRegistrationResult(newControllerEpoch, setDataResult.getStat.getVersion)
           case Code.BADVERSION =>
-            info(s"The controller epoch changed $failureSuffix")
-            None
+            info(s"The ZK controller epoch changed $failureSuffix")
+            FailedRegistrationResult()
           case Code.NONODE =>
             info(s"The ephemeral node at ${ControllerZNode.path} went away $failureSuffix")
-            None
+            FailedRegistrationResult()
           case Code.NODEEXISTS =>
             info(s"The ephemeral node at ${ControllerZNode.path} was created by another controller $failureSuffix")
-            None
+            FailedRegistrationResult()
           case code =>
             error(s"ZooKeeper had an error $failureSuffix")
             throw KeeperException.create(code)
@@ -1210,6 +1223,17 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
     }
   }
 
+
+  def getControllerRegistration: Option[ZKControllerRegistration] = {
+    val getDataRequest = GetDataRequest(ControllerZNode.path)
+    val getDataResponse = retryRequestUntilConnected(getDataRequest)
+    getDataResponse.resultCode match {
+      case Code.OK => Some(ControllerZNode.decodeController(getDataResponse.data, getDataResponse.stat.getVersion))
+      case Code.NONODE => None
+      case _ => throw getDataResponse.resultException.get
+    }
+  }
+
   /**
    * Deletes the controller znode.
    * @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala
index 84b767d4d3b..b0337b90062 100644
--- a/core/src/main/scala/kafka/zk/ZkData.scala
+++ b/core/src/main/scala/kafka/zk/ZkData.scala
@@ -59,15 +59,28 @@ object ControllerZNode {
   def path = "/controller"
   def encode(brokerId: Int, timestamp: Long, kraftControllerEpoch: Int = -1): Array[Byte] = {
     Json.encodeAsBytes(Map(
-      "version" -> 2, "brokerid" -> brokerId,
+      "version" -> 2,
+      "brokerid" -> brokerId,
       "timestamp" -> timestamp.toString,
       "kraftControllerEpoch" -> kraftControllerEpoch).asJava)
   }
   def decode(bytes: Array[Byte]): Option[Int] = Json.parseBytes(bytes).map { js =>
     js.asJsonObject("brokerid").to[Int]
   }
+  def decodeController(bytes: Array[Byte], zkVersion: Int): ZKControllerRegistration = Json.tryParseBytes(bytes) match {
+    case Right(json) =>
+      val controller = json.asJsonObject
+      val brokerId = controller("brokerid").to[Int]
+      val kraftControllerEpoch = controller.get("kraftControllerEpoch").map(j => j.to[Int])
+      ZKControllerRegistration(brokerId, kraftControllerEpoch, zkVersion)
+
+    case Left(err) =>
+      throw new KafkaException(s"Failed to parse ZooKeeper registration for controller: ${new String(bytes, UTF_8)}", err)
+  }
 }
 
+case class ZKControllerRegistration(broker: Int, kraftEpoch: Option[Int], zkVersion: Int)
+
 object ControllerEpochZNode {
   def path = "/controller_epoch"
   def encode(epoch: Int): Array[Byte] = epoch.toString.getBytes(UTF_8)
diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
index 77f46b9c794..017f773ee21 100644
--- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
+++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
@@ -53,12 +53,9 @@ class ZkMigrationClient(zkClient: KafkaZkClient) extends MigrationClient with Lo
   }
 
   override def claimControllerLeadership(state: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
-    val epochZkVersionOpt = zkClient.tryRegisterKRaftControllerAsActiveController(
-      state.kraftControllerId(), state.kraftControllerEpoch())
-    if (epochZkVersionOpt.isDefined) {
-      state.withControllerZkVersion(epochZkVersionOpt.get)
-    } else {
-      state.withControllerZkVersion(-1)
+    zkClient.tryRegisterKRaftControllerAsActiveController(state.kraftControllerId(), state.kraftControllerEpoch()) match {
+      case SuccessfulRegistrationResult(_, controllerEpochZkVersion) => state.withControllerZkVersion(controllerEpochZkVersion)
+      case FailedRegistrationResult() => state.withControllerZkVersion(-1)
     }
   }
 
diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala
index 11eae3386f8..416abd23eb2 100644
--- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala
@@ -17,7 +17,7 @@
 package kafka.zk
 
 import java.nio.charset.StandardCharsets.UTF_8
-import java.util.concurrent.{CountDownLatch, TimeUnit}
+import java.util.concurrent.{CountDownLatch, Executors, TimeUnit}
 import java.util.{Collections, Properties}
 import kafka.api.LeaderAndIsr
 import kafka.cluster.{Broker, EndPoint}
@@ -1199,6 +1199,72 @@ class KafkaZkClientTest extends QuorumTestHarness {
       "Updating with wrong ZK version returns BADVERSION")
   }
 
+  @Test
+  def testRegisterZkControllerAfterKRaft(): Unit = {
+    // Register KRaft
+    var controllerEpochZkVersion = -1
+    zkClient.tryRegisterKRaftControllerAsActiveController(3000, 42) match {
+      case SuccessfulRegistrationResult(kraftEpoch, zkVersion) =>
+        assertEquals(2, kraftEpoch)
+        controllerEpochZkVersion = zkVersion
+      case FailedRegistrationResult() => fail("Expected to register KRaft as controller in ZK")
+    }
+    assertEquals(1, controllerEpochZkVersion)
+
+    // Can't register ZK anymore
+    assertThrows(classOf[ControllerMovedException], () => zkClient.registerControllerAndIncrementControllerEpoch(1))
+
+    // Delete controller, and try again
+    zkClient.deleteController(controllerEpochZkVersion)
+    val (newEpoch, newZkVersion) = zkClient.registerControllerAndIncrementControllerEpoch(1)
+    assertEquals(3, newEpoch)
+    assertEquals(2, newZkVersion)
+
+    zkClient.tryRegisterKRaftControllerAsActiveController(3000, 42) match {
+      case SuccessfulRegistrationResult(zkEpoch, zkVersion) =>
+        assertEquals(4, zkEpoch)
+        assertEquals(3, zkVersion)
+      case FailedRegistrationResult() => fail("Expected to register KRaft as controller in ZK")
+    }
+  }
+
+  @Test
+  def testConcurrentKRaftControllerClaim(): Unit = {
+    // Setup three threads to race on registering a KRaft controller in ZK
+    val registeredEpochs = new java.util.concurrent.ConcurrentLinkedQueue[Integer]()
+    val registeringNodes = new java.util.concurrent.ConcurrentHashMap[Integer, Integer]()
+
+    def newThread(nodeId: Int): Runnable = {
+      () => {
+        0.to(999).foreach(epoch =>
+          zkClient.tryRegisterKRaftControllerAsActiveController(nodeId, epoch) match {
+            case SuccessfulRegistrationResult(writtenEpoch, _) =>
+              registeredEpochs.add(writtenEpoch)
+              registeringNodes.compute(nodeId, (_, count) => if (count == null) {
+                0
+              } else {
+                count + 1
+              })
+            case FailedRegistrationResult() =>
+          }
+        )
+      }
+    }
+    val thread1 = newThread(1)
+    val thread2 = newThread(2)
+    val thread3 = newThread(3)
+    val executor = Executors.newFixedThreadPool(3)
+    executor.submit(thread1)
+    executor.submit(thread2)
+    executor.submit(thread3)
+    executor.shutdown()
+    executor.awaitTermination(30, TimeUnit.SECONDS)
+
+    assertEquals(1000, registeredEpochs.size())
+    val uniqueEpochs = registeredEpochs.asScala.toSet
+    assertEquals(1000, uniqueEpochs.size)
+  }
+
   @Test
   def testControllerManagementMethods(): Unit = {
     // No controller
diff --git a/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala b/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala
index 7fae24f650e..a8493d027d5 100644
--- a/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/ZkMigrationClientTest.scala
@@ -42,6 +42,9 @@ import scala.jdk.CollectionConverters._
  */
 class ZkMigrationClientTest extends QuorumTestHarness {
 
+  private val InitialControllerEpoch: Int = 42
+  private val InitialKRaftEpoch: Int = 0
+
   private var migrationClient: ZkMigrationClient = _
 
   private var migrationState: ZkMigrationLeadershipState = _
@@ -58,7 +61,7 @@ class ZkMigrationClientTest extends QuorumTestHarness {
 
   private def initialMigrationState: ZkMigrationLeadershipState = {
     val (_, stat) = zkClient.getControllerEpoch.get
-    new ZkMigrationLeadershipState(3000, 42, 100, 42, Time.SYSTEM.milliseconds(), -1, stat.getVersion)
+    new ZkMigrationLeadershipState(3000, InitialControllerEpoch, 100, InitialKRaftEpoch, Time.SYSTEM.milliseconds(), -1, stat.getVersion)
   }
 
   @Test
@@ -236,16 +239,22 @@ class ZkMigrationClientTest extends QuorumTestHarness {
   def testNonIncreasingKRaftEpoch(): Unit = {
     assertEquals(0, migrationState.migrationZkVersion())
 
+    migrationState = migrationState.withNewKRaftController(3001, InitialControllerEpoch)
     migrationState = migrationClient.claimControllerLeadership(migrationState)
     assertEquals(1, migrationState.controllerZkVersion())
 
-    migrationState = migrationState.withNewKRaftController(3000, 40)
-    val t1 = assertThrows(classOf[IllegalStateException], () => migrationClient.claimControllerLeadership(migrationState))
-    assertEquals("Cannot register KRaft controller 3000 as the active controller in ZK since its epoch 10000040 is not higher than the current ZK epoch 10000042.", t1.getMessage)
+    migrationState = migrationState.withNewKRaftController(3001, InitialControllerEpoch - 1)
+    val t1 = assertThrows(classOf[ControllerMovedException], () => migrationClient.claimControllerLeadership(migrationState))
+    assertEquals("Cannot register KRaft controller 3001 with epoch 41 as the current controller register in ZK has the same or newer epoch 42.", t1.getMessage)
+
+    migrationState = migrationState.withNewKRaftController(3001, InitialControllerEpoch)
+    val t2 = assertThrows(classOf[ControllerMovedException], () => migrationClient.claimControllerLeadership(migrationState))
+    assertEquals("Cannot register KRaft controller 3001 with epoch 42 as the current controller register in ZK has the same or newer epoch 42.", t2.getMessage)
 
-    migrationState = migrationState.withNewKRaftController(3000, 42)
-    val t2 = assertThrows(classOf[IllegalStateException], () => migrationClient.claimControllerLeadership(migrationState))
-    assertEquals("Cannot register KRaft controller 3000 as the active controller in ZK since its epoch 10000042 is not higher than the current ZK epoch 10000042.", t2.getMessage)
+    migrationState = migrationState.withNewKRaftController(3001, 100)
+    migrationState = migrationClient.claimControllerLeadership(migrationState)
+    assertEquals(migrationState.kraftControllerEpoch(), 100)
+    assertEquals(migrationState.kraftControllerId(), 3001)
   }
 
   @Test
@@ -259,8 +268,8 @@ class ZkMigrationClientTest extends QuorumTestHarness {
     migrationState = migrationClient.claimControllerLeadership(migrationState)
     assertEquals(2, migrationState.controllerZkVersion())
     zkClient.getControllerEpoch match {
-      case Some((kraftEpoch, stat)) =>
-        assertEquals(10000042, kraftEpoch)
+      case Some((zkEpoch, stat)) =>
+        assertEquals(3, zkEpoch)
         assertEquals(2, stat.getVersion)
       case None => fail()
     }
@@ -269,7 +278,7 @@ class ZkMigrationClientTest extends QuorumTestHarness {
 
     migrationState = migrationClient.releaseControllerLeadership(migrationState)
     val (epoch1, zkVersion1) = zkClient.registerControllerAndIncrementControllerEpoch(100)
-    assertEquals(epoch1, 10000043)
+    assertEquals(epoch1, 4)
     assertEquals(zkVersion1, 3)
   }
 


[kafka] 11/11: MINOR: ControllerServer should use the new metadata loader and snapshot generator (#12983)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 59bad9e47f2d25f3f197cb1a85fe13380512a100
Author: Colin Patrick McCabe <cm...@apache.org>
AuthorDate: Thu Dec 15 16:53:07 2022 -0800

    MINOR: ControllerServer should use the new metadata loader and snapshot generator (#12983)
    
    This PR introduces the new metadata loader and snapshot generator. For the time being, they are
    only used by the controller, but a PR for the broker will come soon.
    
    The new metadata loader supports adding and removing publishers dynamically. (In contrast, the old
    loader only supported adding a single publisher.) It also passes along more information about each
    new image that is published. This information can be found in the LogDeltaManifest and
    SnapshotManifest classes.
    
    The new snapshot generator replaces the previous logic for generating snapshots in
    QuorumController.java and associated classes. The new generator is intended to be shared between
    the broker and the controller, so it is decoupled from both.
    
    There are a few small changes to the old snapshot generator in this PR. Specifically, we move the
    batch processing time and batch size metrics out of BrokerMetadataListener.scala and into
    BrokerServerMetrics.scala.
    
    Finally, fix a case where we are using 'is' rather than '==' for a numeric comparison in
    snapshot_test.py.
    
    Reviewers: David Arthur <mu...@gmail.com>
---
 checkstyle/import-control.xml                      |   6 +-
 .../main/scala/kafka/server/ControllerServer.scala |   3 +-
 .../src/main/scala/kafka/server/SharedServer.scala |  77 ++-
 .../server/metadata/BrokerMetadataListener.scala   |  39 +-
 .../server/metadata/BrokerServerMetrics.scala      |  59 ++-
 core/src/test/java/kafka/test/MockController.java  |   5 -
 .../server/metadata/BrokerServerMetricsTest.scala  |  21 +-
 .../kafka/server/DynamicConfigChangeTest.scala     |   4 +-
 .../metadata/BrokerMetadataListenerTest.scala      |   8 +-
 .../apache/kafka/controller/AclControlManager.java |  18 -
 .../controller/ClientQuotaControlManager.java      |  42 --
 .../kafka/controller/ClusterControlManager.java    |  60 ---
 .../controller/ConfigurationControlManager.java    |  36 --
 .../org/apache/kafka/controller/Controller.java    |   8 -
 .../kafka/controller/FeatureControlManager.java    |  44 --
 .../apache/kafka/controller/LogReplayTracker.java  |   3 +-
 .../kafka/controller/ProducerIdControlManager.java |  24 +-
 .../apache/kafka/controller/QuorumController.java  | 307 +----------
 .../controller/ReplicationControlManager.java      |  34 --
 .../apache/kafka/controller/SnapshotGenerator.java | 132 -----
 .../kafka/image/loader/LogDeltaManifest.java       | 106 ++++
 .../apache/kafka/image/loader/MetadataLoader.java  | 572 +++++++++++++++++++++
 .../kafka/image/loader/MetadataLoaderMetrics.java  |  46 ++
 .../kafka/image/loader/SnapshotManifest.java       |  77 +++
 .../kafka/image/publisher/MetadataPublisher.java   |  72 +++
 .../kafka/image/publisher/SnapshotEmitter.java     | 123 +++++
 .../kafka/image/publisher/SnapshotGenerator.java   | 275 ++++++++++
 .../kafka/controller/AclControlManagerTest.java    |  24 +-
 .../controller/ClientQuotaControlManagerTest.java  |  59 ++-
 .../controller/ClusterControlManagerTest.java      |  27 +-
 .../ConfigurationControlManagerTest.java           |   8 -
 .../controller/FeatureControlManagerTest.java      |  38 +-
 .../controller/ProducerIdControlManagerTest.java   |  30 +-
 .../kafka/controller/QuorumControllerTest.java     | 401 ++-------------
 .../controller/ReplicationControlManagerTest.java  |  11 -
 .../kafka/controller/SnapshotGeneratorTest.java    | 112 ----
 .../kafka/image/loader/MetadataLoaderTest.java     | 517 +++++++++++++++++++
 .../kafka/image/publisher/SnapshotEmitterTest.java | 206 ++++++++
 .../image/publisher/SnapshotGeneratorTest.java     | 185 +++++++
 .../org/apache/kafka/metalog/LocalLogManager.java  |  19 +-
 .../kafka/metalog/LocalLogManagerTestEnv.java      |   7 +
 tests/kafkatest/tests/core/snapshot_test.py        |   2 +-
 42 files changed, 2486 insertions(+), 1361 deletions(-)

diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index df9a2e9adfd..32f13d0e340 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -225,6 +225,7 @@
     <allow pkg="org.apache.kafka.common.requests" />
     <allow pkg="org.apache.kafka.common.resource" />
     <allow pkg="org.apache.kafka.controller" />
+    <allow pkg="org.apache.kafka.image.writer" />
     <allow pkg="org.apache.kafka.metadata" />
     <allow pkg="org.apache.kafka.metadata.authorizer" />
     <allow pkg="org.apache.kafka.metadata.migration" />
@@ -248,15 +249,18 @@
     <allow pkg="org.apache.kafka.common.metadata" />
     <allow pkg="org.apache.kafka.common.protocol" />
     <allow pkg="org.apache.kafka.common.quota" />
-    <allow pkg="org.apache.kafka.common.resource" />
     <allow pkg="org.apache.kafka.common.requests" />
+    <allow pkg="org.apache.kafka.common.resource" />
     <allow pkg="org.apache.kafka.image" />
     <allow pkg="org.apache.kafka.image.writer" />
     <allow pkg="org.apache.kafka.metadata" />
+    <allow pkg="org.apache.kafka.queue" />
     <allow pkg="org.apache.kafka.raft" />
     <allow pkg="org.apache.kafka.server.common" />
+    <allow pkg="org.apache.kafka.server.fault" />
     <allow pkg="org.apache.kafka.server.util" />
     <allow pkg="org.apache.kafka.snapshot" />
+    <allow pkg="org.apache.kafka.test" />
   </subpackage>
 
   <subpackage name="metadata">
diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
index 03b86b7067e..77f9ba546fd 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -112,6 +112,7 @@ class ControllerServer(
       maybeChangeStatus(STARTING, STARTED)
       this.logIdent = new LogContext(s"[ControllerServer id=${config.nodeId}] ").logPrefix()
 
+
       newGauge("ClusterId", () => clusterId)
       newGauge("yammer-metrics-count", () =>  KafkaYammerMetrics.defaultRegistry.allMetrics.size)
 
@@ -194,8 +195,6 @@ class ControllerServer(
           setDefaultNumPartitions(config.numPartitions.intValue()).
           setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(),
             TimeUnit.MILLISECONDS)).
-          setSnapshotMaxNewRecordBytes(config.metadataSnapshotMaxNewRecordBytes).
-          setSnapshotMaxIntervalMs(config.metadataSnapshotMaxIntervalMs).
           setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs).
           setMaxIdleIntervalNs(maxIdleIntervalNs).
           setMetrics(sharedServer.controllerMetrics).
diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala
index 8b647e7464f..151429a1797 100644
--- a/core/src/main/scala/kafka/server/SharedServer.scala
+++ b/core/src/main/scala/kafka/server/SharedServer.scala
@@ -25,6 +25,8 @@ import kafka.utils.{CoreUtils, Logging}
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time}
 import org.apache.kafka.controller.QuorumControllerMetrics
+import org.apache.kafka.image.loader.MetadataLoader
+import org.apache.kafka.image.publisher.{SnapshotEmitter, SnapshotGenerator}
 import org.apache.kafka.metadata.MetadataRecordSerde
 import org.apache.kafka.raft.RaftConfig.AddressSpec
 import org.apache.kafka.server.common.ApiMessageAndVersion
@@ -32,7 +34,9 @@ import org.apache.kafka.server.fault.{FaultHandler, LoggingFaultHandler, Process
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
 
 import java.util
+import java.util.Collections
 import java.util.concurrent.CompletableFuture
+import java.util.concurrent.atomic.AtomicReference
 
 
 /**
@@ -97,6 +101,10 @@ class SharedServer(
   @volatile var raftManager: KafkaRaftManager[ApiMessageAndVersion] = _
   @volatile var brokerMetrics: BrokerServerMetrics = _
   @volatile var controllerMetrics: QuorumControllerMetrics = _
+  @volatile var loader: MetadataLoader = _
+  val snapshotsDiabledReason = new AtomicReference[String](null)
+  @volatile var snapshotEmitter: SnapshotEmitter = _
+  @volatile var snapshotGenerator: SnapshotGenerator = _
 
   def isUsed(): Boolean = synchronized {
     usedByController || usedByBroker
@@ -145,39 +153,48 @@ class SharedServer(
   /**
    * The fault handler to use when metadata loading fails.
    */
-  def metadataLoaderFaultHandler: FaultHandler = faultHandlerFactory.build("metadata loading",
+  def metadataLoaderFaultHandler: FaultHandler = faultHandlerFactory.build(
+    name = "metadata loading",
     fatal = sharedServerConfig.processRoles.contains(ControllerRole),
     action = () => SharedServer.this.synchronized {
       if (brokerMetrics != null) brokerMetrics.metadataLoadErrorCount.getAndIncrement()
       if (controllerMetrics != null) controllerMetrics.incrementMetadataErrorCount()
+      snapshotsDiabledReason.compareAndSet(null, "metadata loading fault")
     })
 
   /**
    * The fault handler to use when the initial broker metadata load fails.
    */
-  def initialBrokerMetadataLoadFaultHandler: FaultHandler = faultHandlerFactory.build("initial metadata loading",
+  def initialBrokerMetadataLoadFaultHandler: FaultHandler = faultHandlerFactory.build(
+    name = "initial broker metadata loading",
     fatal = true,
     action = () => SharedServer.this.synchronized {
       if (brokerMetrics != null) brokerMetrics.metadataApplyErrorCount.getAndIncrement()
       if (controllerMetrics != null) controllerMetrics.incrementMetadataErrorCount()
+      snapshotsDiabledReason.compareAndSet(null, "initial broker metadata loading fault")
     })
 
   /**
    * The fault handler to use when the QuorumController experiences a fault.
    */
-  def quorumControllerFaultHandler: FaultHandler = faultHandlerFactory.build("quorum controller",
+  def quorumControllerFaultHandler: FaultHandler = faultHandlerFactory.build(
+    name = "quorum controller",
     fatal = true,
-    action = () => {}
-  )
+    action = () => SharedServer.this.synchronized {
+      if (controllerMetrics != null) controllerMetrics.incrementMetadataErrorCount()
+      snapshotsDiabledReason.compareAndSet(null, "quorum controller fault")
+    })
 
   /**
    * The fault handler to use when metadata cannot be published.
    */
-  def metadataPublishingFaultHandler: FaultHandler = faultHandlerFactory.build("metadata publishing",
+  def metadataPublishingFaultHandler: FaultHandler = faultHandlerFactory.build(
+    name = "metadata publishing",
     fatal = false,
     action = () => SharedServer.this.synchronized {
       if (brokerMetrics != null) brokerMetrics.metadataApplyErrorCount.getAndIncrement()
       if (controllerMetrics != null) controllerMetrics.incrementMetadataErrorCount()
+      // Note: snapshot generation does not need to be disabled for a publishing fault.
     })
 
   private def start(): Unit = synchronized {
@@ -210,6 +227,40 @@ class SharedServer(
           threadNamePrefix,
           controllerQuorumVotersFuture)
         raftManager.startup()
+
+        if (sharedServerConfig.processRoles.contains(ControllerRole)) {
+          val loaderBuilder = new MetadataLoader.Builder().
+            setNodeId(metaProps.nodeId).
+            setTime(time).
+            setThreadNamePrefix(threadNamePrefix.getOrElse("")).
+            setFaultHandler(metadataLoaderFaultHandler).
+            setHighWaterMarkAccessor(() => raftManager.client.highWatermark())
+          if (brokerMetrics != null) {
+            loaderBuilder.setMetadataLoaderMetrics(brokerMetrics)
+          }
+          loader = loaderBuilder.build()
+          snapshotEmitter = new SnapshotEmitter.Builder().
+            setNodeId(metaProps.nodeId).
+            setRaftClient(raftManager.client).
+            build()
+          snapshotGenerator = new SnapshotGenerator.Builder(snapshotEmitter).
+            setNodeId(metaProps.nodeId).
+            setTime(time).
+            setFaultHandler(metadataPublishingFaultHandler).
+            setMaxBytesSinceLastSnapshot(sharedServerConfig.metadataSnapshotMaxNewRecordBytes).
+            setMaxTimeSinceLastSnapshotNs(sharedServerConfig.metadataSnapshotMaxIntervalMs).
+            setDisabledReason(snapshotsDiabledReason).
+            build()
+          raftManager.register(loader)
+          try {
+            loader.installPublishers(Collections.singletonList(snapshotGenerator))
+          } catch {
+            case t: Throwable => {
+              error("Unable to install metadata publishers", t)
+              throw new RuntimeException("Unable to install metadata publishers.", t)
+            }
+          }
+        }
         debug("Completed SharedServer startup.")
         started = true
       } catch {
@@ -235,6 +286,20 @@ class SharedServer(
       debug("SharedServer is not running.")
     } else {
       info("Stopping SharedServer")
+      if (loader != null) {
+        CoreUtils.swallow(loader.beginShutdown(), this)
+      }
+      if (snapshotGenerator != null) {
+        CoreUtils.swallow(snapshotGenerator.beginShutdown(), this)
+      }
+      if (loader != null) {
+        CoreUtils.swallow(loader.close(), this)
+        loader = null
+      }
+      if (snapshotGenerator != null) {
+        CoreUtils.swallow(snapshotGenerator.close(), this)
+        snapshotGenerator = null
+      }
       if (raftManager != null) {
         CoreUtils.swallow(raftManager.shutdown(), this)
         raftManager = null
diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
index 21fc126691f..789ae89f049 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala
@@ -18,7 +18,7 @@ package kafka.server.metadata
 
 import java.util
 import java.util.concurrent.atomic.AtomicBoolean
-import java.util.concurrent.{CompletableFuture, TimeUnit}
+import java.util.concurrent.CompletableFuture
 import kafka.metrics.KafkaMetricsGroup
 import org.apache.kafka.common.utils.{LogContext, Time}
 import org.apache.kafka.image.writer.{ImageWriterOptions, RecordListWriter}
@@ -30,14 +30,10 @@ import org.apache.kafka.server.common.ApiMessageAndVersion
 import org.apache.kafka.server.fault.FaultHandler
 import org.apache.kafka.snapshot.SnapshotReader
 
+import java.util.concurrent.TimeUnit.NANOSECONDS
 import scala.compat.java8.OptionConverters._
 
 
-object BrokerMetadataListener {
-  val MetadataBatchProcessingTimeUs = "MetadataBatchProcessingTimeUs"
-  val MetadataBatchSizes = "MetadataBatchSizes"
-}
-
 class BrokerMetadataListener(
   val brokerId: Int,
   time: Time,
@@ -65,16 +61,6 @@ class BrokerMetadataListener(
   private val log = logContext.logger(classOf[BrokerMetadataListener])
   logIdent = logContext.logPrefix()
 
-  /**
-   * A histogram tracking the time in microseconds it took to process batches of events.
-   */
-  private val batchProcessingTimeHist = newHistogram(BrokerMetadataListener.MetadataBatchProcessingTimeUs)
-
-  /**
-   * A histogram tracking the sizes of batches that we have processed.
-   */
-  private val metadataBatchSizeHist = newHistogram(BrokerMetadataListener.MetadataBatchSizes)
-
   /**
    * The highest metadata offset that we've seen.  Written only from the event queue thread.
    */
@@ -293,14 +279,14 @@ class BrokerMetadataListener(
         }
       }
       numBytes = numBytes + batch.sizeInBytes()
-      metadataBatchSizeHist.update(batch.records().size())
+      brokerMetrics.updateBatchSize(batch.records().size())
       numBatches = numBatches + 1
     }
 
     val endTimeNs = time.nanoseconds()
-    val elapsedUs = TimeUnit.MICROSECONDS.convert(endTimeNs - startTimeNs, TimeUnit.NANOSECONDS)
-    batchProcessingTimeHist.update(elapsedUs)
-    BatchLoadResults(numBatches, numRecords, elapsedUs, numBytes)
+    val elapsedNs = endTimeNs - startTimeNs
+    brokerMetrics.updateBatchProcessingTime(elapsedNs)
+    BatchLoadResults(numBatches, numRecords, NANOSECONDS.toMicros(elapsedNs), numBytes)
   }
 
   def startPublishing(publisher: MetadataPublisher): CompletableFuture[Void] = {
@@ -367,8 +353,7 @@ class BrokerMetadataListener(
     publisher.publish(delta, _image)
 
     // Update the metrics since the publisher handled the lastest image
-    brokerMetrics.lastAppliedRecordOffset.set(_highestOffset)
-    brokerMetrics.lastAppliedRecordTimestamp.set(_highestTimestamp)
+    brokerMetrics.updateLastAppliedImageProvenance(_image.provenance())
   }
 
   override def handleLeaderChange(leaderAndEpoch: LeaderAndEpoch): Unit = {
@@ -376,15 +361,7 @@ class BrokerMetadataListener(
   }
 
   override def beginShutdown(): Unit = {
-    eventQueue.beginShutdown("beginShutdown", new ShutdownEvent())
-  }
-
-  class ShutdownEvent extends EventQueue.FailureLoggingEvent(log) {
-    override def run(): Unit = {
-      brokerMetrics.close()
-      removeMetric(BrokerMetadataListener.MetadataBatchProcessingTimeUs)
-      removeMetric(BrokerMetadataListener.MetadataBatchSizes)
-    }
+    eventQueue.beginShutdown("beginShutdown")
   }
 
   def close(): Unit = {
diff --git a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala
index 3e68ae85f92..465b10f1e54 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala
@@ -17,17 +17,48 @@
 
 package kafka.server.metadata
 
-import java.util.concurrent.atomic.AtomicLong
+import kafka.metrics.KafkaMetricsGroup
+
+import java.util.concurrent.atomic.{AtomicLong, AtomicReference}
 import org.apache.kafka.common.MetricName
 import org.apache.kafka.common.metrics.Gauge
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.metrics.MetricConfig
+import org.apache.kafka.image.MetadataProvenance
+import org.apache.kafka.image.loader.MetadataLoaderMetrics
+import org.apache.kafka.server.metrics.KafkaYammerMetrics
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
 
-final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable {
+final class BrokerServerMetrics private (
+  metrics: Metrics
+) extends MetadataLoaderMetrics with KafkaMetricsGroup {
   import BrokerServerMetrics._
 
-  val lastAppliedRecordOffset: AtomicLong = new AtomicLong(0)
-  val lastAppliedRecordTimestamp: AtomicLong = new AtomicLong(0)
+  private val batchProcessingTimeHistName = explicitMetricName("kafka.server",
+    "BrokerMetadataListener",
+    "MetadataBatchProcessingTimeUs",
+    Map.empty)
+
+  /**
+   * A histogram tracking the time in microseconds it took to process batches of events.
+   */
+  private val batchProcessingTimeHist =
+    KafkaYammerMetrics.defaultRegistry().newHistogram(batchProcessingTimeHistName, true)
+
+  private val batchSizeHistName = explicitMetricName("kafka.server",
+    "BrokerMetadataListener",
+    "MetadataBatchSizes",
+    Map.empty)
+
+  /**
+   * A histogram tracking the sizes of batches that we have processed.
+   */
+  private val batchSizeHist =
+    KafkaYammerMetrics.defaultRegistry().newHistogram(batchSizeHistName, true)
+
+  val lastAppliedImageProvenance: AtomicReference[MetadataProvenance] =
+    new AtomicReference[MetadataProvenance](MetadataProvenance.EMPTY)
   val metadataLoadErrorCount: AtomicLong = new AtomicLong(0)
   val metadataApplyErrorCount: AtomicLong = new AtomicLong(0)
 
@@ -62,15 +93,15 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable
   )
 
   addMetric(metrics, lastAppliedRecordOffsetName) { _ =>
-    lastAppliedRecordOffset.get
+    lastAppliedImageProvenance.get.offset()
   }
 
   addMetric(metrics, lastAppliedRecordTimestampName) { _ =>
-    lastAppliedRecordTimestamp.get
+    lastAppliedImageProvenance.get.lastContainedLogTimeMs()
   }
 
   addMetric(metrics, lastAppliedRecordLagMsName) { now =>
-    now - lastAppliedRecordTimestamp.get
+    now - lastAppliedImageProvenance.get.lastContainedLogTimeMs()
   }
 
   addMetric(metrics, metadataLoadErrorCountName) { _ =>
@@ -82,6 +113,8 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable
   }
 
   override def close(): Unit = {
+    KafkaYammerMetrics.defaultRegistry().removeMetric(batchProcessingTimeHistName)
+    KafkaYammerMetrics.defaultRegistry().removeMetric(batchSizeHistName)
     List(
       lastAppliedRecordOffsetName,
       lastAppliedRecordTimestampName,
@@ -90,6 +123,18 @@ final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable
       metadataApplyErrorCountName
     ).foreach(metrics.removeMetric)
   }
+
+  override def updateBatchProcessingTime(elapsedNs: Long): Unit =
+    batchProcessingTimeHist.update(NANOSECONDS.toMicros(elapsedNs))
+
+  override def updateBatchSize(size: Int): Unit = batchSizeHist.update(size)
+
+  override def updateLastAppliedImageProvenance(provenance: MetadataProvenance): Unit =
+    lastAppliedImageProvenance.set(provenance)
+
+  override def lastAppliedOffset(): Long = lastAppliedImageProvenance.get().offset()
+
+  def lastAppliedTimestamp(): Long = lastAppliedImageProvenance.get().lastContainedLogTimeMs()
 }
 
 
diff --git a/core/src/test/java/kafka/test/MockController.java b/core/src/test/java/kafka/test/MockController.java
index ff1154d2119..061e19213e3 100644
--- a/core/src/test/java/kafka/test/MockController.java
+++ b/core/src/test/java/kafka/test/MockController.java
@@ -436,11 +436,6 @@ public class MockController implements Controller {
         return CompletableFuture.completedFuture(results);
     }
 
-    @Override
-    public CompletableFuture<Long> beginWritingSnapshot() {
-        throw new UnsupportedOperationException();
-    }
-
     @Override
     public void beginShutdown() {
         this.active = false;
diff --git a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala
index ea2b439c166..200deed4270 100644
--- a/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala
+++ b/core/src/test/scala/kafka/server/metadata/BrokerServerMetricsTest.scala
@@ -22,9 +22,11 @@ import kafka.utils.TestUtils
 import org.apache.kafka.common.MetricName
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.utils.MockTime
+import org.apache.kafka.image.MetadataProvenance
 import org.junit.jupiter.api.Assertions.assertEquals
 import org.junit.jupiter.api.Assertions.assertTrue
 import org.junit.jupiter.api.Test
+
 import scala.jdk.CollectionConverters._
 
 final class BrokerServerMetricsTest {
@@ -59,11 +61,14 @@ final class BrokerServerMetricsTest {
     val metrics = new Metrics()
     TestUtils.resource(BrokerServerMetrics(metrics)) { brokerMetrics =>
       val offsetMetric = metrics.metrics().get(brokerMetrics.lastAppliedRecordOffsetName)
-      assertEquals(0, offsetMetric.metricValue.asInstanceOf[Long])
+      assertEquals(-1L, offsetMetric.metricValue.asInstanceOf[Long])
 
       // Update metric value and check
       val expectedValue = 1000
-      brokerMetrics.lastAppliedRecordOffset.set(expectedValue)
+      brokerMetrics.updateLastAppliedImageProvenance(new MetadataProvenance(
+        expectedValue,
+        brokerMetrics.lastAppliedImageProvenance.get().epoch(),
+        brokerMetrics.lastAppliedTimestamp()));
       assertEquals(expectedValue, offsetMetric.metricValue.asInstanceOf[Long])
     }
   }
@@ -77,12 +82,16 @@ final class BrokerServerMetricsTest {
       val timestampMetric = metrics.metrics().get(brokerMetrics.lastAppliedRecordTimestampName)
       val lagMetric = metrics.metrics().get(brokerMetrics.lastAppliedRecordLagMsName)
 
-      assertEquals(0, timestampMetric.metricValue.asInstanceOf[Long])
-      assertEquals(time.milliseconds, lagMetric.metricValue.asInstanceOf[Long])
+      assertEquals(-1L, timestampMetric.metricValue.asInstanceOf[Long])
+      assertEquals(time.milliseconds + 1, lagMetric.metricValue.asInstanceOf[Long])
 
       // Update metric value and check
-      val timestamp = 500
-      brokerMetrics.lastAppliedRecordTimestamp.set(timestamp)
+      val timestamp = 500L
+
+      brokerMetrics.updateLastAppliedImageProvenance(new MetadataProvenance(
+        brokerMetrics.lastAppliedOffset(),
+        brokerMetrics.lastAppliedImageProvenance.get().epoch(),
+        timestamp))
       assertEquals(timestamp, timestampMetric.metricValue.asInstanceOf[Long])
       assertEquals(time.milliseconds - timestamp, lagMetric.metricValue.asInstanceOf[Long])
     }
diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
index 84d6f5a2ef9..bd1b506e27c 100644
--- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
@@ -22,7 +22,6 @@ import java.util
 import java.util.Collections.{singletonList, singletonMap}
 import java.util.{Collections, Properties}
 import java.util.concurrent.ExecutionException
-
 import kafka.integration.KafkaServerTestHarness
 import kafka.log.LogConfig._
 import kafka.utils._
@@ -43,7 +42,7 @@ import org.apache.kafka.common.record.{CompressionType, RecordVersion}
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1
 import org.junit.jupiter.api.Assertions._
-import org.junit.jupiter.api.Test
+import org.junit.jupiter.api.{Test, Timeout}
 import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.ValueSource
 import org.mockito.ArgumentMatchers.{any, anyString}
@@ -53,6 +52,7 @@ import scala.annotation.nowarn
 import scala.collection.{Map, Seq}
 import scala.jdk.CollectionConverters._
 
+@Timeout(100)
 class DynamicConfigChangeTest extends KafkaServerTestHarness {
   def generateConfigs = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnectOrNull)))
 
diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala
index a589c0572dd..e559a6b753d 100644
--- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala
@@ -86,8 +86,8 @@ class BrokerMetadataListenerTest {
       val imageRecords = listener.getImageRecords().get()
       assertEquals(0, imageRecords.size())
       assertEquals(100L, listener.highestMetadataOffset)
-      assertEquals(0L, metrics.lastAppliedRecordOffset.get)
-      assertEquals(0L, metrics.lastAppliedRecordTimestamp.get)
+      assertEquals(-1L, metrics.lastAppliedOffset())
+      assertEquals(-1L, metrics.lastAppliedTimestamp())
       assertEquals(0L, metrics.metadataLoadErrorCount.get)
       assertEquals(0L, metrics.metadataApplyErrorCount.get)
 
@@ -121,8 +121,8 @@ class BrokerMetadataListenerTest {
         override def publishedOffset: Long = -1
       }).get()
 
-      assertEquals(fencedLastOffset, metrics.lastAppliedRecordOffset.get)
-      assertEquals(fencedTimestamp, metrics.lastAppliedRecordTimestamp.get)
+      assertEquals(fencedLastOffset, metrics.lastAppliedOffset())
+      assertEquals(fencedTimestamp, metrics.lastAppliedTimestamp())
       assertEquals(0L, metrics.metadataLoadErrorCount.get)
       assertEquals(0L, metrics.metadataApplyErrorCount.get)
     } finally {
diff --git a/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java
index d3fc0fe76ed..313927fc537 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java
@@ -28,7 +28,6 @@ import org.apache.kafka.common.metadata.RemoveAccessControlEntryRecord;
 import org.apache.kafka.common.requests.ApiError;
 import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer;
 import org.apache.kafka.metadata.authorizer.StandardAcl;
-import org.apache.kafka.metadata.authorizer.StandardAclRecordIterator;
 import org.apache.kafka.metadata.authorizer.StandardAclWithId;
 import org.apache.kafka.raft.OffsetAndEpoch;
 import org.apache.kafka.server.authorizer.AclCreateResult;
@@ -42,7 +41,6 @@ import org.apache.kafka.timeline.TimelineHashSet;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -216,20 +214,4 @@ public class AclControlManager {
     Map<Uuid, StandardAcl> idToAcl() {
         return Collections.unmodifiableMap(idToAcl);
     }
-
-    Iterator<List<ApiMessageAndVersion>> iterator(long epoch) {
-        Iterator<Entry<Uuid, StandardAcl>> iterator = idToAcl.entrySet(epoch).iterator();
-        return new StandardAclRecordIterator(new Iterator<StandardAclWithId>() {
-            @Override
-            public boolean hasNext() {
-                return iterator.hasNext();
-            }
-
-            @Override
-            public StandardAclWithId next() {
-                Entry<Uuid, StandardAcl> entry = iterator.next();
-                return new StandardAclWithId(entry.getKey(), entry.getValue());
-            }
-        });
-    }
 }
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java
index 504994b3a5b..b859bbfd65d 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java
@@ -36,11 +36,9 @@ 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.NoSuchElementException;
 import java.util.Objects;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
@@ -288,44 +286,4 @@ public class ClientQuotaControlManager {
 
         return ApiError.NONE;
     }
-
-    class ClientQuotaControlIterator implements Iterator<List<ApiMessageAndVersion>> {
-        private final long epoch;
-        private final Iterator<Entry<ClientQuotaEntity, TimelineHashMap<String, Double>>> iterator;
-
-        ClientQuotaControlIterator(long epoch) {
-            this.epoch = epoch;
-            this.iterator = clientQuotaData.entrySet(epoch).iterator();
-        }
-
-        @Override
-        public boolean hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public List<ApiMessageAndVersion> next() {
-            if (!hasNext()) throw new NoSuchElementException();
-            Entry<ClientQuotaEntity, TimelineHashMap<String, Double>> entry = iterator.next();
-            ClientQuotaEntity entity = entry.getKey();
-            List<ApiMessageAndVersion> records = new ArrayList<>();
-            for (Entry<String, Double> quotaEntry : entry.getValue().entrySet(epoch)) {
-                ClientQuotaRecord record = new ClientQuotaRecord();
-                for (Entry<String, String> entityEntry : entity.entries().entrySet()) {
-                    record.entity().add(new EntityData().
-                        setEntityType(entityEntry.getKey()).
-                        setEntityName(entityEntry.getValue()));
-                }
-                record.setKey(quotaEntry.getKey());
-                record.setValue(quotaEntry.getValue());
-                record.setRemove(false);
-                records.add(new ApiMessageAndVersion(record, (short) 0));
-            }
-            return records;
-        }
-    }
-
-    ClientQuotaControlIterator iterator(long epoch) {
-        return new ClientQuotaControlIterator(epoch);
-    }
 }
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
index 95aae773fb4..ba6c0e1d1c2 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
@@ -29,9 +29,7 @@ import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
 import org.apache.kafka.common.metadata.FenceBrokerRecord;
 import org.apache.kafka.common.metadata.RegisterBrokerRecord;
 import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint;
-import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpointCollection;
 import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerFeature;
-import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerFeatureCollection;
 import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
 import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
 import org.apache.kafka.common.protocol.ApiMessage;
@@ -59,7 +57,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.NoSuchElementException;
 import java.util.Optional;
 import java.util.OptionalLong;
 import java.util.Random;
@@ -68,7 +65,6 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
-import static java.util.Collections.singletonList;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 
 
@@ -654,60 +650,4 @@ public class ClusterControlManager {
             readyBrokersFuture = Optional.empty();
         }
     }
-
-    class ClusterControlIterator implements Iterator<List<ApiMessageAndVersion>> {
-        private final Iterator<Entry<Integer, BrokerRegistration>> iterator;
-        private final MetadataVersion metadataVersion;
-
-        ClusterControlIterator(long epoch) {
-            this.iterator = brokerRegistrations.entrySet(epoch).iterator();
-            this.metadataVersion = featureControl.metadataVersion();
-        }
-
-        @Override
-        public boolean hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public List<ApiMessageAndVersion> next() {
-            if (!hasNext()) throw new NoSuchElementException();
-            Entry<Integer, BrokerRegistration> entry = iterator.next();
-            int brokerId = entry.getKey();
-            BrokerRegistration registration = entry.getValue();
-            BrokerEndpointCollection endpoints = new BrokerEndpointCollection();
-            for (Entry<String, Endpoint> endpointEntry : registration.listeners().entrySet()) {
-                endpoints.add(new BrokerEndpoint().setName(endpointEntry.getKey()).
-                    setHost(endpointEntry.getValue().host()).
-                    setPort(endpointEntry.getValue().port()).
-                    setSecurityProtocol(endpointEntry.getValue().securityProtocol().id));
-            }
-            BrokerFeatureCollection features = new BrokerFeatureCollection();
-            for (Entry<String, VersionRange> featureEntry : registration.supportedFeatures().entrySet()) {
-                features.add(new BrokerFeature().setName(featureEntry.getKey()).
-                    setMaxSupportedVersion(featureEntry.getValue().max()).
-                    setMinSupportedVersion(featureEntry.getValue().min()));
-            }
-            RegisterBrokerRecord record = new RegisterBrokerRecord().
-                setBrokerId(brokerId).
-                setIncarnationId(registration.incarnationId()).
-                setBrokerEpoch(registration.epoch()).
-                setEndPoints(endpoints).
-                setFeatures(features).
-                setRack(registration.rack().orElse(null)).
-                setFenced(registration.fenced());
-            if (metadataVersion.isInControlledShutdownStateSupported()) {
-                record.setInControlledShutdown(registration.inControlledShutdown());
-            }
-            if (metadataVersion.isMigrationSupported()) {
-                record.setIsMigratingZkBroker(registration.isMigratingZkBroker());
-            }
-            return singletonList(new ApiMessageAndVersion(record,
-                metadataVersion.registerBrokerRecordVersion()));
-        }
-    }
-
-    ClusterControlIterator iterator(long epoch) {
-        return new ClusterControlIterator(epoch);
-    }
 }
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
index d569ae1a6ca..b5d71230cf4 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
@@ -43,7 +43,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.NoSuchElementException;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.function.Consumer;
@@ -474,39 +473,4 @@ public class ConfigurationControlManager {
         Map<String, String> result = configData.get(currentController);
         return (result == null) ? Collections.emptyMap() : result;
     }
-
-    class ConfigurationControlIterator implements Iterator<List<ApiMessageAndVersion>> {
-        private final long epoch;
-        private final Iterator<Entry<ConfigResource, TimelineHashMap<String, String>>> iterator;
-
-        ConfigurationControlIterator(long epoch) {
-            this.epoch = epoch;
-            this.iterator = configData.entrySet(epoch).iterator();
-        }
-
-        @Override
-        public boolean hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public List<ApiMessageAndVersion> next() {
-            if (!hasNext()) throw new NoSuchElementException();
-            List<ApiMessageAndVersion> records = new ArrayList<>();
-            Entry<ConfigResource, TimelineHashMap<String, String>> entry = iterator.next();
-            ConfigResource resource = entry.getKey();
-            for (Entry<String, String> configEntry : entry.getValue().entrySet(epoch)) {
-                records.add(new ApiMessageAndVersion(new ConfigRecord().
-                    setResourceName(resource.name()).
-                    setResourceType(resource.type().id()).
-                    setName(configEntry.getKey()).
-                    setValue(configEntry.getValue()), (short) 0));
-            }
-            return records;
-        }
-    }
-
-    ConfigurationControlIterator iterator(long epoch) {
-        return new ConfigurationControlIterator(epoch);
-    }
 }
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 3622fe225dc..ed6c5237533 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java
@@ -316,14 +316,6 @@ public interface Controller extends AclMutator, AutoCloseable {
         UpdateFeaturesRequestData request
     );
 
-    /**
-     * Begin writing a controller snapshot.  If there was already an ongoing snapshot, it
-     * simply returns information about that snapshot rather than starting a new one.
-     *
-     * @return              A future yielding the epoch of the snapshot.
-     */
-    CompletableFuture<Long> beginWritingSnapshot();
-
     /**
      * Create partitions on certain topics.
      *
diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
index b3758586cb8..7b2a9308e96 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
@@ -20,11 +20,9 @@ package org.apache.kafka.controller;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Map;
-import java.util.NoSuchElementException;
 import java.util.Optional;
 import java.util.TreeMap;
 import java.util.function.Consumer;
@@ -306,48 +304,6 @@ public class FeatureControlManager {
         }
     }
 
-    class FeatureControlIterator implements Iterator<List<ApiMessageAndVersion>> {
-        private final Iterator<Entry<String, Short>> iterator;
-        private final MetadataVersion metadataVersion;
-        private boolean wroteVersion = false;
-
-        FeatureControlIterator(long epoch) {
-            this.iterator = finalizedVersions.entrySet(epoch).iterator();
-            this.metadataVersion = FeatureControlManager.this.metadataVersion.get(epoch);
-        }
-
-        @Override
-        public boolean hasNext() {
-            return needsWriteMetadataVersion() || iterator.hasNext();
-        }
-
-        private boolean needsWriteMetadataVersion() {
-            return !wroteVersion && metadataVersion.isAtLeast(minimumBootstrapVersion);
-        }
-
-        @Override
-        public List<ApiMessageAndVersion> next() {
-            // Write the metadata.version first
-            if (needsWriteMetadataVersion()) {
-                wroteVersion = true;
-                return Collections.singletonList(new ApiMessageAndVersion(new FeatureLevelRecord()
-                    .setName(MetadataVersion.FEATURE_NAME)
-                    .setFeatureLevel(metadataVersion.featureLevel()), FEATURE_LEVEL_RECORD.lowestSupportedVersion()));
-            }
-
-            // Then write the rest of the features
-            if (!iterator.hasNext()) throw new NoSuchElementException();
-            Entry<String, Short> entry = iterator.next();
-            return Collections.singletonList(new ApiMessageAndVersion(new FeatureLevelRecord()
-                .setName(entry.getKey())
-                .setFeatureLevel(entry.getValue()), (short) 0));
-        }
-    }
-
-    FeatureControlIterator iterator(long epoch) {
-        return new FeatureControlIterator(epoch);
-    }
-
     boolean isControllerId(int nodeId) {
         return quorumFeatures.isControllerId(nodeId);
     }
diff --git a/metadata/src/main/java/org/apache/kafka/controller/LogReplayTracker.java b/metadata/src/main/java/org/apache/kafka/controller/LogReplayTracker.java
index 2e29a2a52f9..41f881245fc 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/LogReplayTracker.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/LogReplayTracker.java
@@ -24,8 +24,7 @@ import org.slf4j.Logger;
 
 /**
  * The LogReplayTracker manages state associated with replaying the metadata log, such as whether
- * we have seen any records and whether we have seen any metadata version records. It is accessed
- * solely from the quorum controller thread.
+ * we have seen any records. It is accessed solely from the quorum controller thread.
  */
 public class LogReplayTracker {
     public static class Builder {
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java
index 178ef46bdb5..47e4e1b430f 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java
@@ -25,14 +25,10 @@ import org.apache.kafka.timeline.SnapshotRegistry;
 import org.apache.kafka.timeline.TimelineLong;
 import org.apache.kafka.timeline.TimelineObject;
 
-import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
 
 
 public class ProducerIdControlManager {
-
     private final ClusterControlManager clusterControlManager;
     private final TimelineObject<ProducerIdsBlock> nextProducerBlock;
     private final TimelineLong brokerEpoch;
@@ -62,6 +58,11 @@ public class ProducerIdControlManager {
         return ControllerResult.of(Collections.singletonList(new ApiMessageAndVersion(record, (short) 0)), block);
     }
 
+    // VisibleForTesting
+    ProducerIdsBlock nextProducerBlock() {
+        return nextProducerBlock.get();
+    }
+
     void replay(ProducerIdsRecord record) {
         long currentNextProducerId = nextProducerBlock.get().firstProducerId();
         if (record.nextProducerId() <= currentNextProducerId) {
@@ -72,19 +73,4 @@ public class ProducerIdControlManager {
             brokerEpoch.set(record.brokerEpoch());
         }
     }
-
-    Iterator<List<ApiMessageAndVersion>> iterator(long epoch) {
-        List<ApiMessageAndVersion> records = new ArrayList<>(1);
-
-        ProducerIdsBlock producerIdBlock = nextProducerBlock.get(epoch);
-        if (producerIdBlock.firstProducerId() > 0) {
-            records.add(new ApiMessageAndVersion(
-                new ProducerIdsRecord()
-                    .setNextProducerId(producerIdBlock.firstProducerId())
-                    .setBrokerId(producerIdBlock.assignedBrokerId())
-                    .setBrokerEpoch(brokerEpoch.get(epoch)),
-                (short) 0));
-        }
-        return Collections.singleton(records).iterator();
-    }
 }
diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
index 6716044bef5..642d6976dba 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
@@ -72,7 +72,6 @@ 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.controller.SnapshotGenerator.Section;
 import org.apache.kafka.metadata.BrokerHeartbeatReply;
 import org.apache.kafka.metadata.BrokerRegistrationReply;
 import org.apache.kafka.metadata.FinalizedControllerFeatures;
@@ -89,7 +88,6 @@ import org.apache.kafka.raft.BatchReader;
 import org.apache.kafka.raft.LeaderAndEpoch;
 import org.apache.kafka.raft.OffsetAndEpoch;
 import org.apache.kafka.raft.RaftClient;
-import org.apache.kafka.metadata.util.SnapshotReason;
 import org.apache.kafka.server.authorizer.AclCreateResult;
 import org.apache.kafka.server.authorizer.AclDeleteResult;
 import org.apache.kafka.server.common.ApiMessageAndVersion;
@@ -98,7 +96,6 @@ import org.apache.kafka.server.fault.FaultHandler;
 import org.apache.kafka.server.policy.AlterConfigPolicy;
 import org.apache.kafka.server.policy.CreateTopicPolicy;
 import org.apache.kafka.snapshot.SnapshotReader;
-import org.apache.kafka.snapshot.SnapshotWriter;
 import org.apache.kafka.timeline.SnapshotRegistry;
 import org.slf4j.Logger;
 
@@ -107,7 +104,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Map;
@@ -168,8 +164,6 @@ public final class QuorumController implements Controller {
         private short defaultReplicationFactor = 3;
         private int defaultNumPartitions = 1;
         private ReplicaPlacer replicaPlacer = new StripedReplicaPlacer(new Random());
-        private long snapshotMaxNewRecordBytes = Long.MAX_VALUE;
-        private long snapshotMaxIntervalMs = 0;
         private OptionalLong leaderImbalanceCheckIntervalNs = OptionalLong.empty();
         private OptionalLong maxIdleIntervalNs = OptionalLong.empty();
         private long sessionTimeoutNs = ClusterControlManager.DEFAULT_SESSION_TIMEOUT_NS;
@@ -242,16 +236,6 @@ public final class QuorumController implements Controller {
             return this;
         }
 
-        public Builder setSnapshotMaxNewRecordBytes(long value) {
-            this.snapshotMaxNewRecordBytes = value;
-            return this;
-        }
-
-        public Builder setSnapshotMaxIntervalMs(long value) {
-            this.snapshotMaxIntervalMs = value;
-            return this;
-        }
-
         public Builder setLeaderImbalanceCheckIntervalNs(OptionalLong value) {
             this.leaderImbalanceCheckIntervalNs = value;
             return this;
@@ -351,8 +335,6 @@ public final class QuorumController implements Controller {
                     defaultReplicationFactor,
                     defaultNumPartitions,
                     replicaPlacer,
-                    snapshotMaxNewRecordBytes,
-                    snapshotMaxIntervalMs,
                     leaderImbalanceCheckIntervalNs,
                     maxIdleIntervalNs,
                     sessionTimeoutNs,
@@ -521,132 +503,6 @@ public final class QuorumController implements Controller {
         queue.append(event);
     }
 
-    private static final String GENERATE_SNAPSHOT = "generateSnapshot";
-
-    private static final int MAX_BATCHES_PER_GENERATE_CALL = 10;
-
-    class SnapshotGeneratorManager implements Runnable {
-        private SnapshotGenerator generator = null;
-
-        void createSnapshotGenerator(long committedOffset, int committedEpoch, long committedTimestamp) {
-            if (snapshotInProgress()) {
-                throw new IllegalStateException("Snapshot generator already exists");
-            }
-            if (!snapshotRegistry.hasSnapshot(committedOffset)) {
-                throw new IllegalStateException(
-                    String.format(
-                        "Cannot generate a snapshot at committed offset %d because it does not exists in the snapshot registry.",
-                        committedOffset
-                    )
-                );
-            }
-
-            OffsetAndEpoch snapshotId = new OffsetAndEpoch(
-                committedOffset + 1,
-                committedEpoch
-            );
-
-            Optional<SnapshotWriter<ApiMessageAndVersion>> writer = raftClient.createSnapshot(
-                snapshotId,
-                committedTimestamp
-            );
-            if (writer.isPresent()) {
-                generator = new SnapshotGenerator(
-                    logContext,
-                    writer.get(),
-                    MAX_BATCHES_PER_GENERATE_CALL,
-                    Arrays.asList(
-                        new Section("features", featureControl.iterator(committedOffset)),
-                        new Section("cluster", clusterControl.iterator(committedOffset)),
-                        new Section("replication", replicationControl.iterator(committedOffset)),
-                        new Section("configuration", configurationControl.iterator(committedOffset)),
-                        new Section("clientQuotas", clientQuotaControlManager.iterator(committedOffset)),
-                        new Section("producerIds", producerIdControlManager.iterator(committedOffset)),
-                        new Section("acls", aclControlManager.iterator(committedOffset))
-                    )
-                );
-                reschedule(0);
-            } else {
-                log.info(
-                    "Skipping generation of snapshot for committed offset {} and epoch {} since it already exists",
-                    committedOffset,
-                    committedEpoch
-                );
-            }
-        }
-
-        void cancel() {
-            if (!snapshotInProgress()) return;
-            log.error("Cancelling snapshot {}", generator.lastContainedLogOffset());
-            generator.writer().close();
-            generator = null;
-
-            // Delete every in-memory snapshot up to the committed offset. They are not needed since this
-            // snapshot generation was canceled.
-            snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset);
-
-            queue.cancelDeferred(GENERATE_SNAPSHOT);
-        }
-
-        void reschedule(long delayNs) {
-            ControllerEvent event = new ControllerEvent(GENERATE_SNAPSHOT, this);
-            queue.scheduleDeferred(event.name,
-                new EarliestDeadlineFunction(time.nanoseconds() + delayNs), event);
-        }
-
-        void handleSnapshotFinished(Optional<Exception> exception) {
-            if (exception.isPresent()) {
-                log.error("Error while generating snapshot {}", generator.lastContainedLogOffset(), exception.get());
-            } else {
-                log.info("Finished generating snapshot {}.", generator.lastContainedLogOffset());
-            }
-
-            generator.writer().close();
-            generator = null;
-
-            // Delete every in-memory snapshot up to the committed offset. They are not needed since this
-            // snapshot generation finished.
-            snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset);
-
-            // The snapshot counters for size-based and time-based snapshots could have changed to cause a new
-            // snapshot to get generated.
-            maybeGenerateSnapshot();
-        }
-
-        @Override
-        public void run() {
-            if (!snapshotInProgress()) {
-                log.debug("No snapshot is in progress because it was previously canceled");
-                return;
-            }
-
-            OptionalLong nextDelay;
-            try {
-                nextDelay = generator.generateBatches();
-            } catch (Exception e) {
-                handleSnapshotFinished(Optional.of(e));
-                return;
-            }
-
-            if (nextDelay.isPresent()) {
-                reschedule(nextDelay.getAsLong());
-            } else {
-                handleSnapshotFinished(Optional.empty());
-            }
-        }
-
-        OptionalLong snapshotLastOffsetFromLog() {
-            if (!snapshotInProgress()) {
-                return OptionalLong.empty();
-            }
-            return OptionalLong.of(generator.lastContainedLogOffset());
-        }
-
-        public boolean snapshotInProgress() {
-            return generator != null;
-        }
-    }
-
     /**
      * A controller event that reads the committed internal state in order to expose it
      * to an API.
@@ -985,12 +841,8 @@ public final class QuorumController implements Controller {
                             // Complete any events in the purgatory that were waiting for this offset.
                             purgatory.completeUpTo(offset);
 
-                            // Delete all the in-memory snapshots that are no longer needed.
-                            //
-                            // If the active controller has a snapshot in progress, it needs to keep that in-memory
-                            // snapshot. Otherwise, the active controller can delete up to the current committed offset.
-                            snapshotRegistry.deleteSnapshotsUpTo(
-                                snapshotGeneratorManager.snapshotLastOffsetFromLog().orElse(offset));
+                            // The active controller can delete up to the current committed offset.
+                            snapshotRegistry.deleteSnapshotsUpTo(offset);
                         } else {
                             // If the controller is a standby, replay the records that were
                             // created by the active controller.
@@ -1022,8 +874,7 @@ public final class QuorumController implements Controller {
                         updateLastCommittedState(
                             offset,
                             epoch,
-                            batch.appendTimestamp(),
-                            committedBytesSinceLastSnapshot + batch.sizeInBytes()
+                            batch.appendTimestamp()
                         );
 
                         if (offset >= raftClient.latestSnapshotId().map(OffsetAndEpoch::offset).orElse(0L)) {
@@ -1033,8 +884,6 @@ public final class QuorumController implements Controller {
                             );
                         }
                     }
-
-                    maybeGenerateSnapshot();
                 } finally {
                     reader.close();
                 }
@@ -1086,12 +935,9 @@ public final class QuorumController implements Controller {
                             i++;
                         }
                     }
-                    updateLastCommittedState(
-                        reader.lastContainedLogOffset(),
+                    updateLastCommittedState(reader.lastContainedLogOffset(),
                         reader.lastContainedLogEpoch(),
-                        reader.lastContainedLogTimestamp(),
-                        0
-                    );
+                        reader.lastContainedLogTimestamp());
                     snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset);
                     authorizer.ifPresent(a -> a.loadSnapshot(aclControlManager.idToAcl()));
                 } finally {
@@ -1247,13 +1093,11 @@ public final class QuorumController implements Controller {
     private void updateLastCommittedState(
         long offset,
         int epoch,
-        long timestamp,
-        long bytesSinceLastSnapshot
+        long timestamp
     ) {
         lastCommittedOffset = offset;
         lastCommittedEpoch = epoch;
         lastCommittedTimestamp = timestamp;
-        committedBytesSinceLastSnapshot = bytesSinceLastSnapshot;
 
         controllerMetrics.setLastCommittedRecordOffset(offset);
         if (!isActiveController()) {
@@ -1426,38 +1270,6 @@ public final class QuorumController implements Controller {
         queue.cancelDeferred(WRITE_NO_OP_RECORD);
     }
 
-    private static final String MAYBE_GENERATE_SNAPSHOT = "maybeGenerateSnapshot";
-
-    private void maybeScheduleNextGenerateSnapshot() {
-        if (!generateSnapshotScheduled) {
-            long now = time.milliseconds();
-            long delayMs = Math.min(
-                0,
-                snapshotMaxIntervalMs + oldestNonSnapshottedTimestamp - now
-            );
-
-            log.debug(
-                "Scheduling write event for {} because snapshotMaxIntervalMs ({}), " +
-                "oldestNonSnapshottedTimestamp ({}) and now ({})",
-                MAYBE_GENERATE_SNAPSHOT,
-                snapshotMaxIntervalMs,
-                oldestNonSnapshottedTimestamp,
-                now
-            );
-
-            ControllerEvent event = new ControllerEvent(MAYBE_GENERATE_SNAPSHOT, this::maybeGenerateSnapshot);
-
-            long scheduleNs = time.nanoseconds() + TimeUnit.MILLISECONDS.toNanos(delayMs);
-            queue.scheduleDeferred(MAYBE_GENERATE_SNAPSHOT, new EarliestDeadlineFunction(scheduleNs), event);
-            generateSnapshotScheduled = true;
-        }
-    }
-
-    private void cancelNextGenerateSnapshot() {
-        queue.cancelDeferred(MAYBE_GENERATE_SNAPSHOT);
-        generateSnapshotScheduled = false;
-    }
-
     private void handleFeatureControlChange() {
         // The feature control maybe have changed. On the active controller cancel or schedule noop
         // record writes accordingly.
@@ -1536,72 +1348,13 @@ public final class QuorumController implements Controller {
         }
     }
 
-    private void maybeGenerateSnapshot() {
-        if (snapshotGeneratorManager.snapshotInProgress()) {
-            /* Skip snapshot generation if there is a snaphshot in progress.
-             *
-             * When the in-progress snapshot completes it will call this method to check if the controller should
-             * generate another snapshot due to any of the reasons supported by this method.
-             */
-        } else {
-            Set<SnapshotReason> snapshotReasons = new HashSet<>();
-            // Check if a snapshot should be generated because of committed bytes
-            if (committedBytesSinceLastSnapshot >= snapshotMaxNewRecordBytes) {
-                snapshotReasons.add(
-                    SnapshotReason.maxBytesExceeded(committedBytesSinceLastSnapshot, snapshotMaxNewRecordBytes)
-                );
-            }
-
-            // Check if a snapshot should be generated because of committed append times
-            if (snapshotMaxIntervalMs > 0) {
-                // Time base snasphots are enabled
-                long snapshotIntervalMs = time.milliseconds() - oldestNonSnapshottedTimestamp;
-                if (snapshotIntervalMs >= snapshotMaxIntervalMs) {
-                    snapshotReasons.add(SnapshotReason.maxIntervalExceeded(snapshotIntervalMs, snapshotMaxIntervalMs));
-                } else {
-                    maybeScheduleNextGenerateSnapshot();
-                }
-            }
-
-            if (!snapshotReasons.isEmpty()) {
-                if (!isActiveController()) {
-                    // The inactive controllers only create an in-memory snapshot when generating a snapshot. This is
-                    // unlike the active controller which creates in-memory snapshots every time an uncommitted batch
-                    // gets replayed.
-                    snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset);
-                }
-
-                log.info(
-                    "Generating a snapshot that includes (epoch={}, offset={}) because: {}",
-                    lastCommittedEpoch,
-                    lastCommittedOffset,
-                    SnapshotReason.stringFromReasons(snapshotReasons)
-                );
-
-                snapshotGeneratorManager.createSnapshotGenerator(
-                    lastCommittedOffset,
-                    lastCommittedEpoch,
-                    lastCommittedTimestamp
-                );
-
-                // Reset all of the snapshot counters
-                committedBytesSinceLastSnapshot = 0;
-                oldestNonSnapshottedTimestamp = Long.MAX_VALUE;
-
-                // Starting a snapshot invalidates any scheduled snapshot generation
-                cancelNextGenerateSnapshot();
-            }
-        }
-    }
-
     /**
      * Clear all data structures and reset all KRaft state.
      */
     private void resetToEmptyState() {
-        snapshotGeneratorManager.cancel();
         snapshotRegistry.reset();
 
-        updateLastCommittedState(-1, -1, -1, 0);
+        updateLastCommittedState(-1, -1, -1);
     }
 
     /**
@@ -1715,11 +1468,6 @@ public final class QuorumController implements Controller {
      */
     private final LogReplayTracker logReplayTracker;
 
-    /**
-     * Manages generating controller snapshots.
-     */
-    private final SnapshotGeneratorManager snapshotGeneratorManager = new SnapshotGeneratorManager();
-
     /**
      * The interface that we use to mutate the Raft log.
      */
@@ -1766,21 +1514,6 @@ public final class QuorumController implements Controller {
      */
     private long writeOffset;
 
-    /**
-     * Maximum number of bytes processed through handling commits before generating a snapshot.
-     */
-    private final long snapshotMaxNewRecordBytes;
-
-    /**
-     * Number of bytes processed through handling commits since the last snapshot was generated.
-     */
-    private long committedBytesSinceLastSnapshot = 0;
-
-    /**
-     * Maximum amount of to wait for a record in the log to get included in a snapshot.
-     */
-    private final long snapshotMaxIntervalMs;
-
     /**
      * Timestamp for the oldest record that was committed but not included in a snapshot.
      */
@@ -1843,8 +1576,6 @@ public final class QuorumController implements Controller {
         short defaultReplicationFactor,
         int defaultNumPartitions,
         ReplicaPlacer replicaPlacer,
-        long snapshotMaxNewRecordBytes,
-        long snapshotMaxIntervalMs,
         OptionalLong leaderImbalanceCheckIntervalNs,
         OptionalLong maxIdleIntervalNs,
         long sessionTimeoutNs,
@@ -1903,8 +1634,6 @@ public final class QuorumController implements Controller {
             setZkMigrationEnabled(zkMigrationEnabled).
             build();
         this.producerIdControlManager = new ProducerIdControlManager(clusterControl, snapshotRegistry);
-        this.snapshotMaxNewRecordBytes = snapshotMaxNewRecordBytes;
-        this.snapshotMaxIntervalMs = snapshotMaxIntervalMs;
         this.leaderImbalanceCheckIntervalNs = leaderImbalanceCheckIntervalNs;
         this.maxIdleIntervalNs = maxIdleIntervalNs;
         this.replicationControl = new ReplicationControlManager.Builder().
@@ -2243,28 +1972,6 @@ public final class QuorumController implements Controller {
         });
     }
 
-    @Override
-    public CompletableFuture<Long> beginWritingSnapshot() {
-        CompletableFuture<Long> future = new CompletableFuture<>();
-        appendControlEvent("beginWritingSnapshot", () -> {
-            if (!snapshotGeneratorManager.snapshotInProgress()) {
-                log.info(
-                    "Generating a snapshot that includes (epoch={}, offset={}) because, {}.",
-                    lastCommittedEpoch,
-                    lastCommittedOffset,
-                    SnapshotReason.UNKNOWN
-                );
-                snapshotGeneratorManager.createSnapshotGenerator(
-                    lastCommittedOffset,
-                    lastCommittedEpoch,
-                    lastCommittedTimestamp
-                );
-            }
-            future.complete(snapshotGeneratorManager.generator.lastContainedLogOffset());
-        });
-        return future;
-    }
-
     @Override
     public CompletableFuture<List<AclCreateResult>> createAcls(
         ControllerRequestContext context,
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
index dbe23218d48..5fd9af85e06 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
@@ -108,7 +108,6 @@ import java.util.List;
 import java.util.ListIterator;
 import java.util.Map.Entry;
 import java.util.Map;
-import java.util.NoSuchElementException;
 import java.util.Optional;
 import java.util.OptionalInt;
 import java.util.Set;
@@ -1911,39 +1910,6 @@ public class ReplicationControlManager {
             setReplicas(Replicas.toList(partition.replicas)));
     }
 
-    class ReplicationControlIterator implements Iterator<List<ApiMessageAndVersion>> {
-        private final long epoch;
-        private final Iterator<TopicControlInfo> iterator;
-
-        ReplicationControlIterator(long epoch) {
-            this.epoch = epoch;
-            this.iterator = topics.values(epoch).iterator();
-        }
-
-        @Override
-        public boolean hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public List<ApiMessageAndVersion> next() {
-            if (!hasNext()) throw new NoSuchElementException();
-            TopicControlInfo topic = iterator.next();
-            List<ApiMessageAndVersion> records = new ArrayList<>();
-            records.add(new ApiMessageAndVersion(new TopicRecord().
-                setName(topic.name).
-                setTopicId(topic.id), (short) 0));
-            for (Entry<Integer, PartitionRegistration> entry : topic.parts.entrySet(epoch)) {
-                records.add(entry.getValue().toRecord(topic.id, entry.getKey()));
-            }
-            return records;
-        }
-    }
-
-    ReplicationControlIterator iterator(long epoch) {
-        return new ReplicationControlIterator(epoch);
-    }
-
     private static final class IneligibleReplica {
         private final int replicaId;
         private final String reason;
diff --git a/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java b/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java
deleted file mode 100644
index d34696ef44f..00000000000
--- a/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * 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.List;
-import java.util.OptionalLong;
-
-import org.apache.kafka.common.utils.LogContext;
-import org.apache.kafka.server.common.ApiMessageAndVersion;
-import org.apache.kafka.snapshot.SnapshotWriter;
-import org.slf4j.Logger;
-
-
-final class SnapshotGenerator {
-    static class Section {
-        private final String name;
-        private final Iterator<List<ApiMessageAndVersion>> iterator;
-
-        Section(String name, Iterator<List<ApiMessageAndVersion>> iterator) {
-            this.name = name;
-            this.iterator = iterator;
-        }
-
-        String name() {
-            return name;
-        }
-
-        Iterator<List<ApiMessageAndVersion>> iterator() {
-            return iterator;
-        }
-    }
-
-    private final Logger log;
-    private final SnapshotWriter<ApiMessageAndVersion> writer;
-    private final int maxBatchesPerGenerateCall;
-    private final List<Section> sections;
-    private final Iterator<Section> sectionIterator;
-    private Iterator<List<ApiMessageAndVersion>> batchIterator;
-    private List<ApiMessageAndVersion> batch;
-    private Section section;
-    private long numRecords;
-
-    SnapshotGenerator(LogContext logContext,
-                      SnapshotWriter<ApiMessageAndVersion> writer,
-                      int maxBatchesPerGenerateCall,
-                      List<Section> sections) {
-        this.log = logContext.logger(SnapshotGenerator.class);
-        this.writer = writer;
-        this.maxBatchesPerGenerateCall = maxBatchesPerGenerateCall;
-        this.sections = sections;
-        this.sectionIterator = this.sections.iterator();
-        this.batchIterator = Collections.emptyIterator();
-        this.batch = null;
-        this.section = null;
-        this.numRecords = 0;
-    }
-
-    /**
-     * Returns the last offset from the log that will be included in the snapshot.
-     */
-    long lastContainedLogOffset() {
-        return writer.lastContainedLogOffset();
-    }
-
-    SnapshotWriter writer() {
-        return writer;
-    }
-
-    /**
-     * Generate and write the next batch of records.
-     *
-     * @return true if the last batch was generated, otherwise false
-     */
-    private boolean generateBatch() throws Exception {
-        if (batch == null) {
-            while (!batchIterator.hasNext()) {
-                if (section != null) {
-                    log.info("Generated {} record(s) for the {} section of snapshot {}.",
-                             numRecords, section.name(), writer.snapshotId());
-                    section = null;
-                    numRecords = 0;
-                }
-                if (!sectionIterator.hasNext()) {
-                    writer.freeze();
-                    return true;
-                }
-                section = sectionIterator.next();
-                log.info("Generating records for the {} section of snapshot {}.",
-                         section.name(), writer.snapshotId());
-                batchIterator = section.iterator();
-            }
-            batch = batchIterator.next();
-        }
-
-        writer.append(batch);
-        numRecords += batch.size();
-        batch = null;
-        return false;
-    }
-
-    /**
-     * Generate the next few batches of records.
-     *
-     * @return  The number of nanoseconds to delay before rescheduling the
-     *          generateBatches event, or empty if the snapshot is done.
-     */
-    OptionalLong generateBatches() throws Exception {
-        for (int numBatches = 0; numBatches < maxBatchesPerGenerateCall; numBatches++) {
-            if (generateBatch()) {
-                return OptionalLong.empty();
-            }
-        }
-        return OptionalLong.of(0);
-    }
-}
diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java b/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java
new file mode 100644
index 00000000000..c3ab72b3b79
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/loader/LogDeltaManifest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.image.loader;
+
+import org.apache.kafka.image.MetadataProvenance;
+
+import java.util.Objects;
+
+
+/**
+ * Contains information about a set of changes that were loaded from the metadata log.
+ */
+public class LogDeltaManifest {
+    /**
+     * The highest offset and epoch included in this delta, inclusive.
+     */
+    private final MetadataProvenance provenance;
+
+    /**
+     * The number of batches that were loaded.
+     */
+    private final int numBatches;
+
+    /**
+     * The time in nanoseconds that it took to load the changes.
+     */
+    private final long elapsedNs;
+
+    /**
+     * The total size of the records in bytes that we read while creating the delta.
+     */
+    private final long numBytes;
+
+    public LogDeltaManifest(
+        MetadataProvenance provenance,
+        int numBatches,
+        long elapsedNs,
+        long numBytes
+    ) {
+        this.provenance = provenance;
+        this.numBatches = numBatches;
+        this.elapsedNs = elapsedNs;
+        this.numBytes = numBytes;
+    }
+
+
+    public MetadataProvenance provenance() {
+        return provenance;
+    }
+
+    public int numBatches() {
+        return numBatches;
+    }
+
+    public long elapsedNs() {
+        return elapsedNs;
+    }
+
+    public long numBytes() {
+        return numBytes;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                provenance,
+                numBatches,
+                elapsedNs,
+                numBytes);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null || !o.getClass().equals(this.getClass())) return false;
+        LogDeltaManifest other = (LogDeltaManifest) o;
+        return provenance.equals(other.provenance) &&
+                numBatches == other.numBatches &&
+                elapsedNs == other.elapsedNs &&
+                numBytes == other.numBytes;
+    }
+
+    @Override
+    public String toString() {
+        return "LogDeltaManifest(" +
+                "provenance=" + provenance +
+                ", numBatches=" + numBatches +
+                ", elapsedNs=" + elapsedNs +
+                ", numBytes=" + numBytes +
+                ")";
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java
new file mode 100644
index 00000000000..9bfe5a5884f
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java
@@ -0,0 +1,572 @@
+/*
+ * 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.image.loader;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.MetadataProvenance;
+import org.apache.kafka.image.publisher.MetadataPublisher;
+import org.apache.kafka.image.writer.ImageReWriter;
+import org.apache.kafka.image.writer.ImageWriterOptions;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.raft.Batch;
+import org.apache.kafka.raft.BatchReader;
+import org.apache.kafka.raft.LeaderAndEpoch;
+import org.apache.kafka.raft.RaftClient;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.server.fault.FaultHandler;
+import org.apache.kafka.server.fault.FaultHandlerException;
+import org.apache.kafka.snapshot.SnapshotReader;
+import org.slf4j.Logger;
+
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+
+/**
+ * The MetadataLoader follows changes provided by a RaftClient, and packages them into metadata
+ * deltas and images that can be consumed by publishers.
+ *
+ * The Loader maintains its own thread, which is used to make all callbacks into publishers. If a
+ * publisher A is installed before B, A will receive all callbacks before B. This is also true if
+ * A and B are installed as part of a list [A, B].
+ *
+ * Publishers should not modify any data structures passed to them.
+ *
+ * It is possible to change the list of publishers dynamically over time. Whenever a new publisher is
+ * added, it receives a catch-up delta which contains the full state. Any publisher installed when the
+ * loader is closed will itself be closed.
+ */
+public class MetadataLoader implements RaftClient.Listener<ApiMessageAndVersion>, AutoCloseable {
+    public static class Builder {
+        private int nodeId = -1;
+        private Time time = Time.SYSTEM;
+        private LogContext logContext = null;
+        private String threadNamePrefix = "";
+        private FaultHandler faultHandler = (m, e) -> new FaultHandlerException(m, e);
+        private MetadataLoaderMetrics metrics = new MetadataLoaderMetrics() {
+            private volatile long lastAppliedOffset = -1L;
+
+            @Override
+            public void updateBatchProcessingTime(long elapsedNs) { }
+
+            @Override
+            public void updateBatchSize(int size) { }
+
+            @Override
+            public void updateLastAppliedImageProvenance(MetadataProvenance provenance) {
+                this.lastAppliedOffset = provenance.offset();
+            }
+
+            @Override
+            public long lastAppliedOffset() {
+                return lastAppliedOffset;
+            }
+
+            @Override
+            public void close() throws Exception { }
+        };
+        private Supplier<OptionalLong> highWaterMarkAccessor = null;
+
+        public Builder setNodeId(int nodeId) {
+            this.nodeId = nodeId;
+            return this;
+        }
+
+        public Builder setTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder setThreadNamePrefix(String threadNamePrefix) {
+            this.threadNamePrefix = threadNamePrefix;
+            return this;
+        }
+
+        public Builder setFaultHandler(FaultHandler faultHandler) {
+            this.faultHandler = faultHandler;
+            return this;
+        }
+
+        public Builder setMetadataLoaderMetrics(MetadataLoaderMetrics metrics) {
+            this.metrics = metrics;
+            return this;
+        }
+
+        public Builder setHighWaterMarkAccessor(Supplier<OptionalLong> highWaterMarkAccessor) {
+            this.highWaterMarkAccessor = highWaterMarkAccessor;
+            return this;
+        }
+
+        public MetadataLoader build() {
+            if (logContext == null) {
+                logContext = new LogContext("[MetadataLoader " + nodeId + "] ");
+            }
+            if (highWaterMarkAccessor == null) {
+                throw new RuntimeException("You must set the high water mark accessor.");
+            }
+            return new MetadataLoader(
+                time,
+                logContext,
+                threadNamePrefix,
+                faultHandler,
+                metrics,
+                highWaterMarkAccessor);
+        }
+    }
+
+    /**
+     * The log4j logger for this loader.
+     */
+    private final Logger log;
+
+    /**
+     * The clock used by this loader.
+     */
+    private final Time time;
+
+    /**
+     * The fault handler to use if metadata loading fails.
+     */
+    private final FaultHandler faultHandler;
+
+    /**
+     * Callbacks for updating metrics.
+     */
+    private final MetadataLoaderMetrics metrics;
+
+    /**
+     * A function which supplies the current high water mark, or empty if it is not known.
+     */
+    private final Supplier<OptionalLong> highWaterMarkAccessor;
+
+    /**
+     * Publishers which haven't been initialized yet.
+     */
+    private final LinkedHashMap<String, MetadataPublisher> uninitializedPublishers;
+
+    /**
+     * Publishers which are receiving updates.
+     */
+    private final LinkedHashMap<String, MetadataPublisher> publishers;
+
+    /**
+     * True if we have caught up with the initial high water mark.
+     */
+    private boolean catchingUp = false;
+
+    /**
+     * The current leader and epoch.
+     */
+    private LeaderAndEpoch currentLeaderAndEpoch = LeaderAndEpoch.UNKNOWN;
+
+    /**
+     * The current metadata image. Accessed only from the event queue thread.
+     */
+    private MetadataImage image;
+
+    /**
+     * The event queue which runs this loader.
+     */
+    private final KafkaEventQueue eventQueue;
+
+    private MetadataLoader(
+        Time time,
+        LogContext logContext,
+        String threadNamePrefix,
+        FaultHandler faultHandler,
+        MetadataLoaderMetrics metrics,
+        Supplier<OptionalLong> highWaterMarkAccessor
+    ) {
+        this.log = logContext.logger(MetadataLoader.class);
+        this.time = time;
+        this.faultHandler = faultHandler;
+        this.metrics = metrics;
+        this.highWaterMarkAccessor = highWaterMarkAccessor;
+        this.uninitializedPublishers = new LinkedHashMap<>();
+        this.publishers = new LinkedHashMap<>();
+        this.image = MetadataImage.EMPTY;
+        this.eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix);
+    }
+
+    private boolean stillNeedToCatchUp(long offset) {
+        if (!catchingUp) {
+            log.trace("We are not in the initial catching up state.");
+            return false;
+        }
+        OptionalLong highWaterMark = highWaterMarkAccessor.get();
+        if (!highWaterMark.isPresent()) {
+            log.info("The loader is still catching up because we still don't know the high " +
+                    "water mark yet.");
+            return true;
+        }
+        if (highWaterMark.getAsLong() > offset) {
+            log.info("The loader is still catching up because we have loaded up to offset " +
+                    offset + ", but the high water mark is " + highWaterMark.getAsLong());
+            return true;
+        }
+        log.info("The loader finished catch up to the current high water mark of " +
+                highWaterMark.getAsLong());
+        catchingUp = true;
+        return false;
+    }
+
+    private void maybeInitializeNewPublishers() {
+        if (uninitializedPublishers.isEmpty()) {
+            log.trace("There are no uninitialized publishers to initialize.");
+            return;
+        }
+        long startNs = time.nanoseconds();
+        MetadataDelta delta = new MetadataDelta.Builder().
+                setImage(image).
+                build();
+        ImageReWriter writer = new ImageReWriter(delta);
+        image.write(writer, new ImageWriterOptions.Builder().
+                setMetadataVersion(image.features().metadataVersion()).
+                build());
+        SnapshotManifest manifest = new SnapshotManifest(
+                image.provenance(),
+                time.nanoseconds() - startNs);
+        for (Iterator<MetadataPublisher> iter = uninitializedPublishers.values().iterator();
+                iter.hasNext(); ) {
+            MetadataPublisher publisher = iter.next();
+            iter.remove();
+            try {
+                log.info("Publishing initial snapshot at offset {} to {}",
+                        image.highestOffsetAndEpoch().offset(), publisher.name());
+                publisher.publishSnapshot(delta, image, manifest);
+                publishers.put(publisher.name(), publisher);
+            } catch (Throwable e) {
+                faultHandler.handleFault("Unhandled error publishing the initial metadata " +
+                        "image from snapshot at offset " + image.highestOffsetAndEpoch().offset() +
+                        " with publisher " + publisher.name(), e);
+            }
+        }
+    }
+
+    @Override
+    public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                LogDeltaManifest manifest = loadLogDelta(delta, reader);
+                if (log.isDebugEnabled()) {
+                    log.debug("Generated a metadata delta between {} and {} from {} batch(es) " +
+                            "in {} us.", image.offset(), manifest.provenance().offset(),
+                            manifest.numBatches(), NANOSECONDS.toMicros(manifest.elapsedNs()));
+                }
+                try {
+                    image = delta.apply(manifest.provenance());
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error generating new metadata image from " +
+                        "metadata delta between offset " + image.offset() +
+                            " and " + manifest.provenance().offset(), e);
+                    return;
+                }
+                if (stillNeedToCatchUp(manifest.provenance().offset())) {
+                    return;
+                }
+                log.debug("Publishing new image with provenance {}.", image.provenance());
+                for (MetadataPublisher publisher : publishers.values()) {
+                    try {
+                        publisher.publishLogDelta(delta, image, manifest);
+                    } catch (Throwable e) {
+                        faultHandler.handleFault("Unhandled error publishing the new metadata " +
+                            "image ending at " + manifest.provenance().offset() +
+                                " with publisher " + publisher.name(), e);
+                    }
+                }
+                maybeInitializeNewPublishers();
+                metrics.updateLastAppliedImageProvenance(image.provenance());
+            } catch (Throwable e) {
+                // This is a general catch-all block where we don't expect to end up;
+                // failure-prone operations should have individual try/catch blocks around them.
+                faultHandler.handleFault("Unhandled fault in MetadataLoader#handleCommit. " +
+                    "Last image offset was " + image.offset(), e);
+            } finally {
+                reader.close();
+            }
+        });
+    }
+
+    /**
+     * Load some  batches of records from the log. We have to do some bookkeeping here to
+     * translate between batch offsets and record offsets, and track the number of bytes we
+     * have read. Additionally, there is the chance that one of the records is a metadata
+     * version change which needs to be handled differently.
+     *
+     * @param delta     The metadata delta we are preparing.
+     * @param reader    The reader which yields the batches.
+     * @return          A manifest of what was loaded.
+     */
+    LogDeltaManifest loadLogDelta(
+        MetadataDelta delta,
+        BatchReader<ApiMessageAndVersion> reader
+    ) {
+        long startNs = time.nanoseconds();
+        int numBatches = 0;
+        long numBytes = 0L;
+        long lastOffset = image.provenance().offset();
+        int lastEpoch = image.provenance().epoch();
+        long lastContainedLogTimeMs = image.provenance().lastContainedLogTimeMs();
+
+        while (reader.hasNext()) {
+            Batch<ApiMessageAndVersion> batch = reader.next();
+            int indexWithinBatch = 0;
+            for (ApiMessageAndVersion record : batch.records()) {
+                try {
+                    delta.replay(record.message());
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error loading metadata log record from offset " +
+                            batch.baseOffset() + indexWithinBatch, e);
+                }
+                indexWithinBatch++;
+            }
+            metrics.updateBatchSize(batch.records().size());
+            lastOffset = batch.lastOffset();
+            lastEpoch = batch.epoch();
+            lastContainedLogTimeMs = batch.appendTimestamp();
+            numBytes += batch.sizeInBytes();
+            numBatches++;
+        }
+        MetadataProvenance provenance =
+                new MetadataProvenance(lastOffset, lastEpoch, lastContainedLogTimeMs);
+        long elapsedNs = time.nanoseconds() - startNs;
+        metrics.updateBatchProcessingTime(elapsedNs);
+        return new LogDeltaManifest(provenance,
+                numBatches,
+                elapsedNs,
+                numBytes);
+    }
+
+    @Override
+    public void handleSnapshot(SnapshotReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                SnapshotManifest manifest = loadSnapshot(delta, reader);
+                if (log.isDebugEnabled()) {
+                    log.debug("Generated a metadata delta from a snapshot at offset {} " +
+                            "in {} us.", manifest.provenance().offset(),
+                            NANOSECONDS.toMicros(manifest.elapsedNs()));
+                }
+                try {
+                    image = delta.apply(manifest.provenance());
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error generating new metadata image from " +
+                            "snapshot at offset " + reader.lastContainedLogOffset(), e);
+                    return;
+                }
+                if (stillNeedToCatchUp(manifest.provenance().offset())) {
+                    return;
+                }
+                log.debug("Publishing new snapshot image with provenance {}.", image.provenance());
+                for (MetadataPublisher publisher : publishers.values()) {
+                    try {
+                        publisher.publishSnapshot(delta, image, manifest);
+                    } catch (Throwable e) {
+                        faultHandler.handleFault("Unhandled error publishing the new metadata " +
+                                "image from snapshot at offset " + reader.lastContainedLogOffset() +
+                                    " with publisher " + publisher.name(), e);
+                    }
+                }
+                maybeInitializeNewPublishers();
+                metrics.updateLastAppliedImageProvenance(image.provenance());
+            } catch (Throwable e) {
+                // This is a general catch-all block where we don't expect to end up;
+                // failure-prone operations should have individual try/catch blocks around them.
+                faultHandler.handleFault("Unhandled fault in MetadataLoader#handleSnapshot. " +
+                        "Snapshot offset was " + reader.lastContainedLogOffset(), e);
+            } finally {
+                reader.close();
+            }
+        });
+    }
+
+    /**
+     * Load a snapshot. This is relatively straightforward since we don't track as many things as
+     * we do in loadLogDelta. The main complication here is that we have to maintain an index
+     * of what record we are processing so that we can give useful error messages.
+     *
+     * @param delta     The metadata delta we are preparing.
+     * @param reader    The reader which yields the snapshot batches.
+     * @return          A manifest of what was loaded.
+     */
+    SnapshotManifest loadSnapshot(
+            MetadataDelta delta,
+            SnapshotReader<ApiMessageAndVersion> reader
+    ) {
+        long startNs = time.nanoseconds();
+        int snapshotIndex = 0;
+        while (reader.hasNext()) {
+            Batch<ApiMessageAndVersion> batch = reader.next();
+            for (ApiMessageAndVersion record : batch.records()) {
+                try {
+                    delta.replay(record.message());
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error loading metadata log record " + snapshotIndex +
+                            " in snapshot at offset " + reader.lastContainedLogOffset(), e);
+                }
+                snapshotIndex++;
+            }
+        }
+        MetadataProvenance provenance = new MetadataProvenance(reader.lastContainedLogOffset(),
+                reader.lastContainedLogEpoch(), reader.lastContainedLogTimestamp());
+        return new SnapshotManifest(provenance,
+                time.nanoseconds() - startNs);
+    }
+
+    @Override
+    public void handleLeaderChange(LeaderAndEpoch leaderAndEpoch) {
+        eventQueue.append(() -> {
+            currentLeaderAndEpoch = leaderAndEpoch;
+        });
+    }
+
+    /**
+     * Install a list of publishers. When a publisher is installed, we will publish a MetadataDelta
+     * to it which contains the entire current image.
+     *
+     * @param newPublishers     The publishers to install.
+     *
+     * @return                  A future which yields null when the publishers have been added, or
+     *                          an exception if the installation failed.
+     */
+    public CompletableFuture<Void> installPublishers(List<? extends MetadataPublisher> newPublishers) {
+        if (newPublishers.isEmpty()) return CompletableFuture.completedFuture(null);
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        eventQueue.append(() -> {
+            try {
+                installNewPublishers(newPublishers);
+                future.complete(null);
+            } catch (Throwable e) {
+                future.completeExceptionally(faultHandler.handleFault("Unhandled fault in " +
+                        "MetadataLoader#installPublishers", e));
+            }
+        });
+        return future;
+    }
+
+    void installNewPublishers(
+        List<? extends MetadataPublisher> newPublishers
+    ) {
+        // Publishers can't be re-installed if they're already present.
+        for (MetadataPublisher newPublisher : newPublishers) {
+            MetadataPublisher prev = publishers.get(newPublisher.name());
+            if (prev == null) {
+                prev = uninitializedPublishers.get(newPublisher.name());
+            }
+            if (prev != null) {
+                if (prev == newPublisher) {
+                    throw faultHandler.handleFault("Attempted to install publisher " +
+                            newPublisher.name() + ", which is already installed.");
+                } else {
+                    throw faultHandler.handleFault("Attempted to install a new publisher " +
+                            "named " + newPublisher.name() + ", but there is already a publisher " +
+                            "with that name.");
+                }
+            }
+            uninitializedPublishers.put(newPublisher.name(), newPublisher);
+        }
+    }
+
+    // VisibleForTesting
+    void waitForAllEventsToBeHandled() throws Exception {
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        eventQueue.append(() -> future.complete(null));
+        future.get();
+    }
+
+    /**
+     * Remove a publisher and close it.
+     *
+     * @param publisher         The publisher to remove and close.
+     *
+     * @return                  A future which yields null when the publisher has been removed
+     *                          and closed, or an exception if the removal failed.
+     */
+    public CompletableFuture<Void> removeAndClosePublisher(MetadataPublisher publisher) {
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        eventQueue.append(() -> {
+            try {
+                if (!publishers.remove(publisher.name(), publisher)) {
+                    if (!uninitializedPublishers.remove(publisher.name(), publisher)) {
+                        throw faultHandler.handleFault("Attempted to remove publisher " + publisher.name() +
+                                ", which is not installed.");
+                    }
+                }
+                closePublisher(publisher);
+                future.complete(null);
+            } catch (Throwable e) {
+                future.completeExceptionally(e);
+            }
+        });
+        return future;
+    }
+
+    public long lastAppliedOffset() {
+        return metrics.lastAppliedOffset();
+    }
+
+    @Override
+    public void beginShutdown() {
+        eventQueue.beginShutdown("beginShutdown", () -> {
+            for (Iterator<MetadataPublisher> iter = uninitializedPublishers.values().iterator();
+                    iter.hasNext(); ) {
+                closePublisher(iter.next());
+                iter.remove();
+            }
+            for (Iterator<MetadataPublisher> iter = publishers.values().iterator();
+                 iter.hasNext(); ) {
+                closePublisher(iter.next());
+                iter.remove();
+            }
+        });
+    }
+
+    Time time() {
+        return time;
+    }
+
+    private void closePublisher(MetadataPublisher publisher) {
+        try {
+            publisher.close();
+        } catch (Throwable e) {
+            faultHandler.handleFault("Got unexpected exception while closing " +
+                    "publisher " + publisher.name(), e);
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        beginShutdown();
+        eventQueue.close();
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoaderMetrics.java b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoaderMetrics.java
new file mode 100644
index 00000000000..654bc9dd505
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoaderMetrics.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.image.loader;
+
+import org.apache.kafka.image.MetadataProvenance;
+
+
+/**
+ * An interface for the metadata loader metrics.
+ */
+public interface MetadataLoaderMetrics extends AutoCloseable {
+    /**
+     * Update the batch processing time histogram.
+     */
+    void updateBatchProcessingTime(long elapsedNs);
+
+    /**
+     * Update the batch size histogram.
+     */
+    void updateBatchSize(int size);
+
+    /**
+     * Set the provenance of the last image which has been processed by all publishers.
+     */
+    void updateLastAppliedImageProvenance(MetadataProvenance provenance);
+
+    /**
+     * Retrieve the last offset which has been processed by all publishers.
+     */
+    long lastAppliedOffset();
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java b/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.java
new file mode 100644
index 00000000000..b6c6dcce4d5
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/loader/SnapshotManifest.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.image.loader;
+
+import org.apache.kafka.image.MetadataProvenance;
+
+import java.util.Objects;
+
+
+/**
+ * Contains information about a snapshot that was loaded.
+ */
+public class SnapshotManifest {
+    /**
+     * The source of this snapshot.
+     */
+    private final MetadataProvenance provenance;
+
+    /**
+     * The time in microseconds that it took to load the snapshot.
+     */
+    private final long elapsedNs;
+
+    public SnapshotManifest(
+        MetadataProvenance provenance,
+        long elapsedNs
+    ) {
+        this.provenance = provenance;
+        this.elapsedNs = elapsedNs;
+    }
+
+    public MetadataProvenance provenance() {
+        return provenance;
+    }
+
+    public long elapsedNs() {
+        return elapsedNs;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                provenance,
+                elapsedNs);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null || !o.getClass().equals(this.getClass())) return false;
+        SnapshotManifest other = (SnapshotManifest) o;
+        return provenance.equals(other.provenance) &&
+                elapsedNs == other.elapsedNs;
+    }
+
+    @Override
+    public String toString() {
+        return "SnapshotManifest(" +
+                "provenance=" + provenance +
+                ", elapsedNs=" + elapsedNs +
+                ")";
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java b/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java
new file mode 100644
index 00000000000..8dfba7a99ab
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java
@@ -0,0 +1,72 @@
+/*
+ * 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.image.publisher;
+
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.loader.LogDeltaManifest;
+import org.apache.kafka.image.loader.SnapshotManifest;
+
+
+/**
+ * Publishes metadata deltas which we have loaded from the log and snapshots.
+ *
+ * Publishers receive a stream of callbacks from the metadata loader which keeps them notified
+ * of the latest cluster metadata. This interface abstracts away some of the complications of
+ * following the cluster metadata. For example, if the loader needs to read a snapshot, it will
+ * present the contents of the snapshot in the form of a delta from the previous state.
+ */
+public interface MetadataPublisher extends AutoCloseable {
+    /**
+     * Returns the name of this publisher.
+     *
+     * @return The publisher name.
+     */
+    String name();
+
+    /**
+     * Publish a new cluster metadata snapshot that we loaded.
+     *
+     * @param delta    The delta between the previous state and the new one.
+     * @param newImage The complete new state.
+     * @param manifest The contents of what was published.
+     */
+    void publishSnapshot(
+            MetadataDelta delta,
+            MetadataImage newImage,
+            SnapshotManifest manifest
+    );
+
+    /**
+     * Publish a change to the cluster metadata.
+     *
+     * @param delta    The delta between the previous state and the new one.
+     * @param newImage The complete new state.
+     * @param manifest The contents of what was published.
+     */
+    void publishLogDelta(
+            MetadataDelta delta,
+            MetadataImage newImage,
+            LogDeltaManifest manifest
+    );
+
+    /**
+     * Close this metadata publisher.
+     */
+    void close() throws Exception;
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java
new file mode 100644
index 00000000000..31ac2169be4
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java
@@ -0,0 +1,123 @@
+/*
+ * 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.image.publisher;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.MetadataProvenance;
+import org.apache.kafka.image.writer.ImageWriterOptions;
+import org.apache.kafka.image.writer.RaftSnapshotWriter;
+import org.apache.kafka.raft.RaftClient;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.snapshot.SnapshotWriter;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+
+
+public class SnapshotEmitter implements SnapshotGenerator.Emitter {
+    /**
+     * The maximum number of records we will put in each snapshot batch by default.
+     *
+     * From the perspective of the Raft layer, the limit on batch size is specified in terms of
+     * bytes, not number of records. See MAX_BATCH_SIZE_BYTES in KafkaRaftClient for details.
+     * However, it's more convenient to limit the batch size here in terms of number of records.
+     * So we chose a low number that will not cause problems.
+     */
+    private final static int DEFAULT_BATCH_SIZE = 1024;
+
+    public static class Builder {
+        private int nodeId = 0;
+        private RaftClient<ApiMessageAndVersion> raftClient = null;
+        private int batchSize = DEFAULT_BATCH_SIZE;
+
+        public Builder setNodeId(int nodeId) {
+            this.nodeId = nodeId;
+            return this;
+        }
+
+        public Builder setRaftClient(RaftClient<ApiMessageAndVersion> raftClient) {
+            this.raftClient = raftClient;
+            return this;
+        }
+
+        public Builder setBatchSize(int batchSize) {
+            this.batchSize = batchSize;
+            return this;
+        }
+
+        public SnapshotEmitter build() {
+            if (raftClient == null) throw new RuntimeException("You must set the raftClient.");
+            return new SnapshotEmitter(nodeId,
+                    raftClient,
+                    batchSize);
+        }
+    }
+
+    /**
+     * The slf4j logger to use.
+     */
+    private final Logger log;
+
+    /**
+     * The RaftClient to use.
+     */
+    private final RaftClient<ApiMessageAndVersion> raftClient;
+
+    /**
+     * The maximum number of records to put in each batch.
+     */
+    private final int batchSize;
+
+    private SnapshotEmitter(
+            int nodeId,
+            RaftClient<ApiMessageAndVersion> raftClient,
+            int batchSize
+    ) {
+        this.log = new LogContext("[SnapshotEmitter id=" + nodeId + "] ").logger(SnapshotEmitter.class);
+        this.raftClient = raftClient;
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    public void maybeEmit(MetadataImage image) {
+        MetadataProvenance provenance = image.provenance();
+        Optional<SnapshotWriter<ApiMessageAndVersion>> snapshotWriter =
+            raftClient.createSnapshot(provenance.offsetAndEpoch(),
+                    provenance.lastContainedLogTimeMs());
+        if (!snapshotWriter.isPresent()) {
+            log.error("Not generating {} because it already exists.", provenance.snapshotName());
+            return;
+        }
+        RaftSnapshotWriter writer = new RaftSnapshotWriter(snapshotWriter.get(), batchSize);
+        try {
+            image.write(writer, new ImageWriterOptions.Builder().
+                    setMetadataVersion(image.features().metadataVersion()).
+                    build());
+            writer.close(true);
+        } catch (Throwable e) {
+            log.error("Encountered error while writing {}", provenance.snapshotName(), e);
+            throw e;
+        } finally {
+            Utils.closeQuietly(writer, "RaftSnapshotWriter");
+            Utils.closeQuietly(snapshotWriter.get(), "SnapshotWriter");
+        }
+        log.info("Successfully wrote {}", provenance.snapshotName());
+    }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java
new file mode 100644
index 00000000000..43809de3898
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.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.image.publisher;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.loader.LogDeltaManifest;
+import org.apache.kafka.image.loader.SnapshotManifest;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.server.fault.FaultHandler;
+import org.slf4j.Logger;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+
+/**
+ * A metadata publisher that generates snapshots when appropriate.
+ */
+public class SnapshotGenerator implements MetadataPublisher {
+    public static class Builder {
+        private final Emitter emitter;
+        private int nodeId = 0;
+        private Time time = Time.SYSTEM;
+        private FaultHandler faultHandler = (m, e) -> null;
+        private long maxBytesSinceLastSnapshot = 100 * 1024L * 1024L;
+        private long maxTimeSinceLastSnapshotNs = TimeUnit.DAYS.toNanos(1);
+        private AtomicReference<String> disabledReason = null;
+
+        public Builder(Emitter emitter) {
+            this.emitter = emitter;
+        }
+
+        public Builder setNodeId(int nodeId) {
+            this.nodeId = nodeId;
+            return this;
+        }
+
+        public Builder setTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder setFaultHandler(FaultHandler faultHandler) {
+            this.faultHandler = faultHandler;
+            return this;
+        }
+
+        public Builder setMaxBytesSinceLastSnapshot(long maxBytesSinceLastSnapshot) {
+            this.maxBytesSinceLastSnapshot = maxBytesSinceLastSnapshot;
+            return this;
+        }
+
+        public Builder setMaxTimeSinceLastSnapshotNs(long maxTimeSinceLastSnapshotNs) {
+            this.maxTimeSinceLastSnapshotNs = maxTimeSinceLastSnapshotNs;
+            return this;
+        }
+
+        public Builder setDisabledReason(AtomicReference<String> disabledReason) {
+            this.disabledReason = disabledReason;
+            return this;
+        }
+
+        public SnapshotGenerator build() {
+            if (disabledReason == null) {
+                disabledReason = new AtomicReference<>();
+            }
+            return new SnapshotGenerator(
+                nodeId,
+                time,
+                emitter,
+                faultHandler,
+                maxBytesSinceLastSnapshot,
+                maxTimeSinceLastSnapshotNs,
+                disabledReason
+            );
+        }
+    }
+
+    /**
+     * The callback which actually generates the snapshot.
+     */
+    public interface Emitter {
+        /**
+         * Emit a snapshot for the given image.
+         *
+         * Note: if a snapshot has already been emitted for the given offset and epoch pair, this
+         * function will not recreate it.
+         *
+         * @param image     The metadata image to emit.
+         */
+        void maybeEmit(MetadataImage image);
+    }
+
+    /**
+     * The node ID.
+     */
+    private final int nodeId;
+
+    /**
+     * The clock to use.
+     */
+    private final Time time;
+
+    /**
+     * The emitter callback, which actually generates the snapshot.
+     */
+    private final Emitter emitter;
+
+    /**
+     * The slf4j logger to use.
+     */
+    private final Logger log;
+
+    /**
+     * The fault handler to use.
+     */
+    private final FaultHandler faultHandler;
+
+    /**
+     * The maximum number of bytes we will wait to see before triggering a new snapshot.
+     */
+    private final long maxBytesSinceLastSnapshot;
+
+    /**
+     * The maximum amount of time we will wait before triggering a snapshot, or 0 to disable
+     * time-based snapshotting.
+     */
+    private final long maxTimeSinceLastSnapshotNs;
+
+    /**
+     * If non-null, the reason why snapshots have been disabled.
+     */
+    private final AtomicReference<String> disabledReason;
+
+    /**
+     * The event queue used to schedule emitting snapshots.
+     */
+    private final EventQueue eventQueue;
+
+    /**
+     * The log bytes that we have read since the last snapshot.
+     */
+    private long bytesSinceLastSnapshot;
+
+    /**
+     * The time at which we created the last snapshot.
+     */
+    private long lastSnapshotTimeNs;
+
+    private SnapshotGenerator(
+        int nodeId,
+        Time time,
+        Emitter emitter,
+        FaultHandler faultHandler,
+        long maxBytesSinceLastSnapshot,
+        long maxTimeSinceLastSnapshotNs,
+        AtomicReference<String> disabledReason
+    ) {
+        this.nodeId = nodeId;
+        this.time = time;
+        this.emitter = emitter;
+        this.faultHandler = faultHandler;
+        this.maxBytesSinceLastSnapshot = maxBytesSinceLastSnapshot;
+        this.maxTimeSinceLastSnapshotNs = maxTimeSinceLastSnapshotNs;
+        LogContext logContext = new LogContext("[SnapshotGenerator " + nodeId + "] ");
+        this.log = logContext.logger(SnapshotGenerator.class);
+        this.disabledReason = disabledReason;
+        this.eventQueue = new KafkaEventQueue(time, logContext, "SnapshotGenerator" + nodeId);
+        resetSnapshotCounters();
+        log.debug("Starting SnapshotGenerator.");
+    }
+
+    @Override
+    public String name() {
+        return "SnapshotGenerator";
+    }
+
+    void resetSnapshotCounters() {
+        this.bytesSinceLastSnapshot = 0L;
+        this.lastSnapshotTimeNs = time.nanoseconds();
+    }
+
+    @Override
+    public void publishSnapshot(
+        MetadataDelta delta,
+        MetadataImage newImage,
+        SnapshotManifest manifest
+    ) {
+        log.debug("Resetting the snapshot counters because we just read a snapshot at offset {}.",
+                newImage.provenance().offset());
+        resetSnapshotCounters();
+    }
+
+    @Override
+    public void publishLogDelta(
+        MetadataDelta delta,
+        MetadataImage newImage,
+        LogDeltaManifest manifest
+    ) {
+        bytesSinceLastSnapshot += manifest.numBytes();
+        if (bytesSinceLastSnapshot >= maxBytesSinceLastSnapshot) {
+            if (eventQueue.isEmpty()) {
+                scheduleEmit("we have replayed at least " + maxBytesSinceLastSnapshot +
+                    " bytes", newImage);
+            } else if (log.isTraceEnabled()) {
+                log.trace("Not scheduling bytes-based snapshot because event queue is not empty yet.");
+            }
+        } else if (maxTimeSinceLastSnapshotNs != 0 &&
+                (time.nanoseconds() - lastSnapshotTimeNs >= maxTimeSinceLastSnapshotNs)) {
+            if (eventQueue.isEmpty()) {
+                scheduleEmit("we have waited at least " +
+                    TimeUnit.NANOSECONDS.toMinutes(maxTimeSinceLastSnapshotNs) + " minute(s)", newImage);
+            } else if (log.isTraceEnabled()) {
+                log.trace("Not scheduling time-based snapshot because event queue is not empty yet.");
+            }
+        } else if (log.isTraceEnabled()) {
+            log.trace("Neither time-based nor bytes-based criteria are met; not scheduling snapshot.");
+        }
+    }
+
+    void scheduleEmit(
+        String reason,
+        MetadataImage image
+    ) {
+        resetSnapshotCounters();
+        eventQueue.append(() -> {
+            String currentDisabledReason = disabledReason.get();
+            if (currentDisabledReason != null) {
+                log.error("Not emitting {} despite the fact that {} because snapshots are " +
+                    "disabled; {}", image.provenance().snapshotName(), reason,
+                        currentDisabledReason);
+            } else {
+                log.info("Creating new KRaft snapshot file {} because {}.",
+                        image.provenance().snapshotName(), reason);
+                try {
+                    emitter.maybeEmit(image);
+                } catch (Throwable e) {
+                    faultHandler.handleFault("KRaft snapshot file generation error", e);
+                }
+            }
+        });
+    }
+
+    public void beginShutdown() {
+        log.debug("Beginning shutdown of SnapshotGenerator.");
+        this.disabledReason.compareAndSet(null, "we are shutting down");
+        eventQueue.beginShutdown("beginShutdown");
+    }
+
+    @Override
+    public void close() throws InterruptedException {
+        eventQueue.beginShutdown("close");
+        log.debug("Closing SnapshotGenerator.");
+        eventQueue.close();
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java
index fdc03276451..566fa4acd54 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java
@@ -215,11 +215,9 @@ public class AclControlManagerTest {
 
         // Verify that the ACLs stored in the AclControlManager match the ones we expect.
         Set<ApiMessageAndVersion> foundAcls = new HashSet<>();
-        for (Iterator<List<ApiMessageAndVersion>> iterator = manager.iterator(Long.MAX_VALUE);
-                 iterator.hasNext(); ) {
-            for (ApiMessageAndVersion apiMessageAndVersion : iterator.next()) {
-                assertTrue(foundAcls.add(apiMessageAndVersion));
-            }
+        for (Map.Entry<Uuid, StandardAcl> entry : manager.idToAcl().entrySet()) {
+            foundAcls.add(new ApiMessageAndVersion(
+                    new StandardAclWithId(entry.getKey(), entry.getValue()).toRecord(), (short) 0));
         }
         assertEquals(loadedAcls, foundAcls);
 
@@ -233,7 +231,7 @@ public class AclControlManagerTest {
         // a cluster metadata authorizer.
         snapshotRegistry.revertToSnapshot(0);
         authorizer.loadSnapshot(manager.idToAcl());
-        assertFalse(manager.iterator(Long.MAX_VALUE).hasNext());
+        assertTrue(manager.idToAcl().isEmpty());
     }
 
     @Test
@@ -243,11 +241,9 @@ public class AclControlManagerTest {
         MockClusterMetadataAuthorizer authorizer = new MockClusterMetadataAuthorizer();
         authorizer.loadSnapshot(manager.idToAcl());
         manager.replay(StandardAclWithIdTest.TEST_ACLS.get(0).toRecord(), Optional.empty());
-        assertEquals(new ApiMessageAndVersion(TEST_ACLS.get(0).toRecord(), (short) 0),
-            manager.iterator(Long.MAX_VALUE).next().get(0));
         manager.replay(new RemoveAccessControlEntryRecord().
             setId(TEST_ACLS.get(0).id()), Optional.empty());
-        assertFalse(manager.iterator(Long.MAX_VALUE).hasNext());
+        assertTrue(manager.idToAcl().isEmpty());
     }
 
     @Test
@@ -284,7 +280,7 @@ public class AclControlManagerTest {
             }
         }
         RecordTestUtils.replayAll(manager, createResult.records());
-        assertTrue(manager.iterator(Long.MAX_VALUE).hasNext());
+        assertFalse(manager.idToAcl().isEmpty());
 
         ControllerResult<List<AclDeleteResult>> deleteResult =
             manager.deleteAcls(Arrays.asList(
@@ -308,12 +304,8 @@ public class AclControlManagerTest {
             deleteResult.response().get(1).exception().get().getClass());
         RecordTestUtils.replayAll(manager, deleteResult.records());
 
-        Iterator<List<ApiMessageAndVersion>> iterator = manager.iterator(Long.MAX_VALUE);
-        assertTrue(iterator.hasNext());
-        List<ApiMessageAndVersion> list = iterator.next();
-        assertEquals(1, list.size());
-        assertEquals(TEST_ACLS.get(1).toBinding(), StandardAcl.fromRecord(
-            (AccessControlEntryRecord) list.get(0).message()).toBinding());
+        Iterator<Map.Entry<Uuid, StandardAcl>> iterator = manager.idToAcl().entrySet().iterator();
+        assertEquals(TEST_ACLS.get(1).acl(), iterator.next().getValue());
         assertFalse(iterator.hasNext());
     }
 
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java
index b915db3fe0d..1fb81cbf7a4 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java
@@ -205,49 +205,50 @@ public class ClientQuotaControlManagerTest {
 
         List<ClientQuotaAlteration> alters = new ArrayList<>();
         quotasToTest.forEach((entity, quota) -> entityQuotaToAlterations(entity, quota, alters::add));
-        alterQuotas(alters, manager);
-
-        RecordTestUtils.assertBatchIteratorContains(Arrays.asList(
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+        List<ApiMessageAndVersion> records = alterQuotas(alters, manager);
+        List<ApiMessageAndVersion> expectedRecords = Arrays.asList(
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName("user-1"),
                 new EntityData().setEntityType("client-id").setEntityName("client-id-1"))).
-                    setKey("request_percentage").setValue(50.5).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(50.5).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName("user-2"),
                 new EntityData().setEntityType("client-id").setEntityName("client-id-1"))).
-                    setKey("request_percentage").setValue(51.51).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(51.51).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName("user-3"),
                 new EntityData().setEntityType("client-id").setEntityName("client-id-2"))).
-                    setKey("request_percentage").setValue(52.52).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(52.52).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName(null),
                 new EntityData().setEntityType("client-id").setEntityName("client-id-1"))).
-                    setKey("request_percentage").setValue(53.53).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(53.53).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName("user-1"),
                 new EntityData().setEntityType("client-id").setEntityName(null))).
-                    setKey("request_percentage").setValue(54.54).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(54.54).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName("user-3"),
                 new EntityData().setEntityType("client-id").setEntityName(null))).
-                    setKey("request_percentage").setValue(55.55).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(55.55).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName("user-1"))).
-                    setKey("request_percentage").setValue(56.56).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(56.56).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName("user-2"))).
-                    setKey("request_percentage").setValue(57.57).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(57.57).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName("user-3"))).
-                    setKey("request_percentage").setValue(58.58).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(58.58).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("user").setEntityName(null))).
-                    setKey("request_percentage").setValue(59.59).setRemove(false), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
+                    setKey("request_percentage").setValue(59.59).setRemove(false), (short) 0),
+            new ApiMessageAndVersion(new ClientQuotaRecord().setEntity(Arrays.asList(
                 new EntityData().setEntityType("client-id").setEntityName("client-id-2"))).
-                    setKey("request_percentage").setValue(60.60).setRemove(false), (short) 0))),
-            manager.iterator(Long.MAX_VALUE));
+                    setKey("request_percentage").setValue(60.60).setRemove(false), (short) 0));
+        RecordTestUtils.deepSortRecords(records);
+        RecordTestUtils.deepSortRecords(expectedRecords);
+        assertEquals(expectedRecords, records);
     }
 
     static void entityQuotaToAlterations(ClientQuotaEntity entity, Map<String, Double> quota,
@@ -258,11 +259,15 @@ public class ClientQuotaControlManagerTest {
         acceptor.accept(new ClientQuotaAlteration(entity, ops));
     }
 
-    static void alterQuotas(List<ClientQuotaAlteration> alterations, ClientQuotaControlManager manager) {
+    static List<ApiMessageAndVersion> 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((ClientQuotaRecord) apiMessageAndVersion.message()));
+        return result.records();
     }
 
     static Map<String, Double> quotas(String key, Double value) {
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java
index 72eea5427e8..86918d5513c 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java
@@ -34,12 +34,12 @@ 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.image.writer.ImageWriterOptions;
 import org.apache.kafka.metadata.BrokerRegistration;
 import org.apache.kafka.metadata.BrokerRegistrationFencingChange;
 import org.apache.kafka.metadata.BrokerRegistrationInControlledShutdownChange;
 import org.apache.kafka.metadata.BrokerRegistrationReply;
 import org.apache.kafka.metadata.FinalizedControllerFeatures;
-import org.apache.kafka.metadata.RecordTestUtils;
 import org.apache.kafka.metadata.VersionRange;
 import org.apache.kafka.metadata.placement.PartitionAssignment;
 import org.apache.kafka.metadata.placement.PlacementSpec;
@@ -418,7 +418,7 @@ public class ClusterControlManagerTest {
 
     @ParameterizedTest
     @EnumSource(value = MetadataVersion.class, names = {"IBP_3_3_IV2", "IBP_3_3_IV3"})
-    public void testIterator(MetadataVersion metadataVersion) throws Exception {
+    public void testRegistrationsToRecords(MetadataVersion metadataVersion) throws Exception {
         MockTime time = new MockTime(0, 0, 0);
         SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
         FeatureControlManager featureControl = new FeatureControlManager.Builder().
@@ -460,8 +460,12 @@ public class ClusterControlManagerTest {
                     IN_CONTROLLED_SHUTDOWN.value());
         clusterControl.replay(registrationChangeRecord);
         short expectedVersion = metadataVersion.registerBrokerRecordVersion();
-        RecordTestUtils.assertBatchIteratorContains(Arrays.asList(
-            Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord().
+
+        ImageWriterOptions options = new ImageWriterOptions.Builder().
+                setMetadataVersion(metadataVersion).
+                setLossHandler(__ -> { }).
+                build();
+        assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord().
                 setBrokerEpoch(100).setBrokerId(0).setRack(null).
                 setEndPoints(new BrokerEndpointCollection(Collections.singleton(
                     new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id).
@@ -469,27 +473,28 @@ public class ClusterControlManagerTest {
                         setName("PLAINTEXT").
                         setHost("example.com")).iterator())).
                 setInControlledShutdown(metadataVersion.isInControlledShutdownStateSupported()).
-                setFenced(false), expectedVersion)),
-            Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord().
+                setFenced(false), expectedVersion),
+            clusterControl.brokerRegistrations().get(0).toRecord(options));
+        assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord().
                 setBrokerEpoch(100).setBrokerId(1).setRack(null).
                 setEndPoints(new BrokerEndpointCollection(Collections.singleton(
                     new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id).
                         setPort((short) 9093).
                         setName("PLAINTEXT").
                         setHost("example.com")).iterator())).
-                setFenced(false), expectedVersion)),
-            Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord().
+                setFenced(false), expectedVersion),
+            clusterControl.brokerRegistrations().get(1).toRecord(options));
+        assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord().
                 setBrokerEpoch(100).setBrokerId(2).setRack(null).
                 setEndPoints(new BrokerEndpointCollection(Collections.singleton(
                     new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id).
                         setPort((short) 9094).
                         setName("PLAINTEXT").
                         setHost("example.com")).iterator())).
-                setFenced(true), expectedVersion))),
-                clusterControl.iterator(Long.MAX_VALUE));
+                        setFenced(true), expectedVersion),
+            clusterControl.brokerRegistrations().get(2).toRecord(options));
     }
 
-
     @Test
     public void testRegistrationWithUnsupportedMetadataVersion() {
         SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java
index 02bd6e4d7e9..bbca7bfbf6b 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java
@@ -135,14 +135,6 @@ public class ConfigurationControlManagerTest {
             setName("def").setValue("blah"));
         assertEquals(toMap(entry("abc", "x,y,z"), entry("def", "blah")),
             manager.getConfigs(MYTOPIC));
-        RecordTestUtils.assertBatchIteratorContains(asList(
-            asList(new ApiMessageAndVersion(new ConfigRecord().
-                    setResourceType(TOPIC.id()).setResourceName("mytopic").
-                    setName("abc").setValue("x,y,z"), CONFIG_RECORD.highestSupportedVersion()),
-                new ApiMessageAndVersion(new ConfigRecord().
-                    setResourceType(TOPIC.id()).setResourceName("mytopic").
-                    setName("def").setValue("blah"), CONFIG_RECORD.highestSupportedVersion()))),
-            manager.iterator(Long.MAX_VALUE));
     }
 
     @Test
diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
index 2478f4ce164..8345b4e3149 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java
@@ -21,6 +21,7 @@ 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 java.util.Optional;
@@ -202,25 +203,7 @@ public class FeatureControlManagerTest {
     }
 
     @Test
-    public void testFeatureControlIteratorWithOldMetadataVersion() throws Exception {
-        // We require minimum of IBP_3_3_IV0 to write metadata version in the snapshot.
-
-        LogContext logContext = new LogContext();
-        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
-        FeatureControlManager manager = new FeatureControlManager.Builder()
-            .setLogContext(logContext)
-            .setSnapshotRegistry(snapshotRegistry)
-            .setMetadataVersion(MetadataVersion.IBP_3_2_IV0)
-            .build();
-
-        RecordTestUtils.assertBatchIteratorContains(
-            Collections.emptyList(),
-            manager.iterator(Long.MAX_VALUE)
-        );
-    }
-
-    @Test
-    public void testFeatureControlIterator() throws Exception {
+    public void testReplayRecords() throws Exception {
         LogContext logContext = new LogContext();
         SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
         FeatureControlManager manager = new FeatureControlManager.Builder().
@@ -233,17 +216,12 @@ public class FeatureControlManagerTest {
             updateFeatures(updateMap("foo", 5, "bar", 1),
                 Collections.emptyMap(), Collections.emptyMap(), false);
         RecordTestUtils.replayAll(manager, result.records());
-        RecordTestUtils.assertBatchIteratorContains(Arrays.asList(
-            Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord().
-                    setName("metadata.version").
-                    setFeatureLevel((short) 4), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord().
-                setName("foo").
-                setFeatureLevel((short) 5), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord().
-                setName("bar").
-                setFeatureLevel((short) 1), (short) 0))),
-            manager.iterator(Long.MAX_VALUE));
+        assertEquals(MetadataVersion.IBP_3_3_IV0, manager.metadataVersion());
+        assertEquals(Optional.of((short) 5), manager.finalizedFeatures(Long.MAX_VALUE).get("foo"));
+        assertEquals(Optional.of((short) 1), manager.finalizedFeatures(Long.MAX_VALUE).get("bar"));
+        assertEquals(new HashSet<>(Arrays.asList(
+            MetadataVersion.FEATURE_NAME, "foo", "bar")),
+                manager.finalizedFeatures(Long.MAX_VALUE).featureNames());
     }
 
     private static final FeatureControlManager.Builder TEST_MANAGER_BUILDER1 =
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java
index 80c5c505ae0..d0c16d1e3b0 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java
@@ -26,20 +26,14 @@ import org.apache.kafka.common.metadata.RegisterBrokerRecord;
 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.server.common.ApiMessageAndVersion;
 import org.apache.kafka.server.common.MetadataVersion;
 import org.apache.kafka.server.common.ProducerIdsBlock;
 import org.apache.kafka.timeline.SnapshotRegistry;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
-import java.util.Iterator;
-import java.util.List;
-
 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;
 
 
 public class ProducerIdControlManagerTest {
@@ -149,29 +143,11 @@ public class ProducerIdControlManagerTest {
     }
 
     @Test
-    public void testSnapshotIterator() {
-        ProducerIdsBlock range = null;
+    public void testGenerateProducerIds() {
         for (int i = 0; i < 100; i++) {
-            range = generateProducerIds(producerIdControlManager, i % 4, 100);
-        }
-
-        Iterator<List<ApiMessageAndVersion>> snapshotIterator = producerIdControlManager.iterator(Long.MAX_VALUE);
-        assertTrue(snapshotIterator.hasNext());
-        List<ApiMessageAndVersion> batch = snapshotIterator.next();
-        assertEquals(1, batch.size(), "Producer IDs record batch should only contain a single record");
-        assertEquals(range.firstProducerId() + range.size(), ((ProducerIdsRecord) batch.get(0).message()).nextProducerId());
-        assertFalse(snapshotIterator.hasNext(), "Producer IDs iterator should only contain a single batch");
-
-        ProducerIdControlManager newProducerIdManager = new ProducerIdControlManager(clusterControl, snapshotRegistry);
-        snapshotIterator = producerIdControlManager.iterator(Long.MAX_VALUE);
-        while (snapshotIterator.hasNext()) {
-            snapshotIterator.next().forEach(message -> newProducerIdManager.replay((ProducerIdsRecord) message.message()));
+            generateProducerIds(producerIdControlManager, i % 4, 100);
         }
-
-        // Verify that after reloading state from this "snapshot", we don't produce any overlapping IDs
-        long lastProducerID = range.firstProducerId() + range.size() - 1;
-        range = generateProducerIds(producerIdControlManager, 1, 100);
-        assertTrue(range.firstProducerId() > lastProducerID);
+        assertEquals(new ProducerIdsBlock(3, 100000, 1000), producerIdControlManager.nextProducerBlock());
     }
 
     static ProducerIdsBlock generateProducerIds(
diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
index 10921c77fbd..9d0508d876a 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
@@ -31,8 +31,6 @@ import java.util.Optional;
 import java.util.OptionalInt;
 import java.util.OptionalLong;
 import java.util.Set;
-import java.util.Spliterator;
-import java.util.Spliterators;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
@@ -42,7 +40,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
-import java.util.stream.StreamSupport;
 
 import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
@@ -85,27 +82,22 @@ import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpointColle
 import org.apache.kafka.common.metadata.TopicRecord;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.requests.ApiError;
-import org.apache.kafka.common.utils.BufferSupplier;
 import org.apache.kafka.controller.QuorumController.ConfigResourceExistenceChecker;
 import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationFencingChange;
 import org.apache.kafka.metadata.BrokerRegistrationReply;
 import org.apache.kafka.metadata.FinalizedControllerFeatures;
-import org.apache.kafka.metadata.MetadataRecordSerde;
 import org.apache.kafka.metadata.PartitionRegistration;
-import org.apache.kafka.metadata.RecordTestUtils;
 import org.apache.kafka.metadata.authorizer.StandardAuthorizer;
 import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
 import org.apache.kafka.metadata.util.BatchFileWriter;
 import org.apache.kafka.metalog.LocalLogManager;
 import org.apache.kafka.metalog.LocalLogManagerTestEnv;
-import org.apache.kafka.raft.Batch;
 import org.apache.kafka.raft.OffsetAndEpoch;
 import org.apache.kafka.server.common.ApiMessageAndVersion;
 import org.apache.kafka.server.common.MetadataVersion;
 import org.apache.kafka.snapshot.FileRawSnapshotReader;
-import org.apache.kafka.snapshot.SnapshotReader;
 import org.apache.kafka.snapshot.RawSnapshotReader;
-import org.apache.kafka.snapshot.RecordsSnapshotReader;
 import org.apache.kafka.snapshot.Snapshots;
 import org.apache.kafka.test.TestUtils;
 import org.junit.jupiter.api.Disabled;
@@ -128,7 +120,6 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals;
 import static org.junit.jupiter.api.Assertions.assertSame;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
 
 
 @Timeout(value = 40)
@@ -669,324 +660,43 @@ public class QuorumControllerTest {
             fooId = fooData.topics().find("foo").topicId();
             active.allocateProducerIds(ANONYMOUS_CONTEXT,
                 new AllocateProducerIdsRequestData().setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0))).get();
-            long snapshotLogOffset = active.beginWritingSnapshot().get();
-            reader = logEnv.waitForSnapshot(snapshotLogOffset);
-            SnapshotReader<ApiMessageAndVersion> snapshot = createSnapshotReader(reader);
-            assertEquals(snapshotLogOffset, snapshot.lastContainedLogOffset());
-            checkSnapshotContent(expectedSnapshotContent(fooId, brokerEpochs), snapshot);
-        }
-
-        try (
-            LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).
-                setSnapshotReader(reader).
-                build();
-            QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv).
-                setControllerBuilderInitializer(controllerBuilder -> {
-                    controllerBuilder.setConfigSchema(SCHEMA);
-                }).
-                build();
-        ) {
-            QuorumController active = controlEnv.activeController();
-            long snapshotLogOffset = active.beginWritingSnapshot().get();
-            SnapshotReader<ApiMessageAndVersion> snapshot = createSnapshotReader(
-                logEnv.waitForSnapshot(snapshotLogOffset)
-            );
-            assertEquals(snapshotLogOffset, snapshot.lastContainedLogOffset());
-            checkSnapshotContent(expectedSnapshotContent(fooId, brokerEpochs), snapshot);
-        }
-    }
-
-    @Test
-    public void testSnapshotConfiguration() throws Throwable {
-        final int numBrokers = 4;
-        final int maxNewRecordBytes = 4;
-        Map<Integer, Long> brokerEpochs = new HashMap<>();
-        Uuid fooId;
-        try (
-            LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).
-                build();
-            QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv).
-                setControllerBuilderInitializer(controllerBuilder -> {
-                    controllerBuilder.setConfigSchema(SCHEMA);
-                    controllerBuilder.setSnapshotMaxNewRecordBytes(maxNewRecordBytes);
-                    controllerBuilder.setBootstrapMetadata(SIMPLE_BOOTSTRAP);
-                }).
-                build();
-        ) {
-            QuorumController active = controlEnv.activeController();
-            for (int i = 0; i < numBrokers; i++) {
-                BrokerRegistrationReply reply = active.registerBroker(ANONYMOUS_CONTEXT,
-                    new BrokerRegistrationRequestData().
-                        setBrokerId(i).
-                        setRack(null).
-                        setClusterId(active.clusterId()).
-                        setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_3_IV3)).
-                        setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)).
-                        setListeners(new ListenerCollection(Arrays.asList(new Listener().
-                            setName("PLAINTEXT").setHost("localhost").
-                            setPort(9092 + i)).iterator()))).get();
-                brokerEpochs.put(i, reply.epoch());
-            }
-            for (int i = 0; i < numBrokers - 1; i++) {
-                assertEquals(new BrokerHeartbeatReply(true, false, false, false),
-                    active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData().
-                        setWantFence(false).setBrokerEpoch(brokerEpochs.get(i)).
-                        setBrokerId(i).setCurrentMetadataOffset(100000L)).get());
-            }
-            CreateTopicsResponseData fooData = active.createTopics(ANONYMOUS_CONTEXT,
-                new CreateTopicsRequestData().setTopics(
-                    new CreatableTopicCollection(Collections.singleton(
-                        new CreatableTopic().setName("foo").setNumPartitions(-1).
-                            setReplicationFactor((short) -1).
-                            setAssignments(new CreatableReplicaAssignmentCollection(
-                                Arrays.asList(new CreatableReplicaAssignment().
-                                    setPartitionIndex(0).
-                                    setBrokerIds(Arrays.asList(0, 1, 2)),
-                                new CreatableReplicaAssignment().
-                                    setPartitionIndex(1).
-                                    setBrokerIds(Arrays.asList(1, 2, 0))).
-                                        iterator()))).iterator())),
-                Collections.singleton("foo")).get();
-            fooId = fooData.topics().find("foo").topicId();
-            active.allocateProducerIds(ANONYMOUS_CONTEXT,
-                    new AllocateProducerIdsRequestData().setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0))).get();
-
-            SnapshotReader<ApiMessageAndVersion> snapshot = createSnapshotReader(logEnv.waitForLatestSnapshot());
-            checkSnapshotSubcontent(
-                expectedSnapshotContent(fooId, brokerEpochs),
-                snapshot
-            );
+            controlEnv.close();
+            assertEquals(generateTestRecords(fooId, brokerEpochs), logEnv.allRecords());
         }
     }
 
-    @Test
-    public void testSnapshotOnlyAfterConfiguredMinBytes() throws Throwable {
-        final int numBrokers = 4;
-        final int maxNewRecordBytes = 1000;
-        Map<Integer, Long> brokerEpochs = new HashMap<>();
-        try (
-            LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).
-                build();
-            QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv).
-                setControllerBuilderInitializer(controllerBuilder -> {
-                    controllerBuilder.setConfigSchema(SCHEMA);
-                    controllerBuilder.setSnapshotMaxNewRecordBytes(maxNewRecordBytes);
-                }).
-                build();
-        ) {
-            QuorumController active = controlEnv.activeController();
-            for (int i = 0; i < numBrokers; i++) {
-                BrokerRegistrationReply reply = active.registerBroker(ANONYMOUS_CONTEXT,
-                    new BrokerRegistrationRequestData().
-                        setBrokerId(i).
-                        setRack(null).
-                        setClusterId(active.clusterId()).
-                        setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_4_IV0)).
-                        setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)).
-                        setListeners(new ListenerCollection(Arrays.asList(new Listener().
-                            setName("PLAINTEXT").setHost("localhost").
-                            setPort(9092 + i)).iterator()))).get();
-                brokerEpochs.put(i, reply.epoch());
-                assertEquals(new BrokerHeartbeatReply(true, false, false, false),
-                    active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData().
-                        setWantFence(false).setBrokerEpoch(brokerEpochs.get(i)).
-                        setBrokerId(i).setCurrentMetadataOffset(100000L)).get());
-            }
-
-            assertTrue(logEnv.appendedBytes() < maxNewRecordBytes,
-                String.format("%s appended bytes is not less than %s max new record bytes",
-                    logEnv.appendedBytes(),
-                    maxNewRecordBytes));
-
-            // Keep creating topic until we reached the max bytes limit
-            int counter = 0;
-            while (logEnv.appendedBytes() < maxNewRecordBytes) {
-                counter += 1;
-                String topicName = String.format("foo-%s", counter);
-                active.createTopics(ANONYMOUS_CONTEXT, new CreateTopicsRequestData().setTopics(
-                        new CreatableTopicCollection(Collections.singleton(
-                            new CreatableTopic().setName(topicName).setNumPartitions(-1).
-                                setReplicationFactor((short) -1).
-                                setAssignments(new CreatableReplicaAssignmentCollection(
-                                    Arrays.asList(new CreatableReplicaAssignment().
-                                        setPartitionIndex(0).
-                                        setBrokerIds(Arrays.asList(0, 1, 2)),
-                                    new CreatableReplicaAssignment().
-                                        setPartitionIndex(1).
-                                        setBrokerIds(Arrays.asList(1, 2, 0))).
-                                            iterator()))).iterator())),
-                    Collections.singleton(topicName)).get(60, TimeUnit.SECONDS);
-            }
-            logEnv.waitForLatestSnapshot();
-        }
-    }
-
-    @Test
-    public void testSnapshotAfterConfiguredMaxInterval() throws Throwable {
-        final int numBrokers = 4;
-        Map<Integer, Long> brokerEpochs = new HashMap<>();
-        try (
-            LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).
-                build();
-            QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv).
-                setControllerBuilderInitializer(controllerBuilder -> {
-                    controllerBuilder.setConfigSchema(SCHEMA);
-                    controllerBuilder.setSnapshotMaxIntervalMs(100);
-                    // Disable snapshot generation due to bytes committed
-                    controllerBuilder.setSnapshotMaxNewRecordBytes(Long.MAX_VALUE);
-                }).
-                build();
-        ) {
-            QuorumController active = controlEnv.activeController();
-            for (int i = 0; i < numBrokers; i++) {
-                BrokerRegistrationReply reply = active.registerBroker(ANONYMOUS_CONTEXT,
-                    new BrokerRegistrationRequestData().
-                        setBrokerId(i).
-                        setRack(null).
-                        setClusterId(active.clusterId()).
-                        setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_4_IV0)).
-                        setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)).
-                        setListeners(new ListenerCollection(Arrays.asList(new Listener().
-                            setName("PLAINTEXT").setHost("localhost").
-                            setPort(9092 + i)).iterator()))).get();
-                brokerEpochs.put(i, reply.epoch());
-                assertEquals(new BrokerHeartbeatReply(true, false, false, false),
-                    active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData().
-                        setWantFence(false).setBrokerEpoch(brokerEpochs.get(i)).
-                        setBrokerId(i).setCurrentMetadataOffset(100000L)).get());
-            }
-
-            logEnv.waitForLatestSnapshot();
-        }
-    }
-
-    @Test
-    public void testSnapshotAfterRepeatedResign() throws Throwable {
-        final int numBrokers = 4;
-        final int maxNewRecordBytes = 1000;
-        Map<Integer, Long> brokerEpochs = new HashMap<>();
-        try (
-            LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).
-                build();
-            QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv).
-                setControllerBuilderInitializer(controllerBuilder -> {
-                    controllerBuilder.setConfigSchema(SCHEMA);
-                    controllerBuilder.setSnapshotMaxNewRecordBytes(maxNewRecordBytes);
-                }).
-                build();
-        ) {
-            QuorumController active = controlEnv.activeController();
-            for (int i = 0; i < numBrokers; i++) {
-                BrokerRegistrationReply reply = active.registerBroker(ANONYMOUS_CONTEXT,
-                    new BrokerRegistrationRequestData().
-                        setBrokerId(i).
-                        setRack(null).
-                        setClusterId(active.clusterId()).
-                        setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_4_IV0)).
-                        setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)).
-                        setListeners(new ListenerCollection(Arrays.asList(new Listener().
-                            setName("PLAINTEXT").setHost("localhost").
-                            setPort(9092 + i)).iterator()))).get();
-                brokerEpochs.put(i, reply.epoch());
-                assertEquals(new BrokerHeartbeatReply(true, false, false, false),
-                    active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData().
-                        setWantFence(false).setBrokerEpoch(brokerEpochs.get(i)).
-                        setBrokerId(i).setCurrentMetadataOffset(100000L)).get());
-            }
-
-            assertTrue(logEnv.appendedBytes() < maxNewRecordBytes,
-                String.format("%s appended bytes is not less than %s max new record bytes",
-                    logEnv.appendedBytes(),
-                    maxNewRecordBytes));
-
-            // Keep creating topic and resign leader until we reached the max bytes limit
-            int counter = 0;
-            while (logEnv.appendedBytes() < maxNewRecordBytes) {
-                active = controlEnv.activeController();
-
-                counter += 1;
-                String topicName = String.format("foo-%s", counter);
-                active.createTopics(ANONYMOUS_CONTEXT, new CreateTopicsRequestData().setTopics(
-                        new CreatableTopicCollection(Collections.singleton(
-                            new CreatableTopic().setName(topicName).setNumPartitions(-1).
-                                setReplicationFactor((short) -1).
-                                setAssignments(new CreatableReplicaAssignmentCollection(
-                                    Arrays.asList(new CreatableReplicaAssignment().
-                                        setPartitionIndex(0).
-                                        setBrokerIds(Arrays.asList(0, 1, 2)),
-                                    new CreatableReplicaAssignment().
-                                        setPartitionIndex(1).
-                                        setBrokerIds(Arrays.asList(1, 2, 0))).
-                                            iterator()))).iterator())),
-                    Collections.singleton(topicName)).get(60, TimeUnit.SECONDS);
-
-                LocalLogManager activeLocalLogManager = logEnv.logManagers().get(active.nodeId());
-                activeLocalLogManager.resign(activeLocalLogManager.leaderAndEpoch().epoch());
-            }
-            logEnv.waitForLatestSnapshot();
-        }
-    }
-
-    private SnapshotReader<ApiMessageAndVersion> createSnapshotReader(RawSnapshotReader reader) {
-        return RecordsSnapshotReader.of(
-            reader,
-            new MetadataRecordSerde(),
-            BufferSupplier.create(),
-            Integer.MAX_VALUE,
-            true
-        );
-    }
-
-    private List<ApiMessageAndVersion> expectedSnapshotContent(Uuid fooId, Map<Integer, Long> brokerEpochs) {
+    private List<ApiMessageAndVersion> generateTestRecords(Uuid fooId, Map<Integer, Long> brokerEpochs) {
         return Arrays.asList(
             new ApiMessageAndVersion(new FeatureLevelRecord().
                 setName(MetadataVersion.FEATURE_NAME).
                 setFeatureLevel(MetadataVersion.IBP_3_3_IV3.featureLevel()), (short) 0),
-            new ApiMessageAndVersion(new TopicRecord().
-                setName("foo").setTopicId(fooId), (short) 0),
-            new ApiMessageAndVersion(new PartitionRecord().setPartitionId(0).
-                setTopicId(fooId).setReplicas(Arrays.asList(0, 1, 2)).
-                setIsr(Arrays.asList(0, 1, 2)).setRemovingReplicas(Collections.emptyList()).
-                setAddingReplicas(Collections.emptyList()).setLeader(0).setLeaderEpoch(0).
-                setPartitionEpoch(0), (short) 0),
-            new ApiMessageAndVersion(new PartitionRecord().setPartitionId(1).
-                setTopicId(fooId).setReplicas(Arrays.asList(1, 2, 0)).
-                setIsr(Arrays.asList(1, 2, 0)).setRemovingReplicas(Collections.emptyList()).
-                setAddingReplicas(Collections.emptyList()).setLeader(1).setLeaderEpoch(0).
-                setPartitionEpoch(0), (short) 0),
             new ApiMessageAndVersion(new RegisterBrokerRecord().
                 setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0)).
                 setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB0")).
-                setEndPoints(
-                    new BrokerEndpointCollection(
-                        Arrays.asList(
-                            new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
-                            setPort(9092).setSecurityProtocol((short) 0)).iterator())).
+                setEndPoints(new BrokerEndpointCollection(
+                    Arrays.asList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
+                        setPort(9092).setSecurityProtocol((short) 0)).iterator())).
                 setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_3_IV3)).
                 setRack(null).
-                setFenced(false), (short) 1),
+                setFenced(true), (short) 1),
             new ApiMessageAndVersion(new RegisterBrokerRecord().
                 setBrokerId(1).setBrokerEpoch(brokerEpochs.get(1)).
                 setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB1")).
-                setEndPoints(
-                    new BrokerEndpointCollection(
-                        Arrays.asList(
-                            new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
-                            setPort(9093).setSecurityProtocol((short) 0)).iterator())).
+                setEndPoints(new BrokerEndpointCollection(Arrays.asList(
+                    new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
+                        setPort(9093).setSecurityProtocol((short) 0)).iterator())).
                 setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_3_IV3)).
                 setRack(null).
-                setFenced(false), (short) 1),
+                setFenced(true), (short) 1),
             new ApiMessageAndVersion(new RegisterBrokerRecord().
                 setBrokerId(2).setBrokerEpoch(brokerEpochs.get(2)).
                 setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB2")).
-                setEndPoints(
-                    new BrokerEndpointCollection(
-                        Arrays.asList(
-                            new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
-                            setPort(9094).setSecurityProtocol((short) 0)).iterator())).
+                setEndPoints(new BrokerEndpointCollection(
+                    Arrays.asList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
+                        setPort(9094).setSecurityProtocol((short) 0)).iterator())).
                 setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_3_IV3)).
                 setRack(null).
-                setFenced(false), (short) 1),
+                setFenced(true), (short) 1),
             new ApiMessageAndVersion(new RegisterBrokerRecord().
                 setBrokerId(3).setBrokerEpoch(brokerEpochs.get(3)).
                 setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB3")).
@@ -994,59 +704,36 @@ public class QuorumControllerTest {
                     new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost").
                         setPort(9095).setSecurityProtocol((short) 0)).iterator())).
                 setFeatures(registrationFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_3_IV3)).
-                setRack(null), (short) 1),
+                setRack(null).
+                setFenced(true), (short) 1),
+            new ApiMessageAndVersion(new BrokerRegistrationChangeRecord().
+                setBrokerId(0).
+                setBrokerEpoch(brokerEpochs.get(0)).
+                setFenced(BrokerRegistrationFencingChange.UNFENCE.value()), (short) 0),
+            new ApiMessageAndVersion(new BrokerRegistrationChangeRecord().
+                setBrokerId(1).
+                setBrokerEpoch(brokerEpochs.get(1)).
+                setFenced(BrokerRegistrationFencingChange.UNFENCE.value()), (short) 0),
+            new ApiMessageAndVersion(new BrokerRegistrationChangeRecord().
+                setBrokerId(2).
+                setBrokerEpoch(brokerEpochs.get(2)).
+                setFenced(BrokerRegistrationFencingChange.UNFENCE.value()), (short) 0),
+            new ApiMessageAndVersion(new TopicRecord().
+                setName("foo").setTopicId(fooId), (short) 0),
+            new ApiMessageAndVersion(new PartitionRecord().setPartitionId(0).
+                setTopicId(fooId).setReplicas(Arrays.asList(0, 1, 2)).
+                setIsr(Arrays.asList(0, 1, 2)).setRemovingReplicas(Collections.emptyList()).
+                setAddingReplicas(Collections.emptyList()).setLeader(0).setLeaderEpoch(0).
+                setPartitionEpoch(0), (short) 0),
+            new ApiMessageAndVersion(new PartitionRecord().setPartitionId(1).
+                setTopicId(fooId).setReplicas(Arrays.asList(1, 2, 0)).
+                setIsr(Arrays.asList(1, 2, 0)).setRemovingReplicas(Collections.emptyList()).
+                setAddingReplicas(Collections.emptyList()).setLeader(1).setLeaderEpoch(0).
+                setPartitionEpoch(0), (short) 0),
             new ApiMessageAndVersion(new ProducerIdsRecord().
                 setBrokerId(0).
                 setBrokerEpoch(brokerEpochs.get(0)).
-                setNextProducerId(1000), (short) 0)
-        );
-    }
-
-    private void checkSnapshotContent(
-        List<ApiMessageAndVersion> expected,
-        Iterator<Batch<ApiMessageAndVersion>> iterator
-    ) throws Exception {
-        RecordTestUtils.assertBatchIteratorContains(
-            Arrays.asList(expected),
-            Arrays.asList(
-                StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
-                             .flatMap(batch ->  batch.records().stream())
-                             .collect(Collectors.toList())
-            ).iterator()
-        );
-    }
-
-    /**
-     * This function checks that the iterator is a subset of the expected list.
-     *
-     * This is needed because when generating snapshots through configuration is difficult to control exactly when a
-     * snapshot will be generated and which committed offset will be included in the snapshot.
-     */
-    private void checkSnapshotSubcontent(
-        List<ApiMessageAndVersion> expected,
-        Iterator<Batch<ApiMessageAndVersion>> iterator
-    ) throws Exception {
-        RecordTestUtils.deepSortRecords(expected);
-
-        List<ApiMessageAndVersion> actual = StreamSupport
-            .stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
-            .flatMap(batch ->  batch.records().stream())
-            .collect(Collectors.toList());
-
-        RecordTestUtils.deepSortRecords(actual);
-
-        int expectedIndex = 0;
-        for (ApiMessageAndVersion current : actual) {
-            while (expectedIndex < expected.size() && !expected.get(expectedIndex).equals(current)) {
-                expectedIndex += 1;
-            }
-
-            if (expectedIndex >= expected.size()) {
-                fail("Failed to find record " + current + " in the expected record set: " + expected);
-            }
-
-            expectedIndex += 1;
-        }
+                setNextProducerId(1000), (short) 0));
     }
 
     /**
@@ -1380,7 +1067,7 @@ public class QuorumControllerTest {
                 }).
                 build()
         ) {
-            logEnv.appendInitialRecords(expectedSnapshotContent(FOO_ID, ALL_ZERO_BROKER_EPOCHS));
+            logEnv.appendInitialRecords(generateTestRecords(FOO_ID, ALL_ZERO_BROKER_EPOCHS));
             logEnv.logManagers().forEach(m -> m.setMaxReadOffset(2));
             try (
                 QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv).
diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
index c81f25e4c00..0997792cd90 100644
--- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
+++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java
@@ -58,7 +58,6 @@ import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.On
 import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
 import org.apache.kafka.common.metadata.ConfigRecord;
 import org.apache.kafka.common.metadata.PartitionChangeRecord;
-import org.apache.kafka.common.metadata.PartitionRecord;
 import org.apache.kafka.common.metadata.RegisterBrokerRecord;
 import org.apache.kafka.common.metadata.RemoveTopicRecord;
 import org.apache.kafka.common.metadata.TopicRecord;
@@ -519,16 +518,6 @@ public class ReplicationControlManagerTest {
                 setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code()).
                 setErrorMessage("Topic 'foo' already exists."));
         assertEquals(expectedResponse4, result4.response());
-        Uuid fooId = result3.response().topics().find("foo").topicId();
-        RecordTestUtils.assertBatchIteratorContains(asList(
-            asList(new ApiMessageAndVersion(new PartitionRecord().
-                    setPartitionId(0).setTopicId(fooId).
-                    setReplicas(asList(1, 2, 0)).setIsr(asList(1, 2, 0)).
-                    setRemovingReplicas(Collections.emptyList()).setAddingReplicas(Collections.emptyList()).setLeader(1).
-                    setLeaderEpoch(0).setPartitionEpoch(0), (short) 0),
-                new ApiMessageAndVersion(new TopicRecord().
-                    setTopicId(fooId).setName("foo"), (short) 0))),
-            ctx.replicationControl.iterator(Long.MAX_VALUE));
     }
 
     @Test
diff --git a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java
deleted file mode 100644
index f7fa18f20a4..00000000000
--- a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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.config.ConfigResource;
-import org.apache.kafka.common.memory.MemoryPool;
-import org.apache.kafka.common.metadata.ConfigRecord;
-import org.apache.kafka.common.metadata.TopicRecord;
-import org.apache.kafka.common.record.CompressionType;
-import org.apache.kafka.common.utils.LogContext;
-import org.apache.kafka.common.utils.MockTime;
-import org.apache.kafka.controller.SnapshotGenerator.Section;
-import org.apache.kafka.metadata.MetadataRecordSerde;
-import org.apache.kafka.raft.OffsetAndEpoch;
-import org.apache.kafka.server.common.ApiMessageAndVersion;
-import org.apache.kafka.snapshot.SnapshotWriter;
-import org.apache.kafka.snapshot.MockRawSnapshotWriter;
-import org.apache.kafka.snapshot.RawSnapshotWriter;
-import org.apache.kafka.snapshot.RecordsSnapshotWriter;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.OptionalLong;
-import java.util.Optional;
-
-import static org.apache.kafka.raft.KafkaRaftClient.MAX_BATCH_SIZE_BYTES;
-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 SnapshotGeneratorTest {
-    private static final List<List<ApiMessageAndVersion>> BATCHES;
-
-    static {
-        BATCHES = Arrays.asList(
-            Arrays.asList(new ApiMessageAndVersion(new TopicRecord().
-                setName("foo").setTopicId(Uuid.randomUuid()), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new TopicRecord().
-                setName("bar").setTopicId(Uuid.randomUuid()), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new TopicRecord().
-                setName("baz").setTopicId(Uuid.randomUuid()), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ConfigRecord().
-                    setResourceName("foo").setResourceType(ConfigResource.Type.TOPIC.id()).
-                    setName("retention.ms").setValue("10000000"), (short) 0),
-                new ApiMessageAndVersion(new ConfigRecord().
-                    setResourceName("foo").setResourceType(ConfigResource.Type.TOPIC.id()).
-                    setName("max.message.bytes").setValue("100000000"), (short) 0)),
-            Arrays.asList(new ApiMessageAndVersion(new ConfigRecord().
-                setResourceName("bar").setResourceType(ConfigResource.Type.TOPIC.id()).
-                setName("retention.ms").setValue("5000000"), (short) 0)));
-    }
-
-    @Test
-    public void testGenerateBatches() throws Exception {
-        SnapshotWriter<ApiMessageAndVersion> writer = createSnapshotWriter(123, 0);
-        List<Section> sections = Arrays.asList(new Section("replication",
-                Arrays.asList(BATCHES.get(0), BATCHES.get(1), BATCHES.get(2)).iterator()),
-            new Section("configuration",
-                Arrays.asList(BATCHES.get(3), BATCHES.get(4)).iterator()));
-        SnapshotGenerator generator = new SnapshotGenerator(new LogContext(),
-            writer, 2, sections);
-        assertFalse(writer.isFrozen());
-        assertEquals(123L, generator.lastContainedLogOffset());
-        assertEquals(writer, generator.writer());
-        assertEquals(OptionalLong.of(0L), generator.generateBatches());
-        assertEquals(OptionalLong.of(0L), generator.generateBatches());
-        assertFalse(writer.isFrozen());
-        assertEquals(OptionalLong.empty(), generator.generateBatches());
-        assertTrue(writer.isFrozen());
-    }
-
-    private SnapshotWriter<ApiMessageAndVersion> createSnapshotWriter(
-        long committedOffset,
-        long lastContainedLogTime
-    ) {
-        return RecordsSnapshotWriter.createWithHeader(
-            () -> createNewSnapshot(new OffsetAndEpoch(committedOffset + 1, 1)),
-            MAX_BATCH_SIZE_BYTES,
-            MemoryPool.NONE,
-            new MockTime(),
-            lastContainedLogTime,
-            CompressionType.NONE,
-            new MetadataRecordSerde()
-        ).get();
-    }
-
-    private Optional<RawSnapshotWriter> createNewSnapshot(
-            OffsetAndEpoch snapshotId
-    ) {
-        return Optional.of(new MockRawSnapshotWriter(snapshotId, buffer -> { }));
-    }
-}
diff --git a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java
new file mode 100644
index 00000000000..585f1dc40a7
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java
@@ -0,0 +1,517 @@
+/*
+ * 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.image.loader;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.metadata.FeatureLevelRecord;
+import org.apache.kafka.common.metadata.RemoveTopicRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.MetadataProvenance;
+import org.apache.kafka.image.publisher.MetadataPublisher;
+import org.apache.kafka.raft.Batch;
+import org.apache.kafka.raft.BatchReader;
+import org.apache.kafka.raft.OffsetAndEpoch;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.fault.MockFaultHandler;
+import org.apache.kafka.snapshot.SnapshotReader;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static java.util.Arrays.asList;
+import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV1;
+import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV2;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+@Timeout(value = 40)
+public class MetadataLoaderTest {
+    @Test
+    public void testCreateAndClose() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("testCreateAndClose");
+        try (MetadataLoader loader = new MetadataLoader.Builder().
+                setFaultHandler(faultHandler).
+                setHighWaterMarkAccessor(() -> OptionalLong.empty()).
+                build()) {
+            assertEquals(-1L, loader.lastAppliedOffset());
+        }
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    static class MockPublisher implements MetadataPublisher {
+        private final String name;
+        MetadataDelta latestDelta = null;
+        MetadataImage latestImage = null;
+        LogDeltaManifest latestLogDeltaManifest = null;
+        SnapshotManifest latestSnapshotManifest = null;
+        boolean closed = false;
+
+        MockPublisher() {
+            this("MockPublisher");
+        }
+
+        MockPublisher(String name) {
+            this.name = name;
+        }
+
+        @Override
+        public String name() {
+            return name;
+        }
+
+        @Override
+        public void publishSnapshot(
+            MetadataDelta delta,
+            MetadataImage newImage,
+            SnapshotManifest manifest
+        ) {
+            latestDelta = delta;
+            latestImage = newImage;
+            latestSnapshotManifest = manifest;
+        }
+
+        @Override
+        public void publishLogDelta(
+            MetadataDelta delta,
+            MetadataImage newImage,
+            LogDeltaManifest manifest
+        ) {
+            latestDelta = delta;
+            latestImage = newImage;
+            latestLogDeltaManifest = manifest;
+        }
+
+        @Override
+        public void close() throws Exception {
+            closed = true;
+        }
+    }
+
+    /**
+     * Install 2 publishers and check that the publishers that were installed are closed when the
+     * loader is closed.
+     */
+    @Test
+    public void testInstallPublishers() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("testInstallPublishers");
+        List<MockPublisher> publishers = asList(new MockPublisher("a"),
+                new MockPublisher("b"),
+                new MockPublisher("c"));
+        try (MetadataLoader loader = new MetadataLoader.Builder().
+                setFaultHandler(faultHandler).
+                setHighWaterMarkAccessor(() -> OptionalLong.empty()).
+                build()) {
+            loader.installPublishers(publishers.subList(0, 2)).get();
+        }
+        assertTrue(publishers.get(0).closed);
+        assertNull(publishers.get(0).latestImage);
+        assertTrue(publishers.get(1).closed);
+        assertNull(publishers.get(1).latestImage);
+        assertFalse(publishers.get(2).closed);
+        assertNull(publishers.get(2).latestImage);
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    static class MockSnapshotReader implements SnapshotReader<ApiMessageAndVersion> {
+        private final MetadataProvenance provenance;
+        private final Iterator<Batch<ApiMessageAndVersion>> iterator;
+        private MockTime time = null;
+        boolean closed = false;
+
+        static MockSnapshotReader fromRecordLists(
+            MetadataProvenance provenance,
+            List<List<ApiMessageAndVersion>> lists
+        ) {
+            List<Batch<ApiMessageAndVersion>> batches = new ArrayList<>();
+            lists.forEach(records -> batches.add(Batch.data(
+                provenance.offset(),
+                provenance.epoch(),
+                provenance.lastContainedLogTimeMs(),
+                0,
+                records)));
+            return new MockSnapshotReader(provenance, batches);
+        }
+
+        MockSnapshotReader(
+            MetadataProvenance provenance,
+            List<Batch<ApiMessageAndVersion>> batches
+        ) {
+            this.provenance = provenance;
+            this.iterator = batches.iterator();
+        }
+
+        MockSnapshotReader setTime(MockTime time) {
+            this.time = time;
+            return this;
+        }
+
+        @Override
+        public OffsetAndEpoch snapshotId() {
+            return provenance.offsetAndEpoch();
+        }
+
+        @Override
+        public long lastContainedLogOffset() {
+            return provenance.offset();
+        }
+
+        @Override
+        public int lastContainedLogEpoch() {
+            return provenance.epoch();
+        }
+
+        @Override
+        public long lastContainedLogTimestamp() {
+            return provenance.lastContainedLogTimeMs();
+        }
+
+        @Override
+        public void close() {
+            closed = true;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (time != null) time.sleep(1);
+            return iterator.hasNext();
+        }
+
+        @Override
+        public Batch<ApiMessageAndVersion> next() {
+            if (time != null) time.sleep(1);
+            return iterator.next();
+        }
+    }
+
+    /**
+     * Test that a publisher cannot be installed more than once.
+     */
+    @ParameterizedTest
+    @CsvSource(value = {"false,false", "false,true", "true,false", "true,true"})
+    public void testPublisherCannotBeInstalledMoreThanOnce(
+        boolean loadSnapshot,
+        boolean sameObject
+    ) throws Exception {
+        MockFaultHandler faultHandler =
+                new MockFaultHandler("testPublisherCannotBeInstalledMoreThanOnce");
+        MockPublisher publisher = new MockPublisher();
+        try (MetadataLoader loader = new MetadataLoader.Builder().
+                setFaultHandler(faultHandler).
+                setHighWaterMarkAccessor(() -> OptionalLong.of(0L)).
+                build()) {
+            loader.installPublishers(asList(publisher)).get();
+            if (loadSnapshot) {
+                MockSnapshotReader snapshotReader = new MockSnapshotReader(
+                        new MetadataProvenance(200, 100, 4000),
+                        asList(Batch.control(200, 100, 4000, 10, 200)));
+                loader.handleSnapshot(snapshotReader);
+            }
+            loader.waitForAllEventsToBeHandled();
+            if (sameObject) {
+                assertEquals("testPublisherCannotBeInstalledMoreThanOnce: Attempted to install " +
+                    "publisher MockPublisher, which is already installed.",
+                        assertThrows(ExecutionException.class,
+                                () -> loader.installPublishers(asList(publisher)).get()).
+                                getCause().getMessage());
+            } else {
+                assertEquals("testPublisherCannotBeInstalledMoreThanOnce: Attempted to install " +
+                    "a new publisher named MockPublisher, but there is already a publisher with that name.",
+                        assertThrows(ExecutionException.class,
+                                () -> loader.installPublishers(asList(new MockPublisher())).get()).
+                                getCause().getMessage());
+            }
+        }
+    }
+
+    /**
+     * Install 2 publishers and remove one.
+     */
+    @Test
+    public void testRemovePublisher() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("testRemovePublisher");
+        List<MockPublisher> publishers = asList(new MockPublisher("a"),
+                new MockPublisher("b"),
+                new MockPublisher("c"));
+        try (MetadataLoader loader = new MetadataLoader.Builder().
+                setFaultHandler(faultHandler).
+                setHighWaterMarkAccessor(() -> OptionalLong.of(0L)).
+                build()) {
+            loader.installPublishers(publishers.subList(0, 2)).get();
+            loader.removeAndClosePublisher(publishers.get(1)).get();
+            MockSnapshotReader snapshotReader = MockSnapshotReader.fromRecordLists(
+                new MetadataProvenance(100, 50, 2000),
+                asList(asList(new ApiMessageAndVersion(
+                    new FeatureLevelRecord().
+                        setName(MetadataVersion.FEATURE_NAME).
+                        setFeatureLevel(IBP_3_3_IV2.featureLevel()), (short) 0))));
+            assertFalse(snapshotReader.closed);
+            loader.handleSnapshot(snapshotReader);
+            loader.waitForAllEventsToBeHandled();
+            assertTrue(snapshotReader.closed);
+            loader.removeAndClosePublisher(publishers.get(0)).get();
+        }
+        assertTrue(publishers.get(0).closed);
+        assertEquals(IBP_3_3_IV2,
+                publishers.get(0).latestImage.features().metadataVersion());
+        assertTrue(publishers.get(1).closed);
+        assertNull(publishers.get(1).latestImage);
+        assertFalse(publishers.get(2).closed);
+        assertNull(publishers.get(2).latestImage);
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    /**
+     * Test loading a snapshot with 0 records.
+     */
+    @Test
+    public void testLoadEmptySnapshot() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("testLoadEmptySnapshot");
+        MockTime time = new MockTime();
+        List<MockPublisher> publishers = asList(new MockPublisher());
+        try (MetadataLoader loader = new MetadataLoader.Builder().
+                setFaultHandler(faultHandler).
+                setTime(time).
+                setHighWaterMarkAccessor(() -> OptionalLong.of(0L)).
+                build()) {
+            loader.installPublishers(publishers).get();
+            loadEmptySnapshot(loader, 200);
+            assertEquals(200L, loader.lastAppliedOffset());
+            loadEmptySnapshot(loader, 300);
+            assertEquals(300L, loader.lastAppliedOffset());
+            assertEquals(new SnapshotManifest(new MetadataProvenance(300, 100, 4000), 3000000L),
+                publishers.get(0).latestSnapshotManifest);
+        }
+        assertTrue(publishers.get(0).closed);
+        assertEquals(MetadataVersion.IBP_3_0_IV1,
+                publishers.get(0).latestImage.features().metadataVersion());
+        assertTrue(publishers.get(0).latestImage.isEmpty());
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    private void loadEmptySnapshot(
+        MetadataLoader loader,
+        long offset
+    ) throws Exception {
+        MockSnapshotReader snapshotReader = new MockSnapshotReader(
+                new MetadataProvenance(offset, 100, 4000),
+                asList(Batch.control(200, 100, 4000, 10, 200)));
+        if (loader.time() instanceof MockTime) {
+            snapshotReader.setTime((MockTime) loader.time());
+        }
+        loader.handleSnapshot(snapshotReader);
+        loader.waitForAllEventsToBeHandled();
+    }
+
+    static class MockBatchReader implements BatchReader<ApiMessageAndVersion> {
+        private final long baseOffset;
+        private final Iterator<Batch<ApiMessageAndVersion>> iterator;
+        private boolean closed = false;
+        private MockTime time = null;
+
+        static Batch<ApiMessageAndVersion> newBatch(
+            long batchBaseOffset,
+            int epoch,
+            List<ApiMessageAndVersion> records
+        ) {
+            return Batch.data(batchBaseOffset, epoch, 0, 0, records);
+        }
+
+        MockBatchReader(
+            long baseOffset,
+            List<Batch<ApiMessageAndVersion>> batches
+        ) {
+            this.baseOffset = baseOffset;
+            this.iterator = batches.iterator();
+        }
+
+        private MockBatchReader setTime(MockTime time) {
+            this.time = time;
+            return this;
+        }
+
+        @Override
+        public long baseOffset() {
+            return baseOffset;
+        }
+
+        @Override
+        public OptionalLong lastOffset() {
+            return OptionalLong.empty();
+        }
+
+        @Override
+        public void close() {
+            this.closed = true;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (time != null) time.sleep(1);
+            return iterator.hasNext();
+        }
+
+        @Override
+        public Batch<ApiMessageAndVersion> next() {
+            if (time != null) time.sleep(1);
+            return iterator.next();
+        }
+    }
+
+    /**
+     * Test loading a batch with 0 records.
+     */
+    @Test
+    public void testLoadEmptyBatch() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("testLoadEmptyBatch");
+        MockTime time = new MockTime();
+        List<MockPublisher> publishers = asList(new MockPublisher());
+        try (MetadataLoader loader = new MetadataLoader.Builder().
+                setFaultHandler(faultHandler).
+                setTime(time).
+                setHighWaterMarkAccessor(() -> OptionalLong.of(0L)).
+                build()) {
+            loader.installPublishers(publishers).get();
+            loadTestSnapshot(loader, 200);
+            MockBatchReader batchReader = new MockBatchReader(300, asList(
+                Batch.control(300, 100, 4000, 10, 400))).
+                    setTime(time);
+            loader.handleCommit(batchReader);
+            loader.waitForAllEventsToBeHandled();
+            assertTrue(batchReader.closed);
+            assertEquals(400L, loader.lastAppliedOffset());
+        }
+        assertTrue(publishers.get(0).closed);
+        assertEquals(new LogDeltaManifest(new MetadataProvenance(400, 100, 4000), 1,
+                        3000000L, 10),
+            publishers.get(0).latestLogDeltaManifest);
+        assertEquals(MetadataVersion.IBP_3_3_IV1,
+            publishers.get(0).latestImage.features().metadataVersion());
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    /**
+     * Test that the lastAppliedOffset moves forward as expected.
+     */
+    @Test
+    public void testLastAppliedOffset() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("testLastAppliedOffset");
+        List<MockPublisher> publishers = asList(new MockPublisher("a"),
+                new MockPublisher("b"));
+        try (MetadataLoader loader = new MetadataLoader.Builder().
+                setFaultHandler(faultHandler).
+                setHighWaterMarkAccessor(() -> OptionalLong.of(0L)).
+                build()) {
+            loader.installPublishers(publishers).get();
+            loader.handleSnapshot(MockSnapshotReader.fromRecordLists(
+                new MetadataProvenance(200, 100, 4000), asList(
+                    asList(new ApiMessageAndVersion(new FeatureLevelRecord().
+                        setName(MetadataVersion.FEATURE_NAME).
+                        setFeatureLevel(IBP_3_3_IV1.featureLevel()), (short) 0)),
+                    asList(new ApiMessageAndVersion(new TopicRecord().
+                        setName("foo").
+                        setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0))
+                )));
+            loader.waitForAllEventsToBeHandled();
+            assertEquals(200L, loader.lastAppliedOffset());
+            loader.handleCommit(new MockBatchReader(201, asList(
+                MockBatchReader.newBatch(201, 100, asList(
+                    new ApiMessageAndVersion(new RemoveTopicRecord().
+                        setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0))))));
+            loader.waitForAllEventsToBeHandled();
+            assertEquals(201L, loader.lastAppliedOffset());
+        }
+        for (int i = 0; i < 2; i++) {
+            assertTrue(publishers.get(i).closed);
+            assertTrue(publishers.get(i).closed);
+            assertEquals(IBP_3_3_IV1,
+                    publishers.get(i).latestImage.features().metadataVersion());
+        }
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    /**
+     * Test that we do not leave the catchingUp state state until we have loaded up to the high
+     * water mark.
+     */
+    public void testCatchingUpState() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("testLastAppliedOffset");
+        List<MockPublisher> publishers = asList(new MockPublisher("a"),
+                new MockPublisher("b"));
+        AtomicReference<OptionalLong> highWaterMark = new AtomicReference<>(OptionalLong.empty());
+        try (MetadataLoader loader = new MetadataLoader.Builder().
+                setFaultHandler(faultHandler).
+                setHighWaterMarkAccessor(() -> highWaterMark.get()).
+                build()) {
+            loader.installPublishers(publishers).get();
+            loadTestSnapshot(loader, 200);
+
+            // We don't update lastAppliedOffset because we're still in catchingUp state due to
+            // highWaterMark being OptionalLong.empty (aka unknown).
+            assertEquals(-1L, loader.lastAppliedOffset());
+
+            // Setting the high water mark here doesn't do anything because we only check it when
+            // we're publishing an update. This is OK because we know that we'll get updates
+            // frequently. If there is no other activity, there will at least be NoOpRecords.
+            highWaterMark.set(OptionalLong.of(0));
+            assertEquals(-1L, loader.lastAppliedOffset());
+
+            // This still doesn't advance lastAppliedOffset since the high water mark at 220
+            // is greater than our snapshot at 210.
+            highWaterMark.set(OptionalLong.of(220));
+            loadTestSnapshot(loader, 210);
+            assertEquals(-1L, loader.lastAppliedOffset());
+
+            // Loading a test snapshot at 220 allows us to leave catchUp state.
+            loadTestSnapshot(loader, 220);
+            assertEquals(220L, loader.lastAppliedOffset());
+        }
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    private void loadTestSnapshot(
+        MetadataLoader loader,
+        long offset
+    ) throws Exception {
+        loader.handleSnapshot(MockSnapshotReader.fromRecordLists(
+                new MetadataProvenance(offset, 100, 4000), asList(
+                        asList(new ApiMessageAndVersion(new FeatureLevelRecord().
+                                setName(MetadataVersion.FEATURE_NAME).
+                                setFeatureLevel(IBP_3_3_IV1.featureLevel()), (short) 0)),
+                        asList(new ApiMessageAndVersion(new TopicRecord().
+                                setName("foo").
+                                setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0))
+                )));
+        loader.waitForAllEventsToBeHandled();
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java
new file mode 100644
index 00000000000..ef40d714604
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.image.publisher;
+
+import org.apache.kafka.image.MetadataImageTest;
+import org.apache.kafka.raft.LeaderAndEpoch;
+import org.apache.kafka.raft.OffsetAndEpoch;
+import org.apache.kafka.raft.RaftClient;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.snapshot.SnapshotWriter;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+@Timeout(value = 40)
+public class SnapshotEmitterTest {
+    static class MockRaftClient implements RaftClient<ApiMessageAndVersion> {
+        TreeMap<OffsetAndEpoch, MockSnapshotWriter> writers = new TreeMap<>();
+
+        @Override
+        public void initialize() {
+            // nothing to do
+        }
+
+        @Override
+        public void register(Listener<ApiMessageAndVersion> listener) {
+            // nothing to do
+        }
+
+        @Override
+        public void unregister(Listener<ApiMessageAndVersion> listener) {
+            // nothing to do
+        }
+
+        @Override
+        public OptionalLong highWatermark() {
+            return OptionalLong.empty();
+        }
+
+        @Override
+        public LeaderAndEpoch leaderAndEpoch() {
+            return LeaderAndEpoch.UNKNOWN;
+        }
+
+        @Override
+        public OptionalInt nodeId() {
+            return OptionalInt.empty();
+        }
+
+        @Override
+        public long scheduleAppend(int epoch, List<ApiMessageAndVersion> records) {
+            return 0;
+        }
+
+        @Override
+        public long scheduleAtomicAppend(int epoch, List<ApiMessageAndVersion> records) {
+            return 0;
+        }
+
+        @Override
+        public CompletableFuture<Void> shutdown(int timeoutMs) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+        @Override
+        public void resign(int epoch) {
+            // nothing to do
+        }
+
+        @Override
+        public Optional<SnapshotWriter<ApiMessageAndVersion>> createSnapshot(
+            OffsetAndEpoch snapshotId,
+            long lastContainedLogTime
+        ) {
+            if (writers.containsKey(snapshotId)) {
+                return Optional.empty();
+            }
+            MockSnapshotWriter writer = new MockSnapshotWriter(snapshotId);
+            writers.put(snapshotId, writer);
+            return Optional.of(writer);
+        }
+
+        @Override
+        public Optional<OffsetAndEpoch> latestSnapshotId() {
+            NavigableSet<OffsetAndEpoch> descendingSet = writers.descendingKeySet();
+            if (descendingSet.isEmpty()) {
+                return Optional.empty();
+            } else {
+                return Optional.of(descendingSet.first());
+            }
+        }
+
+        @Override
+        public void close() throws Exception {
+            // nothing to do
+        }
+    }
+
+    static class MockSnapshotWriter implements SnapshotWriter<ApiMessageAndVersion> {
+        private final OffsetAndEpoch snapshotId;
+        private boolean frozen = false;
+        private boolean closed = false;
+        private final List<List<ApiMessageAndVersion>> batches;
+
+        MockSnapshotWriter(OffsetAndEpoch snapshotId) {
+            this.snapshotId = snapshotId;
+            this.batches = new ArrayList<>();
+        }
+
+        @Override
+        public OffsetAndEpoch snapshotId() {
+            return snapshotId;
+        }
+
+        @Override
+        public long lastContainedLogOffset() {
+            return snapshotId.offset();
+        }
+
+        @Override
+        public int lastContainedLogEpoch() {
+            return snapshotId.epoch();
+        }
+
+        @Override
+        public boolean isFrozen() {
+            return frozen;
+        }
+
+        @Override
+        public void append(List<ApiMessageAndVersion> records) {
+            batches.add(records);
+        }
+
+        List<List<ApiMessageAndVersion>> batches() {
+            List<List<ApiMessageAndVersion>> results = new ArrayList<>();
+            batches.forEach(batch -> results.add(new ArrayList<>(batch)));
+            return results;
+        }
+
+        @Override
+        public void freeze() {
+            frozen = true;
+        }
+
+        @Override
+        public void close() {
+            closed = true;
+        }
+
+        boolean isClosed() {
+            return closed;
+        }
+    }
+
+    @Test
+    public void testEmit() throws Exception {
+        MockRaftClient mockRaftClient = new MockRaftClient();
+        SnapshotEmitter emitter = new SnapshotEmitter.Builder().
+            setBatchSize(2).
+            setRaftClient(mockRaftClient).
+            build();
+        emitter.maybeEmit(MetadataImageTest.IMAGE1);
+        MockSnapshotWriter writer = mockRaftClient.writers.get(
+                MetadataImageTest.IMAGE1.highestOffsetAndEpoch());
+        assertNotNull(writer);
+        assertEquals(MetadataImageTest.IMAGE1.highestOffsetAndEpoch().offset(),
+                writer.lastContainedLogOffset());
+        assertEquals(MetadataImageTest.IMAGE1.highestOffsetAndEpoch().epoch(),
+                writer.lastContainedLogEpoch());
+        assertTrue(writer.isFrozen());
+        assertTrue(writer.isClosed());
+
+        // Second call to emit does nothing because we already have a snapshot at that offset and epoch.
+        emitter.maybeEmit(MetadataImageTest.IMAGE1);
+        assertEquals(1, mockRaftClient.writers.size());
+    }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java
new file mode 100644
index 00000000000..42fb8d92f6e
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java
@@ -0,0 +1,185 @@
+/*
+ * 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.image.publisher;
+
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.MetadataProvenance;
+import org.apache.kafka.image.loader.LogDeltaManifest;
+import org.apache.kafka.metadata.RecordTestUtils;
+import org.apache.kafka.server.fault.FaultHandlerException;
+import org.apache.kafka.server.fault.MockFaultHandler;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+
+@Timeout(value = 40)
+public class SnapshotGeneratorTest {
+    static class MockEmitter implements SnapshotGenerator.Emitter {
+        private final CountDownLatch latch = new CountDownLatch(1);
+        private final List<MetadataImage> images = new ArrayList<>();
+        private RuntimeException problem = null;
+
+        MockEmitter setReady() {
+            latch.countDown();
+            return this;
+        }
+
+        synchronized MockEmitter setProblem(RuntimeException problem) {
+            this.problem = problem;
+            return this;
+        }
+
+        @Override
+        public synchronized void maybeEmit(MetadataImage image) {
+            RuntimeException currentProblem = problem;
+            if (currentProblem != null) {
+                throw currentProblem;
+            }
+            try {
+                latch.await();
+            } catch (Throwable e) {
+                throw new RuntimeException(e);
+            }
+            images.add(image);
+        }
+
+        synchronized List<MetadataImage> images() {
+            return new ArrayList<>(images);
+        }
+    }
+
+    private final static MetadataDelta TEST_DELTA;
+
+    static {
+        TEST_DELTA = new MetadataDelta.Builder().
+                setImage(MetadataImage.EMPTY).
+                build();
+        TEST_DELTA.replay(RecordTestUtils.testRecord(0).message());
+    }
+
+    private final static MetadataImage TEST_IMAGE = TEST_DELTA.apply(MetadataProvenance.EMPTY);
+
+    @Test
+    public void testCreateSnapshot() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("SnapshotGenerator");
+        MockEmitter emitter = new MockEmitter();
+        try (SnapshotGenerator generator = new SnapshotGenerator.Builder(emitter).
+                setFaultHandler(faultHandler).
+                setMaxBytesSinceLastSnapshot(200).
+                setMaxTimeSinceLastSnapshotNs(TimeUnit.DAYS.toNanos(10)).
+                build()) {
+            // Publish a log delta batch. This one will not trigger a snapshot yet.
+            generator.publishLogDelta(TEST_DELTA, TEST_IMAGE,
+                    new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 100));
+            // Publish a log delta batch. This will trigger a snapshot.
+            generator.publishLogDelta(TEST_DELTA, TEST_IMAGE,
+                    new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 100));
+            // Publish a log delta batch. This one will be ignored because there are other images
+            // queued for writing.
+            generator.publishLogDelta(TEST_DELTA, TEST_IMAGE,
+                    new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 2000));
+            assertEquals(Collections.emptyList(), emitter.images());
+            emitter.setReady();
+        }
+        assertEquals(Arrays.asList(TEST_IMAGE), emitter.images());
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    @Test
+    public void testSnapshotsDisabled() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("SnapshotGenerator");
+        MockEmitter emitter = new MockEmitter().setReady();
+        AtomicReference<String> disabledReason = new AtomicReference<>();
+        try (SnapshotGenerator generator = new SnapshotGenerator.Builder(emitter).
+                setFaultHandler(faultHandler).
+                setMaxBytesSinceLastSnapshot(1).
+                setMaxTimeSinceLastSnapshotNs(0).
+                setDisabledReason(disabledReason).
+                build()) {
+            disabledReason.compareAndSet(null, "we are testing disable()");
+            // No snapshots are generated because snapshots are disabled.
+            generator.publishLogDelta(TEST_DELTA, TEST_IMAGE,
+                    new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 100));
+        }
+        assertEquals(Collections.emptyList(), emitter.images());
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    @Test
+    public void testTimeBasedSnapshots() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("SnapshotGenerator");
+        MockEmitter emitter = new MockEmitter().setReady();
+        MockTime mockTime = new MockTime();
+        try (SnapshotGenerator generator = new SnapshotGenerator.Builder(emitter).
+                setTime(mockTime).
+                setFaultHandler(faultHandler).
+                setMaxBytesSinceLastSnapshot(200).
+                setMaxTimeSinceLastSnapshotNs(TimeUnit.MINUTES.toNanos(30)).
+                build()) {
+            // This image isn't published yet.
+            generator.publishLogDelta(TEST_DELTA, TEST_IMAGE,
+                    new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 50));
+            assertEquals(Collections.emptyList(), emitter.images());
+            mockTime.sleep(TimeUnit.MINUTES.toNanos(40));
+            // Next image is published because of the time delay.
+            generator.publishLogDelta(TEST_DELTA, TEST_IMAGE,
+                    new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 50));
+            TestUtils.waitForCondition(() -> emitter.images().size() == 1, "images.size == 1");
+            // bytesSinceLastSnapshot was reset to 0 by the previous snapshot,
+            // so this does not trigger a new snapshot.
+            generator.publishLogDelta(TEST_DELTA, TEST_IMAGE,
+                    new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 100, 150));
+        }
+        assertEquals(Arrays.asList(TEST_IMAGE), emitter.images());
+        faultHandler.maybeRethrowFirstException();
+    }
+
+    @Test
+    public void testEmitterProblem() throws Exception {
+        MockFaultHandler faultHandler = new MockFaultHandler("SnapshotGenerator");
+        MockEmitter emitter = new MockEmitter().setProblem(new RuntimeException("oops"));
+        try (SnapshotGenerator generator = new SnapshotGenerator.Builder(emitter).
+                setFaultHandler(faultHandler).
+                setMaxBytesSinceLastSnapshot(200).
+                build()) {
+            for (int i = 0; i < 2; i++) {
+                generator.publishLogDelta(TEST_DELTA, TEST_IMAGE,
+                        new LogDeltaManifest(MetadataProvenance.EMPTY, 1, 10000, 50000));
+            }
+        }
+        assertEquals(Collections.emptyList(), emitter.images());
+        assertNotNull(faultHandler.firstException());
+        assertEquals(FaultHandlerException.class, faultHandler.firstException().getClass());
+        assertEquals("SnapshotGenerator: KRaft snapshot file generation error: oops",
+                faultHandler.firstException().getMessage());
+    }
+}
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 3bc07c06af0..a90e2687200 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
@@ -46,6 +46,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.IdentityHashMap;
@@ -318,9 +319,9 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
         }
 
         /**
-         * Returns the snapshot whos last offset is the committed offset.
+         * Returns the snapshot whose last offset is the committed offset.
          *
-         * If such snapshot doesn't exists, it waits until it does.
+         * If such snapshot doesn't exist, it waits until it does.
          */
         synchronized RawSnapshotReader waitForSnapshot(long committedOffset) throws InterruptedException {
             while (true) {
@@ -380,6 +381,20 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
         public long initialMaxReadOffset() {
             return initialMaxReadOffset;
         }
+
+        /**
+         * Return all records in the log as a list.
+         */
+        public synchronized List<ApiMessageAndVersion> allRecords() {
+            List<ApiMessageAndVersion> allRecords = new ArrayList<>();
+            for (LocalBatch batch : batches.values()) {
+                if (batch instanceof LocalRecordBatch) {
+                    LocalRecordBatch recordBatch = (LocalRecordBatch) batch;
+                    allRecords.addAll(recordBatch.records);
+                }
+            }
+            return allRecords;
+        }
     }
 
     private static class MetaLogListenerData {
diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java
index 1693b62be1a..d72b7557b48 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java
@@ -140,6 +140,13 @@ public class LocalLogManagerTestEnv implements AutoCloseable {
         this.logManagers = newLogManagers;
     }
 
+    /**
+     * Return all records in the log as a list.
+     */
+    public List<ApiMessageAndVersion> allRecords() {
+        return shared.allRecords();
+    }
+
     /**
      * Append some records to the log. This method is meant to be called before the
      * controllers are started, to simulate a pre-existing metadata log.
diff --git a/tests/kafkatest/tests/core/snapshot_test.py b/tests/kafkatest/tests/core/snapshot_test.py
index 1218cff319c..cb62cb93f55 100644
--- a/tests/kafkatest/tests/core/snapshot_test.py
+++ b/tests/kafkatest/tests/core/snapshot_test.py
@@ -118,7 +118,7 @@ class TestSnapshots(ProduceConsumeValidateTest):
         cmd = "ls %s" % file_path
         files = node.account.ssh_output(cmd, allow_fail=True, combine_stderr=False)
 
-        if len(files) is 0:
+        if len(files) == 0:
             self.logger.debug("File %s does not exist" % file_path)
             return False
         else:


[kafka] 10/11: KAFKA-14446: API forwarding support from zkBrokers to the Controller (#12961)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit aa532b44199f9610f1b6e587acecc50a9e1a9b9c
Author: Akhilesh C <ak...@users.noreply.github.com>
AuthorDate: Thu Dec 15 14:16:41 2022 -0800

    KAFKA-14446: API forwarding support from zkBrokers to the Controller (#12961)
    
    This PR enables brokers which are upgrading from ZK mode to KRaft mode to forward certain metadata
    change requests to the controller instead of applying them directly through ZK. To faciliate this,
    we now support EnvelopeRequest on zkBrokers (instead of only on KRaft nodes.)
    
    In BrokerToControllerChannelManager, we can now reinitialize our NetworkClient. This is needed to
    handle the case when we transition from forwarding requests to a ZK-based broker over the
    inter-broker listener, to forwarding requests to a quorum node over the controller listener.
    
    In MetadataCache.scala, distinguish between KRaft and ZK controller nodes with a new type,
    CachedControllerId.
    
    In LeaderAndIsrRequest, StopReplicaRequest, and UpdateMetadataRequest, switch from sending both a
    zk and a KRaft controller ID to sending a single controller ID plus a boolean to express whether it
    is KRaft. The previous scheme was ambiguous as to whether the system was in KRaft or ZK mode when
    both IDs were -1 (although this case is unlikely to come up in practice). The new scheme avoids
    this ambiguity and is simpler to understand.
    
    Reviewers: dengziming <de...@gmail.com>, David Arthur <mu...@gmail.com>, Colin P. McCabe <cm...@apache.org>
---
 .../common/requests/AbstractControlRequest.java    |  10 +-
 .../kafka/common/requests/LeaderAndIsrRequest.java |  18 ++-
 .../kafka/common/requests/StopReplicaRequest.java  |  18 ++-
 .../common/requests/UpdateMetadataRequest.java     |  26 ++++-
 .../resources/common/message/EnvelopeRequest.json  |   2 +-
 .../common/message/LeaderAndIsrRequest.json        |   4 +-
 .../common/message/StopReplicaRequest.json         |   4 +-
 .../common/message/UpdateMetadataRequest.json      |   4 +-
 .../common/requests/ApiVersionsResponseTest.java   |   1 -
 core/src/main/scala/kafka/Kafka.scala              |   8 +-
 .../scala/kafka/common/InterBrokerSendThread.scala |   2 +-
 .../server/BrokerToControllerChannelManager.scala  | 121 ++++++++++++++-------
 core/src/main/scala/kafka/server/KafkaApis.scala   |  18 ++-
 core/src/main/scala/kafka/server/KafkaServer.scala |  22 +++-
 .../main/scala/kafka/server/MetadataCache.scala    |  18 ++-
 .../kafka/server/metadata/KRaftMetadataCache.scala |  11 +-
 .../kafka/server/metadata/ZkMetadataCache.scala    |  58 ++++++++--
 .../kafka/api/IntegrationTestHarness.scala         |   4 +
 .../kafka/api/PlaintextAdminIntegrationTest.scala  |   2 +-
 .../kafka/server/QuorumTestHarness.scala           |   4 +
 .../BrokerToControllerRequestThreadTest.scala      |  52 ++++++---
 .../src/test/scala/kafka/utils/TestInfoUtils.scala |   8 ++
 .../kafka/integration/KafkaServerTestHarness.scala |   3 +-
 .../unit/kafka/server/ApiVersionManagerTest.scala  |   6 +-
 .../kafka/server/BrokerLifecycleManagerTest.scala  |   9 +-
 .../server/BrokerRegistrationRequestTest.scala     |  15 ++-
 .../kafka/server/CreateTopicsRequestTest.scala     |  10 +-
 .../kafka/server/DeleteTopicsRequestTest.scala     |   5 +-
 .../unit/kafka/server/ForwardingManagerTest.scala  |  24 ++--
 .../scala/unit/kafka/server/KafkaApisTest.scala    |   2 +-
 .../MockBrokerToControllerChannelManager.scala     |   2 +-
 .../scala/unit/kafka/server/RequestQuotaTest.scala |   8 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala    |   7 +-
 .../jmh/fetcher/ReplicaFetcherThreadBenchmark.java |   3 +-
 .../jmh/metadata/MetadataRequestBenchmark.java     |   3 +-
 .../apache/kafka/jmh/server/CheckpointBench.java   |   4 +-
 .../kafka/jmh/server/PartitionCreationBench.java   |   6 +-
 .../kafka/server/common/MetadataVersion.java       |   4 +
 38 files changed, 383 insertions(+), 143 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractControlRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractControlRequest.java
index dc4a1e21e8d..789a4abeaba 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractControlRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractControlRequest.java
@@ -27,14 +27,20 @@ public abstract class AbstractControlRequest extends AbstractRequest {
         protected final int controllerId;
         protected final int controllerEpoch;
         protected final long brokerEpoch;
+        protected final boolean kraftController;
 
         protected Builder(ApiKeys api, short version, int controllerId, int controllerEpoch, long brokerEpoch) {
+            this(api, version, controllerId, controllerEpoch, brokerEpoch, false);
+        }
+
+        protected Builder(ApiKeys api, short version, int controllerId, int controllerEpoch,
+                          long brokerEpoch, boolean kraftController) {
             super(api, version);
             this.controllerId = controllerId;
             this.controllerEpoch = controllerEpoch;
             this.brokerEpoch = brokerEpoch;
+            this.kraftController = kraftController;
         }
-
     }
 
     protected AbstractControlRequest(ApiKeys api, short version) {
@@ -43,6 +49,8 @@ public abstract class AbstractControlRequest extends AbstractRequest {
 
     public abstract int controllerId();
 
+    public abstract boolean isKRaftController();
+
     public abstract int controllerEpoch();
 
     public abstract long brokerEpoch();
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
index d7382863176..257d8e78bfc 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
@@ -51,7 +51,14 @@ public class LeaderAndIsrRequest extends AbstractControlRequest {
         public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
                        List<LeaderAndIsrPartitionState> partitionStates, Map<String, Uuid> topicIds,
                        Collection<Node> liveLeaders) {
-            super(ApiKeys.LEADER_AND_ISR, version, controllerId, controllerEpoch, brokerEpoch);
+            this(version, controllerId, controllerEpoch, brokerEpoch, partitionStates, topicIds,
+                liveLeaders, false);
+        }
+
+        public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
+                       List<LeaderAndIsrPartitionState> partitionStates, Map<String, Uuid> topicIds,
+                       Collection<Node> liveLeaders, boolean kraftController) {
+            super(ApiKeys.LEADER_AND_ISR, version, controllerId, controllerEpoch, brokerEpoch, kraftController);
             this.partitionStates = partitionStates;
             this.topicIds = topicIds;
             this.liveLeaders = liveLeaders;
@@ -71,6 +78,10 @@ public class LeaderAndIsrRequest extends AbstractControlRequest {
                 .setBrokerEpoch(brokerEpoch)
                 .setLiveLeaders(leaders);
 
+            if (version >= 7) {
+                data.setIsKRaftController(kraftController);
+            }
+
             if (version >= 2) {
                 Map<String, LeaderAndIsrTopicState> topicStatesMap = groupByTopic(partitionStates, topicIds);
                 data.setTopicStates(new ArrayList<>(topicStatesMap.values()));
@@ -168,6 +179,11 @@ public class LeaderAndIsrRequest extends AbstractControlRequest {
         return data.controllerId();
     }
 
+    @Override
+    public boolean isKRaftController() {
+        return data.isKRaftController();
+    }
+
     @Override
     public int controllerEpoch() {
         return data.controllerEpoch();
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
index 4326aaffd87..df746b56c84 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java
@@ -45,7 +45,14 @@ public class StopReplicaRequest extends AbstractControlRequest {
 
         public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
                        boolean deletePartitions, List<StopReplicaTopicState> topicStates) {
-            super(ApiKeys.STOP_REPLICA, version, controllerId, controllerEpoch, brokerEpoch);
+            this(version, controllerId, controllerEpoch, brokerEpoch, deletePartitions,
+                topicStates, false);
+        }
+
+        public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
+                       boolean deletePartitions, List<StopReplicaTopicState> topicStates,
+                       boolean kraftController) {
+            super(ApiKeys.STOP_REPLICA, version, controllerId, controllerEpoch, brokerEpoch, kraftController);
             this.deletePartitions = deletePartitions;
             this.topicStates = topicStates;
         }
@@ -56,6 +63,10 @@ public class StopReplicaRequest extends AbstractControlRequest {
                 .setControllerEpoch(controllerEpoch)
                 .setBrokerEpoch(brokerEpoch);
 
+            if (version >= 4) {
+                data.setIsKRaftController(kraftController);
+            }
+
             if (version >= 3) {
                 data.setTopicStates(topicStates);
             } else if (version >= 1) {
@@ -196,6 +207,11 @@ public class StopReplicaRequest extends AbstractControlRequest {
         return data.controllerId();
     }
 
+    @Override
+    public boolean isKRaftController() {
+        return data.isKRaftController();
+    }
+
     @Override
     public int controllerEpoch() {
         return data.controllerEpoch();
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
index 845bdd92111..c0fd3000cc5 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
@@ -51,7 +51,14 @@ public class UpdateMetadataRequest extends AbstractControlRequest {
         public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
                        List<UpdateMetadataPartitionState> partitionStates, List<UpdateMetadataBroker> liveBrokers,
                        Map<String, Uuid> topicIds) {
-            super(ApiKeys.UPDATE_METADATA, version, controllerId, controllerEpoch, brokerEpoch);
+            this(version, controllerId, controllerEpoch, brokerEpoch, partitionStates,
+                liveBrokers, topicIds, false);
+        }
+
+        public Builder(short version, int controllerId, int controllerEpoch, long brokerEpoch,
+                       List<UpdateMetadataPartitionState> partitionStates, List<UpdateMetadataBroker> liveBrokers,
+                       Map<String, Uuid> topicIds, boolean kraftController) {
+            super(ApiKeys.UPDATE_METADATA, version, controllerId, controllerEpoch, brokerEpoch, kraftController);
             this.partitionStates = partitionStates;
             this.liveBrokers = liveBrokers;
             this.topicIds = topicIds;
@@ -81,10 +88,14 @@ public class UpdateMetadataRequest extends AbstractControlRequest {
             }
 
             UpdateMetadataRequestData data = new UpdateMetadataRequestData()
-                    .setControllerId(controllerId)
-                    .setControllerEpoch(controllerEpoch)
-                    .setBrokerEpoch(brokerEpoch)
-                    .setLiveBrokers(liveBrokers);
+                .setControllerId(controllerId)
+                .setControllerEpoch(controllerEpoch)
+                .setBrokerEpoch(brokerEpoch)
+                .setLiveBrokers(liveBrokers);
+
+            if (version >= 8) {
+                data.setIsKRaftController(kraftController);
+            }
 
             if (version >= 5) {
                 Map<String, UpdateMetadataTopicState> topicStatesMap = groupByTopic(topicIds, partitionStates);
@@ -180,6 +191,11 @@ public class UpdateMetadataRequest extends AbstractControlRequest {
         return data.controllerId();
     }
 
+    @Override
+    public boolean isKRaftController() {
+        return data.isKRaftController();
+    }
+
     @Override
     public int controllerEpoch() {
         return data.controllerEpoch();
diff --git a/clients/src/main/resources/common/message/EnvelopeRequest.json b/clients/src/main/resources/common/message/EnvelopeRequest.json
index 1f6ff62de8d..a30a50ba684 100644
--- a/clients/src/main/resources/common/message/EnvelopeRequest.json
+++ b/clients/src/main/resources/common/message/EnvelopeRequest.json
@@ -16,7 +16,7 @@
 {
   "apiKey": 58,
   "type": "request",
-  "listeners": ["controller"],
+  "listeners": ["controller", "zkBroker"],
   "name": "EnvelopeRequest",
   // Request struct for forwarding.
   "validVersions": "0",
diff --git a/clients/src/main/resources/common/message/LeaderAndIsrRequest.json b/clients/src/main/resources/common/message/LeaderAndIsrRequest.json
index 7042ec84d5b..e049d088c53 100644
--- a/clients/src/main/resources/common/message/LeaderAndIsrRequest.json
+++ b/clients/src/main/resources/common/message/LeaderAndIsrRequest.json
@@ -36,8 +36,8 @@
   "fields": [
     { "name": "ControllerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The current controller ID." },
-    { "name": "KRaftControllerId", "type": "int32", "versions": "7+", "entityType": "brokerId", "default": "-1",
-      "about": "The KRaft controller id, used during migration. See KIP-866" },
+    { "name": "isKRaftController", "type": "bool", "versions": "7+", "default": "false",
+      "about": "If KRaft controller id is used during migration. See KIP-866" },
     { "name": "ControllerEpoch", "type": "int32", "versions": "0+",
       "about": "The current controller epoch." },
     { "name": "BrokerEpoch", "type": "int64", "versions": "2+", "ignorable": true, "default": "-1",
diff --git a/clients/src/main/resources/common/message/StopReplicaRequest.json b/clients/src/main/resources/common/message/StopReplicaRequest.json
index 67ed752a555..7c82c97aa71 100644
--- a/clients/src/main/resources/common/message/StopReplicaRequest.json
+++ b/clients/src/main/resources/common/message/StopReplicaRequest.json
@@ -31,8 +31,8 @@
   "fields": [
     { "name": "ControllerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The controller id." },
-    { "name": "KRaftControllerId", "type": "int32", "versions": "4+", "entityType": "brokerId", "default": "-1",
-      "about": "The KRaft controller id, used during migration. See KIP-866" },
+    { "name": "isKRaftController", "type": "bool", "versions": "4+", "default": "false",
+      "about": "If KRaft controller id is used during migration. See KIP-866" },
     { "name": "ControllerEpoch", "type": "int32", "versions": "0+",
       "about": "The controller epoch." },
     { "name": "BrokerEpoch", "type": "int64", "versions": "1+", "default": "-1", "ignorable": true,
diff --git a/clients/src/main/resources/common/message/UpdateMetadataRequest.json b/clients/src/main/resources/common/message/UpdateMetadataRequest.json
index 287b0ed1a4b..e876caa2bac 100644
--- a/clients/src/main/resources/common/message/UpdateMetadataRequest.json
+++ b/clients/src/main/resources/common/message/UpdateMetadataRequest.json
@@ -36,8 +36,8 @@
   "fields": [
     { "name": "ControllerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
       "about": "The controller id." },
-    { "name": "KRaftControllerId", "type": "int32", "versions": "8+", "entityType": "brokerId",
-      "about": "The KRaft controller id, used during migration." },
+    { "name": "isKRaftController", "type": "bool", "versions": "8+", "default": "false",
+      "about": "If KRaft controller id is used during migration. See KIP-866" },
     { "name": "ControllerEpoch", "type": "int32", "versions": "0+",
       "about": "The controller epoch." },
     { "name": "BrokerEpoch", "type": "int64", "versions": "5+", "ignorable": true, "default": "-1",
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java
index 62571c6986a..2e498339256 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java
@@ -188,7 +188,6 @@ public class ApiVersionsResponseTest {
 
         // Ensure that APIs needed for the KRaft mode are not exposed through ApiVersions until we are ready for them
         HashSet<ApiKeys> exposedApis = apiKeysInResponse(response);
-        assertFalse(exposedApis.contains(ApiKeys.ENVELOPE));
         assertFalse(exposedApis.contains(ApiKeys.VOTE));
         assertFalse(exposedApis.contains(ApiKeys.BEGIN_QUORUM_EPOCH));
         assertFalse(exposedApis.contains(ApiKeys.END_QUORUM_EPOCH));
diff --git a/core/src/main/scala/kafka/Kafka.scala b/core/src/main/scala/kafka/Kafka.scala
index e1bd575f0c2..dad462dcad6 100755
--- a/core/src/main/scala/kafka/Kafka.scala
+++ b/core/src/main/scala/kafka/Kafka.scala
@@ -63,6 +63,12 @@ object Kafka extends Logging {
     props
   }
 
+  // For Zk mode, the API forwarding is currently enabled only under migration flag. We can
+  // directly do a static IBP check to see API forwarding is enabled here because IBP check is
+  // static in Zk mode.
+  private def enableApiForwarding(config: KafkaConfig) =
+    config.migrationEnabled && config.interBrokerProtocolVersion.isApiForwardingEnabled
+
   private def buildServer(props: Properties): Server = {
     val config = KafkaConfig.fromProps(props, false)
     if (config.requiresZookeeper) {
@@ -70,7 +76,7 @@ object Kafka extends Logging {
         config,
         Time.SYSTEM,
         threadNamePrefix = None,
-        enableForwarding = false
+        enableForwarding = enableApiForwarding(config)
       )
     } else {
       new KafkaRaftServer(
diff --git a/core/src/main/scala/kafka/common/InterBrokerSendThread.scala b/core/src/main/scala/kafka/common/InterBrokerSendThread.scala
index c2724e24f1d..e1bea72ddd0 100644
--- a/core/src/main/scala/kafka/common/InterBrokerSendThread.scala
+++ b/core/src/main/scala/kafka/common/InterBrokerSendThread.scala
@@ -33,7 +33,7 @@ import scala.jdk.CollectionConverters._
  */
 abstract class InterBrokerSendThread(
   name: String,
-  networkClient: KafkaClient,
+  var networkClient: KafkaClient,
   requestTimeoutMs: Int,
   time: Time,
   isInterruptible: Boolean = true
diff --git a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala
index 3d1e5d3f63c..3446f83b647 100644
--- a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala
@@ -21,6 +21,7 @@ import java.util.concurrent.LinkedBlockingDeque
 import java.util.concurrent.atomic.AtomicReference
 import kafka.common.{InterBrokerSendThread, RequestAndCompletionHandler}
 import kafka.raft.RaftManager
+import kafka.server.metadata.ZkMetadataCache
 import kafka.utils.Logging
 import org.apache.kafka.clients._
 import org.apache.kafka.common.{Node, Reconfigurable}
@@ -37,42 +38,55 @@ import scala.collection.Seq
 import scala.compat.java8.OptionConverters._
 import scala.jdk.CollectionConverters._
 
-trait ControllerNodeProvider {
-  def get(): Option[Node]
-  def listenerName: ListenerName
-  def securityProtocol: SecurityProtocol
-  def saslMechanism: String
-}
+case class ControllerInformation(node: Option[Node],
+                                 listenerName: ListenerName,
+                                 securityProtocol: SecurityProtocol,
+                                 saslMechanism: String,
+                                 isZkController: Boolean)
 
-object MetadataCacheControllerNodeProvider {
-  def apply(
-    config: KafkaConfig,
-    metadataCache: kafka.server.MetadataCache
-  ): MetadataCacheControllerNodeProvider = {
-    val listenerName = config.controlPlaneListenerName
-      .getOrElse(config.interBrokerListenerName)
-
-    val securityProtocol = config.controlPlaneSecurityProtocol
-      .getOrElse(config.interBrokerSecurityProtocol)
-
-    new MetadataCacheControllerNodeProvider(
-      metadataCache,
-      listenerName,
-      securityProtocol,
-      config.saslMechanismInterBrokerProtocol
-    )
-  }
+trait ControllerNodeProvider {
+  def getControllerInfo(): ControllerInformation
 }
 
 class MetadataCacheControllerNodeProvider(
-  val metadataCache: kafka.server.MetadataCache,
-  val listenerName: ListenerName,
-  val securityProtocol: SecurityProtocol,
-  val saslMechanism: String
+  val metadataCache: ZkMetadataCache,
+  val config: KafkaConfig
 ) extends ControllerNodeProvider {
-  override def get(): Option[Node] = {
-    metadataCache.getControllerId
-      .flatMap(metadataCache.getAliveBrokerNode(_, listenerName))
+
+  private val zkControllerListenerName = config.controlPlaneListenerName.getOrElse(config.interBrokerListenerName)
+  private val zkControllerSecurityProtocol = config.controlPlaneSecurityProtocol.getOrElse(config.interBrokerSecurityProtocol)
+  private val zkControllerSaslMechanism = config.saslMechanismInterBrokerProtocol
+
+  private val kraftControllerListenerName = if (config.controllerListenerNames.nonEmpty)
+    new ListenerName(config.controllerListenerNames.head) else null
+  private val kraftControllerSecurityProtocol = Option(kraftControllerListenerName)
+    .map( listener => config.effectiveListenerSecurityProtocolMap.getOrElse(
+      listener, SecurityProtocol.forName(kraftControllerListenerName.value())))
+    .orNull
+  private val kraftControllerSaslMechanism = config.saslMechanismControllerProtocol
+
+  private val emptyZkControllerInfo =  ControllerInformation(
+    None,
+    zkControllerListenerName,
+    zkControllerSecurityProtocol,
+    zkControllerSaslMechanism,
+    isZkController = true)
+
+  override def getControllerInfo(): ControllerInformation = {
+    metadataCache.getControllerId.map {
+      case ZkCachedControllerId(id) => ControllerInformation(
+        metadataCache.getAliveBrokerNode(id, zkControllerListenerName),
+        zkControllerListenerName,
+        zkControllerSecurityProtocol,
+        zkControllerSaslMechanism,
+        isZkController = true)
+      case KRaftCachedControllerId(id) => ControllerInformation(
+        metadataCache.getAliveBrokerNode(id, kraftControllerListenerName),
+        kraftControllerListenerName,
+        kraftControllerSecurityProtocol,
+        kraftControllerSaslMechanism,
+        isZkController = false)
+    }.getOrElse(emptyZkControllerInfo)
   }
 }
 
@@ -108,9 +122,9 @@ class RaftControllerNodeProvider(
 ) extends ControllerNodeProvider with Logging {
   val idToNode = controllerQuorumVoterNodes.map(node => node.id() -> node).toMap
 
-  override def get(): Option[Node] = {
-    raftManager.leaderAndEpoch.leaderId.asScala.map(idToNode)
-  }
+  override def getControllerInfo(): ControllerInformation =
+    ControllerInformation(raftManager.leaderAndEpoch.leaderId.asScala.map(idToNode),
+      listenerName, securityProtocol, saslMechanism, isZkController = false)
 }
 
 object BrokerToControllerChannelManager {
@@ -176,13 +190,13 @@ class BrokerToControllerChannelManagerImpl(
   }
 
   private[server] def newRequestThread = {
-    val networkClient = {
+    def networkClient(controllerInfo: ControllerInformation) = {
       val channelBuilder = ChannelBuilders.clientChannelBuilder(
-        controllerNodeProvider.securityProtocol,
+        controllerInfo.securityProtocol,
         JaasContext.Type.SERVER,
         config,
-        controllerNodeProvider.listenerName,
-        controllerNodeProvider.saslMechanism,
+        controllerInfo.listenerName,
+        controllerInfo.saslMechanism,
         time,
         config.saslInterBrokerHandshakeRequestEnable,
         logContext
@@ -276,17 +290,38 @@ case class BrokerToControllerQueueItem(
 )
 
 class BrokerToControllerRequestThread(
-  networkClient: KafkaClient,
+  networkClientFactory: ControllerInformation => KafkaClient,
   metadataUpdater: ManualMetadataUpdater,
   controllerNodeProvider: ControllerNodeProvider,
   config: KafkaConfig,
   time: Time,
   threadName: String,
   retryTimeoutMs: Long
-) extends InterBrokerSendThread(threadName, networkClient, Math.min(Int.MaxValue, Math.min(config.controllerSocketTimeoutMs, retryTimeoutMs)).toInt, time, isInterruptible = false) {
+) extends InterBrokerSendThread(threadName, null, Math.min(Int.MaxValue, Math.min(config.controllerSocketTimeoutMs, retryTimeoutMs)).toInt, time, isInterruptible = false) {
+
+  var isZkController = false
+  private def maybeResetNetworkClient(controllerInformation: ControllerInformation,
+                                      initialize: Boolean = false): Unit = {
+    if (initialize || isZkController != controllerInformation.isZkController) {
+      if (!initialize) {
+        debug("Controller changed to " + (if (isZkController) "zk" else "kraft") + " mode. " +
+          "Resetting network client")
+      }
+      // Close existing network client.
+      if (networkClient != null) {
+        networkClient.initiateClose()
+        networkClient.close()
+      }
+      isZkController = controllerInformation.isZkController
+      updateControllerAddress(controllerInformation.node.orNull)
+      controllerInformation.node.foreach(n => metadataUpdater.setNodes(Seq(n).asJava))
+      networkClient = networkClientFactory(controllerInformation)
+    }
+  }
 
   private val requestQueue = new LinkedBlockingDeque[BrokerToControllerQueueItem]()
   private val activeController = new AtomicReference[Node](null)
+  maybeResetNetworkClient(controllerNodeProvider.getControllerInfo(), initialize = true)
 
   // Used for testing
   @volatile
@@ -364,11 +399,13 @@ class BrokerToControllerRequestThread(
   }
 
   override def doWork(): Unit = {
+    val controllerInformation = controllerNodeProvider.getControllerInfo()
+    maybeResetNetworkClient(controllerInformation)
     if (activeControllerAddress().isDefined) {
       super.pollOnce(Long.MaxValue)
     } else {
-      debug("Controller isn't known, checking with controller provider")
-      controllerNodeProvider.get() match {
+      debug("Controller isn't cached, looking for local metadata changes")
+      controllerInformation.node match {
         case Some(controllerNode) =>
           info(s"Recorded new controller, from now on will use node $controllerNode")
           updateControllerAddress(controllerNode)
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 4839e00f8c2..ebe208d6e8f 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -80,7 +80,6 @@ import java.util
 import java.util.concurrent.{CompletableFuture, ConcurrentHashMap}
 import java.util.concurrent.atomic.AtomicInteger
 import java.util.{Collections, Optional}
-
 import scala.annotation.nowarn
 import scala.collection.{Map, Seq, Set, immutable, mutable}
 import scala.jdk.CollectionConverters._
@@ -1317,6 +1316,12 @@ class KafkaApis(val requestChannel: RequestChannel,
 
     trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(completeTopicMetadata.mkString(","),
       brokers.mkString(","), request.header.correlationId, request.header.clientId))
+    val controllerId = {
+      metadataCache.getControllerId.flatMap {
+        case ZkCachedControllerId(id) => Some(id)
+        case KRaftCachedControllerId(_) => metadataCache.getRandomAliveBrokerId
+      }
+    }
 
     requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
        MetadataResponse.prepareResponse(
@@ -1324,7 +1329,7 @@ class KafkaApis(val requestChannel: RequestChannel,
          requestThrottleMs,
          brokers.toList.asJava,
          clusterId,
-         metadataCache.getControllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID),
+         controllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID),
          completeTopicMetadata.asJava,
          clusterAuthorizedOperations
       ))
@@ -3332,13 +3337,18 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
 
     val brokers = metadataCache.getAliveBrokerNodes(request.context.listenerName)
-    val controllerId = metadataCache.getControllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID)
+    val controllerId = {
+      metadataCache.getControllerId.flatMap {
+        case ZkCachedControllerId(id) => Some(id)
+        case KRaftCachedControllerId(_) => metadataCache.getRandomAliveBrokerId
+      }
+    }
 
     requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => {
       val data = new DescribeClusterResponseData()
         .setThrottleTimeMs(requestThrottleMs)
         .setClusterId(clusterId)
-        .setControllerId(controllerId)
+        .setControllerId(controllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID))
         .setClusterAuthorizedOperations(clusterAuthorizedOperations)
 
 
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index 79a621c6b54..2f880a118e1 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -147,6 +147,7 @@ class KafkaServer(
 
   var kafkaScheduler: KafkaScheduler = _
 
+  var kraftControllerNodes: Seq[Node] = Seq.empty
   @volatile var metadataCache: ZkMetadataCache = _
   var quotaManagers: QuotaFactory.QuotaManagers = _
 
@@ -272,8 +273,16 @@ class KafkaServer(
         _brokerState = BrokerState.RECOVERY
         logManager.startup(zkClient.getAllTopicsInCluster())
 
-        metadataCache = MetadataCache.zkMetadataCache(config.brokerId, config.interBrokerProtocolVersion, brokerFeatures)
-        val controllerNodeProvider = MetadataCacheControllerNodeProvider(config, metadataCache)
+        if (config.migrationEnabled) {
+          kraftControllerNodes = RaftConfig.voterConnectionsToNodes(
+            RaftConfig.parseVoterConnections(config.quorumVoters)).asScala
+        }
+        metadataCache = MetadataCache.zkMetadataCache(
+          config.brokerId,
+          config.interBrokerProtocolVersion,
+          brokerFeatures,
+          kraftControllerNodes)
+        val controllerNodeProvider = new MetadataCacheControllerNodeProvider(metadataCache, config)
 
         /* initialize feature change listener */
         _featureChangeListener = new FinalizedFeatureChangeListener(metadataCache, _zkClient)
@@ -614,7 +623,14 @@ class KafkaServer(
   private def controlledShutdown(): Unit = {
     val socketTimeoutMs = config.controllerSocketTimeoutMs
 
+    // TODO (KAFKA-14447): Handle controlled shutdown for zkBroker when we have KRaft controller.
     def doControlledShutdown(retries: Int): Boolean = {
+      if (config.requiresZookeeper &&
+        metadataCache.getControllerId.exists(_.isInstanceOf[KRaftCachedControllerId])) {
+        info("ZkBroker currently has a KRaft controller. Controlled shutdown will be handled " +
+          "through broker life cycle manager")
+        return false
+      }
       val metadataUpdater = new ManualMetadataUpdater()
       val networkClient = {
         val channelBuilder = ChannelBuilders.clientChannelBuilder(
@@ -668,7 +684,7 @@ class KafkaServer(
 
           // 1. Find the controller and establish a connection to it.
           // If the controller id or the broker registration are missing, we sleep and retry (if there are remaining retries)
-          metadataCache.getControllerId match {
+          metadataCache.getControllerId.filter(_.isInstanceOf[ZkCachedControllerId]).map(_.id) match {
             case Some(controllerId) =>
               metadataCache.getAliveBrokerNode(controllerId, config.interBrokerListenerName) match {
                 case Some(broker) =>
diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala
index b20d4f6414c..e0501ef1ebe 100755
--- a/core/src/main/scala/kafka/server/MetadataCache.scala
+++ b/core/src/main/scala/kafka/server/MetadataCache.scala
@@ -32,6 +32,13 @@ case class FinalizedFeaturesAndEpoch(features: Map[String, Short], epoch: Long)
   }
 }
 
+sealed trait CachedControllerId {
+  val id: Int
+}
+
+case class ZkCachedControllerId(id: Int) extends CachedControllerId
+case class KRaftCachedControllerId(id: Int) extends CachedControllerId
+
 trait MetadataCache {
 
   /**
@@ -92,7 +99,7 @@ trait MetadataCache {
 
   def getPartitionReplicaEndpoints(tp: TopicPartition, listenerName: ListenerName): Map[Int, Node]
 
-  def getControllerId: Option[Int]
+  def getControllerId: Option[CachedControllerId]
 
   def getClusterMetadata(clusterId: String, listenerName: ListenerName): Cluster
 
@@ -103,13 +110,18 @@ trait MetadataCache {
   def metadataVersion(): MetadataVersion
 
   def features(): FinalizedFeaturesAndEpoch
+
+  def getRandomAliveBrokerId: Option[Int]
 }
 
 object MetadataCache {
   def zkMetadataCache(brokerId: Int,
                       metadataVersion: MetadataVersion,
-                      brokerFeatures: BrokerFeatures = BrokerFeatures.createEmpty()): ZkMetadataCache = {
-    new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures)
+                      brokerFeatures: BrokerFeatures = BrokerFeatures.createEmpty(),
+                      kraftControllerNodes: collection.Seq[Node] = null)
+  : ZkMetadataCache = {
+    new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures,
+      Option(kraftControllerNodes).getOrElse(collection.Seq.empty[Node]))
   }
 
   def kRaftMetadataCache(brokerId: Int): KRaftMetadataCache = {
diff --git a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala
index 7bd1c6343d9..7e6ad7bfd09 100644
--- a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala
+++ b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala
@@ -18,7 +18,7 @@
 package kafka.server.metadata
 
 import kafka.controller.StateChangeLogger
-import kafka.server.{FinalizedFeaturesAndEpoch, MetadataCache}
+import kafka.server.{CachedControllerId, FinalizedFeaturesAndEpoch, KRaftCachedControllerId, MetadataCache}
 import kafka.utils.Logging
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic}
@@ -287,14 +287,19 @@ class KRaftMetadataCache(val brokerId: Int) extends MetadataCache with Logging w
     result.toMap
   }
 
-  override def getControllerId: Option[Int] = getRandomAliveBroker(_currentImage)
-
   /**
    * Choose a random broker node to report as the controller. We do this because we want
    * the client to send requests destined for the controller to a random broker.
    * Clients do not have direct access to the controller in the KRaft world, as explained
    * in KIP-590.
    */
+  override def getControllerId: Option[CachedControllerId] =
+    getRandomAliveBroker(_currentImage).map(KRaftCachedControllerId)
+
+  override def getRandomAliveBrokerId: Option[Int] = {
+    getRandomAliveBroker(_currentImage)
+  }
+
   private def getRandomAliveBroker(image: MetadataImage): Option[Int] = {
     val aliveBrokers = getAliveBrokers(image).toList
     if (aliveBrokers.isEmpty) {
diff --git a/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala
index d69785f90f6..d774cd41a5c 100755
--- a/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala
+++ b/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala
@@ -27,7 +27,7 @@ import scala.jdk.CollectionConverters._
 import kafka.cluster.{Broker, EndPoint}
 import kafka.api._
 import kafka.controller.StateChangeLogger
-import kafka.server.{BrokerFeatures, FinalizedFeaturesAndEpoch, MetadataCache}
+import kafka.server.{BrokerFeatures, CachedControllerId, FinalizedFeaturesAndEpoch, KRaftCachedControllerId, MetadataCache, ZkCachedControllerId}
 import kafka.utils.CoreUtils._
 import kafka.utils.Logging
 import kafka.utils.Implicits._
@@ -42,7 +42,7 @@ import org.apache.kafka.common.requests.{ApiVersionsResponse, MetadataResponse,
 import org.apache.kafka.common.security.auth.SecurityProtocol
 import org.apache.kafka.server.common.MetadataVersion
 
-import java.util.concurrent.TimeUnit
+import java.util.concurrent.{ThreadLocalRandom, TimeUnit}
 import scala.concurrent.TimeoutException
 import scala.math.max
 
@@ -60,7 +60,11 @@ trait ZkFinalizedFeatureCache {
  *  A cache for the state (e.g., current leader) of each partition. This cache is updated through
  *  UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously.
  */
-class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFeatures: BrokerFeatures)
+class ZkMetadataCache(
+  brokerId: Int,
+  metadataVersion: MetadataVersion,
+  brokerFeatures: BrokerFeatures,
+  kraftControllerNodes: Seq[Node] = Seq.empty)
   extends MetadataCache with ZkFinalizedFeatureCache with Logging {
 
   private val partitionMetadataLock = new ReentrantReadWriteLock()
@@ -68,8 +72,12 @@ class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFea
   //replace the value with a completely new one. this means reads (which are not under any lock) need to grab
   //the value of this var (into a val) ONCE and retain that read copy for the duration of their operation.
   //multiple reads of this value risk getting different snapshots.
-  @volatile private var metadataSnapshot: MetadataSnapshot = MetadataSnapshot(partitionStates = mutable.AnyRefMap.empty,
-    topicIds = Map.empty, controllerId = None, aliveBrokers = mutable.LongMap.empty, aliveNodes = mutable.LongMap.empty)
+  @volatile private var metadataSnapshot: MetadataSnapshot = MetadataSnapshot(
+    partitionStates = mutable.AnyRefMap.empty,
+    topicIds = Map.empty,
+    controllerId = None,
+    aliveBrokers = mutable.LongMap.empty,
+    aliveNodes = mutable.LongMap.empty)
 
   this.logIdent = s"[MetadataCache brokerId=$brokerId] "
   private val stateChangeLogger = new StateChangeLogger(brokerId, inControllerContext = false, None)
@@ -79,6 +87,8 @@ class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFea
   private val featureLock = new ReentrantLock()
   private val featureCond = featureLock.newCondition()
 
+  private val kraftControllerNodeMap = kraftControllerNodes.map(node => node.id() -> node).toMap
+
   // This method is the main hotspot when it comes to the performance of metadata requests,
   // we should be careful about adding additional logic here. Relatedly, `brokers` is
   // `List[Integer]` instead of `List[Int]` to avoid a collection copy.
@@ -248,7 +258,12 @@ class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFea
   }
 
   override def getAliveBrokerNode(brokerId: Int, listenerName: ListenerName): Option[Node] = {
-    metadataSnapshot.aliveBrokers.get(brokerId).flatMap(_.getNode(listenerName))
+    val snapshot = metadataSnapshot
+    brokerId match {
+      case id if snapshot.controllerId.filter(_.isInstanceOf[KRaftCachedControllerId]).exists(_.id == id) =>
+        kraftControllerNodeMap.get(id)
+      case _ => snapshot.aliveBrokers.get(brokerId).flatMap(_.getNode(listenerName))
+    }
   }
 
   override def getAliveBrokerNodes(listenerName: ListenerName): Iterable[Node] = {
@@ -315,7 +330,14 @@ class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFea
     }.getOrElse(Map.empty[Int, Node])
   }
 
-  def getControllerId: Option[Int] = metadataSnapshot.controllerId
+  def getControllerId: Option[CachedControllerId] = {
+    metadataSnapshot.controllerId
+  }
+
+  def getRandomAliveBrokerId: Option[Int] = {
+    val aliveBrokers = metadataSnapshot.aliveBrokers.values.toList
+    Some(aliveBrokers(ThreadLocalRandom.current().nextInt(aliveBrokers.size)).id)
+  }
 
   def getClusterMetadata(clusterId: String, listenerName: ListenerName): Cluster = {
     val snapshot = metadataSnapshot
@@ -329,6 +351,13 @@ class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFea
       nodes.getOrElse(id.toLong, new Node(id, "", -1))
     }
 
+    def controllerId(snapshot: MetadataSnapshot): Option[Node] = {
+      snapshot.controllerId.flatMap {
+        case ZkCachedControllerId(id) => getAliveBrokerNode(id, listenerName)
+        case KRaftCachedControllerId(_) => getRandomAliveBrokerId.flatMap(getAliveBrokerNode(_, listenerName))
+      }
+    }
+
     val partitions = getAllPartitions(snapshot)
       .filter { case (_, state) => state.leader != LeaderAndIsr.LeaderDuringDelete }
       .map { case (tp, state) =>
@@ -342,7 +371,7 @@ class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFea
     new Cluster(clusterId, nodes.values.toBuffer.asJava,
       partitions.toBuffer.asJava,
       unauthorizedTopics, internalTopics,
-      snapshot.controllerId.map(id => node(id)).orNull)
+      controllerId(snapshot).orNull)
   }
 
   // This method returns the deleted TopicPartitions received from UpdateMetadataRequest
@@ -351,9 +380,13 @@ class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFea
 
       val aliveBrokers = new mutable.LongMap[Broker](metadataSnapshot.aliveBrokers.size)
       val aliveNodes = new mutable.LongMap[collection.Map[ListenerName, Node]](metadataSnapshot.aliveNodes.size)
-      val controllerIdOpt = updateMetadataRequest.controllerId match {
+      val controllerIdOpt: Option[CachedControllerId] = updateMetadataRequest.controllerId match {
         case id if id < 0 => None
-        case id => Some(id)
+        case id =>
+          if (updateMetadataRequest.isKRaftController)
+            Some(KRaftCachedControllerId(id))
+          else
+            Some(ZkCachedControllerId(id))
       }
 
       updateMetadataRequest.liveBrokers.forEach { broker =>
@@ -386,7 +419,8 @@ class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFea
 
       val deletedPartitions = new mutable.ArrayBuffer[TopicPartition]
       if (!updateMetadataRequest.partitionStates.iterator.hasNext) {
-        metadataSnapshot = MetadataSnapshot(metadataSnapshot.partitionStates, topicIds.toMap, controllerIdOpt, aliveBrokers, aliveNodes)
+        metadataSnapshot = MetadataSnapshot(metadataSnapshot.partitionStates, topicIds.toMap,
+          controllerIdOpt, aliveBrokers, aliveNodes)
       } else {
         //since kafka may do partial metadata updates, we start by copying the previous state
         val partitionStates = new mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]](metadataSnapshot.partitionStates.size)
@@ -446,7 +480,7 @@ class ZkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFea
 
   case class MetadataSnapshot(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]],
                               topicIds: Map[String, Uuid],
-                              controllerId: Option[Int],
+                              controllerId: Option[CachedControllerId],
                               aliveBrokers: mutable.LongMap[Broker],
                               aliveNodes: mutable.LongMap[collection.Map[ListenerName, Node]]) {
     val topicNames: Map[Uuid, String] = topicIds.map { case (topicName, topicId) => (topicId, topicName) }
diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
index 3b459875b60..f4ab8e041be 100644
--- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
@@ -60,10 +60,14 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
   }
 
   override def generateConfigs: Seq[KafkaConfig] = {
+
     val cfgs = TestUtils.createBrokerConfigs(brokerCount, zkConnectOrNull, interBrokerSecurityProtocol = Some(securityProtocol),
       trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties, logDirCount = logDirCount)
     configureListeners(cfgs)
     modifyConfigs(cfgs)
+    if (isZkMigrationTest()) {
+      cfgs.foreach(_.setProperty(KafkaConfig.MigrationEnabledProp, "true"))
+    }
     insertControllerListenersIfNeeded(cfgs)
     cfgs.map(KafkaConfig.fromProps)
   }
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
index 80f6cd758b3..2a3e0a18117 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
@@ -215,7 +215,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
       val brokerIds = brokers.map(_.config.brokerId).toSet
       assertTrue(brokerIds.contains(controller.id))
     } else {
-      assertEquals(brokers.head.dataPlaneRequestProcessor.metadataCache.getControllerId.
+      assertEquals(brokers.head.dataPlaneRequestProcessor.metadataCache.getControllerId.map(_.id).
         getOrElse(MetadataResponse.NO_CONTROLLER_ID), controller.id)
     }
 
diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
index f746cec53d7..51720db924a 100755
--- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
@@ -160,6 +160,10 @@ abstract class QuorumTestHarness extends Logging {
     TestInfoUtils.isKRaft(testInfo)
   }
 
+  def isZkMigrationTest(): Boolean = {
+    TestInfoUtils.isZkMigrationTest(testInfo)
+  }
+
   def checkIsZKTest(): Unit = {
     if (isKRaftTest()) {
       throw new RuntimeException("This function can't be accessed when running the test " +
diff --git a/core/src/test/scala/kafka/server/BrokerToControllerRequestThreadTest.scala b/core/src/test/scala/kafka/server/BrokerToControllerRequestThreadTest.scala
index bee1aefaca2..eea5c7517a0 100644
--- a/core/src/test/scala/kafka/server/BrokerToControllerRequestThreadTest.scala
+++ b/core/src/test/scala/kafka/server/BrokerToControllerRequestThreadTest.scala
@@ -25,9 +25,10 @@ import kafka.utils.TestUtils.TestControllerRequestCompletionHandler
 import org.apache.kafka.clients.{ClientResponse, ManualMetadataUpdater, Metadata, MockClient, NodeApiVersions}
 import org.apache.kafka.common.Node
 import org.apache.kafka.common.message.{EnvelopeResponseData, MetadataRequestData}
+import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
 import org.apache.kafka.common.requests.{AbstractRequest, EnvelopeRequest, EnvelopeResponse, MetadataRequest, RequestTestUtils}
-import org.apache.kafka.common.security.auth.KafkaPrincipal
+import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
 import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
 import org.apache.kafka.common.utils.MockTime
 import org.junit.jupiter.api.Assertions._
@@ -37,6 +38,14 @@ import org.mockito.Mockito._
 
 class BrokerToControllerRequestThreadTest {
 
+  private def controllerInfo(node: Option[Node]): ControllerInformation = {
+    ControllerInformation(node, new ListenerName(""), SecurityProtocol.PLAINTEXT, "", isZkController = true)
+  }
+
+  private def emptyControllerInfo: ControllerInformation = {
+    controllerInfo(None)
+  }
+
   @Test
   def testRetryTimeoutWhileControllerNotAvailable(): Unit = {
     val time = new MockTime()
@@ -45,10 +54,10 @@ class BrokerToControllerRequestThreadTest {
     val mockClient = new MockClient(time, metadata)
     val controllerNodeProvider = mock(classOf[ControllerNodeProvider])
 
-    when(controllerNodeProvider.get()).thenReturn(None)
+    when(controllerNodeProvider.getControllerInfo()).thenReturn(emptyControllerInfo)
 
     val retryTimeoutMs = 30000
-    val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
+    val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
       config, time, "", retryTimeoutMs)
     testRequestThread.started = true
 
@@ -82,10 +91,10 @@ class BrokerToControllerRequestThreadTest {
     val controllerNodeProvider = mock(classOf[ControllerNodeProvider])
     val activeController = new Node(controllerId, "host", 1234)
 
-    when(controllerNodeProvider.get()).thenReturn(Some(activeController))
+    when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo(Some(activeController)))
 
     val expectedResponse = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap("a", 2))
-    val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
+    val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
       config, time, "", retryTimeoutMs = Long.MaxValue)
     testRequestThread.started = true
     mockClient.prepareResponse(expectedResponse)
@@ -124,10 +133,11 @@ class BrokerToControllerRequestThreadTest {
     val oldController = new Node(oldControllerId, "host1", 1234)
     val newController = new Node(newControllerId, "host2", 1234)
 
-    when(controllerNodeProvider.get()).thenReturn(Some(oldController), Some(newController))
+    when(controllerNodeProvider.getControllerInfo()).thenReturn(
+      emptyControllerInfo, controllerInfo(Some(oldController)), controllerInfo(Some(newController)))
 
     val expectedResponse = RequestTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", 2))
-    val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(),
+    val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(),
       controllerNodeProvider, config, time, "", retryTimeoutMs = Long.MaxValue)
     testRequestThread.started = true
 
@@ -171,13 +181,14 @@ class BrokerToControllerRequestThreadTest {
     val oldController = new Node(oldControllerId, "host1", port)
     val newController = new Node(newControllerId, "host2", port)
 
-    when(controllerNodeProvider.get()).thenReturn(Some(oldController), Some(newController))
+    when(controllerNodeProvider.getControllerInfo()).thenReturn(
+      emptyControllerInfo, controllerInfo(Some(oldController)), controllerInfo(Some(newController)))
 
     val responseWithNotControllerError = RequestTestUtils.metadataUpdateWith("cluster1", 2,
       Collections.singletonMap("a", Errors.NOT_CONTROLLER),
       Collections.singletonMap("a", 2))
     val expectedResponse = RequestTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", 2))
-    val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
+    val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
       config, time, "", retryTimeoutMs = Long.MaxValue)
     testRequestThread.started = true
 
@@ -231,7 +242,11 @@ class BrokerToControllerRequestThreadTest {
     val oldController = new Node(oldControllerId, "host1", port)
     val newController = new Node(newControllerId, "host2", port)
 
-    when(controllerNodeProvider.get()).thenReturn(Some(oldController), Some(newController))
+    when(controllerNodeProvider.getControllerInfo()).thenReturn(
+      emptyControllerInfo,                  // call to create network client.
+      controllerInfo(Some(oldController)),
+      controllerInfo(Some(newController))
+    )
 
     // create an envelopeResponse with NOT_CONTROLLER error
     val envelopeResponseWithNotControllerError = new EnvelopeResponse(
@@ -240,7 +255,7 @@ class BrokerToControllerRequestThreadTest {
     // response for retry request after receiving NOT_CONTROLLER error
     val expectedResponse = RequestTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", 2))
 
-    val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
+    val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
       config, time, "", retryTimeoutMs = Long.MaxValue)
     testRequestThread.started = true
 
@@ -296,13 +311,13 @@ class BrokerToControllerRequestThreadTest {
     val controllerNodeProvider = mock(classOf[ControllerNodeProvider])
     val controller = new Node(controllerId, "host1", 1234)
 
-    when(controllerNodeProvider.get()).thenReturn(Some(controller))
+    when(controllerNodeProvider.getControllerInfo()).thenReturn(emptyControllerInfo, controllerInfo(Some(controller)))
 
     val retryTimeoutMs = 30000
     val responseWithNotControllerError = RequestTestUtils.metadataUpdateWith("cluster1", 2,
       Collections.singletonMap("a", Errors.NOT_CONTROLLER),
       Collections.singletonMap("a", 2))
-    val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
+    val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
       config, time, "", retryTimeoutMs)
     testRequestThread.started = true
 
@@ -344,7 +359,7 @@ class BrokerToControllerRequestThreadTest {
     val controllerNodeProvider = mock(classOf[ControllerNodeProvider])
     val activeController = new Node(controllerId, "host", 1234)
 
-    when(controllerNodeProvider.get()).thenReturn(Some(activeController))
+    when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo(Some(activeController)))
 
     val callbackResponse = new AtomicReference[ClientResponse]()
     val completionHandler = new ControllerRequestCompletionHandler {
@@ -360,7 +375,7 @@ class BrokerToControllerRequestThreadTest {
 
     mockClient.prepareUnsupportedVersionResponse(request => request.apiKey == ApiKeys.METADATA)
 
-    val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
+    val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
       config, time, "", retryTimeoutMs = Long.MaxValue)
     testRequestThread.started = true
 
@@ -381,7 +396,7 @@ class BrokerToControllerRequestThreadTest {
     val controllerNodeProvider = mock(classOf[ControllerNodeProvider])
     val activeController = new Node(controllerId, "host", 1234)
 
-    when(controllerNodeProvider.get()).thenReturn(Some(activeController))
+    when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo(Some(activeController)))
 
     val callbackResponse = new AtomicReference[ClientResponse]()
     val completionHandler = new ControllerRequestCompletionHandler {
@@ -397,7 +412,7 @@ class BrokerToControllerRequestThreadTest {
 
     mockClient.createPendingAuthenticationError(activeController, 50)
 
-    val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
+    val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
       config, time, "", retryTimeoutMs = Long.MaxValue)
     testRequestThread.started = true
 
@@ -416,8 +431,9 @@ class BrokerToControllerRequestThreadTest {
     val mockClient = new MockClient(time, metadata)
 
     val controllerNodeProvider = mock(classOf[ControllerNodeProvider])
+    when(controllerNodeProvider.getControllerInfo()).thenReturn(emptyControllerInfo)
 
-    val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
+    val testRequestThread = new BrokerToControllerRequestThread(_ => mockClient, new ManualMetadataUpdater(), controllerNodeProvider,
       config, time, "", retryTimeoutMs = Long.MaxValue)
 
     val completionHandler = new TestControllerRequestCompletionHandler(None)
diff --git a/core/src/test/scala/kafka/utils/TestInfoUtils.scala b/core/src/test/scala/kafka/utils/TestInfoUtils.scala
index fa48024f313..ba93fa36b99 100644
--- a/core/src/test/scala/kafka/utils/TestInfoUtils.scala
+++ b/core/src/test/scala/kafka/utils/TestInfoUtils.scala
@@ -43,5 +43,13 @@ object TestInfoUtils {
       false
     }
   }
+
+  def isZkMigrationTest(testInfo: TestInfo): Boolean = {
+    if (!isKRaft(testInfo)) {
+      false
+    } else {
+      testInfo.getDisplayName().contains("quorum=zkMigration")
+    }
+  }
   final val TestWithParameterizedQuorumName = "{displayName}.quorum={0}"
 }
diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
index 5dbac1bb4dc..1ef4d47995a 100755
--- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
@@ -357,7 +357,8 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness {
         config,
         time = brokerTime(config.brokerId),
         threadNamePrefix = None,
-        startup = false
+        startup = false,
+        enableZkApiForwarding = isZkMigrationTest()
       )
     }
   }
diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala
index 9936f8deaed..5d94319ed35 100644
--- a/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala
@@ -124,11 +124,11 @@ class ApiVersionManagerTest {
       features = brokerFeatures,
       metadataCache = metadataCache
     )
-    assertFalse(versionManager.isApiEnabled(ApiKeys.ENVELOPE))
-    assertFalse(versionManager.enabledApis.contains(ApiKeys.ENVELOPE))
+    assertTrue(versionManager.isApiEnabled(ApiKeys.ENVELOPE))
+    assertTrue(versionManager.enabledApis.contains(ApiKeys.ENVELOPE))
 
     val apiVersionsResponse = versionManager.apiVersionResponse(throttleTimeMs = 0)
-    assertNull(apiVersionsResponse.data.apiKeys.find(ApiKeys.ENVELOPE.id))
+    assertNotNull(apiVersionsResponse.data.apiKeys.find(ApiKeys.ENVELOPE.id))
   }
 
 }
diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
index 1a0fac443c0..da6d9a8aa80 100644
--- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
@@ -57,13 +57,14 @@ class BrokerLifecycleManagerTest {
   class SimpleControllerNodeProvider extends ControllerNodeProvider {
     val node = new AtomicReference[Node](null)
 
-    override def get(): Option[Node] = Option(node.get())
+    def listenerName: ListenerName = new ListenerName("PLAINTEXT")
 
-    override def listenerName: ListenerName = new ListenerName("PLAINTEXT")
+    def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT;
 
-    override def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT;
+    def saslMechanism: String = SaslConfigs.DEFAULT_SASL_MECHANISM
 
-    override def saslMechanism: String = SaslConfigs.DEFAULT_SASL_MECHANISM
+    override def getControllerInfo(): ControllerInformation = ControllerInformation(Option(node.get()),
+      listenerName, securityProtocol, saslMechanism, isZkController = false)
   }
 
   class BrokerLifecycleManagerTestContext(properties: Properties) {
diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala
index 2bb1314a789..cab586f23e4 100644
--- a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala
@@ -17,7 +17,7 @@
 
 package unit.kafka.server
 
-import kafka.server.{BrokerToControllerChannelManager, ControllerNodeProvider, ControllerRequestCompletionHandler}
+import kafka.server.{BrokerToControllerChannelManager, ControllerInformation, ControllerNodeProvider, ControllerRequestCompletionHandler}
 import kafka.test.ClusterInstance
 import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, Type}
 import kafka.test.junit.ClusterTestExtensions
@@ -48,17 +48,22 @@ class BrokerRegistrationRequestTest {
   def brokerToControllerChannelManager(clusterInstance: ClusterInstance): BrokerToControllerChannelManager = {
     BrokerToControllerChannelManager(
       new ControllerNodeProvider() {
-        override def get(): Option[Node] = Some(new Node(
+        def node: Option[Node] = Some(new Node(
           clusterInstance.anyControllerSocketServer().config.nodeId,
           "127.0.0.1",
           clusterInstance.anyControllerSocketServer().boundPort(clusterInstance.controllerListenerName().get()),
         ))
 
-        override def listenerName: ListenerName = clusterInstance.controllerListenerName().get()
+        def listenerName: ListenerName = clusterInstance.controllerListenerName().get()
 
-        override def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT
+        val securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT
 
-        override def saslMechanism: String = ""
+        val saslMechanism: String = ""
+
+        def isZkController: Boolean = !clusterInstance.isKRaftTest
+
+        override def getControllerInfo(): ControllerInformation =
+          ControllerInformation(node, listenerName, securityProtocol, saslMechanism, isZkController)
       },
       Time.SYSTEM,
       new Metrics(),
diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala
index a193db284c4..c6892566968 100644
--- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala
@@ -22,15 +22,16 @@ import org.apache.kafka.common.Uuid
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.message.CreateTopicsRequestData
 import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection
-import org.apache.kafka.common.protocol.ApiKeys
-import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
 import org.apache.kafka.common.requests.CreateTopicsRequest
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.ValueSource
+
 import scala.jdk.CollectionConverters._
 
 class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
+
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
   @ValueSource(strings = Array("zk", "kraft"))
   def testValidCreateTopicsRequests(quorum: String): Unit = {
@@ -148,13 +149,14 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
   }
 
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
-  @ValueSource(strings = Array("zk"))
+  @ValueSource(strings = Array("zk", "zkMigration"))
   def testNotController(quorum: String): Unit = {
     // Note: we don't run this test when in KRaft mode, because KRaft doesn't have this
     // behavior of returning NOT_CONTROLLER. Instead, the request is forwarded.
     val req = topicsReq(Seq(topicReq("topic1")))
     val response = sendCreateTopicRequest(req, notControllerSocketServer)
-    assertEquals(1, response.errorCounts().get(Errors.NOT_CONTROLLER))
+    val error = if (isZkMigrationTest()) Errors.NONE else Errors.NOT_CONTROLLER
+    assertEquals(1, response.errorCounts().get(error))
   }
 
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala
index 629f203169d..ff14323b019 100644
--- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala
@@ -221,7 +221,7 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
    * Instead, the request is forwarded.
    */
   @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
-  @ValueSource(strings = Array("zk"))
+  @ValueSource(strings = Array("zk", "zkMigration"))
   def testNotController(quorum: String): Unit = {
     val request = new DeleteTopicsRequest.Builder(
         new DeleteTopicsRequestData()
@@ -229,8 +229,9 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
           .setTimeoutMs(1000)).build()
     val response = sendDeleteTopicsRequest(request, notControllerSocketServer)
 
+    val expectedError = if (isZkMigrationTest()) Errors.NONE else Errors.NOT_CONTROLLER
     val error = response.data.responses.find("not-controller").errorCode()
-    assertEquals(Errors.NOT_CONTROLLER.code,  error, "Expected controller error when routed incorrectly")
+    assertEquals(expectedError.code(),  error)
   }
 
   private def validateTopicIsDeleted(topic: String): Unit = {
diff --git a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala
index d0fc30fbdec..21c7d0d9807 100644
--- a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala
@@ -60,6 +60,14 @@ class ForwardingManagerTest {
     NodeApiVersions.create(List(envelopeApiVersion).asJava)
   }
 
+  private def controllerInfo = {
+    ControllerInformation(Some(new Node(0, "host", 1234)), new ListenerName(""), SecurityProtocol.PLAINTEXT, "", isZkController = true)
+  }
+
+  private def emptyControllerInfo = {
+    ControllerInformation(None, new ListenerName(""), SecurityProtocol.PLAINTEXT, "", isZkController = true)
+  }
+
   @Test
   def testResponseCorrelationIdMismatch(): Unit = {
     val requestCorrelationId = 27
@@ -71,7 +79,7 @@ class ForwardingManagerTest {
     val responseBuffer = RequestTestUtils.serializeResponseWithHeader(responseBody, requestHeader.apiVersion,
       requestCorrelationId + 1)
 
-    Mockito.when(controllerNodeProvider.get()).thenReturn(Some(new Node(0, "host", 1234)))
+    Mockito.when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo)
     val isEnvelopeRequest: RequestMatcher = request => request.isInstanceOf[EnvelopeRequest]
     client.prepareResponse(isEnvelopeRequest, new EnvelopeResponse(responseBuffer, Errors.NONE));
 
@@ -95,7 +103,7 @@ class ForwardingManagerTest {
     val responseBuffer = RequestTestUtils.serializeResponseWithHeader(responseBody,
       requestHeader.apiVersion, requestCorrelationId)
 
-    Mockito.when(controllerNodeProvider.get()).thenReturn(Some(new Node(0, "host", 1234)))
+    Mockito.when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo)
     val isEnvelopeRequest: RequestMatcher = request => request.isInstanceOf[EnvelopeRequest]
     client.prepareResponse(isEnvelopeRequest, new EnvelopeResponse(responseBuffer, Errors.UNSUPPORTED_VERSION));
 
@@ -112,7 +120,7 @@ class ForwardingManagerTest {
     val (requestHeader, requestBuffer) = buildRequest(testAlterConfigRequest, requestCorrelationId)
     val request = buildRequest(requestHeader, requestBuffer, clientPrincipal)
 
-    Mockito.when(controllerNodeProvider.get()).thenReturn(None)
+    Mockito.when(controllerNodeProvider.getControllerInfo()).thenReturn(emptyControllerInfo)
 
     val response = new AtomicReference[AbstractResponse]()
     forwardingManager.forwardRequest(request, res => res.foreach(response.set))
@@ -136,7 +144,7 @@ class ForwardingManagerTest {
     val (requestHeader, requestBuffer) = buildRequest(testAlterConfigRequest, requestCorrelationId)
     val request = buildRequest(requestHeader, requestBuffer, clientPrincipal)
 
-    Mockito.when(controllerNodeProvider.get()).thenReturn(Some(new Node(0, "host", 1234)))
+    Mockito.when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo)
 
     val response = new AtomicReference[AbstractResponse]()
     forwardingManager.forwardRequest(request, res => res.foreach(response.set))
@@ -162,8 +170,7 @@ class ForwardingManagerTest {
     val (requestHeader, requestBuffer) = buildRequest(testAlterConfigRequest, requestCorrelationId)
     val request = buildRequest(requestHeader, requestBuffer, clientPrincipal)
 
-    val controllerNode = new Node(0, "host", 1234)
-    Mockito.when(controllerNodeProvider.get()).thenReturn(Some(controllerNode))
+    Mockito.when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo)
 
     client.prepareUnsupportedVersionResponse(req => req.apiKey == requestHeader.apiKey)
 
@@ -183,10 +190,9 @@ class ForwardingManagerTest {
     val (requestHeader, requestBuffer) = buildRequest(testAlterConfigRequest, requestCorrelationId)
     val request = buildRequest(requestHeader, requestBuffer, clientPrincipal)
 
-    val controllerNode = new Node(0, "host", 1234)
-    Mockito.when(controllerNodeProvider.get()).thenReturn(Some(controllerNode))
+    Mockito.when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo)
 
-    client.createPendingAuthenticationError(controllerNode, 50)
+    client.createPendingAuthenticationError(controllerInfo.node.get, 50)
 
     val response = new AtomicReference[AbstractResponse]()
     forwardingManager.forwardRequest(request, res => res.foreach(response.set))
diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
index 5019a57b0ed..a8e5d5464a4 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
@@ -3571,7 +3571,7 @@ class KafkaApisTest {
     val capturedResponse = verifyNoThrottling(request)
     val describeClusterResponse = capturedResponse.getValue.asInstanceOf[DescribeClusterResponse]
 
-    assertEquals(metadataCache.getControllerId.get, describeClusterResponse.data.controllerId)
+    assertEquals(metadataCache.getControllerId.get.id, describeClusterResponse.data.controllerId)
     assertEquals(clusterId, describeClusterResponse.data.clusterId)
     assertEquals(8096, describeClusterResponse.data.clusterAuthorizedOperations)
     assertEquals(metadataCache.getAliveBrokerNodes(plaintextListener).toSet,
diff --git a/core/src/test/scala/unit/kafka/server/MockBrokerToControllerChannelManager.scala b/core/src/test/scala/unit/kafka/server/MockBrokerToControllerChannelManager.scala
index febd06f354d..1752e2c6644 100644
--- a/core/src/test/scala/unit/kafka/server/MockBrokerToControllerChannelManager.scala
+++ b/core/src/test/scala/unit/kafka/server/MockBrokerToControllerChannelManager.scala
@@ -73,7 +73,7 @@ class MockBrokerToControllerChannelManager(
         queueItem.callback.onTimeout()
         unsentIterator.remove()
       } else {
-        controllerNodeProvider.get() match {
+        controllerNodeProvider.getControllerInfo().node match {
           case Some(controller) if client.ready(controller, time.milliseconds()) =>
             val clientRequest = client.newClientRequest(
               controller.idString,
diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
index 82c19949e3b..8679706777d 100644
--- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
+++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
@@ -159,7 +159,8 @@ class RequestQuotaTest extends BaseRequestTest {
 
   @Test
   def testExemptRequestTime(): Unit = {
-    for (apiKey <- RequestQuotaTest.ClusterActions -- RequestQuotaTest.ClusterActionsWithThrottle) {
+    val actions = RequestQuotaTest.ClusterActions -- RequestQuotaTest.ClusterActionsWithThrottle -- RequestQuotaTest.Envelope
+    for (apiKey <- actions) {
       submitTest(apiKey, () => checkExemptRequestMetric(apiKey))
     }
 
@@ -170,7 +171,7 @@ class RequestQuotaTest extends BaseRequestTest {
   def testUnauthorizedThrottle(): Unit = {
     RequestQuotaTest.principal = RequestQuotaTest.UnauthorizedPrincipal
 
-    for (apiKey <- ApiKeys.zkBrokerApis.asScala) {
+    for (apiKey <- ApiKeys.zkBrokerApis.asScala.toSet -- RequestQuotaTest.Envelope) {
       submitTest(apiKey, () => checkUnauthorizedRequestThrottle(apiKey))
     }
 
@@ -765,7 +766,8 @@ object RequestQuotaTest {
   val ClusterActions = ApiKeys.zkBrokerApis.asScala.filter(_.clusterAction).toSet
   val ClusterActionsWithThrottle = Set(ApiKeys.ALLOCATE_PRODUCER_IDS, ApiKeys.UPDATE_FEATURES)
   val SaslActions = Set(ApiKeys.SASL_HANDSHAKE, ApiKeys.SASL_AUTHENTICATE)
-  val ClientActions = ApiKeys.zkBrokerApis.asScala.toSet -- ClusterActions -- SaslActions
+  val Envelope = Set(ApiKeys.ENVELOPE)
+  val ClientActions = ApiKeys.zkBrokerApis.asScala.toSet -- ClusterActions -- SaslActions -- Envelope
 
   val UnauthorizedPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "Unauthorized")
   // Principal used for all client connections. This is modified by tests which
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 0162ebdd5e3..7110237ce4c 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -179,7 +179,12 @@ object TestUtils extends Logging {
   }
 
   def createServer(config: KafkaConfig, time: Time, threadNamePrefix: Option[String], startup: Boolean): KafkaServer = {
-    val server = new KafkaServer(config, time, threadNamePrefix, enableForwarding = false)
+    createServer(config, time, threadNamePrefix, startup, enableZkApiForwarding = false)
+  }
+
+  def createServer(config: KafkaConfig, time: Time, threadNamePrefix: Option[String],
+                   startup: Boolean, enableZkApiForwarding: Boolean) = {
+    val server = new KafkaServer(config, time, threadNamePrefix, enableForwarding = enableZkApiForwarding)
     if (startup) server.startup()
     server
   }
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java
index 3e57f29f8d4..6cbf2438c8c 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java
@@ -218,7 +218,8 @@ public class ReplicaFetcherThreadBenchmark {
                 0, 0, 0, updatePartitionState, Collections.emptyList(), topicIds).build();
 
         // TODO: fix to support raft
-        ZkMetadataCache metadataCache = new ZkMetadataCache(0, config.interBrokerProtocolVersion(), BrokerFeatures.createEmpty());
+        ZkMetadataCache metadataCache = MetadataCache.zkMetadataCache(0,
+            config.interBrokerProtocolVersion(), BrokerFeatures.createEmpty(), null);
         metadataCache.updateMetadata(0, updateMetadataRequest);
 
         replicaManager = new ReplicaManagerBuilder().
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java
index 80376948b89..f65fdcdaa24 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java
@@ -110,7 +110,8 @@ public class MetadataRequestBenchmark {
     private KafkaZkClient kafkaZkClient = Mockito.mock(KafkaZkClient.class);
     private Metrics metrics = new Metrics();
     private int brokerId = 1;
-    private ZkMetadataCache metadataCache = MetadataCache.zkMetadataCache(brokerId, MetadataVersion.latest(), BrokerFeatures.createEmpty());
+    private ZkMetadataCache metadataCache = MetadataCache.zkMetadataCache(brokerId,
+        MetadataVersion.latest(), BrokerFeatures.createEmpty(), null);
     private ClientQuotaManager clientQuotaManager = Mockito.mock(ClientQuotaManager.class);
     private ClientRequestQuotaManager clientRequestQuotaManager = Mockito.mock(ClientRequestQuotaManager.class);
     private ControllerMutationQuotaManager controllerMutationQuotaManager = Mockito.mock(ControllerMutationQuotaManager.class);
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
index 9a60efbeabd..d83f748bcad 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
@@ -112,7 +112,9 @@ public class CheckpointBench {
         scheduler.startup();
         final BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
         final MetadataCache metadataCache =
-                MetadataCache.zkMetadataCache(this.brokerProperties.brokerId(), this.brokerProperties.interBrokerProtocolVersion(), BrokerFeatures.createEmpty());
+                MetadataCache.zkMetadataCache(this.brokerProperties.brokerId(),
+                    this.brokerProperties.interBrokerProtocolVersion(),
+                    BrokerFeatures.createEmpty(), null);
         this.quotaManagers =
                 QuotaFactory.instantiate(this.brokerProperties,
                         this.metrics,
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java
index efb69007888..3a8343772cb 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java
@@ -26,6 +26,7 @@ import kafka.server.BrokerFeatures;
 import kafka.server.BrokerTopicStats;
 import kafka.server.KafkaConfig;
 import kafka.server.LogDirFailureChannel;
+import kafka.server.MetadataCache;
 import kafka.server.QuotaFactory;
 import kafka.server.ReplicaManager;
 import kafka.server.builders.LogManagerBuilder;
@@ -33,7 +34,6 @@ import kafka.server.builders.ReplicaManagerBuilder;
 import kafka.server.checkpoints.OffsetCheckpoints;
 import kafka.server.metadata.ConfigRepository;
 import kafka.server.metadata.MockConfigRepository;
-import kafka.server.metadata.ZkMetadataCache;
 import kafka.utils.KafkaScheduler;
 import kafka.utils.Scheduler;
 import kafka.utils.TestUtils;
@@ -160,7 +160,9 @@ public class PartitionCreationBench {
             setLogManager(logManager).
             setQuotaManagers(quotaManagers).
             setBrokerTopicStats(brokerTopicStats).
-            setMetadataCache(new ZkMetadataCache(this.brokerProperties.brokerId(), this.brokerProperties.interBrokerProtocolVersion(), BrokerFeatures.createEmpty())).
+            setMetadataCache(MetadataCache.zkMetadataCache(this.brokerProperties.brokerId(),
+                this.brokerProperties.interBrokerProtocolVersion(), BrokerFeatures.createEmpty(),
+                null)).
             setLogDirFailureChannel(failureChannel).
             setAlterPartitionManager(alterPartitionManager).
             build();
diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java
index 9fb7c0ff4ea..15f8fee9bde 100644
--- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java
+++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java
@@ -243,6 +243,10 @@ public enum MetadataVersion {
         return this.isAtLeast(IBP_3_3_IV1);
     }
 
+    public boolean isApiForwardingEnabled() {
+        return this.isAtLeast(IBP_3_4_IV0);
+    }
+
     public boolean isKRaftSupported() {
         return this.featureLevel > 0;
     }


[kafka] 08/11: KAFKA-14448 Let ZK brokers register with KRaft controller (#12965)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1d528e395a9ec30311b980aeb1ae20c34aa623ed
Author: David Arthur <mu...@gmail.com>
AuthorDate: Tue Dec 13 16:15:21 2022 -0500

    KAFKA-14448 Let ZK brokers register with KRaft controller (#12965)
    
    Prior to starting a KIP-866 migration, the ZK brokers must register themselves with the active
    KRaft controller. The controller waits for all brokers to register in order to verify that all the
    brokers can
    
    A) Communicate with the quorum
    B) Have the migration config enabled
    C) Have the proper IBP set
    
    This patch uses the new isMigratingZkBroker field in BrokerRegistrationRequest and
    RegisterBrokerRecord. The type was changed from int8 to bool for BrokerRegistrationRequest (a
    mistake from #12860). The ZK brokers use the existing BrokerLifecycleManager class to register and
    heartbeat with the controllers.
    
    Reviewers: Mickael Maison <mi...@gmail.com>, Colin P. McCabe <cm...@apache.org>
---
 .../common/requests/BrokerRegistrationRequest.java |   9 +
 .../common/message/BrokerRegistrationRequest.json  |   4 +-
 .../common/message/BrokerRegistrationResponse.json |   2 +-
 .../kafka/server/BrokerLifecycleManager.scala      |  20 ++-
 .../server/BrokerToControllerChannelManager.scala  |   4 +-
 .../main/scala/kafka/server/ControllerServer.scala |   3 +-
 core/src/main/scala/kafka/server/KafkaConfig.scala |  11 +-
 core/src/main/scala/kafka/server/KafkaServer.scala | 111 ++++++++++--
 .../server/metadata/OffsetTrackingListener.scala   |  48 +++++
 core/src/main/scala/kafka/zk/KafkaZkClient.scala   |  19 +-
 core/src/test/java/kafka/test/ClusterInstance.java |   2 +
 .../test/junit/RaftClusterInvocationContext.java   |   8 +
 .../test/junit/ZkClusterInvocationContext.java     |   8 +-
 .../server/KafkaServerKRaftRegistrationTest.scala  |  85 +++++++++
 .../server/BrokerRegistrationRequestTest.scala     | 197 +++++++++++++++++++++
 .../kafka/controller/ClusterControlManager.java    |  33 +++-
 .../apache/kafka/controller/QuorumController.java  |  13 +-
 .../apache/kafka/metadata/BrokerRegistration.java  |  55 +++++-
 .../kafka/metadata/BrokerRegistrationTest.java     |   4 +-
 .../kafka/server/common/MetadataVersion.java       |   9 +-
 .../kafka/server/common/MetadataVersionTest.java   |  10 +-
 21 files changed, 608 insertions(+), 47 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java
index 2ba1529e72a..18d6a070d05 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java
@@ -34,6 +34,15 @@ public class BrokerRegistrationRequest extends AbstractRequest {
             this.data = data;
         }
 
+        @Override
+        public short oldestAllowedVersion() {
+            if (data.isMigratingZkBroker()) {
+                return (short) 1;
+            } else {
+                return (short) 0;
+            }
+        }
+
         @Override
         public BrokerRegistrationRequest build(short version) {
             return new BrokerRegistrationRequest(data, version);
diff --git a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
index c29e190dd01..19ad8f249b3 100644
--- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
+++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
@@ -18,7 +18,7 @@
   "type": "request",
   "listeners": ["controller"],
   "name": "BrokerRegistrationRequest",
-  "validVersions": "0",
+  "validVersions": "0-1",
   "flexibleVersions": "0+",
   "fields": [
     { "name": "BrokerId", "type": "int32", "versions": "0+", "entityType": "brokerId",
@@ -51,7 +51,7 @@
     },
     { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+",
       "about": "The rack which this broker is in." },
-    { "name": "IsMigratingZkBroker", "type": "int8", "versions": "0+", "taggedVersions": "0+", "tag": 0, "ignorable": true,
+    { "name": "IsMigratingZkBroker", "type": "bool", "versions": "1+", "default": "false",
       "about": "Set by a ZK broker if the required configurations for ZK migration are present." }
   ]
 }
diff --git a/clients/src/main/resources/common/message/BrokerRegistrationResponse.json b/clients/src/main/resources/common/message/BrokerRegistrationResponse.json
index 1f12123b04b..7515d5ee4bc 100644
--- a/clients/src/main/resources/common/message/BrokerRegistrationResponse.json
+++ b/clients/src/main/resources/common/message/BrokerRegistrationResponse.json
@@ -17,7 +17,7 @@
   "apiKey": 62,
   "type": "response",
   "name": "BrokerRegistrationResponse",
-  "validVersions": "0",
+  "validVersions": "0-1",
   "flexibleVersions": "0+",
   "fields": [
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
index 39dff71ad11..9f361275d4e 100644
--- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
+++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
@@ -51,9 +51,13 @@ import scala.jdk.CollectionConverters._
  * In some cases we expose a volatile variable which can be read from any thread, but only
  * written from the event queue thread.
  */
-class BrokerLifecycleManager(val config: KafkaConfig,
-                             val time: Time,
-                             val threadNamePrefix: Option[String]) extends Logging {
+class BrokerLifecycleManager(
+  val config: KafkaConfig,
+  val time: Time,
+  val threadNamePrefix: Option[String],
+  val isZkBroker: Boolean = false
+) extends Logging {
+
   val logContext = new LogContext(s"[BrokerLifecycleManager id=${config.nodeId}] ")
 
   this.logIdent = logContext.logPrefix()
@@ -266,9 +270,12 @@ class BrokerLifecycleManager(val config: KafkaConfig,
       _clusterId = clusterId
       _advertisedListeners = advertisedListeners.duplicate()
       _supportedFeatures = new util.HashMap[String, VersionRange](supportedFeatures)
-      eventQueue.scheduleDeferred("initialRegistrationTimeout",
-        new DeadlineFunction(time.nanoseconds() + initialTimeoutNs),
-        new RegistrationTimeoutEvent())
+      if (!isZkBroker) {
+        // ZK brokers don't block on registration during startup
+        eventQueue.scheduleDeferred("initialRegistrationTimeout",
+          new DeadlineFunction(time.nanoseconds() + initialTimeoutNs),
+          new RegistrationTimeoutEvent())
+      }
       sendBrokerRegistration()
       info(s"Incarnation $incarnationId of broker $nodeId in cluster $clusterId " +
         "is now STARTING.")
@@ -285,6 +292,7 @@ class BrokerLifecycleManager(val config: KafkaConfig,
     }
     val data = new BrokerRegistrationRequestData().
         setBrokerId(nodeId).
+        setIsMigratingZkBroker(isZkBroker).
         setClusterId(_clusterId).
         setFeatures(features).
         setIncarnationId(incarnationId).
diff --git a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala
index 92754a793f5..3d1e5d3f63c 100644
--- a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala
@@ -367,7 +367,7 @@ class BrokerToControllerRequestThread(
     if (activeControllerAddress().isDefined) {
       super.pollOnce(Long.MaxValue)
     } else {
-      debug("Controller isn't cached, looking for local metadata changes")
+      debug("Controller isn't known, checking with controller provider")
       controllerNodeProvider.get() match {
         case Some(controllerNode) =>
           info(s"Recorded new controller, from now on will use node $controllerNode")
@@ -375,7 +375,7 @@ class BrokerToControllerRequestThread(
           metadataUpdater.setNodes(Seq(controllerNode).asJava)
         case None =>
           // need to backoff to avoid tight loops
-          debug("No controller defined in metadata cache, retrying after backoff")
+          debug("No controller provided, retrying after backoff")
           super.pollOnce(maxTimeoutMs = 100)
       }
     }
diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
index 2bd518cde2a..03b86b7067e 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -204,7 +204,8 @@ class ControllerServer(
           setConfigurationValidator(new ControllerConfigurationValidator()).
           setStaticConfig(config.originals).
           setBootstrapMetadata(bootstrapMetadata).
-          setFatalFaultHandler(sharedServer.quorumControllerFaultHandler)
+          setFatalFaultHandler(sharedServer.quorumControllerFaultHandler).
+          setZkMigrationEnabled(config.migrationEnabled)
       }
       authorizer match {
         case Some(a: ClusterMetadataAuthorizer) => controllerBuilder.setAuthorizer(a)
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index 43ac98f53c8..7be37ee9e71 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -2230,8 +2230,15 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
       validateAdvertisedListenersNonEmptyForBroker()
     } else {
       // ZK-based
-      // controller listener names must be empty when not in KRaft mode
-      require(controllerListenerNames.isEmpty, s"${KafkaConfig.ControllerListenerNamesProp} must be empty when not running in KRaft mode: ${controllerListenerNames.asJava}")
+      if (migrationEnabled) {
+        validateNonEmptyQuorumVotersForKRaft()
+        require(controllerListenerNames.nonEmpty,
+          s"${KafkaConfig.ControllerListenerNamesProp} must not be empty when running in ZK migration mode: ${controllerListenerNames.asJava}")
+      } else {
+        // controller listener names must be empty when not in KRaft mode
+        require(controllerListenerNames.isEmpty,
+          s"${KafkaConfig.ControllerListenerNamesProp} must be empty when not running in KRaft mode: ${controllerListenerNames.asJava}")
+      }
       validateAdvertisedListenersNonEmptyForBroker()
     }
 
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index bc13a0986e9..79a621c6b54 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -21,7 +21,6 @@ import java.io.{File, IOException}
 import java.net.{InetAddress, SocketTimeoutException}
 import java.util.concurrent._
 import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
-
 import kafka.cluster.{Broker, EndPoint}
 import kafka.common.{GenerateBrokerIdException, InconsistentBrokerIdException, InconsistentClusterIdException}
 import kafka.controller.KafkaController
@@ -30,13 +29,15 @@ import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator}
 import kafka.log.LogManager
 import kafka.metrics.KafkaMetricsReporter
 import kafka.network.{ControlPlaneAcceptor, DataPlaneAcceptor, RequestChannel, SocketServer}
+import kafka.raft.KafkaRaftManager
 import kafka.security.CredentialProvider
-import kafka.server.metadata.{ZkConfigRepository, ZkMetadataCache}
+import kafka.server.metadata.{OffsetTrackingListener, ZkConfigRepository, ZkMetadataCache}
 import kafka.utils._
 import kafka.zk.{AdminZkClient, BrokerInfo, KafkaZkClient}
 import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient, NetworkClientUtils}
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.message.ApiMessageType.ListenerType
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.{Listener, ListenerCollection}
 import org.apache.kafka.common.message.ControlledShutdownRequestData
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.network._
@@ -47,8 +48,10 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok
 import org.apache.kafka.common.security.{JaasContext, JaasUtils}
 import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time, Utils}
 import org.apache.kafka.common.{Endpoint, Node}
-import org.apache.kafka.metadata.BrokerState
+import org.apache.kafka.metadata.{BrokerState, MetadataRecordSerde, VersionRange}
+import org.apache.kafka.raft.RaftConfig
 import org.apache.kafka.server.authorizer.Authorizer
+import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
 import org.apache.kafka.server.common.MetadataVersion._
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
 import org.apache.zookeeper.client.ZKClientConfig
@@ -181,6 +184,8 @@ class KafkaServer(
 
   def kafkaController: KafkaController = _kafkaController
 
+  var lifecycleManager: BrokerLifecycleManager = _
+
   /**
    * Start up API for bringing up a single instance of the Kafka server.
    * Instantiates the LogManager, the SocketServer and the request handlers - KafkaRequestHandlers
@@ -199,6 +204,11 @@ class KafkaServer(
       if (canStartup) {
         _brokerState = BrokerState.STARTING
 
+        lifecycleManager = new BrokerLifecycleManager(config,
+          time,
+          threadNamePrefix,
+          isZkBroker = true)
+
         /* setup zookeeper */
         initZkClient(time)
         configRepository = new ZkConfigRepository(new AdminZkClient(zkClient))
@@ -335,7 +345,65 @@ class KafkaServer(
         val brokerEpoch = zkClient.registerBroker(brokerInfo)
 
         // Now that the broker is successfully registered, checkpoint its metadata
-        checkpointBrokerMetadata(ZkMetaProperties(clusterId, config.brokerId))
+        val zkMetaProperties = ZkMetaProperties(clusterId, config.brokerId)
+        checkpointBrokerMetadata(zkMetaProperties)
+
+        if (config.migrationEnabled) {
+          // If the ZK broker is in migration mode, start up a RaftManager to learn about the new KRaft controller
+          val kraftMetaProps = MetaProperties(zkMetaProperties.clusterId, zkMetaProperties.brokerId)
+          val controllerQuorumVotersFuture = CompletableFuture.completedFuture(
+            RaftConfig.parseVoterConnections(config.quorumVoters))
+          val raftManager = new KafkaRaftManager[ApiMessageAndVersion](
+            kraftMetaProps,
+            config,
+            new MetadataRecordSerde,
+            KafkaRaftServer.MetadataPartition,
+            KafkaRaftServer.MetadataTopicId,
+            time,
+            metrics,
+            threadNamePrefix,
+            controllerQuorumVotersFuture
+          )
+          val controllerNodes = RaftConfig.voterConnectionsToNodes(controllerQuorumVotersFuture.get()).asScala
+          val quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes)
+          val brokerToQuorumChannelManager = BrokerToControllerChannelManager(
+            controllerNodeProvider = quorumControllerNodeProvider,
+            time = time,
+            metrics = metrics,
+            config = config,
+            channelName = "quorum",
+            threadNamePrefix = threadNamePrefix,
+            retryTimeoutMs = config.requestTimeoutMs.longValue
+          )
+
+          val listener = new OffsetTrackingListener()
+          raftManager.register(listener)
+
+          val networkListeners = new ListenerCollection()
+          config.effectiveAdvertisedListeners.foreach { ep =>
+            networkListeners.add(new Listener().
+              setHost(if (Utils.isBlank(ep.host)) InetAddress.getLocalHost.getCanonicalHostName else ep.host).
+              setName(ep.listenerName.value()).
+              setPort(if (ep.port == 0) socketServer.boundPort(ep.listenerName) else ep.port).
+              setSecurityProtocol(ep.securityProtocol.id))
+          }
+
+          // Even though ZK brokers don't use "metadata.version" feature, we send our IBP here as part of the broker registration
+          // so the KRaft controller can verify that all brokers are on the same IBP before starting the migration.
+          val ibpAsFeature =
+           java.util.Collections.singletonMap(MetadataVersion.FEATURE_NAME,
+             VersionRange.of(config.interBrokerProtocolVersion.featureLevel(), config.interBrokerProtocolVersion.featureLevel()))
+
+          lifecycleManager.start(
+            () => listener.highestOffset,
+            brokerToQuorumChannelManager,
+            kraftMetaProps.clusterId,
+            networkListeners,
+            ibpAsFeature
+          )
+
+          raftManager.startup()
+        }
 
         /* start token manager */
         tokenManager = new DelegationTokenManager(config, tokenCache, time , zkClient)
@@ -453,6 +521,17 @@ class KafkaServer(
         dynamicConfigManager = new ZkConfigManager(zkClient, dynamicConfigHandlers)
         dynamicConfigManager.startup()
 
+        if (config.migrationEnabled && lifecycleManager != null) {
+          lifecycleManager.initialCatchUpFuture.whenComplete { case (_, t) =>
+            if (t != null) {
+              fatal("Encountered an exception when waiting to catch up with KRaft metadata log", t)
+              shutdown()
+            } else {
+              info("Finished catching up on KRaft metadata log, requesting that the KRaft controller unfence this broker")
+              lifecycleManager.setReadyToUnfence()
+            }
+          }
+        }
         socketServer.enableRequestProcessing(authorizerFutures)
 
         _brokerState = BrokerState.RUNNING
@@ -491,17 +570,7 @@ class KafkaServer(
 
   private def initZkClient(time: Time): Unit = {
     info(s"Connecting to zookeeper on ${config.zkConnect}")
-
-    val secureAclsEnabled = config.zkEnableSecureAcls
-    val isZkSecurityEnabled = JaasUtils.isZkSaslEnabled() || KafkaConfig.zkTlsClientAuthEnabled(zkClientConfig)
-
-    if (secureAclsEnabled && !isZkSecurityEnabled)
-      throw new java.lang.SecurityException(s"${KafkaConfig.ZkEnableSecureAclsProp} is true, but ZooKeeper client TLS configuration identifying at least $KafkaConfig.ZkSslClientEnableProp, $KafkaConfig.ZkClientCnxnSocketProp, and $KafkaConfig.ZkSslKeyStoreLocationProp was not present and the " +
-        s"verification of the JAAS login file failed ${JaasUtils.zkSecuritySysConfigString}")
-
-    _zkClient = KafkaZkClient(config.zkConnect, secureAclsEnabled, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs,
-      config.zkMaxInFlightRequests, time, name = "Kafka server", zkClientConfig = zkClientConfig,
-      createChrootIfNecessary = true)
+    _zkClient = KafkaZkClient.createZkClient("Kafka server", time, config, zkClientConfig)
     _zkClient.createTopLevelPaths()
   }
 
@@ -532,6 +601,7 @@ class KafkaServer(
     )
 
     val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt
+
     BrokerInfo(
       Broker(config.brokerId, updatedEndpoints, config.rack, brokerFeatures.supportedFeatures),
       config.interBrokerProtocolVersion,
@@ -690,6 +760,14 @@ class KafkaServer(
 
       _brokerState = BrokerState.PENDING_CONTROLLED_SHUTDOWN
 
+      if (config.migrationEnabled && lifecycleManager != null) {
+        // TODO KAFKA-14447 Only use KRaft controlled shutdown (when in migration mode)
+        // For now we'll send the heartbeat with WantShutDown set so the KRaft controller can see a broker
+        // shutting down without waiting for the heartbeat to time out.
+        info("Notifying KRaft of controlled shutdown")
+        lifecycleManager.beginControlledShutdown()
+      }
+
       val shutdownSucceeded = doControlledShutdown(config.controlledShutdownMaxRetries.intValue)
 
       if (!shutdownSucceeded)
@@ -793,6 +871,9 @@ class KafkaServer(
         // Clear all reconfigurable instances stored in DynamicBrokerConfig
         config.dynamicConfig.clear()
 
+        if (lifecycleManager != null) {
+          lifecycleManager.close()
+        }
         _brokerState = BrokerState.NOT_RUNNING
 
         startupComplete.set(false)
diff --git a/core/src/main/scala/kafka/server/metadata/OffsetTrackingListener.scala b/core/src/main/scala/kafka/server/metadata/OffsetTrackingListener.scala
new file mode 100644
index 00000000000..056b6f0fec1
--- /dev/null
+++ b/core/src/main/scala/kafka/server/metadata/OffsetTrackingListener.scala
@@ -0,0 +1,48 @@
+/*
+ * 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 kafka.server.metadata
+
+import org.apache.kafka.raft.{BatchReader, RaftClient}
+import org.apache.kafka.server.common.ApiMessageAndVersion
+import org.apache.kafka.snapshot.SnapshotReader
+
+/**
+ *  A simple Raft listener that only keeps track of the highest offset seen. Used for registration of ZK
+ *  brokers with the KRaft controller during a KIP-866 migration.
+ */
+class OffsetTrackingListener extends RaftClient.Listener[ApiMessageAndVersion] {
+  @volatile var _highestOffset = 0L
+
+  def highestOffset: Long = _highestOffset
+
+  override def handleCommit(reader: BatchReader[ApiMessageAndVersion]): Unit = {
+    reader.lastOffset()
+    var index = 0
+    while (reader.hasNext) {
+      index += 1
+      reader.next()
+    }
+    _highestOffset = reader.lastOffset().orElse(reader.baseOffset() + index)
+    reader.close()
+  }
+
+  override def handleSnapshot(reader: SnapshotReader[ApiMessageAndVersion]): Unit = {
+    _highestOffset = reader.lastContainedLogOffset()
+    reader.close()
+  }
+}
diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala
index 12f4bfb2c3e..115446572e1 100644
--- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala
+++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala
@@ -25,12 +25,13 @@ import kafka.log.LogConfig
 import kafka.metrics.KafkaMetricsGroup
 import kafka.security.authorizer.AclAuthorizer.{NoAcls, VersionedAcls}
 import kafka.security.authorizer.AclEntry
-import kafka.server.ConfigType
+import kafka.server.{ConfigType, KafkaConfig}
 import kafka.utils.Logging
 import kafka.zk.TopicZNode.TopicIdReplicaAssignment
 import kafka.zookeeper._
 import org.apache.kafka.common.errors.ControllerMovedException
 import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType}
+import org.apache.kafka.common.security.JaasUtils
 import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation}
 import org.apache.kafka.common.utils.{Time, Utils}
 import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
@@ -2312,4 +2313,20 @@ object KafkaZkClient {
       case _ => throw new IllegalStateException(s"Cannot unwrap $response because it is not a MultiResponse")
     }
   }
+
+  def createZkClient(name: String, time: Time, config: KafkaConfig, zkClientConfig: ZKClientConfig): KafkaZkClient = {
+    val secureAclsEnabled = config.zkEnableSecureAcls
+    val isZkSecurityEnabled = JaasUtils.isZkSaslEnabled || KafkaConfig.zkTlsClientAuthEnabled(zkClientConfig)
+
+    if (secureAclsEnabled && !isZkSecurityEnabled)
+      throw new java.lang.SecurityException(
+        s"${KafkaConfig.ZkEnableSecureAclsProp} is true, but ZooKeeper client TLS configuration identifying at least " +
+          s"${KafkaConfig.ZkSslClientEnableProp}, ${KafkaConfig.ZkClientCnxnSocketProp}, and " +
+          s"${KafkaConfig.ZkSslKeyStoreLocationProp} was not present and the verification of the JAAS login file failed " +
+          s"${JaasUtils.zkSecuritySysConfigString}")
+
+    KafkaZkClient(config.zkConnect, secureAclsEnabled, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs,
+      config.zkMaxInFlightRequests, time, name = name, zkClientConfig = zkClientConfig,
+      createChrootIfNecessary = true)
+  }
 }
diff --git a/core/src/test/java/kafka/test/ClusterInstance.java b/core/src/test/java/kafka/test/ClusterInstance.java
index 9058508fa94..f149c82e7c3 100644
--- a/core/src/test/java/kafka/test/ClusterInstance.java
+++ b/core/src/test/java/kafka/test/ClusterInstance.java
@@ -115,6 +115,8 @@ public interface ClusterInstance {
      */
     Map<Integer, BrokerFeatures> brokerFeatures();
 
+    String clusterId();
+
     /**
      * The underlying object which is responsible for setting up and tearing down the cluster.
      */
diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
index f5c281ff249..f7eb0a50179 100644
--- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
+++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
@@ -183,6 +183,14 @@ public class RaftClusterInvocationContext implements TestTemplateInvocationConte
             ));
         }
 
+        @Override
+        public String clusterId() {
+            return controllers().findFirst().map(ControllerServer::clusterId).orElse(
+                brokers().findFirst().map(BrokerServer::clusterId).orElseThrow(
+                    () -> new RuntimeException("No controllers or brokers!"))
+            );
+        }
+
         public Collection<ControllerServer> controllerServers() {
             return controllers().collect(Collectors.toList());
         }
diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java
index 18a85e2d7bf..e77ef4a3ce8 100644
--- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java
+++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java
@@ -237,7 +237,7 @@ public class ZkClusterInvocationContext implements TestTemplateInvocationContext
                 .filter(broker -> broker.kafkaController().isActive())
                 .map(KafkaServer::socketServer)
                 .findFirst()
-                .orElseThrow(() -> new RuntimeException("No broker SocketServers found"));
+                .orElseThrow(() -> new RuntimeException("No controller SocketServers found"));
         }
 
         @Override
@@ -248,6 +248,12 @@ public class ZkClusterInvocationContext implements TestTemplateInvocationContext
             ));
         }
 
+        @Override
+        public String clusterId() {
+            return servers().findFirst().map(KafkaServer::clusterId).orElseThrow(
+                () -> new RuntimeException("No broker instances found"));
+        }
+
         @Override
         public ClusterType clusterType() {
             return ClusterType.ZK;
diff --git a/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala b/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala
new file mode 100644
index 00000000000..8b85a8f81cc
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala
@@ -0,0 +1,85 @@
+/*
+ * 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 kafka.server
+
+import kafka.test.ClusterInstance
+import kafka.test.annotation.{ClusterTest, Type}
+import kafka.test.junit.ClusterTestExtensions
+import kafka.testkit.{KafkaClusterTestKit, TestKitNodes}
+import org.apache.kafka.common.Uuid
+import org.apache.kafka.raft.RaftConfig
+import org.apache.kafka.server.common.MetadataVersion
+import org.junit.jupiter.api.Assertions.fail
+import org.junit.jupiter.api.extension.ExtendWith
+import org.junit.jupiter.api.{Tag, Timeout}
+
+import java.util.concurrent.{TimeUnit, TimeoutException}
+import scala.jdk.CollectionConverters._
+
+
+/**
+ * This test creates a full ZK cluster and a controller-only KRaft cluster and configures the ZK brokers to register
+ * themselves with the KRaft controller. This is mainly a happy-path test since the only way to reliably test the
+ * failure paths is to use timeouts. See {@link unit.kafka.server.BrokerRegistrationRequestTest} for integration test
+ * of just the broker registration path.
+ */
+@Timeout(120)
+@Tag("integration")
+@ExtendWith(value = Array(classOf[ClusterTestExtensions]))
+class KafkaServerKRaftRegistrationTest {
+
+  @ClusterTest(clusterType = Type.ZK, brokers = 3, metadataVersion = MetadataVersion.IBP_3_4_IV0)
+  def testRegisterZkBrokerInKraft1(zkCluster: ClusterInstance): Unit = {
+    val clusterId = zkCluster.clusterId()
+
+    // Bootstrap the ZK cluster ID into KRaft
+    val kraftCluster = new KafkaClusterTestKit.Builder(
+      new TestKitNodes.Builder().
+        setBootstrapMetadataVersion(MetadataVersion.IBP_3_4_IV0).
+        setClusterId(Uuid.fromString(clusterId)).
+        setNumBrokerNodes(0).
+        setNumControllerNodes(1).build())
+      .setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
+      .build()
+    try {
+      kraftCluster.format()
+      kraftCluster.startup()
+      val readyFuture = kraftCluster.controllers().values().asScala.head.controller.waitForReadyBrokers(3)
+
+      // Enable migration configs and restart brokers
+      val props = kraftCluster.controllerClientProperties()
+      val voters = props.get(RaftConfig.QUORUM_VOTERS_CONFIG)
+      zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true")
+      zkCluster.config().serverProperties().put(RaftConfig.QUORUM_VOTERS_CONFIG, voters)
+      zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
+      zkCluster.config().serverProperties().put(KafkaConfig.ListenerSecurityProtocolMapProp, "CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT")
+      zkCluster.rollingBrokerRestart()
+      zkCluster.waitForReadyBrokers()
+
+      try {
+        // Wait until all three ZK brokers are registered with KRaft controller
+        readyFuture.get(30, TimeUnit.SECONDS)
+      } catch {
+        case _: TimeoutException => fail("Did not see 3 brokers within 30 seconds")
+        case t: Throwable => fail("Had some other error waiting for brokers", t)
+      }
+    } finally {
+      kraftCluster.close()
+    }
+  }
+}
diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala
new file mode 100644
index 00000000000..2bb1314a789
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala
@@ -0,0 +1,197 @@
+/*
+ * 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 unit.kafka.server
+
+import kafka.server.{BrokerToControllerChannelManager, ControllerNodeProvider, ControllerRequestCompletionHandler}
+import kafka.test.ClusterInstance
+import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, Type}
+import kafka.test.junit.ClusterTestExtensions
+import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.message.{BrokerRegistrationRequestData, BrokerRegistrationResponseData}
+import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.network.ListenerName
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{BrokerRegistrationRequest, BrokerRegistrationResponse}
+import org.apache.kafka.common.security.auth.SecurityProtocol
+import org.apache.kafka.common.utils.Time
+import org.apache.kafka.common.{Node, Uuid}
+import org.apache.kafka.server.common.MetadataVersion
+import org.junit.jupiter.api.Assertions.assertEquals
+import org.junit.jupiter.api.extension.ExtendWith
+import org.junit.jupiter.api.{Tag, Timeout}
+
+import java.util.concurrent.{CompletableFuture, TimeUnit, TimeoutException}
+
+/**
+ * This test simulates a broker registering with the KRaft quorum under different configurations.
+ */
+@Timeout(120)
+@Tag("integration")
+@ExtendWith(value = Array(classOf[ClusterTestExtensions]))
+class BrokerRegistrationRequestTest {
+
+  def brokerToControllerChannelManager(clusterInstance: ClusterInstance): BrokerToControllerChannelManager = {
+    BrokerToControllerChannelManager(
+      new ControllerNodeProvider() {
+        override def get(): Option[Node] = Some(new Node(
+          clusterInstance.anyControllerSocketServer().config.nodeId,
+          "127.0.0.1",
+          clusterInstance.anyControllerSocketServer().boundPort(clusterInstance.controllerListenerName().get()),
+        ))
+
+        override def listenerName: ListenerName = clusterInstance.controllerListenerName().get()
+
+        override def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT
+
+        override def saslMechanism: String = ""
+      },
+      Time.SYSTEM,
+      new Metrics(),
+      clusterInstance.anyControllerSocketServer().config,
+      "heartbeat",
+      Some("heartbeat"),
+      10000
+    )
+  }
+
+  def sendAndRecieve(
+    channelManager: BrokerToControllerChannelManager,
+    req: BrokerRegistrationRequestData
+  ): BrokerRegistrationResponseData = {
+    val responseFuture = new CompletableFuture[BrokerRegistrationResponseData]()
+    channelManager.sendRequest(new BrokerRegistrationRequest.Builder(req), new ControllerRequestCompletionHandler() {
+      override def onTimeout(): Unit = responseFuture.completeExceptionally(new TimeoutException())
+
+      override def onComplete(response: ClientResponse): Unit =
+        responseFuture.complete(response.responseBody().asInstanceOf[BrokerRegistrationResponse].data())
+    })
+    responseFuture.get(30, TimeUnit.SECONDS)
+  }
+
+  def registerBroker(
+    channelManager: BrokerToControllerChannelManager,
+    clusterId: String,
+    brokerId: Int,
+    zk: Boolean,
+    ibpToSend: Option[(MetadataVersion, MetadataVersion)]
+  ): Errors = {
+    val features = new BrokerRegistrationRequestData.FeatureCollection()
+
+    ibpToSend foreach {
+      case (min, max) =>
+        features.add(new BrokerRegistrationRequestData.Feature()
+          .setName(MetadataVersion.FEATURE_NAME)
+          .setMinSupportedVersion(min.featureLevel())
+          .setMaxSupportedVersion(max.featureLevel())
+        )
+    }
+
+    val req = new BrokerRegistrationRequestData()
+      .setBrokerId(brokerId)
+      .setClusterId(clusterId)
+      .setIncarnationId(Uuid.randomUuid())
+      .setIsMigratingZkBroker(zk)
+      .setFeatures(features)
+
+    Errors.forCode(sendAndRecieve(channelManager, req).errorCode())
+  }
+
+  @ClusterTest(clusterType = Type.KRAFT, brokers = 0, controllers = 1, metadataVersion = MetadataVersion.IBP_3_4_IV0,
+    serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "false")))
+  def testRegisterZkWithKRaftMigrationDisabled(clusterInstance: ClusterInstance): Unit = {
+    val clusterId = clusterInstance.clusterId()
+    val channelManager = brokerToControllerChannelManager(clusterInstance)
+    try {
+      channelManager.start()
+
+      assertEquals(
+        Errors.BROKER_ID_NOT_REGISTERED,
+        registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_3_IV0, MetadataVersion.IBP_3_3_IV0))))
+
+      assertEquals(
+        Errors.BROKER_ID_NOT_REGISTERED,
+        registerBroker(channelManager, clusterId, 100, true, None))
+
+      assertEquals(
+        Errors.BROKER_ID_NOT_REGISTERED,
+        registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0))))
+
+      assertEquals(
+        Errors.NONE,
+        registerBroker(channelManager, clusterId, 100, false, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0))))
+    } finally {
+      channelManager.shutdown()
+    }
+  }
+
+  @ClusterTest(clusterType = Type.KRAFT, brokers = 0, controllers = 1, metadataVersion = MetadataVersion.IBP_3_3_IV3,
+    serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true")))
+  def testRegisterZkWithKRaftOldMetadataVersion(clusterInstance: ClusterInstance): Unit = {
+    val clusterId = clusterInstance.clusterId()
+    val channelManager = brokerToControllerChannelManager(clusterInstance)
+    try {
+      channelManager.start()
+
+      assertEquals(
+        Errors.BROKER_ID_NOT_REGISTERED,
+        registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_3_IV0, MetadataVersion.IBP_3_3_IV0))))
+
+      assertEquals(
+        Errors.BROKER_ID_NOT_REGISTERED,
+        registerBroker(channelManager, clusterId, 100, true, None))
+
+      assertEquals(
+        Errors.BROKER_ID_NOT_REGISTERED,
+        registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0))))
+
+      assertEquals(
+        Errors.NONE,
+        registerBroker(channelManager, clusterId, 100, false, Some((MetadataVersion.IBP_3_3_IV3, MetadataVersion.IBP_3_4_IV0))))
+    } finally {
+      channelManager.shutdown()
+    }
+  }
+
+  @ClusterTest(clusterType = Type.KRAFT, brokers = 0, controllers = 1, metadataVersion = MetadataVersion.IBP_3_4_IV0,
+    serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true")))
+  def testRegisterZkWithKRaftMigrationEnabled(clusterInstance: ClusterInstance): Unit = {
+    val clusterId = clusterInstance.clusterId()
+    val channelManager = brokerToControllerChannelManager(clusterInstance)
+    try {
+      channelManager.start()
+
+      assertEquals(
+        Errors.NONE,
+        registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0))))
+
+      assertEquals(
+        Errors.UNSUPPORTED_VERSION,
+        registerBroker(channelManager, clusterId, 100, true, None))
+
+      assertEquals(
+        Errors.UNSUPPORTED_VERSION,
+        registerBroker(channelManager, clusterId, 100, true, Some((MetadataVersion.IBP_3_3_IV3, MetadataVersion.IBP_3_3_IV3))))
+
+      assertEquals(
+        Errors.NONE,
+        registerBroker(channelManager, clusterId, 100, false, Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0))))
+    } finally {
+      channelManager.shutdown()
+    }
+  }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
index 5bc5bd2c3af..95aae773fb4 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
@@ -19,6 +19,7 @@ package org.apache.kafka.controller;
 
 import org.apache.kafka.common.Endpoint;
 import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
 import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
 import org.apache.kafka.common.errors.InconsistentClusterIdException;
 import org.apache.kafka.common.errors.StaleBrokerEpochException;
@@ -88,6 +89,7 @@ public class ClusterControlManager {
         private ReplicaPlacer replicaPlacer = null;
         private ControllerMetrics controllerMetrics = null;
         private FeatureControlManager featureControl = null;
+        private boolean zkMigrationEnabled = false;
 
         Builder setLogContext(LogContext logContext) {
             this.logContext = logContext;
@@ -129,6 +131,11 @@ public class ClusterControlManager {
             return this;
         }
 
+        Builder setZkMigrationEnabled(boolean zkMigrationEnabled) {
+            this.zkMigrationEnabled = zkMigrationEnabled;
+            return this;
+        }
+
         ClusterControlManager build() {
             if (logContext == null) {
                 logContext = new LogContext();
@@ -155,7 +162,8 @@ public class ClusterControlManager {
                 sessionTimeoutNs,
                 replicaPlacer,
                 controllerMetrics,
-                featureControl
+                featureControl,
+                zkMigrationEnabled
             );
         }
     }
@@ -247,6 +255,8 @@ public class ClusterControlManager {
      */
     private final FeatureControlManager featureControl;
 
+    private final boolean zkMigrationEnabled;
+
     private ClusterControlManager(
         LogContext logContext,
         String clusterId,
@@ -255,7 +265,8 @@ public class ClusterControlManager {
         long sessionTimeoutNs,
         ReplicaPlacer replicaPlacer,
         ControllerMetrics metrics,
-        FeatureControlManager featureControl
+        FeatureControlManager featureControl,
+        boolean zkMigrationEnabled
     ) {
         this.logContext = logContext;
         this.clusterId = clusterId;
@@ -269,6 +280,7 @@ public class ClusterControlManager {
         this.readyBrokersFuture = Optional.empty();
         this.controllerMetrics = metrics;
         this.featureControl = featureControl;
+        this.zkMigrationEnabled = zkMigrationEnabled;
     }
 
     ReplicaPlacer replicaPlacer() {
@@ -311,6 +323,10 @@ public class ClusterControlManager {
             .collect(Collectors.toSet());
     }
 
+    boolean zkRegistrationAllowed() {
+        return zkMigrationEnabled && featureControl.metadataVersion().isMigrationSupported();
+    }
+
     /**
      * Process an incoming broker registration request.
      */
@@ -341,7 +357,13 @@ public class ClusterControlManager {
             }
         }
 
-        RegisterBrokerRecord record = new RegisterBrokerRecord().setBrokerId(brokerId).
+        if (request.isMigratingZkBroker() && !zkRegistrationAllowed()) {
+            throw new BrokerIdNotRegisteredException("Controller does not support registering ZK brokers.");
+        }
+
+        RegisterBrokerRecord record = new RegisterBrokerRecord().
+            setBrokerId(brokerId).
+            setIsMigratingZkBroker(request.isMigratingZkBroker()).
             setIncarnationId(request.incarnationId()).
             setBrokerEpoch(brokerEpoch).
             setRack(request.rack());
@@ -423,7 +445,7 @@ public class ClusterControlManager {
                 new BrokerRegistration(brokerId, record.brokerEpoch(),
                     record.incarnationId(), listeners, features,
                     Optional.ofNullable(record.rack()), record.fenced(),
-                    record.inControlledShutdown()));
+                    record.inControlledShutdown(), record.isMigratingZkBroker()));
         updateMetrics(prevRegistration, brokerRegistrations.get(brokerId));
         if (heartbeatManager != null) {
             if (prevRegistration != null) heartbeatManager.remove(brokerId);
@@ -677,6 +699,9 @@ public class ClusterControlManager {
             if (metadataVersion.isInControlledShutdownStateSupported()) {
                 record.setInControlledShutdown(registration.inControlledShutdown());
             }
+            if (metadataVersion.isMigrationSupported()) {
+                record.setIsMigratingZkBroker(registration.isMigratingZkBroker());
+            }
             return singletonList(new ApiMessageAndVersion(record,
                 metadataVersion.registerBrokerRecordVersion()));
         }
diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
index 3dba5b401d6..6716044bef5 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
@@ -181,6 +181,7 @@ public final class QuorumController implements Controller {
         private Map<String, Object> staticConfig = Collections.emptyMap();
         private BootstrapMetadata bootstrapMetadata = null;
         private int maxRecordsPerBatch = MAX_RECORDS_PER_BATCH;
+        private boolean zkMigrationEnabled = false;
 
         public Builder(int nodeId, String clusterId) {
             this.nodeId = nodeId;
@@ -306,6 +307,11 @@ public final class QuorumController implements Controller {
             return this;
         }
 
+        public Builder setZkMigrationEnabled(boolean zkMigrationEnabled) {
+            this.zkMigrationEnabled = zkMigrationEnabled;
+            return this;
+        }
+
         @SuppressWarnings("unchecked")
         public QuorumController build() throws Exception {
             if (raftClient == null) {
@@ -357,7 +363,8 @@ public final class QuorumController implements Controller {
                     authorizer,
                     staticConfig,
                     bootstrapMetadata,
-                    maxRecordsPerBatch
+                    maxRecordsPerBatch,
+                    zkMigrationEnabled
                 );
             } catch (Exception e) {
                 Utils.closeQuietly(queue, "event queue");
@@ -1848,7 +1855,8 @@ public final class QuorumController implements Controller {
         Optional<ClusterMetadataAuthorizer> authorizer,
         Map<String, Object> staticConfig,
         BootstrapMetadata bootstrapMetadata,
-        int maxRecordsPerBatch
+        int maxRecordsPerBatch,
+        boolean zkMigrationEnabled
     ) {
         this.fatalFaultHandler = fatalFaultHandler;
         this.logContext = logContext;
@@ -1892,6 +1900,7 @@ public final class QuorumController implements Controller {
             setReplicaPlacer(replicaPlacer).
             setControllerMetrics(controllerMetrics).
             setFeatureControlManager(featureControl).
+            setZkMigrationEnabled(zkMigrationEnabled).
             build();
         this.producerIdControlManager = new ProducerIdControlManager(clusterControl, snapshotRegistry);
         this.snapshotMaxNewRecordBytes = snapshotMaxNewRecordBytes;
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java
index bf895fb8a11..6aa40195070 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java
@@ -57,7 +57,9 @@ public class BrokerRegistration {
     private final Optional<String> rack;
     private final boolean fenced;
     private final boolean inControlledShutdown;
+    private final boolean isMigratingZkBroker;
 
+    // Visible for testing
     public BrokerRegistration(int id,
                               long epoch,
                               Uuid incarnationId,
@@ -67,9 +69,23 @@ public class BrokerRegistration {
                               boolean fenced,
                               boolean inControlledShutdown) {
         this(id, epoch, incarnationId, listenersToMap(listeners), supportedFeatures, rack,
-            fenced, inControlledShutdown);
+            fenced, inControlledShutdown, false);
     }
 
+    public BrokerRegistration(int id,
+                              long epoch,
+                              Uuid incarnationId,
+                              List<Endpoint> listeners,
+                              Map<String, VersionRange> supportedFeatures,
+                              Optional<String> rack,
+                              boolean fenced,
+                              boolean inControlledShutdown,
+                              boolean isMigratingZkBroker) {
+        this(id, epoch, incarnationId, listenersToMap(listeners), supportedFeatures, rack,
+            fenced, inControlledShutdown, isMigratingZkBroker);
+    }
+
+    // Visible for testing
     public BrokerRegistration(int id,
                               long epoch,
                               Uuid incarnationId,
@@ -78,6 +94,18 @@ public class BrokerRegistration {
                               Optional<String> rack,
                               boolean fenced,
                               boolean inControlledShutdown) {
+        this(id, epoch, incarnationId, listeners, supportedFeatures, rack, fenced, inControlledShutdown, false);
+    }
+
+    public BrokerRegistration(int id,
+                              long epoch,
+                              Uuid incarnationId,
+                              Map<String, Endpoint> listeners,
+                              Map<String, VersionRange> supportedFeatures,
+                              Optional<String> rack,
+                              boolean fenced,
+                              boolean inControlledShutdown,
+                              boolean isMigratingZkBroker) {
         this.id = id;
         this.epoch = epoch;
         this.incarnationId = incarnationId;
@@ -95,6 +123,7 @@ public class BrokerRegistration {
         this.rack = rack;
         this.fenced = fenced;
         this.inControlledShutdown = inControlledShutdown;
+        this.isMigratingZkBroker = isMigratingZkBroker;
     }
 
     public static BrokerRegistration fromRecord(RegisterBrokerRecord record) {
@@ -117,7 +146,8 @@ public class BrokerRegistration {
             supportedFeatures,
             Optional.ofNullable(record.rack()),
             record.fenced(),
-            record.inControlledShutdown());
+            record.inControlledShutdown(),
+            record.isMigratingZkBroker());
     }
 
     public int id() {
@@ -160,6 +190,10 @@ public class BrokerRegistration {
         return inControlledShutdown;
     }
 
+    public boolean isMigratingZkBroker() {
+        return isMigratingZkBroker;
+    }
+
     public ApiMessageAndVersion toRecord(ImageWriterOptions options) {
         RegisterBrokerRecord registrationRecord = new RegisterBrokerRecord().
             setBrokerId(id).
@@ -176,6 +210,14 @@ public class BrokerRegistration {
             }
         }
 
+        if (isMigratingZkBroker) {
+            if (options.metadataVersion().isMigrationSupported()) {
+                registrationRecord.setIsMigratingZkBroker(isMigratingZkBroker);
+            } else {
+                options.handleLoss("the isMigratingZkBroker state of one or more brokers");
+            }
+        }
+
         for (Entry<String, Endpoint> entry : listeners.entrySet()) {
             Endpoint endpoint = entry.getValue();
             registrationRecord.endPoints().add(new BrokerEndpoint().
@@ -199,7 +241,7 @@ public class BrokerRegistration {
     @Override
     public int hashCode() {
         return Objects.hash(id, epoch, incarnationId, listeners, supportedFeatures,
-            rack, fenced, inControlledShutdown);
+            rack, fenced, inControlledShutdown, isMigratingZkBroker);
     }
 
     @Override
@@ -213,7 +255,8 @@ public class BrokerRegistration {
             other.supportedFeatures.equals(supportedFeatures) &&
             other.rack.equals(rack) &&
             other.fenced == fenced &&
-            other.inControlledShutdown == inControlledShutdown;
+            other.inControlledShutdown == inControlledShutdown &&
+            other.isMigratingZkBroker == isMigratingZkBroker;
     }
 
     @Override
@@ -234,6 +277,7 @@ public class BrokerRegistration {
         bld.append(", rack=").append(rack);
         bld.append(", fenced=").append(fenced);
         bld.append(", inControlledShutdown=").append(inControlledShutdown);
+        bld.append(", isMigratingZkBroker=").append(isMigratingZkBroker);
         bld.append(")");
         return bld.toString();
     }
@@ -256,7 +300,8 @@ public class BrokerRegistration {
             supportedFeatures,
             rack,
             newFenced,
-            newInControlledShutdownChange
+            newInControlledShutdownChange,
+            isMigratingZkBroker
         );
     }
 }
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java
index 4844b765007..afdb15e7272 100644
--- a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java
+++ b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java
@@ -80,13 +80,13 @@ public class BrokerRegistrationTest {
             "incarnationId=3MfdxWlNSn2UDYsmDP1pYg, listeners=[Endpoint(" +
             "listenerName='INTERNAL', securityProtocol=PLAINTEXT, " +
             "host='localhost', port=9091)], supportedFeatures={foo: 1-2}, " +
-            "rack=Optional.empty, fenced=true, inControlledShutdown=false)",
+            "rack=Optional.empty, fenced=true, inControlledShutdown=false, isMigratingZkBroker=false)",
             REGISTRATIONS.get(1).toString());
         assertEquals("BrokerRegistration(id=2, epoch=0, " +
             "incarnationId=eY7oaG1RREie5Kk9uy1l6g, listeners=[Endpoint(" +
             "listenerName='INTERNAL', securityProtocol=PLAINTEXT, " +
             "host='localhost', port=9092)], supportedFeatures={bar: 1-4, foo: 2-3}, " +
-            "rack=Optional[myrack], fenced=false, inControlledShutdown=true)",
+            "rack=Optional[myrack], fenced=false, inControlledShutdown=true, isMigratingZkBroker=false)",
             REGISTRATIONS.get(2).toString());
     }
 
diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java
index ffe7f1e316f..9fb7c0ff4ea 100644
--- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java
+++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java
@@ -265,8 +265,15 @@ public enum MetadataVersion {
         return this.isAtLeast(IBP_3_3_IV3);
     }
 
+    public boolean isMigrationSupported() {
+        return this.isAtLeast(MetadataVersion.IBP_3_4_IV0);
+    }
+
     public short registerBrokerRecordVersion() {
-        if (isInControlledShutdownStateSupported()) {
+        if (isMigrationSupported()) {
+            // new isMigrationZkBroker field
+            return (short) 2;
+        } else if (isInControlledShutdownStateSupported()) {
             return (short) 1;
         } else {
             return (short) 0;
diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java
index 51933c5ea01..6c76a82bfa6 100644
--- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java
+++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java
@@ -332,8 +332,14 @@ class MetadataVersionTest {
     @ParameterizedTest
     @EnumSource(value = MetadataVersion.class)
     public void testRegisterBrokerRecordVersion(MetadataVersion metadataVersion) {
-        short expectedVersion = metadataVersion.isAtLeast(IBP_3_3_IV3) ?
-            (short) 1 : (short) 0;
+        final short expectedVersion;
+        if (metadataVersion.isAtLeast(MetadataVersion.IBP_3_4_IV0)) {
+            expectedVersion = 2;
+        } else if (metadataVersion.isAtLeast(IBP_3_3_IV3)) {
+            expectedVersion = 1;
+        } else {
+            expectedVersion = 0;
+        }
         assertEquals(expectedVersion, metadataVersion.registerBrokerRecordVersion());
     }
 }


[kafka] 03/11: MINOR: Move MetadataQuorumCommand from `core` to `tools` (#12951)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e49132326e14992b1d1462e225ddb322c973ed0f
Author: Ismael Juma <is...@juma.me.uk>
AuthorDate: Fri Dec 9 09:22:58 2022 -0800

    MINOR: Move MetadataQuorumCommand from `core` to `tools` (#12951)
    
    `core` should only be  used for legacy cli tools and tools that require
    access to `core` classes instead of communicating via the kafka protocol
    (typically by using the client classes).
    
    Summary of changes:
    1. Convert the command implementation and tests to Java and move it to
        the `tools` module.
    2. Introduce mechanism to capture stdout and stderr from tests.
    3. Change `kafka-metadata-quorum.sh` to point to the new command class.
    4. Adjusted the test classpath of the `tools` module so that it supports tests
        that rely on the `@ClusterTests` annotation.
    5. Improved error handling when an exception different from `TerseFailure` is
        thrown.
    6. Changed `ToolsUtils` to avoid usage of arrays in favor of `List`.
    
    Reviewers: dengziming <de...@gmail.com>
---
 bin/kafka-metadata-quorum.sh                       |   2 +-
 bin/windows/kafka-metatada-quorum.bat              |   2 +-
 build.gradle                                       |   7 +-
 checkstyle/import-control.xml                      |   1 +
 .../java/org/apache/kafka/common/utils/Utils.java  |  14 +-
 .../org/apache/kafka/common/utils/UtilsTest.java   |   7 +
 .../scala/kafka/admin/MetadataQuorumCommand.scala  | 172 ------------------
 .../kafka/admin/MetadataQuorumCommandTest.scala    | 192 --------------------
 .../org/apache/kafka/server/util/ToolsUtils.java   |  19 +-
 .../apache/kafka/tools/MetadataQuorumCommand.java  | 195 +++++++++++++++++++++
 .../org/apache/kafka/tools/TerseException.java     |  33 ++++
 .../apache/kafka/tools/TransactionsCommand.java    |  42 ++---
 .../tools/MetadataQuorumCommandErrorTest.java      |  48 +++++
 .../kafka/tools/MetadataQuorumCommandTest.java     | 161 +++++++++++++++++
 .../org/apache/kafka/tools/ToolsTestUtils.java     |  51 ++++++
 .../kafka/tools/TransactionsCommandTest.java       |  12 +-
 16 files changed, 553 insertions(+), 405 deletions(-)

diff --git a/bin/kafka-metadata-quorum.sh b/bin/kafka-metadata-quorum.sh
index 24bedbded1e..3b25c7d159b 100755
--- a/bin/kafka-metadata-quorum.sh
+++ b/bin/kafka-metadata-quorum.sh
@@ -14,4 +14,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-exec $(dirname $0)/kafka-run-class.sh kafka.admin.MetadataQuorumCommand "$@"
+exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.MetadataQuorumCommand "$@"
diff --git a/bin/windows/kafka-metatada-quorum.bat b/bin/windows/kafka-metatada-quorum.bat
index 4ea8e3109f9..7942115422b 100644
--- a/bin/windows/kafka-metatada-quorum.bat
+++ b/bin/windows/kafka-metatada-quorum.bat
@@ -14,4 +14,4 @@ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 rem See the License for the specific language governing permissions and
 rem limitations under the License.
 
-"%~dp0kafka-run-class.bat" kafka.admin.MetadataQuorumCommand %*
+"%~dp0kafka-run-class.bat" org.apache.kafka.tools.MetadataQuorumCommand %*
diff --git a/build.gradle b/build.gradle
index 142bf70f465..63d47e522fd 100644
--- a/build.gradle
+++ b/build.gradle
@@ -1768,10 +1768,15 @@ project(':tools') {
     implementation libs.jacksonJaxrsJsonProvider
 
     testImplementation project(':clients')
-    testImplementation libs.junitJupiter
     testImplementation project(':clients').sourceSets.test.output
+    testImplementation project(':core')
+    testImplementation project(':core').sourceSets.test.output
+    testImplementation project(':server-common')
+    testImplementation project(':server-common').sourceSets.test.output
+    testImplementation libs.junitJupiter
     testImplementation libs.mockitoInline // supports mocking static methods, final classes, etc.
     testImplementation libs.mockitoJunitJupiter // supports MockitoExtension
+    testImplementation libs.bcpkix // required by the clients test module, but we have to specify it explicitly as gradle does not include the transitive test dependency automatically
     testRuntimeOnly libs.slf4jlog4j
   }
 
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 7a62b671f84..df9a2e9adfd 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -396,6 +396,7 @@
     <allow pkg="org.jose4j" />
     <allow pkg="net.sourceforge.argparse4j" />
     <allow pkg="org.apache.log4j" />
+    <allow pkg="kafka.test" />
   </subpackage>
 
   <subpackage name="trogdor">
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
index 42dcb60357b..a9c510bac3f 100755
--- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -598,13 +598,25 @@ public final class Utils {
      */
     public static <T> String join(Collection<T> collection, String separator) {
         Objects.requireNonNull(collection);
+        return mkString(collection.stream(), "", "", separator);
+    }
+
+    /**
+     * Create a string representation of a stream surrounded by `begin` and `end` and joined by `separator`.
+     *
+     * @return The string representation.
+     */
+    public static <T> String mkString(Stream<T> stream, String begin, String end, String separator) {
+        Objects.requireNonNull(stream);
         StringBuilder sb = new StringBuilder();
-        Iterator<T> iter = collection.iterator();
+        sb.append(begin);
+        Iterator<T> iter = stream.iterator();
         while (iter.hasNext()) {
             sb.append(iter.next());
             if (iter.hasNext())
                 sb.append(separator);
         }
+        sb.append(end);
         return sb.toString();
     }
 
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
index fbb322a0945..d10fd37a71b 100755
--- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
@@ -162,6 +162,13 @@ public class UtilsTest {
         assertEquals("1,2,3", Utils.join(asList(1, 2, 3), ","));
     }
 
+    @Test
+    public void testMkString() {
+        assertEquals("[]", Utils.mkString(Stream.empty(), "[", "]", ","));
+        assertEquals("(1)", Utils.mkString(Stream.of("1"), "(", ")", ","));
+        assertEquals("{1,2,3}", Utils.mkString(Stream.of(1, 2, 3), "{", "}", ","));
+    }
+
     @Test
     public void testAbs() {
         assertEquals(0, Utils.abs(Integer.MIN_VALUE));
diff --git a/core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala b/core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala
deleted file mode 100644
index c92988d97fa..00000000000
--- a/core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala
+++ /dev/null
@@ -1,172 +0,0 @@
-/**
- * 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 kafka.admin
-
-import kafka.tools.TerseFailure
-import kafka.utils.Exit
-import net.sourceforge.argparse4j.ArgumentParsers
-import net.sourceforge.argparse4j.impl.Arguments.{fileType, storeTrue}
-import net.sourceforge.argparse4j.inf.Subparsers
-import org.apache.kafka.clients._
-import org.apache.kafka.clients.admin.{Admin, QuorumInfo}
-import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.server.util.ToolsUtils.prettyPrintTable
-
-import java.io.File
-import java.util.Properties
-import scala.jdk.CollectionConverters._
-
-/**
- * A tool for describing quorum status
- */
-object MetadataQuorumCommand {
-
-  def main(args: Array[String]): Unit = {
-    val res = mainNoExit(args)
-    Exit.exit(res)
-  }
-
-  def mainNoExit(args: Array[String]): Int = {
-    val parser = ArgumentParsers
-      .newArgumentParser("kafka-metadata-quorum")
-      .defaultHelp(true)
-      .description("This tool describes kraft metadata quorum status.")
-    parser
-      .addArgument("--bootstrap-server")
-      .help("A comma-separated list of host:port pairs to use for establishing the connection to the Kafka cluster.")
-      .required(true)
-
-    parser
-      .addArgument("--command-config")
-      .`type`(fileType())
-      .help("Property file containing configs to be passed to Admin Client.")
-    val subparsers = parser.addSubparsers().dest("command")
-    addDescribeParser(subparsers)
-
-    var admin: Admin = null
-    try {
-      val namespace = parser.parseArgsOrFail(args)
-      val command = namespace.getString("command")
-
-      val commandConfig = namespace.get[File]("command_config")
-      val props = if (commandConfig != null) {
-        if (!commandConfig.exists()) {
-          throw new TerseFailure(s"Properties file ${commandConfig.getPath} does not exists!")
-        }
-        Utils.loadProps(commandConfig.getPath)
-      } else {
-        new Properties()
-      }
-      props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, namespace.getString("bootstrap_server"))
-      admin = Admin.create(props)
-
-      if (command == "describe") {
-        if (namespace.getBoolean("status") && namespace.getBoolean("replication")) {
-          throw new TerseFailure(s"Only one of --status or --replication should be specified with describe sub-command")
-        } else if (namespace.getBoolean("replication")) {
-          handleDescribeReplication(admin)
-        } else if (namespace.getBoolean("status")) {
-          handleDescribeStatus(admin)
-        } else {
-          throw new TerseFailure(s"One of --status or --replication must be specified with describe sub-command")
-        }
-      } else {
-        throw new IllegalStateException(s"Unknown command: $command, only 'describe' is supported")
-      }
-      0
-    } catch {
-      case e: TerseFailure =>
-        Console.err.println(e.getMessage)
-        1
-    } finally {
-      if (admin != null) {
-        admin.close()
-      }
-    }
-  }
-
-  def addDescribeParser(subparsers: Subparsers): Unit = {
-    val describeParser = subparsers
-      .addParser("describe")
-      .help("Describe the metadata quorum info")
-
-    val statusArgs = describeParser.addArgumentGroup("Status")
-    statusArgs
-      .addArgument("--status")
-      .help(
-        "A short summary of the quorum status and the other provides detailed information about the status of replication.")
-      .action(storeTrue())
-    val replicationArgs = describeParser.addArgumentGroup("Replication")
-    replicationArgs
-      .addArgument("--replication")
-      .help("Detailed information about the status of replication")
-      .action(storeTrue())
-  }
-
-  private def handleDescribeReplication(admin: Admin): Unit = {
-    val quorumInfo = admin.describeMetadataQuorum.quorumInfo.get
-    val leaderId = quorumInfo.leaderId
-    val leader = quorumInfo.voters.asScala.filter(_.replicaId == leaderId).head
-
-    def convertQuorumInfo(infos: Seq[QuorumInfo.ReplicaState], status: String): Seq[Array[String]] =
-      infos.map { info =>
-        Array(info.replicaId,
-              info.logEndOffset,
-              leader.logEndOffset - info.logEndOffset,
-              info.lastFetchTimestamp.orElse(-1),
-              info.lastCaughtUpTimestamp.orElse(-1),
-              status
-        ).map(_.toString)
-      }
-    prettyPrintTable(
-      Array("NodeId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"),
-      (convertQuorumInfo(Seq(leader), "Leader")
-        ++ convertQuorumInfo(quorumInfo.voters.asScala.filter(_.replicaId != leaderId).toSeq, "Follower")
-        ++ convertQuorumInfo(quorumInfo.observers.asScala.toSeq, "Observer")).asJava,
-      scala.Console.out
-    )
-  }
-
-  private def handleDescribeStatus(admin: Admin): Unit = {
-    val clusterId = admin.describeCluster.clusterId.get
-    val quorumInfo = admin.describeMetadataQuorum.quorumInfo.get
-    val leaderId = quorumInfo.leaderId
-    val leader = quorumInfo.voters.asScala.filter(_.replicaId == leaderId).head
-    val maxLagFollower = quorumInfo.voters.asScala
-      .minBy(_.logEndOffset)
-    val maxFollowerLag = leader.logEndOffset - maxLagFollower.logEndOffset
-    val maxFollowerLagTimeMs =
-      if (leader == maxLagFollower) {
-        0
-      } else if (leader.lastCaughtUpTimestamp.isPresent && maxLagFollower.lastCaughtUpTimestamp.isPresent) {
-        leader.lastCaughtUpTimestamp.getAsLong - maxLagFollower.lastCaughtUpTimestamp.getAsLong
-      } else {
-        -1
-      }
-    println(
-      s"""|ClusterId:              $clusterId
-          |LeaderId:               ${quorumInfo.leaderId}
-          |LeaderEpoch:            ${quorumInfo.leaderEpoch}
-          |HighWatermark:          ${quorumInfo.highWatermark}
-          |MaxFollowerLag:         $maxFollowerLag
-          |MaxFollowerLagTimeMs:   $maxFollowerLagTimeMs
-          |CurrentVoters:          ${quorumInfo.voters.asScala.map(_.replicaId).mkString("[", ",", "]")}
-          |CurrentObservers:       ${quorumInfo.observers.asScala.map(_.replicaId).mkString("[", ",", "]")}
-          |""".stripMargin
-    )
-  }
-}
diff --git a/core/src/test/scala/unit/kafka/admin/MetadataQuorumCommandTest.scala b/core/src/test/scala/unit/kafka/admin/MetadataQuorumCommandTest.scala
deleted file mode 100644
index 24b6616cb1e..00000000000
--- a/core/src/test/scala/unit/kafka/admin/MetadataQuorumCommandTest.scala
+++ /dev/null
@@ -1,192 +0,0 @@
-/**
- * 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 kafka.admin
-
-import kafka.test.ClusterInstance
-import kafka.test.annotation.{ClusterTest, ClusterTestDefaults, ClusterTests, Type}
-import kafka.test.junit.ClusterTestExtensions
-import kafka.utils.TestUtils
-import org.apache.kafka.common.errors.UnsupportedVersionException
-import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue}
-import org.junit.jupiter.api.{Tag, Test}
-import org.junit.jupiter.api.extension.ExtendWith
-
-import java.util.concurrent.ExecutionException
-
-@ExtendWith(value = Array(classOf[ClusterTestExtensions]))
-@ClusterTestDefaults(clusterType = Type.KRAFT)
-@Tag("integration")
-class MetadataQuorumCommandTest(cluster: ClusterInstance) {
-
-  /**
-   * 1. The same number of broker controllers
-   * 2. More brokers than controllers
-   * 3. Fewer brokers than controllers
-   */
-  @ClusterTests(
-    Array(
-      new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3),
-      new ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3),
-      new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2),
-      new ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2),
-      new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3),
-      new ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3)
-    ))
-  def testDescribeQuorumReplicationSuccessful(): Unit = {
-    cluster.waitForReadyBrokers()
-    val describeOutput = TestUtils.grabConsoleOutput(
-      MetadataQuorumCommand.mainNoExit(
-        Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication"))
-    )
-
-    val leaderPattern = """\d+\s+\d+\s+\d+\s+\d+\s+[-]?\d+\s+Leader\s+""".r
-    val followerPattern = """\d+\s+\d+\s+\d+\s+\d+\s+[-]?\d+\s+Follower\s+""".r
-    val observerPattern = """\d+\s+\d+\s+\d+\s+\d+\s+[-]?\d+\s+Observer\s+""".r
-    val outputs = describeOutput.split("\n").tail
-    if (cluster.config().clusterType() == Type.CO_KRAFT) {
-      assertEquals(Math.max(cluster.config().numControllers(), cluster.config().numBrokers()), outputs.length)
-    } else {
-      assertEquals(cluster.config().numBrokers() + cluster.config().numControllers(), outputs.length)
-    }
-    // `matches` is not supported in scala 2.12, use `findFirstIn` instead.
-    assertTrue(leaderPattern.findFirstIn(outputs.head).nonEmpty)
-    assertEquals(1, outputs.count(leaderPattern.findFirstIn(_).nonEmpty))
-    assertEquals(cluster.config().numControllers() - 1, outputs.count(followerPattern.findFirstIn(_).nonEmpty))
-
-    if (cluster.config().clusterType() == Type.CO_KRAFT) {
-      assertEquals(Math.max(0, cluster.config().numBrokers() - cluster.config().numControllers()), outputs.count(observerPattern.findFirstIn(_).nonEmpty))
-    } else {
-      assertEquals(cluster.config().numBrokers(), outputs.count(observerPattern.findFirstIn(_).nonEmpty))
-    }
-  }
-
-  /**
-   * 1. The same number of broker controllers
-   * 2. More brokers than controllers
-   * 3. Fewer brokers than controllers
-   */
-  @ClusterTests(
-    Array(
-      new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3),
-      new ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3),
-      new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2),
-      new ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2),
-      new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3),
-      new ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3)
-    ))
-  def testDescribeQuorumStatusSuccessful(): Unit = {
-    cluster.waitForReadyBrokers()
-    val describeOutput = TestUtils.grabConsoleOutput(
-      MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status"))
-    )
-    val outputs = describeOutput.split("\n")
-
-    assertTrue("""ClusterId:\s+\S{22}""".r.findFirstIn(outputs(0)).nonEmpty)
-    assertTrue("""LeaderId:\s+\d+""".r.findFirstIn(outputs(1)).nonEmpty)
-    assertTrue("""LeaderEpoch:\s+\d+""".r.findFirstIn(outputs(2)).nonEmpty)
-    // HighWatermark may be -1
-    assertTrue("""HighWatermark:\s+[-]?\d+""".r.findFirstIn(outputs(3)).nonEmpty)
-    assertTrue("""MaxFollowerLag:\s+\d+""".r.findFirstIn(outputs(4)).nonEmpty)
-    assertTrue("""MaxFollowerLagTimeMs:\s+[-]?\d+""".r.findFirstIn(outputs(5)).nonEmpty)
-    assertTrue("""CurrentVoters:\s+\[\d+(,\d+)*\]""".r.findFirstIn(outputs(6)).nonEmpty)
-
-    // There are no observers if we have fewer brokers than controllers
-    if (cluster.config().clusterType() == Type.CO_KRAFT
-        && cluster.config().numBrokers() <= cluster.config().numControllers()) {
-      assertTrue("""CurrentObservers:\s+\[\]""".r.findFirstIn(outputs(7)).nonEmpty)
-    } else {
-      assertTrue("""CurrentObservers:\s+\[\d+(,\d+)*\]""".r.findFirstIn(outputs(7)).nonEmpty)
-    }
-  }
-
-  @ClusterTests(
-    Array(new ClusterTest(clusterType = Type.CO_KRAFT, brokers = 1, controllers = 1),
-          new ClusterTest(clusterType = Type.KRAFT, brokers = 1, controllers = 1)))
-  def testOnlyOneBrokerAndOneController(): Unit = {
-    val statusOutput = TestUtils.grabConsoleOutput(
-      MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status"))
-    )
-    assertEquals("MaxFollowerLag:         0", statusOutput.split("\n")(4))
-    assertEquals("MaxFollowerLagTimeMs:   0", statusOutput.split("\n")(5))
-
-    val replicationOutput = TestUtils.grabConsoleOutput(
-      MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication"))
-    )
-    assertEquals("0", replicationOutput.split("\n")(1).split("\\s+")(2))
-  }
-
-  @ClusterTest(clusterType = Type.ZK, brokers = 3)
-  def testDescribeQuorumInZkMode(): Unit = {
-    assertTrue(
-      assertThrows(
-        classOf[ExecutionException],
-        () =>
-          MetadataQuorumCommand.mainNoExit(
-            Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status"))
-      ).getCause.isInstanceOf[UnsupportedVersionException]
-    )
-    assertTrue(
-      assertThrows(
-        classOf[ExecutionException],
-        () =>
-          MetadataQuorumCommand.mainNoExit(
-            Array("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication"))
-      ).getCause.isInstanceOf[UnsupportedVersionException]
-    )
-  }
-}
-
-class MetadataQuorumCommandErrorTest {
-
-  @Test
-  def testPropertiesFileDoesNotExists(): Unit = {
-    assertEquals(1,
-                 MetadataQuorumCommand.mainNoExit(
-                   Array("--bootstrap-server", "localhost:9092", "--command-config", "admin.properties", "describe")))
-    assertEquals(
-      "Properties file admin.properties does not exists!",
-      TestUtils
-        .grabConsoleError(
-          MetadataQuorumCommand.mainNoExit(
-            Array("--bootstrap-server", "localhost:9092", "--command-config", "admin.properties", "describe")))
-        .trim
-    )
-  }
-
-  @Test
-  def testDescribeOptions(): Unit = {
-    assertEquals(1, MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", "localhost:9092", "describe")))
-    assertEquals(
-      "One of --status or --replication must be specified with describe sub-command",
-      TestUtils
-        .grabConsoleError(MetadataQuorumCommand.mainNoExit(Array("--bootstrap-server", "localhost:9092", "describe")))
-        .trim
-    )
-
-    assertEquals(1,
-                 MetadataQuorumCommand.mainNoExit(
-                   Array("--bootstrap-server", "localhost:9092", "describe", "--status", "--replication")))
-    assertEquals(
-      "Only one of --status or --replication should be specified with describe sub-command",
-      TestUtils
-        .grabConsoleError(
-          MetadataQuorumCommand.mainNoExit(
-            Array("--bootstrap-server", "localhost:9092", "describe", "--status", "--replication")))
-        .trim
-    )
-  }
-}
diff --git a/server-common/src/main/java/org/apache/kafka/server/util/ToolsUtils.java b/server-common/src/main/java/org/apache/kafka/server/util/ToolsUtils.java
index 0c923cd66c1..b14f079b15c 100644
--- a/server-common/src/main/java/org/apache/kafka/server/util/ToolsUtils.java
+++ b/server-common/src/main/java/org/apache/kafka/server/util/ToolsUtils.java
@@ -20,7 +20,6 @@ import org.apache.kafka.common.Metric;
 import org.apache.kafka.common.MetricName;
 
 import java.io.PrintStream;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -70,13 +69,13 @@ public class ToolsUtils {
 
     private static void printRow(
         List<Integer> columnLengths,
-        String[] row,
+        List<String> row,
         PrintStream out
     ) {
         StringBuilder rowBuilder = new StringBuilder();
-        for (int i = 0; i < row.length; i++) {
+        for (int i = 0; i < row.size(); i++) {
             Integer columnLength = columnLengths.get(i);
-            String columnValue = row[i];
+            String columnValue = row.get(i);
             appendColumnValue(rowBuilder, columnValue, columnLength);
             rowBuilder.append('\t');
         }
@@ -84,17 +83,17 @@ public class ToolsUtils {
     }
 
     public static void prettyPrintTable(
-        String[] headers,
-        List<String[]> rows,
+        List<String> headers,
+        List<List<String>> rows,
         PrintStream out
     ) {
-        List<Integer> columnLengths = Arrays.stream(headers)
+        List<Integer> columnLengths = headers.stream()
             .map(String::length)
             .collect(Collectors.toList());
 
-        for (String[] row : rows) {
-            for (int i = 0; i < headers.length; i++) {
-                columnLengths.set(i, Math.max(columnLengths.get(i), row[i].length()));
+        for (List<String> row : rows) {
+            for (int i = 0; i < headers.size(); i++) {
+                columnLengths.set(i, Math.max(columnLengths.get(i), row.get(i).length()));
             }
         }
 
diff --git a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java
new file mode 100644
index 00000000000..6cafea81b52
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java
@@ -0,0 +1,195 @@
+/*
+ * 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.tools;
+
+import net.sourceforge.argparse4j.ArgumentParsers;
+import net.sourceforge.argparse4j.impl.Arguments;
+import net.sourceforge.argparse4j.inf.ArgumentGroup;
+import net.sourceforge.argparse4j.inf.ArgumentParser;
+import net.sourceforge.argparse4j.inf.Namespace;
+import net.sourceforge.argparse4j.inf.Subparser;
+import net.sourceforge.argparse4j.inf.Subparsers;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.QuorumInfo;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.util.ToolsUtils;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static java.util.Arrays.asList;
+
+/**
+ * A tool for describing quorum status
+ */
+public class MetadataQuorumCommand {
+
+    public static void main(String... args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        ArgumentParser parser = ArgumentParsers
+            .newArgumentParser("kafka-metadata-quorum")
+            .defaultHelp(true)
+            .description("This tool describes kraft metadata quorum status.");
+        parser
+            .addArgument("--bootstrap-server")
+            .help("A comma-separated list of host:port pairs to use for establishing the connection to the Kafka cluster.")
+            .required(true);
+
+        parser
+            .addArgument("--command-config")
+            .type(Arguments.fileType())
+            .help("Property file containing configs to be passed to Admin Client.");
+        Subparsers subparsers = parser.addSubparsers().dest("command");
+        addDescribeParser(subparsers);
+
+        Admin admin = null;
+        try {
+            Namespace namespace = parser.parseArgsOrFail(args);
+            String command = namespace.getString("command");
+
+            File commandConfig = namespace.get("command_config");
+            Properties props = new Properties();
+            if (commandConfig != null) {
+                if (!commandConfig.exists())
+                    throw new TerseException("Properties file " + commandConfig.getPath() + " does not exists!");
+
+                Utils.loadProps(commandConfig.getPath());
+            }
+            props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, namespace.getString("bootstrap_server"));
+            admin = Admin.create(props);
+
+            if (command.equals("describe")) {
+                if (namespace.getBoolean("status") && namespace.getBoolean("replication")) {
+                    throw new TerseException("Only one of --status or --replication should be specified with describe sub-command");
+                } else if (namespace.getBoolean("replication")) {
+                    handleDescribeReplication(admin);
+                } else if (namespace.getBoolean("status")) {
+                    handleDescribeStatus(admin);
+                } else {
+                    throw new TerseException("One of --status or --replication must be specified with describe sub-command");
+                }
+            } else {
+                throw new IllegalStateException("Unknown command: " + command + ", only 'describe' is supported");
+            }
+        } finally {
+            if (admin != null)
+                admin.close();
+        }
+    }
+
+    private static void addDescribeParser(Subparsers subparsers) {
+        Subparser describeParser = subparsers
+            .addParser("describe")
+            .help("Describe the metadata quorum info");
+
+        ArgumentGroup statusArgs = describeParser.addArgumentGroup("Status");
+        statusArgs
+            .addArgument("--status")
+            .help("A short summary of the quorum status and the other provides detailed information about the status of replication.")
+            .action(Arguments.storeTrue());
+        ArgumentGroup replicationArgs = describeParser.addArgumentGroup("Replication");
+        replicationArgs
+            .addArgument("--replication")
+            .help("Detailed information about the status of replication")
+            .action(Arguments.storeTrue());
+    }
+
+    private static void handleDescribeReplication(Admin admin) throws ExecutionException, InterruptedException {
+        QuorumInfo quorumInfo = admin.describeMetadataQuorum().quorumInfo().get();
+        int leaderId = quorumInfo.leaderId();
+        QuorumInfo.ReplicaState leader = quorumInfo.voters().stream().filter(voter -> voter.replicaId() == leaderId).findFirst().get();
+
+        List<List<String>> rows = new ArrayList<>();
+        rows.addAll(quorumInfoToRows(leader, Stream.of(leader), "Leader"));
+        rows.addAll(quorumInfoToRows(leader, quorumInfo.voters().stream().filter(v -> v.replicaId() != leaderId), "Follower"));
+        rows.addAll(quorumInfoToRows(leader, quorumInfo.observers().stream(), "Observer"));
+
+        ToolsUtils.prettyPrintTable(
+            asList("NodeId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"),
+            rows,
+            System.out
+        );
+    }
+
+    private static List<List<String>> quorumInfoToRows(QuorumInfo.ReplicaState leader, Stream<QuorumInfo.ReplicaState> infos, String status) {
+        return infos.map(info ->
+            Stream.of(
+                info.replicaId(),
+                info.logEndOffset(),
+                leader.logEndOffset() - info.logEndOffset(),
+                info.lastFetchTimestamp().orElse(-1),
+                info.lastCaughtUpTimestamp().orElse(-1),
+                status
+            ).map(r -> r.toString()).collect(Collectors.toList())
+        ).collect(Collectors.toList());
+    }
+
+    private static void handleDescribeStatus(Admin admin) throws ExecutionException, InterruptedException {
+        String clusterId = admin.describeCluster().clusterId().get();
+        QuorumInfo quorumInfo = admin.describeMetadataQuorum().quorumInfo().get();
+        int leaderId = quorumInfo.leaderId();
+        QuorumInfo.ReplicaState leader = quorumInfo.voters().stream().filter(voter -> voter.replicaId() == leaderId).findFirst().get();
+        QuorumInfo.ReplicaState maxLagFollower = quorumInfo.voters().stream().min(Comparator.comparingLong(qi -> qi.logEndOffset())).get();
+        long maxFollowerLag = leader.logEndOffset() - maxLagFollower.logEndOffset();
+
+        long maxFollowerLagTimeMs;
+        if (leader == maxLagFollower)
+            maxFollowerLagTimeMs = 0;
+        else if (leader.lastCaughtUpTimestamp().isPresent() && maxLagFollower.lastCaughtUpTimestamp().isPresent()) {
+            maxFollowerLagTimeMs = leader.lastCaughtUpTimestamp().getAsLong() - maxLagFollower.lastCaughtUpTimestamp().getAsLong();
+        } else {
+            maxFollowerLagTimeMs = -1;
+        }
+
+        System.out.println(
+            "ClusterId:              " + clusterId +
+            "\nLeaderId:               " + quorumInfo.leaderId() +
+            "\nLeaderEpoch:            " + quorumInfo.leaderEpoch() +
+            "\nHighWatermark:          " + quorumInfo.highWatermark() +
+            "\nMaxFollowerLag:         " + maxFollowerLag +
+            "\nMaxFollowerLagTimeMs:   " + maxFollowerLagTimeMs +
+            "\nCurrentVoters:          " + Utils.mkString(quorumInfo.voters().stream().map(v -> v.replicaId()), "[", "]", ",") +
+            "\nCurrentObservers:       " + Utils.mkString(quorumInfo.observers().stream().map(v -> v.replicaId()), "[", "]", ",")
+        );
+    }
+
+}
diff --git a/tools/src/main/java/org/apache/kafka/tools/TerseException.java b/tools/src/main/java/org/apache/kafka/tools/TerseException.java
new file mode 100644
index 00000000000..c43f5c62a58
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/tools/TerseException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools;
+
+/**
+ * An exception thrown to indicate that the command has failed, but we don't want to
+ * print a stack trace.
+ */
+public class TerseException extends Exception {
+
+    /**
+     * Create new instance with the provided message.
+     *
+     * @param message The message to print out before exiting.  A stack trace will not be printed.
+     */
+    public TerseException(String message) {
+        super(message);
+    }
+}
diff --git a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java
index 194524d2654..4f7fa9c27bf 100644
--- a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java
+++ b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java
@@ -47,7 +47,6 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -63,6 +62,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
+import static java.util.Arrays.asList;
 import static java.util.Collections.singleton;
 import static java.util.Collections.singletonList;
 import static net.sourceforge.argparse4j.impl.Arguments.store;
@@ -254,14 +254,14 @@ public abstract class TransactionsCommand {
     }
 
     static class DescribeProducersCommand extends TransactionsCommand {
-        static final String[] HEADERS = new String[]{
+        static final List<String> HEADERS = asList(
             "ProducerId",
             "ProducerEpoch",
             "LatestCoordinatorEpoch",
             "LastSequence",
             "LastTimestamp",
             "CurrentTransactionStartOffset"
-        };
+        );
 
         DescribeProducersCommand(Time time) {
             super(time);
@@ -320,20 +320,20 @@ public abstract class TransactionsCommand {
                 return;
             }
 
-            List<String[]> rows = result.activeProducers().stream().map(producerState -> {
+            List<List<String>> rows = result.activeProducers().stream().map(producerState -> {
                 String currentTransactionStartOffsetColumnValue =
                     producerState.currentTransactionStartOffset().isPresent() ?
                         String.valueOf(producerState.currentTransactionStartOffset().getAsLong()) :
                         "None";
 
-                return new String[] {
+                return asList(
                     String.valueOf(producerState.producerId()),
                     String.valueOf(producerState.producerEpoch()),
                     String.valueOf(producerState.coordinatorEpoch().orElse(-1)),
                     String.valueOf(producerState.lastSequence()),
                     String.valueOf(producerState.lastTimestamp()),
                     currentTransactionStartOffsetColumnValue
-                };
+                );
             }).collect(Collectors.toList());
 
             prettyPrintTable(HEADERS, rows, out);
@@ -341,7 +341,7 @@ public abstract class TransactionsCommand {
     }
 
     static class DescribeTransactionsCommand extends TransactionsCommand {
-        static final String[] HEADERS = new String[]{
+        static final List<String> HEADERS = asList(
             "CoordinatorId",
             "TransactionalId",
             "ProducerId",
@@ -351,7 +351,7 @@ public abstract class TransactionsCommand {
             "CurrentTransactionStartTimeMs",
             "TransactionDurationMs",
             "TopicPartitions"
-        };
+        );
 
         DescribeTransactionsCommand(Time time) {
             super(time);
@@ -402,7 +402,7 @@ public abstract class TransactionsCommand {
                 transactionDurationMsColumnValue = "None";
             }
 
-            String[] row = new String[]{
+            List<String> row = asList(
                 String.valueOf(result.coordinatorId()),
                 transactionalId,
                 String.valueOf(result.producerId()),
@@ -412,19 +412,19 @@ public abstract class TransactionsCommand {
                 transactionStartTimeMsColumnValue,
                 transactionDurationMsColumnValue,
                 Utils.join(result.topicPartitions(), ",")
-            };
+            );
 
             prettyPrintTable(HEADERS, singletonList(row), out);
         }
     }
 
     static class ListTransactionsCommand extends TransactionsCommand {
-        static final String[] HEADERS = new String[] {
+        static final List<String> HEADERS = asList(
             "TransactionalId",
             "Coordinator",
             "ProducerId",
             "TransactionState"
-        };
+        );
 
         ListTransactionsCommand(Time time) {
             super(time);
@@ -454,18 +454,18 @@ public abstract class TransactionsCommand {
                 return;
             }
 
-            List<String[]> rows = new ArrayList<>();
+            List<List<String>> rows = new ArrayList<>();
             for (Map.Entry<Integer, Collection<TransactionListing>> brokerListingsEntry : result.entrySet()) {
                 String coordinatorIdString = brokerListingsEntry.getKey().toString();
                 Collection<TransactionListing> listings = brokerListingsEntry.getValue();
 
                 for (TransactionListing listing : listings) {
-                    rows.add(new String[] {
+                    rows.add(asList(
                         listing.transactionalId(),
                         coordinatorIdString,
                         String.valueOf(listing.producerId()),
                         listing.state().toString()
-                    });
+                    ));
                 }
             }
 
@@ -476,7 +476,7 @@ public abstract class TransactionsCommand {
     static class FindHangingTransactionsCommand extends TransactionsCommand {
         private static final int MAX_BATCH_SIZE = 500;
 
-        static final String[] HEADERS = new String[] {
+        static final List<String> HEADERS = asList(
             "Topic",
             "Partition",
             "ProducerId",
@@ -485,7 +485,7 @@ public abstract class TransactionsCommand {
             "StartOffset",
             "LastTimestamp",
             "Duration(min)"
-        };
+        );
 
         FindHangingTransactionsCommand(Time time) {
             super(time);
@@ -653,13 +653,13 @@ public abstract class TransactionsCommand {
             PrintStream out
         ) {
             long currentTimeMs = time.milliseconds();
-            List<String[]> rows = new ArrayList<>(hangingTransactions.size());
+            List<List<String>> rows = new ArrayList<>(hangingTransactions.size());
 
             for (OpenTransaction transaction : hangingTransactions) {
                 long transactionDurationMinutes = TimeUnit.MILLISECONDS.toMinutes(
                     currentTimeMs - transaction.producerState.lastTimestamp());
 
-                rows.add(new String[] {
+                rows.add(asList(
                     transaction.topicPartition.topic(),
                     String.valueOf(transaction.topicPartition.partition()),
                     String.valueOf(transaction.producerState.producerId()),
@@ -668,7 +668,7 @@ public abstract class TransactionsCommand {
                     String.valueOf(transaction.producerState.currentTransactionStartOffset().orElse(-1)),
                     String.valueOf(transaction.producerState.lastTimestamp()),
                     String.valueOf(transactionDurationMinutes)
-                });
+                ));
             }
 
             prettyPrintTable(HEADERS, rows, out);
@@ -974,7 +974,7 @@ public abstract class TransactionsCommand {
         PrintStream out,
         Time time
     ) throws Exception {
-        List<TransactionsCommand> commands = Arrays.asList(
+        List<TransactionsCommand> commands = asList(
             new ListTransactionsCommand(time),
             new DescribeTransactionsCommand(time),
             new DescribeProducersCommand(time),
diff --git a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandErrorTest.java b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandErrorTest.java
new file mode 100644
index 00000000000..8503e707b7b
--- /dev/null
+++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandErrorTest.java
@@ -0,0 +1,48 @@
+/*
+ * 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.tools;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class MetadataQuorumCommandErrorTest {
+
+    @Test
+    public void testPropertiesFileDoesNotExists() {
+        assertEquals(1,
+            MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092", "--command-config", "admin.properties", "describe"));
+        assertEquals("Properties file admin.properties does not exists!",
+            ToolsTestUtils.captureStandardErr(() ->
+                MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092", "--command-config", "admin.properties", "describe")));
+    }
+
+    @Test
+    public void testDescribeOptions() {
+        assertEquals(1, MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092", "describe"));
+        assertEquals("One of --status or --replication must be specified with describe sub-command",
+            ToolsTestUtils.captureStandardErr(() ->
+                MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092", "describe")));
+
+        assertEquals(1,
+            MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092", "describe", "--status", "--replication"));
+        assertEquals("Only one of --status or --replication should be specified with describe sub-command",
+            ToolsTestUtils.captureStandardErr(() ->
+                MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092", "describe", "--status", "--replication")));
+    }
+
+}
diff --git a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
new file mode 100644
index 00000000000..7b52b351afd
--- /dev/null
+++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.tools;
+
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.ClusterTests;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import static java.util.Arrays.stream;
+
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.KRAFT)
+@Tag("integration")
+class MetadataQuorumCommandTest {
+
+    private final ClusterInstance cluster;
+    public MetadataQuorumCommandTest(ClusterInstance cluster) {
+        this.cluster = cluster;
+    }
+
+    /**
+     * 1. The same number of broker controllers
+     * 2. More brokers than controllers
+     * 3. Fewer brokers than controllers
+     */
+    @ClusterTests({
+        @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3),
+        @ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3),
+        @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2),
+        @ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2),
+        @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3),
+        @ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3)
+    })
+    public void testDescribeQuorumReplicationSuccessful() throws InterruptedException {
+        cluster.waitForReadyBrokers();
+        String describeOutput = ToolsTestUtils.captureStandardOut(() ->
+            MetadataQuorumCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication")
+        );
+
+        List<String> outputs = stream(describeOutput.split("\n")).skip(1).collect(Collectors.toList());
+        if (cluster.config().clusterType() == Type.CO_KRAFT)
+          assertEquals(Math.max(cluster.config().numControllers(), cluster.config().numBrokers()), outputs.size());
+        else
+          assertEquals(cluster.config().numBrokers() + cluster.config().numControllers(), outputs.size());
+
+        Pattern leaderPattern = Pattern.compile("\\d+\\s+\\d+\\s+\\d+\\s+\\d+\\s+-?\\d+\\s+Leader\\s*");
+        assertTrue(leaderPattern.matcher(outputs.get(0)).find());
+        assertTrue(outputs.stream().skip(1).noneMatch(o -> leaderPattern.matcher(o).find()));
+
+        Pattern followerPattern = Pattern.compile("\\d+\\s+\\d+\\s+\\d+\\s+\\d+\\s+-?\\d+\\s+Follower\\s*");
+        assertEquals(cluster.config().numControllers() - 1, outputs.stream().filter(o -> followerPattern.matcher(o).find()).count());
+
+        Pattern observerPattern = Pattern.compile("\\d+\\s+\\d+\\s+\\d+\\s+\\d+\\s+-?\\d+\\s+Observer\\s*");
+        if (cluster.config().clusterType() == Type.CO_KRAFT)
+            assertEquals(Math.max(0, cluster.config().numBrokers() - cluster.config().numControllers()),
+                outputs.stream().filter(o -> observerPattern.matcher(o).find()).count());
+        else
+            assertEquals(cluster.config().numBrokers(), outputs.stream().filter(o -> observerPattern.matcher(o).find()).count());
+    }
+
+
+    /**
+     * 1. The same number of broker controllers
+     * 2. More brokers than controllers
+     * 3. Fewer brokers than controllers
+     */
+    @ClusterTests({
+        @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3),
+        @ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3),
+        @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2),
+        @ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2),
+        @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3),
+        @ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3)
+    })
+    public void testDescribeQuorumStatusSuccessful() throws InterruptedException {
+        cluster.waitForReadyBrokers();
+        String describeOutput = ToolsTestUtils.captureStandardOut(() ->
+            MetadataQuorumCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status")
+        );
+        String[] outputs = describeOutput.split("\n");
+
+        assertTrue(outputs[0].matches("ClusterId:\\s+\\S{22}"));
+        assertTrue(outputs[1].matches("LeaderId:\\s+\\d+"));
+        assertTrue(outputs[2].matches("LeaderEpoch:\\s+\\d+"));
+        // HighWatermark may be -1
+        assertTrue(outputs[3].matches("HighWatermark:\\s+-?\\d+"));
+        assertTrue(outputs[4].matches("MaxFollowerLag:\\s+\\d+"));
+        assertTrue(outputs[5].matches("MaxFollowerLagTimeMs:\\s+-?\\d+"));
+        assertTrue(outputs[6].matches("CurrentVoters:\\s+\\[\\d+(,\\d+)*]"));
+
+        // There are no observers if we have fewer brokers than controllers
+        if (cluster.config().clusterType() == Type.CO_KRAFT && cluster.config().numBrokers() <= cluster.config().numControllers())
+            assertTrue(outputs[7].matches("CurrentObservers:\\s+\\[]"));
+        else
+            assertTrue(outputs[7].matches("CurrentObservers:\\s+\\[\\d+(,\\d+)*]"));
+    }
+
+    @ClusterTests({
+        @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 1, controllers = 1),
+        @ClusterTest(clusterType = Type.KRAFT, brokers = 1, controllers = 1)
+    })
+    public void testOnlyOneBrokerAndOneController() {
+        String statusOutput = ToolsTestUtils.captureStandardOut(() ->
+            MetadataQuorumCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status")
+        );
+        assertEquals("MaxFollowerLag:         0", statusOutput.split("\n")[4]);
+        assertEquals("MaxFollowerLagTimeMs:   0", statusOutput.split("\n")[5]);
+
+        String replicationOutput = ToolsTestUtils.captureStandardOut(() ->
+            MetadataQuorumCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication")
+        );
+        assertEquals("0", replicationOutput.split("\n")[1].split("\\s+")[2]);
+    }
+
+    @ClusterTest(clusterType = Type.ZK, brokers = 3)
+    public void testDescribeQuorumInZkMode() {
+        assertTrue(
+            assertThrows(
+                ExecutionException.class,
+                () -> MetadataQuorumCommand.execute("--bootstrap-server", cluster.bootstrapServers(), "describe", "--status")
+            ).getCause() instanceof UnsupportedVersionException
+        );
+
+        assertTrue(
+            assertThrows(
+                ExecutionException.class,
+                () -> MetadataQuorumCommand.execute("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication")
+            ).getCause() instanceof UnsupportedVersionException
+        );
+
+    }
+}
diff --git a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java
new file mode 100644
index 00000000000..4664288ae4f
--- /dev/null
+++ b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.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.tools;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+public class ToolsTestUtils {
+
+    public static String captureStandardOut(Runnable runnable) {
+        return captureStandardStream(false, runnable);
+    }
+
+    public static String captureStandardErr(Runnable runnable) {
+        return captureStandardStream(true, runnable);
+    }
+
+    private static String captureStandardStream(boolean isErr, Runnable runnable) {
+        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        PrintStream currentStream = isErr ? System.err : System.out;
+        PrintStream tempStream = new PrintStream(outputStream);
+        if (isErr)
+            System.setErr(tempStream);
+        else
+            System.setOut(tempStream);
+        try {
+            runnable.run();
+            return new String(outputStream.toByteArray()).trim();
+        } finally {
+            if (isErr)
+                System.setErr(currentStream);
+            else
+                System.setOut(currentStream);
+        }
+    }
+
+}
diff --git a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java
index 4b65dc87d3c..3dc77a6d0f6 100644
--- a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java
+++ b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java
@@ -177,7 +177,7 @@ public class TransactionsCommandTest {
         List<List<String>> table = readOutputAsTable();
         assertEquals(3, table.size());
 
-        List<String> expectedHeaders = asList(TransactionsCommand.DescribeProducersCommand.HEADERS);
+        List<String> expectedHeaders = TransactionsCommand.DescribeProducersCommand.HEADERS;
         assertEquals(expectedHeaders, table.get(0));
 
         Set<List<String>> expectedRows = Utils.mkSet(
@@ -213,7 +213,7 @@ public class TransactionsCommandTest {
         assertEquals(4, table.size());
 
         // Assert expected headers
-        List<String> expectedHeaders = asList(TransactionsCommand.ListTransactionsCommand.HEADERS);
+        List<String> expectedHeaders = TransactionsCommand.ListTransactionsCommand.HEADERS;
         assertEquals(expectedHeaders, table.get(0));
 
         Set<List<String>> expectedRows = Utils.mkSet(
@@ -272,7 +272,7 @@ public class TransactionsCommandTest {
         List<List<String>> table = readOutputAsTable();
         assertEquals(2, table.size());
 
-        List<String> expectedHeaders = asList(TransactionsCommand.DescribeTransactionsCommand.HEADERS);
+        List<String> expectedHeaders = TransactionsCommand.DescribeTransactionsCommand.HEADERS;
         assertEquals(expectedHeaders, table.get(0));
 
         List<String> expectedRow = asList(
@@ -703,7 +703,7 @@ public class TransactionsCommandTest {
         List<List<String>> table = readOutputAsTable();
         assertEquals(1, table.size());
 
-        List<String> expectedHeaders = asList(TransactionsCommand.FindHangingTransactionsCommand.HEADERS);
+        List<String> expectedHeaders = TransactionsCommand.FindHangingTransactionsCommand.HEADERS;
         assertEquals(expectedHeaders, table.get(0));
     }
 
@@ -742,7 +742,7 @@ public class TransactionsCommandTest {
         List<List<String>> table = readOutputAsTable();
         assertEquals(1, table.size());
 
-        List<String> expectedHeaders = asList(TransactionsCommand.FindHangingTransactionsCommand.HEADERS);
+        List<String> expectedHeaders = TransactionsCommand.FindHangingTransactionsCommand.HEADERS;
         assertEquals(expectedHeaders, table.get(0));
     }
 
@@ -940,7 +940,7 @@ public class TransactionsCommandTest {
         List<List<String>> table = readOutputAsTable();
         assertEquals(2, table.size());
 
-        List<String> expectedHeaders = asList(TransactionsCommand.FindHangingTransactionsCommand.HEADERS);
+        List<String> expectedHeaders = TransactionsCommand.FindHangingTransactionsCommand.HEADERS;
         assertEquals(expectedHeaders, table.get(0));
 
         long durationMinutes = TimeUnit.MILLISECONDS.toMinutes(time.milliseconds() - lastTimestamp);


[kafka] 07/11: MINOR: Pass snapshot ID directly in `RaftClient.createSnapshot` (#12981)

Posted by cm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2de4451e27e5b0e9f429da160db438ce11df3f88
Author: Jason Gustafson <ja...@confluent.io>
AuthorDate: Tue Dec 13 10:44:56 2022 -0800

    MINOR: Pass snapshot ID directly in `RaftClient.createSnapshot` (#12981)
    
    Let `RaftClient.createSnapshot` take the snapshotId directly instead of the committed offset/epoch (which may not exist).
    
    Reviewers: José Armando García Sancio <js...@apache.org>
---
 .../src/main/scala/kafka/server/BrokerServer.scala |  4 +-
 .../apache/kafka/controller/QuorumController.java  |  9 +++-
 .../org/apache/kafka/metalog/LocalLogManager.java  |  8 ++--
 .../org/apache/kafka/raft/KafkaRaftClient.java     |  5 +--
 .../java/org/apache/kafka/raft/RaftClient.java     | 23 ++++++-----
 .../org/apache/kafka/raft/ReplicatedCounter.java   |  3 +-
 .../kafka/snapshot/RecordsSnapshotWriter.java      | 46 +++++++++++++++------
 .../org/apache/kafka/snapshot/SnapshotWriter.java  |  2 +-
 .../kafka/raft/KafkaRaftClientSnapshotTest.java    | 48 +++++++++++-----------
 .../kafka/snapshot/SnapshotWriterReaderTest.java   |  8 ++--
 10 files changed, 90 insertions(+), 66 deletions(-)

diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index a83da6e0258..623338fd2f1 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -43,6 +43,7 @@ import org.apache.kafka.common.utils.{LogContext, Time, Utils}
 import org.apache.kafka.common.{ClusterResource, Endpoint}
 import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
 import org.apache.kafka.metadata.{BrokerState, VersionRange}
+import org.apache.kafka.raft
 import org.apache.kafka.raft.{RaftClient, RaftConfig}
 import org.apache.kafka.server.authorizer.Authorizer
 import org.apache.kafka.server.common.ApiMessageAndVersion
@@ -59,7 +60,8 @@ class BrokerSnapshotWriterBuilder(raftClient: RaftClient[ApiMessageAndVersion])
   override def build(committedOffset: Long,
                      committedEpoch: Int,
                      lastContainedLogTime: Long): Option[SnapshotWriter[ApiMessageAndVersion]] = {
-    raftClient.createSnapshot(committedOffset, committedEpoch, lastContainedLogTime).asScala
+    val snapshotId = new raft.OffsetAndEpoch(committedOffset + 1, committedEpoch)
+    raftClient.createSnapshot(snapshotId, lastContainedLogTime).asScala
   }
 }
 
diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
index 37e96b294d7..3dba5b401d6 100644
--- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
+++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
@@ -533,9 +533,14 @@ public final class QuorumController implements Controller {
                     )
                 );
             }
+
+            OffsetAndEpoch snapshotId = new OffsetAndEpoch(
+                committedOffset + 1,
+                committedEpoch
+            );
+
             Optional<SnapshotWriter<ApiMessageAndVersion>> writer = raftClient.createSnapshot(
-                committedOffset,
-                committedEpoch,
+                snapshotId,
                 committedTimestamp
             );
             if (writer.isPresent()) {
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 20c5c324dd2..3bc07c06af0 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
@@ -40,8 +40,8 @@ import org.apache.kafka.snapshot.RawSnapshotReader;
 import org.apache.kafka.snapshot.RawSnapshotWriter;
 import org.apache.kafka.snapshot.RecordsSnapshotReader;
 import org.apache.kafka.snapshot.RecordsSnapshotWriter;
-import org.apache.kafka.snapshot.SnapshotWriter;
 import org.apache.kafka.snapshot.SnapshotReader;
+import org.apache.kafka.snapshot.SnapshotWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,8 +51,8 @@ import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map.Entry;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.NavigableMap;
 import java.util.Objects;
 import java.util.Optional;
@@ -774,11 +774,9 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
 
     @Override
     public Optional<SnapshotWriter<ApiMessageAndVersion>> createSnapshot(
-        long committedOffset,
-        int committedEpoch,
+        OffsetAndEpoch snapshotId,
         long lastContainedLogTimestamp
     ) {
-        OffsetAndEpoch snapshotId = new OffsetAndEpoch(committedOffset + 1, committedEpoch);
         return RecordsSnapshotWriter.createWithHeader(
             () -> createNewSnapshot(snapshotId),
             1024,
diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
index 1cd6058eb0c..f21c31eeab9 100644
--- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
+++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
@@ -2342,12 +2342,11 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
 
     @Override
     public Optional<SnapshotWriter<T>> createSnapshot(
-        long committedOffset,
-        int committedEpoch,
+        OffsetAndEpoch snapshotId,
         long lastContainedLogTime
     ) {
         return RecordsSnapshotWriter.createWithHeader(
-                () -> log.createNewSnapshot(new OffsetAndEpoch(committedOffset + 1, committedEpoch)),
+                () -> log.createNewSnapshot(snapshotId),
                 MAX_BATCH_SIZE_BYTES,
                 memoryPool,
                 time,
diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java
index 727bd1bbf80..95805672311 100644
--- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java
+++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java
@@ -216,28 +216,29 @@ public interface RaftClient<T> extends AutoCloseable {
     /**
      * Create a writable snapshot file for a committed offset and epoch.
      *
-     * The RaftClient assumes that the snapshot returned will contain the records up to and
-     * including the committed offset and epoch. See {@link SnapshotWriter} for details on
-     * how to use this object. If a snapshot already exists then returns an
-     * {@link Optional#empty()}.
+     * The RaftClient assumes that the snapshot returned will contain the records up to, but not
+     * including the committed offset and epoch. If no records have been committed, it is possible
+     * to generate an empty snapshot using 0 for both the offset and epoch.
+     *
+     * See {@link SnapshotWriter} for details on how to use this object. If a snapshot already
+     * exists then returns an {@link Optional#empty()}.
      *
-     * @param committedEpoch the epoch of the committed offset
-     * @param committedOffset the last committed offset that will be included in the snapshot
+     * @param snapshotId The ID of the new snapshot, which includes the (exclusive) last committed offset
+     *                   and the last committed epoch.
      * @param lastContainedLogTime The append time of the highest record contained in this snapshot
      * @return a writable snapshot if it doesn't already exists
      * @throws IllegalArgumentException if the committed offset is greater than the high-watermark
      *         or less than the log start offset.
      */
-    Optional<SnapshotWriter<T>> createSnapshot(long committedOffset, int committedEpoch, long lastContainedLogTime);
-
+    Optional<SnapshotWriter<T>> createSnapshot(OffsetAndEpoch snapshotId, long lastContainedLogTime);
 
     /**
-     * The snapshot id for the lastest snapshot.
+     * The snapshot id for the latest snapshot.
      *
-     * Returns the snapshot id of the latest snapshot, if it exists. If a snapshot doesn't exists, returns an
+     * Returns the snapshot id of the latest snapshot, if it exists. If a snapshot doesn't exist, returns an
      * {@link Optional#empty()}.
      *
-     * @return the id of the latest snaphost, if it exists
+     * @return the id of the latest snapshot, if it exists
      */
     Optional<OffsetAndEpoch> latestSnapshotId();
 }
diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
index 27b3be163e0..65a3b6526e9 100644
--- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
+++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
@@ -115,8 +115,7 @@ public class ReplicatedCounter implements RaftClient.Listener<Integer> {
                     lastOffsetSnapshotted
                 );
                 Optional<SnapshotWriter<Integer>> snapshot = client.createSnapshot(
-                    lastCommittedOffset,
-                    lastCommittedEpoch,
+                    new OffsetAndEpoch(lastCommittedOffset + 1, lastCommittedEpoch),
                     lastCommittedTimestamp);
                 if (snapshot.isPresent()) {
                     try {
diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java
index 859a0259445..05d3fde09d0 100644
--- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java
+++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java
@@ -118,19 +118,39 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> {
         CompressionType compressionType,
         RecordSerde<T> serde
     ) {
-        return supplier.get().map(snapshot -> {
-            RecordsSnapshotWriter<T> writer = new RecordsSnapshotWriter<>(
-                    snapshot,
-                    maxBatchSize,
-                    memoryPool,
-                    snapshotTime,
-                    lastContainedLogTimestamp,
-                    compressionType,
-                    serde);
-            writer.initializeSnapshotWithHeader();
-
-            return writer;
-        });
+        return supplier.get().map(writer ->
+            createWithHeader(
+                writer,
+                maxBatchSize,
+                memoryPool,
+                snapshotTime,
+                lastContainedLogTimestamp,
+                compressionType,
+                serde
+            )
+        );
+    }
+
+    public static <T> SnapshotWriter<T> createWithHeader(
+        RawSnapshotWriter rawSnapshotWriter,
+        int maxBatchSize,
+        MemoryPool memoryPool,
+        Time snapshotTime,
+        long lastContainedLogTimestamp,
+        CompressionType compressionType,
+        RecordSerde<T> serde
+    ) {
+        RecordsSnapshotWriter<T> writer = new RecordsSnapshotWriter<>(
+            rawSnapshotWriter,
+            maxBatchSize,
+            memoryPool,
+            snapshotTime,
+            lastContainedLogTimestamp,
+            compressionType,
+            serde
+        );
+        writer.initializeSnapshotWithHeader();
+        return writer;
     }
 
     @Override
diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java
index 77b29d94498..537335c058a 100644
--- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java
+++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java
@@ -32,7 +32,7 @@ import java.util.List;
  * topic partition from offset 0 up to but not including the end offset in the snapshot
  * id.
  *
- * @see org.apache.kafka.raft.KafkaRaftClient#createSnapshot(long, int, long)
+ * @see org.apache.kafka.raft.RaftClient#createSnapshot(OffsetAndEpoch, long)
  */
 public interface SnapshotWriter<T> extends AutoCloseable {
     /**
diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java
index ecef1ffa933..23fe3fd0694 100644
--- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java
+++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java
@@ -232,7 +232,7 @@ final public class KafkaRaftClientSnapshotTest {
 
         // Generate a new snapshot
         OffsetAndEpoch secondSnapshotId = new OffsetAndEpoch(localLogEndOffset, epoch);
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(secondSnapshotId.offset() - 1, secondSnapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(secondSnapshotId, 0).get()) {
             assertEquals(secondSnapshotId, snapshot.snapshotId());
             snapshot.freeze();
         }
@@ -278,7 +278,7 @@ final public class KafkaRaftClientSnapshotTest {
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
         OffsetAndEpoch snapshotId = new OffsetAndEpoch(localLogEndOffset, epoch);
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset() - 1, snapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId, 0).get()) {
             assertEquals(snapshotId, snapshot.snapshotId());
             snapshot.freeze();
         }
@@ -318,7 +318,7 @@ final public class KafkaRaftClientSnapshotTest {
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
         // Create a snapshot at the high watermark
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId, 0).get()) {
             assertEquals(oldestSnapshotId, snapshot.snapshotId());
             snapshot.freeze();
         }
@@ -357,7 +357,7 @@ final public class KafkaRaftClientSnapshotTest {
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
         // Create a snapshot at the high watermark
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId, 0).get()) {
             assertEquals(oldestSnapshotId, snapshot.snapshotId());
             snapshot.freeze();
         }
@@ -400,7 +400,7 @@ final public class KafkaRaftClientSnapshotTest {
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
         // Create a snapshot at the high watermark
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId, 0).get()) {
             assertEquals(oldestSnapshotId, snapshot.snapshotId());
             snapshot.freeze();
         }
@@ -438,7 +438,7 @@ final public class KafkaRaftClientSnapshotTest {
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
         // Create a snapshot at the high watermark
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId, 0).get()) {
             assertEquals(oldestSnapshotId, snapshot.snapshotId());
             snapshot.freeze();
         }
@@ -482,7 +482,7 @@ final public class KafkaRaftClientSnapshotTest {
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
         // Create a snapshot at the high watermark
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId, 0).get()) {
             assertEquals(oldestSnapshotId, snapshot.snapshotId());
             snapshot.freeze();
         }
@@ -572,7 +572,7 @@ final public class KafkaRaftClientSnapshotTest {
 
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset() - 1, snapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId, 0).get()) {
             assertEquals(snapshotId, snapshot.snapshotId());
             snapshot.append(records);
             snapshot.freeze();
@@ -621,7 +621,7 @@ final public class KafkaRaftClientSnapshotTest {
 
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset() - 1, snapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId, 0).get()) {
             assertEquals(snapshotId, snapshot.snapshotId());
             snapshot.append(records);
             snapshot.freeze();
@@ -730,7 +730,7 @@ final public class KafkaRaftClientSnapshotTest {
 
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset() - 1, snapshotId.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId, 0).get()) {
             assertEquals(snapshotId, snapshot.snapshotId());
             snapshot.append(records);
             snapshot.freeze();
@@ -1554,7 +1554,7 @@ final public class KafkaRaftClientSnapshotTest {
         int epoch = 2;
 
         List<String> appendRecords = Arrays.asList("a", "b", "c");
-        OffsetAndEpoch invalidSnapshotId1 = new OffsetAndEpoch(3, epoch);
+        OffsetAndEpoch invalidSnapshotId1 = new OffsetAndEpoch(4, epoch);
 
         RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
                 .appendToLog(epoch, appendRecords)
@@ -1567,7 +1567,7 @@ final public class KafkaRaftClientSnapshotTest {
         // When leader creating snapshot:
         // 1.1 high watermark cannot be empty
         assertEquals(OptionalLong.empty(), context.client.highWatermark());
-        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1.offset(), invalidSnapshotId1.epoch(), 0));
+        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1, 0));
 
         // 1.2 high watermark must larger than or equal to the snapshotId's endOffset
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
@@ -1578,18 +1578,18 @@ final public class KafkaRaftClientSnapshotTest {
         context.client.poll();
         assertEquals(context.log.endOffset().offset, context.client.highWatermark().getAsLong() + newRecords.size());
 
-        OffsetAndEpoch invalidSnapshotId2 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() + 1, currentEpoch);
-        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2.offset(), invalidSnapshotId2.epoch(), 0));
+        OffsetAndEpoch invalidSnapshotId2 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() + 2, currentEpoch);
+        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2, 0));
 
         // 2 the quorum epoch must larger than or equal to the snapshotId's epoch
-        OffsetAndEpoch invalidSnapshotId3 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() - 2, currentEpoch + 1);
-        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3.offset(), invalidSnapshotId3.epoch(), 0));
+        OffsetAndEpoch invalidSnapshotId3 = new OffsetAndEpoch(context.client.highWatermark().getAsLong(), currentEpoch + 1);
+        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3, 0));
 
         // 3 the snapshotId should be validated against endOffsetForEpoch
         OffsetAndEpoch endOffsetForEpoch = context.log.endOffsetForEpoch(epoch);
         assertEquals(epoch, endOffsetForEpoch.epoch());
-        OffsetAndEpoch invalidSnapshotId4 = new OffsetAndEpoch(endOffsetForEpoch.offset() + 1, epoch);
-        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4.offset(), invalidSnapshotId4.epoch(), 0));
+        OffsetAndEpoch invalidSnapshotId4 = new OffsetAndEpoch(endOffsetForEpoch.offset() + 2, epoch);
+        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4, 0));
     }
 
     @Test
@@ -1608,8 +1608,8 @@ final public class KafkaRaftClientSnapshotTest {
         // When follower creating snapshot:
         // 1) The high watermark cannot be empty
         assertEquals(OptionalLong.empty(), context.client.highWatermark());
-        OffsetAndEpoch invalidSnapshotId1 = new OffsetAndEpoch(0, 0);
-        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1.offset(), invalidSnapshotId1.epoch(), 0));
+        OffsetAndEpoch invalidSnapshotId1 = new OffsetAndEpoch(1, 0);
+        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1, 0));
 
         // Poll for our first fetch request
         context.pollUntilRequest();
@@ -1627,11 +1627,11 @@ final public class KafkaRaftClientSnapshotTest {
         // 2) The high watermark must be larger than or equal to the snapshotId's endOffset
         int currentEpoch = context.currentEpoch();
         OffsetAndEpoch invalidSnapshotId2 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() + 1, currentEpoch);
-        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2.offset(), invalidSnapshotId2.epoch(), 0));
+        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2, 0));
 
         // 3) The quorum epoch must be larger than or equal to the snapshotId's epoch
-        OffsetAndEpoch invalidSnapshotId3 = new OffsetAndEpoch(context.client.highWatermark().getAsLong(), currentEpoch + 1);
-        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3.offset(), invalidSnapshotId3.epoch(), 0));
+        OffsetAndEpoch invalidSnapshotId3 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() + 1, currentEpoch + 1);
+        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3, 0));
 
         // The high watermark advances to be larger than log.endOffsetForEpoch(3), to test the case 3
         context.pollUntilRequest();
@@ -1650,7 +1650,7 @@ final public class KafkaRaftClientSnapshotTest {
         OffsetAndEpoch endOffsetForEpoch = context.log.endOffsetForEpoch(3);
         assertEquals(3, endOffsetForEpoch.epoch());
         OffsetAndEpoch invalidSnapshotId4 = new OffsetAndEpoch(endOffsetForEpoch.offset() + 1, epoch);
-        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4.offset(), invalidSnapshotId4.epoch(), 0));
+        assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4, 0));
     }
 
     private static FetchSnapshotRequestData fetchSnapshotRequest(
diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java
index c0fd2dc3f97..bd8258cb708 100644
--- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java
+++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java
@@ -64,7 +64,7 @@ final public class SnapshotWriterReaderTest {
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
         // Create an empty snapshot and freeze it immediately
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset() - 1, id.epoch(), magicTimestamp).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id, magicTimestamp).get()) {
             assertEquals(id, snapshot.snapshotId());
             snapshot.freeze();
         }
@@ -97,7 +97,7 @@ final public class SnapshotWriterReaderTest {
 
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset() - 1, id.epoch(), magicTimestamp).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id, magicTimestamp).get()) {
             assertEquals(id, snapshot.snapshotId());
             expected.forEach(batch -> assertDoesNotThrow(() -> snapshot.append(batch)));
             snapshot.freeze();
@@ -129,7 +129,7 @@ final public class SnapshotWriterReaderTest {
 
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset() - 1, id.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id, 0).get()) {
             assertEquals(id, snapshot.snapshotId());
             expected.forEach(batch -> {
                 assertDoesNotThrow(() -> snapshot.append(batch));
@@ -157,7 +157,7 @@ final public class SnapshotWriterReaderTest {
 
         context.advanceLocalLeaderHighWatermarkToLogEndOffset();
 
-        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset() - 1, id.epoch(), 0).get()) {
+        try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id, 0).get()) {
             assertEquals(id, snapshot.snapshotId());
             expected.forEach(batch -> {
                 assertDoesNotThrow(() -> snapshot.append(batch));