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

[GitHub] [kafka] hachikuji commented on a diff in pull request #12776: KAFKA-14327: Unify KRaft snapshot generation between broker and controller

hachikuji commented on code in PR #12776:
URL: https://github.com/apache/kafka/pull/12776#discussion_r1009809439


##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -467,36 +467,41 @@ public void testIterator(MetadataVersion metadataVersion) throws Exception {
                     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).
                         setPort((short) 9092).
                         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),

Review Comment:
   nit: looks misaligned



##########
server-common/src/test/java/org/apache/kafka/queue/KafkaEventQueueTest.java:
##########
@@ -240,4 +242,29 @@ public void handleException(Throwable e) {
         assertEquals(RejectedExecutionException.class, assertThrows(
             ExecutionException.class, () -> future.get()).getCause().getClass());
     }
-}
+
+    @Test
+    public void testEmpty() throws Exception {
+        KafkaEventQueue queue = new KafkaEventQueue(Time.SYSTEM, new LogContext(),
+                "testEmpty");
+        assertTrue(queue.isEmpty());
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        queue.append(() -> future.get());
+        assertFalse(queue.isEmpty());

Review Comment:
   This assertion is flaky locally. Can we use `waitForCondition` for all cases where we expect the queue to become empty?



##########
core/src/test/java/kafka/testkit/KafkaClusterTestKit.java:
##########
@@ -191,21 +191,36 @@ public KafkaClusterTestKit build() throws Exception {
                     KafkaRaftManager<ApiMessageAndVersion> raftManager = new KafkaRaftManager<>(
                         metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftServer.MetadataTopicId(),
                         Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future);
-                    ControllerServer controller = new ControllerServer(
-                        nodes.controllerProperties(node.id()),
-                        config,
-                        raftManager,
-                        Time.SYSTEM,
-                        new Metrics(),
-                        new MockControllerMetrics(),
-                        Option.apply(threadNamePrefix),
-                        connectFutureManager.future,
-                        KafkaRaftServer.configSchema(),
-                        raftManager.apiVersions(),
-                        bootstrapMetadata,
-                        metadataFaultHandler,
-                        fatalFaultHandler
-                    );
+                    MetadataLoader metadataLoader = null;
+                    ControllerServer controller = null;
+                    try {
+                        metadataLoader = new MetadataLoader.Builder().
+                                setNodeId(node.id()).
+                                setTime(Time.SYSTEM).
+                                setThreadNamePrefix(threadNamePrefix).
+                                setFaultHandler(fatalFaultHandler).
+                                build();
+                        controller = new ControllerServer(
+                                nodes.controllerProperties(node.id()),
+                                config,
+                                raftManager,
+                                Time.SYSTEM,
+                                new Metrics(),
+                                new MockControllerMetrics(),
+                                Option.apply(threadNamePrefix),
+                                connectFutureManager.future,
+                                KafkaRaftServer.configSchema(),
+                                raftManager.apiVersions(),
+                                bootstrapMetadata,
+                                metadataFaultHandler,
+                                fatalFaultHandler,
+                                metadataLoader);
+                    } catch (Throwable e) {
+                        log.error("Error creating controller {}", node.id(), e);
+                        metadataLoader.close();
+                        controller.shutdown();

Review Comment:
   There should probably be a null check. Same for `broker` below.



##########
metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java:
##########
@@ -0,0 +1,504 @@
+/*
+ * 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.MetadataVersionChange;
+import org.apache.kafka.image.MetadataVersionChangeException;
+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.snapshot.SnapshotReader;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * 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 = null;
+        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 { }
+        };
+
+        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 MetadataLoader build() {
+            if (logContext == null) {
+                logContext = new LogContext("[MetadataLoader " + nodeId + "] ");
+            }
+            if (faultHandler == null) throw new RuntimeException("You must set a fault handler.");
+            return new MetadataLoader(
+                time,
+                logContext,
+                threadNamePrefix,
+                faultHandler,
+                metrics);
+        }
+    }
+
+    /**
+     * 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;
+
+    /**
+     * The publishers which should receive cluster metadata updates.
+     */
+    private final List<MetadataPublisher> publishers;
+
+    /**
+     * 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
+    ) {
+        this.log = logContext.logger(MetadataLoader.class);
+        this.time = time;
+        this.faultHandler = faultHandler;
+        this.metrics = metrics;
+        this.publishers = new ArrayList<>();
+        this.image = MetadataImage.EMPTY;
+        this.eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix);
+    }
+
+    @Override
+    public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                LogDeltaManifest manifest = loadLogDelta(delta, reader);
+                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;
+                }
+                for (MetadataPublisher publisher : publishers) {
+                    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);
+                    }
+                }
+                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 (MetadataVersionChangeException e) {
+                    handleMetadataVersionChange(delta,
+                        e.change(),
+                        batch.baseOffset() + indexWithinBatch,
+                        lastEpoch,
+                        lastContainedLogTimeMs);
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error loading metadata log record from offset " +
+                            batch.baseOffset() + indexWithinBatch, e);
+                }
+                lastEpoch = batch.epoch();

Review Comment:
   nit: pull these two lines outside the loop since they do not depend on the record



##########
metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.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 {
+    private final static int DEFAULT_BATCH_SIZE = 100;
+
+    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 emit(MetadataImage image) {
+        MetadataProvenance provenance = image.provenance();
+        Optional<SnapshotWriter<ApiMessageAndVersion>> snapshotWriter =
+            raftClient.createSnapshot(provenance.offset(),
+                    provenance.epoch(),
+                    provenance.lastContainedLogTimeMs());
+        if (!snapshotWriter.isPresent()) {
+            log.error("Not generating {} because it already exists.", provenance.snapshotName());
+            return;
+        }
+        try (RaftSnapshotWriter writer = new RaftSnapshotWriter(snapshotWriter.get(), batchSize)) {
+            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 {
+            try {
+                snapshotWriter.get().close();

Review Comment:
   This would be clearer without using the try-with-resources, which would also end up closing the writer. I do not know which logic will execute first. Perhaps it's best not to depend on the order.



##########
metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java:
##########
@@ -0,0 +1,504 @@
+/*
+ * 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.MetadataVersionChange;
+import org.apache.kafka.image.MetadataVersionChangeException;
+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.snapshot.SnapshotReader;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * 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 = null;
+        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 { }
+        };
+
+        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 MetadataLoader build() {
+            if (logContext == null) {
+                logContext = new LogContext("[MetadataLoader " + nodeId + "] ");
+            }
+            if (faultHandler == null) throw new RuntimeException("You must set a fault handler.");
+            return new MetadataLoader(
+                time,
+                logContext,
+                threadNamePrefix,
+                faultHandler,
+                metrics);
+        }
+    }
+
+    /**
+     * 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;
+
+    /**
+     * The publishers which should receive cluster metadata updates.
+     */
+    private final List<MetadataPublisher> publishers;
+
+    /**
+     * 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
+    ) {
+        this.log = logContext.logger(MetadataLoader.class);
+        this.time = time;
+        this.faultHandler = faultHandler;
+        this.metrics = metrics;
+        this.publishers = new ArrayList<>();
+        this.image = MetadataImage.EMPTY;
+        this.eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix);
+    }
+
+    @Override
+    public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                LogDeltaManifest manifest = loadLogDelta(delta, reader);
+                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;
+                }
+                for (MetadataPublisher publisher : publishers) {
+                    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);
+                    }
+                }
+                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 (MetadataVersionChangeException e) {
+                    handleMetadataVersionChange(delta,
+                        e.change(),
+                        batch.baseOffset() + indexWithinBatch,
+                        lastEpoch,
+                        lastContainedLogTimeMs);
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error loading metadata log record from offset " +
+                            batch.baseOffset() + indexWithinBatch, e);
+                }
+                lastEpoch = batch.epoch();
+                lastContainedLogTimeMs = batch.appendTimestamp();
+                indexWithinBatch++;
+            }
+            metrics.updateBatchSize(batch.records().size());
+            lastOffset = batch.lastOffset();
+            numBytes += batch.sizeInBytes();
+            numBatches++;
+        }
+        MetadataProvenance provenance =
+                new MetadataProvenance(lastOffset, lastEpoch, lastContainedLogTimeMs);
+        long elapsedNs = time.nanoseconds() - startNs;
+        // TODO: this metric should be renamed something like "delta processing time"
+        metrics.updateBatchProcessingTime(elapsedNs);
+        return new LogDeltaManifest(provenance,
+                numBatches,
+                elapsedNs,
+                numBytes);
+    }
+
+    /**
+     * Handle a change in the metadata version.
+     *
+     * @param delta             The metadata delta we're working with.
+     * @param change            The change we're handling.
+     * @param changeOffset      The offset of the change.
+     * @param preChangeEpoch    The log epoch BEFORE the change.
+     */
+    void handleMetadataVersionChange(
+        MetadataDelta delta,
+        MetadataVersionChange change,
+        long changeOffset,
+        int preChangeEpoch,
+        long lastContainedLogTimeMs
+    ) {
+        // First, we materialize the current metadata image and send it to all the publishers that
+        // are interested in preVersionChange images. The most important one is the publisher which
+        // writes snapshots out to disk.
+        MetadataProvenance provenance =
+                new MetadataProvenance(changeOffset - 1, preChangeEpoch, lastContainedLogTimeMs);
+        PreVersionChangeManifest manifest = new PreVersionChangeManifest(provenance, change);
+        MetadataImage preVersionChangeImage = delta.apply(provenance);
+        for (MetadataPublisher publisher : publishers) {
+            try {
+                publisher.publishPreVersionChangeImage(delta, preVersionChangeImage, manifest);
+            } catch (Throwable e) {
+                faultHandler.handleFault("Error publishing pre-version change image at offset " +
+                    provenance.offset() + " with publisher " + publisher.name(), e);
+            }
+        }
+        // Then, we clear the current delta and write out the current image to it in the new format.
+        // If any metadata was lost, we just log it here. We cannot prevent the losses because the
+        // decision to change the metadata version was already taken by the controller.
+        delta.clear();
+        ImageReWriter writer = new ImageReWriter(delta);
+        preVersionChangeImage.write(writer, new ImageWriterOptions.Builder().
+                        setMetadataVersion(change.newVersion()).
+                        setLossHandler(loss -> {
+                            log.warn("{}", loss.getMessage());
+                        }).
+                        build());
+    }
+
+    @Override
+    public void handleSnapshot(SnapshotReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                SnapshotManifest manifest = loadSnapshot(delta, reader);
+                try {
+                    image = delta.apply(manifest.provenance());
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error generating new metadata image from " +
+                            "snapshot at offset " + reader.lastContainedLogOffset(), e);
+                    return;
+                }
+                for (MetadataPublisher publisher : publishers) {
+                    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);
+                    }
+                }
+                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. Additionally, it is an error for a version change to occur unless
+     * it is the very first record. 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.beginShutdown("installPublishers", () -> {

Review Comment:
   Should be `append`, right? Same for `removeAndClosePublisher`. Do we have tests that are failing?



##########
core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataListenerTest.scala:
##########
@@ -1,413 +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.server.metadata
-
-import java.util
-import java.util.concurrent.atomic.AtomicReference
-import java.util.{Collections, Optional}
-import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionChangeRecord, PartitionRecord, RegisterBrokerRecord, TopicRecord}
-import org.apache.kafka.common.metrics.Metrics
-import org.apache.kafka.common.utils.Time
-import org.apache.kafka.common.{Endpoint, Uuid}
-import org.apache.kafka.image.{MetadataDelta, MetadataImage}
-import org.apache.kafka.metadata.util.SnapshotReason
-import org.apache.kafka.metadata.{BrokerRegistration, RecordTestUtils, VersionRange}
-import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
-import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler}
-import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
-import org.junit.jupiter.api.{AfterEach, Test}
-
-import scala.jdk.CollectionConverters._
-
-class BrokerMetadataListenerTest {
-  private val metadataLoadingFaultHandler = new MockFaultHandler("metadata loading")
-
-  @AfterEach
-  def verifyNoFaults(): Unit = {
-    metadataLoadingFaultHandler.maybeRethrowFirstException()
-  }
-
-  private def newBrokerMetadataListener(
-    metrics: BrokerServerMetrics = BrokerServerMetrics(new Metrics()),
-    snapshotter: Option[MetadataSnapshotter] = None,
-    maxBytesBetweenSnapshots: Long = 1000000L,
-    faultHandler: FaultHandler = metadataLoadingFaultHandler
-  ): BrokerMetadataListener = {
-    new BrokerMetadataListener(
-      brokerId = 0,
-      time = Time.SYSTEM,
-      threadNamePrefix = None,
-      maxBytesBetweenSnapshots = maxBytesBetweenSnapshots,
-      snapshotter = snapshotter,
-      brokerMetrics = metrics,
-      _metadataLoadingFaultHandler = faultHandler
-    )
-  }
-
-  @Test
-  def testCreateAndClose(): Unit = {
-    val listener = newBrokerMetadataListener()
-    listener.close()
-  }
-
-  @Test
-  def testPublish(): Unit = {
-    val metrics = BrokerServerMetrics(new Metrics())
-    val listener = newBrokerMetadataListener(metrics = metrics)
-    try {
-      val unfencedTimestamp = 300L
-      listener.handleCommit(
-        RecordTestUtils.mockBatchReader(
-          100,
-          unfencedTimestamp,
-          util.Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord().
-            setBrokerId(0).
-            setBrokerEpoch(100L).
-            setFenced(false).
-            setRack(null).
-            setIncarnationId(Uuid.fromString("GFBwlTcpQUuLYQ2ig05CSg")), 0.toShort))
-        )
-      )
-      val imageRecords = listener.getImageRecords().get()
-      assertEquals(0, imageRecords.size())
-      assertEquals(100L, listener.highestMetadataOffset)
-      assertEquals(0L, metrics.lastAppliedRecordOffset.get)
-      assertEquals(0L, metrics.lastAppliedRecordTimestamp.get)
-      assertEquals(0L, metrics.metadataLoadErrorCount.get)
-      assertEquals(0L, metrics.metadataApplyErrorCount.get)
-
-      val fencedTimestamp = 500L
-      val fencedLastOffset = 200L
-      listener.handleCommit(
-        RecordTestUtils.mockBatchReader(
-          fencedLastOffset,
-          fencedTimestamp,
-          util.Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord().
-            setBrokerId(1).
-            setBrokerEpoch(200L).
-            setFenced(true).
-            setRack(null).
-            setIncarnationId(Uuid.fromString("QkOQtNKVTYatADcaJ28xDg")), 0.toShort))
-        )
-      )
-      listener.startPublishing(new MetadataPublisher {
-        override def publish(delta: MetadataDelta, newImage: MetadataImage): Unit = {
-          assertEquals(200L, newImage.highestOffsetAndEpoch().offset)
-          assertEquals(new BrokerRegistration(0, 100L,
-            Uuid.fromString("GFBwlTcpQUuLYQ2ig05CSg"), Collections.emptyList[Endpoint](),
-            Collections.emptyMap[String, VersionRange](), Optional.empty[String](), false, false),
-            delta.clusterDelta().broker(0))
-          assertEquals(new BrokerRegistration(1, 200L,
-            Uuid.fromString("QkOQtNKVTYatADcaJ28xDg"), Collections.emptyList[Endpoint](),
-            Collections.emptyMap[String, VersionRange](), Optional.empty[String](), true, false),
-            delta.clusterDelta().broker(1))
-        }
-
-        override def publishedOffset: Long = -1
-      }).get()
-
-      assertEquals(fencedLastOffset, metrics.lastAppliedRecordOffset.get)
-      assertEquals(fencedTimestamp, metrics.lastAppliedRecordTimestamp.get)
-      assertEquals(0L, metrics.metadataLoadErrorCount.get)
-      assertEquals(0L, metrics.metadataApplyErrorCount.get)
-    } finally {
-      listener.close()
-    }
-  }
-
-  class MockMetadataSnapshotter extends MetadataSnapshotter {
-    var image = MetadataImage.EMPTY
-    val failure = new AtomicReference[Throwable](null)
-    var activeSnapshotOffset = -1L
-    var prevCommittedOffset = -1L
-    var prevCommittedEpoch = -1
-    var prevLastContainedLogTime = -1L
-
-    override def maybeStartSnapshot(lastContainedLogTime: Long, newImage: MetadataImage, reason: Set[SnapshotReason]): Boolean = {
-      try {
-        if (activeSnapshotOffset == -1L) {
-          assertTrue(prevCommittedOffset <= newImage.highestOffsetAndEpoch().offset)
-          assertTrue(prevCommittedEpoch <= newImage.highestOffsetAndEpoch().epoch)
-          assertTrue(prevLastContainedLogTime <= lastContainedLogTime)
-          prevCommittedOffset = newImage.highestOffsetAndEpoch().offset
-          prevCommittedEpoch = newImage.highestOffsetAndEpoch().epoch
-          prevLastContainedLogTime = lastContainedLogTime
-          image = newImage
-          activeSnapshotOffset = newImage.highestOffsetAndEpoch().offset
-          true
-        } else {
-          false
-        }
-      } catch {
-        case t: Throwable => failure.compareAndSet(null, t)
-      }
-    }
-  }
-
-  class MockMetadataPublisher extends MetadataPublisher {
-    var image = MetadataImage.EMPTY
-
-    override def publish(delta: MetadataDelta, newImage: MetadataImage): Unit = {
-      image = newImage
-    }
-
-    override def publishedOffset: Long = -1
-  }
-
-  private val FOO_ID = Uuid.fromString("jj1G9utnTuCegi_gpnRgYw")
-  private val BAR_ID = Uuid.fromString("SzN5j0LvSEaRIJHrxfMAlg")
-
-  private def generateManyRecords(listener: BrokerMetadataListener,
-                                  endOffset: Long): Unit = {
-    (0 to 10000).foreach { _ =>
-      listener.handleCommit(
-        RecordTestUtils.mockBatchReader(
-          endOffset,
-          0,
-          util.Arrays.asList(
-            new ApiMessageAndVersion(new PartitionChangeRecord().
-              setPartitionId(0).
-              setTopicId(FOO_ID).
-              setRemovingReplicas(Collections.singletonList(1)), 0.toShort),
-            new ApiMessageAndVersion(new PartitionChangeRecord().
-              setPartitionId(0).
-              setTopicId(FOO_ID).
-              setRemovingReplicas(Collections.emptyList()), 0.toShort)
-          )
-        )
-      )
-    }
-    listener.getImageRecords().get()
-  }
-
-  private def generateBadRecords(listener: BrokerMetadataListener,
-                                endOffset: Long): Unit = {
-    listener.handleCommit(
-      RecordTestUtils.mockBatchReader(
-        endOffset,
-        0,
-        util.Arrays.asList(
-          new ApiMessageAndVersion(new PartitionChangeRecord().
-            setPartitionId(0).
-            setTopicId(BAR_ID).
-            setRemovingReplicas(Collections.singletonList(1)), 0.toShort),
-          new ApiMessageAndVersion(new PartitionChangeRecord().
-            setPartitionId(0).
-            setTopicId(BAR_ID).
-            setRemovingReplicas(Collections.emptyList()), 0.toShort)
-        )
-      )
-    )
-    listener.getImageRecords().get()
-  }
-
-  @Test
-  def testHandleCommitsWithNoSnapshotterDefined(): Unit = {
-    val listener = newBrokerMetadataListener(maxBytesBetweenSnapshots = 1000L)
-    try {
-      val brokerIds = 0 to 3
-
-      registerBrokers(listener, brokerIds, endOffset = 100L)
-      createTopicWithOnePartition(listener, replicas = brokerIds, endOffset = 200L)
-      listener.getImageRecords().get()
-      assertEquals(200L, listener.highestMetadataOffset)
-
-      generateManyRecords(listener, endOffset = 1000L)
-      assertEquals(1000L, listener.highestMetadataOffset)
-    } finally {
-      listener.close()
-    }
-  }
-
-  @Test
-  def testCreateSnapshot(): Unit = {
-    val snapshotter = new MockMetadataSnapshotter()
-    val listener = newBrokerMetadataListener(snapshotter = Some(snapshotter),
-      maxBytesBetweenSnapshots = 1000L)
-    try {
-      val brokerIds = 0 to 3
-
-      registerBrokers(listener, brokerIds, endOffset = 100L)
-      createTopicWithOnePartition(listener, replicas = brokerIds, endOffset = 200L)
-      listener.getImageRecords().get()
-      assertEquals(200L, listener.highestMetadataOffset)
-
-      // Check that we generate at least one snapshot once we see enough records.
-      assertEquals(-1L, snapshotter.prevCommittedOffset)
-      generateManyRecords(listener, 1000L)
-      assertEquals(1000L, snapshotter.prevCommittedOffset)
-      assertEquals(1000L, snapshotter.activeSnapshotOffset)
-      snapshotter.activeSnapshotOffset = -1L
-
-      // Test creating a new snapshot after publishing it.
-      val publisher = new MockMetadataPublisher()
-      listener.startPublishing(publisher).get()
-      generateManyRecords(listener, 2000L)
-      listener.getImageRecords().get()
-      assertEquals(2000L, snapshotter.activeSnapshotOffset)
-      assertEquals(2000L, snapshotter.prevCommittedOffset)
-
-      // Test how we handle the snapshotter returning false.
-      generateManyRecords(listener, 3000L)
-      assertEquals(2000L, snapshotter.activeSnapshotOffset)
-      generateManyRecords(listener, 4000L)
-      assertEquals(2000L, snapshotter.activeSnapshotOffset)
-      snapshotter.activeSnapshotOffset = -1L
-      generateManyRecords(listener, 5000L)
-      assertEquals(5000L, snapshotter.activeSnapshotOffset)
-      assertEquals(null, snapshotter.failure.get())
-    } finally {
-      listener.close()
-    }
-  }
-
-  @Test
-  def testNotSnapshotAfterMetadataVersionChangeBeforePublishing(): Unit = {

Review Comment:
   Are all of these tests recreated somewhere?



##########
metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java:
##########
@@ -0,0 +1,504 @@
+/*
+ * 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.MetadataVersionChange;
+import org.apache.kafka.image.MetadataVersionChangeException;
+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.snapshot.SnapshotReader;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * 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 = null;
+        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 { }
+        };
+
+        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 MetadataLoader build() {
+            if (logContext == null) {
+                logContext = new LogContext("[MetadataLoader " + nodeId + "] ");
+            }
+            if (faultHandler == null) throw new RuntimeException("You must set a fault handler.");
+            return new MetadataLoader(
+                time,
+                logContext,
+                threadNamePrefix,
+                faultHandler,
+                metrics);
+        }
+    }
+
+    /**
+     * 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;
+
+    /**
+     * The publishers which should receive cluster metadata updates.
+     */
+    private final List<MetadataPublisher> publishers;
+
+    /**
+     * 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
+    ) {
+        this.log = logContext.logger(MetadataLoader.class);
+        this.time = time;
+        this.faultHandler = faultHandler;
+        this.metrics = metrics;
+        this.publishers = new ArrayList<>();
+        this.image = MetadataImage.EMPTY;
+        this.eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix);
+    }
+
+    @Override
+    public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                LogDeltaManifest manifest = loadLogDelta(delta, reader);
+                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;
+                }
+                for (MetadataPublisher publisher : publishers) {
+                    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);
+                    }
+                }
+                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 (MetadataVersionChangeException e) {
+                    handleMetadataVersionChange(delta,
+                        e.change(),
+                        batch.baseOffset() + indexWithinBatch,
+                        lastEpoch,
+                        lastContainedLogTimeMs);
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error loading metadata log record from offset " +
+                            batch.baseOffset() + indexWithinBatch, e);
+                }
+                lastEpoch = batch.epoch();
+                lastContainedLogTimeMs = batch.appendTimestamp();
+                indexWithinBatch++;
+            }
+            metrics.updateBatchSize(batch.records().size());
+            lastOffset = batch.lastOffset();
+            numBytes += batch.sizeInBytes();
+            numBatches++;
+        }
+        MetadataProvenance provenance =
+                new MetadataProvenance(lastOffset, lastEpoch, lastContainedLogTimeMs);
+        long elapsedNs = time.nanoseconds() - startNs;
+        // TODO: this metric should be renamed something like "delta processing time"
+        metrics.updateBatchProcessingTime(elapsedNs);
+        return new LogDeltaManifest(provenance,
+                numBatches,
+                elapsedNs,
+                numBytes);
+    }
+
+    /**
+     * Handle a change in the metadata version.
+     *
+     * @param delta             The metadata delta we're working with.
+     * @param change            The change we're handling.
+     * @param changeOffset      The offset of the change.
+     * @param preChangeEpoch    The log epoch BEFORE the change.
+     */
+    void handleMetadataVersionChange(
+        MetadataDelta delta,
+        MetadataVersionChange change,
+        long changeOffset,
+        int preChangeEpoch,
+        long lastContainedLogTimeMs
+    ) {
+        // First, we materialize the current metadata image and send it to all the publishers that

Review Comment:
   It would be helpful to add some motivation for this. Is it for recovery if something goes wrong after upgrade/downgrade?



##########
core/src/main/scala/kafka/server/KafkaConfig.scala:
##########
@@ -726,6 +728,7 @@ object KafkaConfig {
   val MetadataLogDirDoc = "This configuration determines where we put the metadata log for clusters in KRaft mode. " +
     "If it is not set, the metadata log is placed in the first log directory from log.dirs."
   val MetadataSnapshotMaxNewRecordBytesDoc = "This is the maximum number of bytes in the log between the latest snapshot and the high-watermark needed before generating a new snapshot."
+  val MetadataSnapshotMaxIntervalMsDoc = "This is the maximum number of milliseconds that can elapse between the previous metadata snapshot and the next one."

Review Comment:
   Perhaps add something like this?
   
   > In other words, metadata snapshots will be generated at least as frequently as specified by this configuration. Note that if `metadata.log.max.record.bytes.between.snapshots` bytes of new data is reached before the full specified interval completes, then the snapshot will still be generated.



##########
core/src/main/scala/kafka/server/ControllerServer.scala:
##########
@@ -258,6 +264,14 @@ class ControllerServer(
         shutdown()
         throw e
     }
+    config.dynamicConfig.addReconfigurables(this)
+    dynamicConfigHandlers = immutable.Map[String, ConfigHandler](
+      ConfigType.Broker -> new BrokerConfigHandler(config, quotaManagers))
+    controllerMetadataPublisher = new ControllerMetadataPublisher(
+      config,
+      dynamicConfigHandlers,
+      fatalFaultHandler)
+    metadataLoader.installPublishers(List().asJava)

Review Comment:
   Don't we need the publisher in here? Do we have any integration tests which verify this all works?



##########
metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java:
##########
@@ -0,0 +1,504 @@
+/*
+ * 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.MetadataVersionChange;
+import org.apache.kafka.image.MetadataVersionChangeException;
+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.snapshot.SnapshotReader;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * 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 = null;
+        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 { }
+        };
+
+        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 MetadataLoader build() {
+            if (logContext == null) {
+                logContext = new LogContext("[MetadataLoader " + nodeId + "] ");
+            }
+            if (faultHandler == null) throw new RuntimeException("You must set a fault handler.");
+            return new MetadataLoader(
+                time,
+                logContext,
+                threadNamePrefix,
+                faultHandler,
+                metrics);
+        }
+    }
+
+    /**
+     * 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;
+
+    /**
+     * The publishers which should receive cluster metadata updates.
+     */
+    private final List<MetadataPublisher> publishers;
+
+    /**
+     * 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
+    ) {
+        this.log = logContext.logger(MetadataLoader.class);
+        this.time = time;
+        this.faultHandler = faultHandler;
+        this.metrics = metrics;
+        this.publishers = new ArrayList<>();
+        this.image = MetadataImage.EMPTY;
+        this.eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix);
+    }
+
+    @Override
+    public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                LogDeltaManifest manifest = loadLogDelta(delta, reader);
+                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;
+                }
+                for (MetadataPublisher publisher : publishers) {
+                    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);
+                    }
+                }
+                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 (MetadataVersionChangeException e) {
+                    handleMetadataVersionChange(delta,
+                        e.change(),
+                        batch.baseOffset() + indexWithinBatch,
+                        lastEpoch,
+                        lastContainedLogTimeMs);
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error loading metadata log record from offset " +
+                            batch.baseOffset() + indexWithinBatch, e);
+                }
+                lastEpoch = batch.epoch();
+                lastContainedLogTimeMs = batch.appendTimestamp();
+                indexWithinBatch++;
+            }
+            metrics.updateBatchSize(batch.records().size());
+            lastOffset = batch.lastOffset();
+            numBytes += batch.sizeInBytes();
+            numBatches++;
+        }
+        MetadataProvenance provenance =
+                new MetadataProvenance(lastOffset, lastEpoch, lastContainedLogTimeMs);
+        long elapsedNs = time.nanoseconds() - startNs;
+        // TODO: this metric should be renamed something like "delta processing time"

Review Comment:
   nit: can we move this comment to a jira? 



##########
metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java:
##########
@@ -0,0 +1,504 @@
+/*
+ * 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.MetadataVersionChange;
+import org.apache.kafka.image.MetadataVersionChangeException;
+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.snapshot.SnapshotReader;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * 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 = null;
+        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 { }
+        };
+
+        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 MetadataLoader build() {
+            if (logContext == null) {
+                logContext = new LogContext("[MetadataLoader " + nodeId + "] ");
+            }
+            if (faultHandler == null) throw new RuntimeException("You must set a fault handler.");
+            return new MetadataLoader(
+                time,
+                logContext,
+                threadNamePrefix,
+                faultHandler,
+                metrics);
+        }
+    }
+
+    /**
+     * 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;
+
+    /**
+     * The publishers which should receive cluster metadata updates.
+     */
+    private final List<MetadataPublisher> publishers;
+
+    /**
+     * The current leader and epoch.
+     */
+    private LeaderAndEpoch currentLeaderAndEpoch = LeaderAndEpoch.UNKNOWN;

Review Comment:
   This field is not accessed. Do we need it?



##########
metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java:
##########
@@ -0,0 +1,504 @@
+/*
+ * 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.MetadataVersionChange;
+import org.apache.kafka.image.MetadataVersionChangeException;
+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.snapshot.SnapshotReader;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * 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 = null;
+        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 { }
+        };
+
+        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 MetadataLoader build() {
+            if (logContext == null) {
+                logContext = new LogContext("[MetadataLoader " + nodeId + "] ");
+            }
+            if (faultHandler == null) throw new RuntimeException("You must set a fault handler.");
+            return new MetadataLoader(
+                time,
+                logContext,
+                threadNamePrefix,
+                faultHandler,
+                metrics);
+        }
+    }
+
+    /**
+     * 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;
+
+    /**
+     * The publishers which should receive cluster metadata updates.
+     */
+    private final List<MetadataPublisher> publishers;
+
+    /**
+     * 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
+    ) {
+        this.log = logContext.logger(MetadataLoader.class);
+        this.time = time;
+        this.faultHandler = faultHandler;
+        this.metrics = metrics;
+        this.publishers = new ArrayList<>();
+        this.image = MetadataImage.EMPTY;
+        this.eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix);
+    }
+
+    @Override
+    public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                LogDeltaManifest manifest = loadLogDelta(delta, reader);
+                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;
+                }
+                for (MetadataPublisher publisher : publishers) {
+                    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);
+                    }
+                }
+                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 (MetadataVersionChangeException e) {
+                    handleMetadataVersionChange(delta,
+                        e.change(),
+                        batch.baseOffset() + indexWithinBatch,
+                        lastEpoch,
+                        lastContainedLogTimeMs);
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error loading metadata log record from offset " +
+                            batch.baseOffset() + indexWithinBatch, e);
+                }
+                lastEpoch = batch.epoch();
+                lastContainedLogTimeMs = batch.appendTimestamp();
+                indexWithinBatch++;
+            }
+            metrics.updateBatchSize(batch.records().size());
+            lastOffset = batch.lastOffset();
+            numBytes += batch.sizeInBytes();
+            numBatches++;
+        }
+        MetadataProvenance provenance =
+                new MetadataProvenance(lastOffset, lastEpoch, lastContainedLogTimeMs);
+        long elapsedNs = time.nanoseconds() - startNs;
+        // TODO: this metric should be renamed something like "delta processing time"
+        metrics.updateBatchProcessingTime(elapsedNs);
+        return new LogDeltaManifest(provenance,
+                numBatches,
+                elapsedNs,
+                numBytes);
+    }
+
+    /**
+     * Handle a change in the metadata version.
+     *
+     * @param delta             The metadata delta we're working with.
+     * @param change            The change we're handling.
+     * @param changeOffset      The offset of the change.
+     * @param preChangeEpoch    The log epoch BEFORE the change.
+     */
+    void handleMetadataVersionChange(
+        MetadataDelta delta,
+        MetadataVersionChange change,
+        long changeOffset,
+        int preChangeEpoch,
+        long lastContainedLogTimeMs
+    ) {
+        // First, we materialize the current metadata image and send it to all the publishers that
+        // are interested in preVersionChange images. The most important one is the publisher which
+        // writes snapshots out to disk.
+        MetadataProvenance provenance =
+                new MetadataProvenance(changeOffset - 1, preChangeEpoch, lastContainedLogTimeMs);
+        PreVersionChangeManifest manifest = new PreVersionChangeManifest(provenance, change);
+        MetadataImage preVersionChangeImage = delta.apply(provenance);
+        for (MetadataPublisher publisher : publishers) {
+            try {
+                publisher.publishPreVersionChangeImage(delta, preVersionChangeImage, manifest);
+            } catch (Throwable e) {
+                faultHandler.handleFault("Error publishing pre-version change image at offset " +
+                    provenance.offset() + " with publisher " + publisher.name(), e);
+            }
+        }
+        // Then, we clear the current delta and write out the current image to it in the new format.
+        // If any metadata was lost, we just log it here. We cannot prevent the losses because the
+        // decision to change the metadata version was already taken by the controller.
+        delta.clear();
+        ImageReWriter writer = new ImageReWriter(delta);
+        preVersionChangeImage.write(writer, new ImageWriterOptions.Builder().
+                        setMetadataVersion(change.newVersion()).
+                        setLossHandler(loss -> {
+                            log.warn("{}", loss.getMessage());
+                        }).
+                        build());
+    }
+
+    @Override
+    public void handleSnapshot(SnapshotReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                SnapshotManifest manifest = loadSnapshot(delta, reader);
+                try {
+                    image = delta.apply(manifest.provenance());
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error generating new metadata image from " +
+                            "snapshot at offset " + reader.lastContainedLogOffset(), e);
+                    return;
+                }
+                for (MetadataPublisher publisher : publishers) {
+                    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);
+                    }
+                }
+                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. Additionally, it is an error for a version change to occur unless
+     * it is the very first record. 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());

Review Comment:
   This may still throw `MetadataVersionChangeException`, but we are not catching it. Am I missing something?



##########
core/src/main/scala/kafka/server/DynamicBrokerConfig.scala:
##########
@@ -269,6 +270,26 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging
     addBrokerReconfigurable(kafkaServer.logManager.producerStateManagerConfig)
   }
 
+  /**
+   * Add reconfigurables to be notified when a dynamic controller config is updated.
+   */
+  def addReconfigurables(controller: ControllerServer): Unit = {
+    controller.authorizer match {
+      case Some(authz: Reconfigurable) => addReconfigurable(authz)
+      case _ =>
+    }
+    if (!kafkaConfig.processRoles.contains(BrokerRole)) {
+      // In combined mode, these elements are shared between broker and controller, and therefore
+      // don't need to be updated here (since the broker will do it.)
+      addReconfigurable(controller.kafkaYammerMetrics)
+      addReconfigurable(new DynamicMetricsReporters(
+        kafkaConfig.brokerId, controller.config, controller.metrics, controller.clusterId))
+    }
+    // TODO: add dynamic thread pool resizing here

Review Comment:
   nit: create jiras. We need one for the quota callback as well.



##########
metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotEmitter.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.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 {
+    private final static int DEFAULT_BATCH_SIZE = 100;
+
+    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 emit(MetadataImage image) {

Review Comment:
   nit: maybe we could call this `maybeEmit` or return a boolean or something. Just to make it clear that a snapshot may or may not be emitted by the call.



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

Review Comment:
   Seems we didn't need this and `elapsedNs` below?



##########
metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.PreVersionChangeManifest;
+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;
+
+
+/**
+ * A metadata publisher that generates snapshots when appropriate.
+ */
+public class SnapshotGenerator implements MetadataPublisher {
+    public static class Builder {
+        private int nodeId = 0;
+        private Time time = Time.SYSTEM;
+        private Emitter emitter = null;
+        private FaultHandler faultHandler = (m, e) -> null;
+        private long minBytesSinceLastSnapshot = 100 * 1024L * 1024L;
+        private long minTimeSinceLastSnapshotNs = TimeUnit.DAYS.toNanos(1);
+
+        public Builder setNodeId(int nodeId) {
+            this.nodeId = nodeId;
+            return this;
+        }
+
+        public Builder setTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder setEmitter(Emitter emitter) {
+            this.emitter = emitter;
+            return this;
+        }
+
+        public Builder setFaultHandler(FaultHandler faultHandler) {
+            this.faultHandler = faultHandler;
+            return this;
+        }
+
+        public Builder setMinBytesSinceLastSnapshot(long minBytesSinceLastSnapshot) {
+            this.minBytesSinceLastSnapshot = minBytesSinceLastSnapshot;
+            return this;
+        }
+
+        public Builder setMinTimeSinceLastSnapshotNs(long minTimeSinceLastSnapshotNs) {
+            this.minTimeSinceLastSnapshotNs = minTimeSinceLastSnapshotNs;
+            return this;
+        }
+
+        public SnapshotGenerator build() {
+            if (emitter == null) throw new RuntimeException("You must set a snapshot emitter.");
+            return new SnapshotGenerator(
+                nodeId,
+                time,
+                emitter,
+                faultHandler,
+                minBytesSinceLastSnapshot,
+                minTimeSinceLastSnapshotNs
+            );
+        }
+    }
+
+    /**
+     * The callback which actually generates the snapshot.
+     */
+    public interface Emitter {
+        void emit(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 minimum number of bytes we will wait to see before emitting a snapshot.
+     */
+    private final long minBytesSinceLastSnapshot;
+
+    /**
+     * The minimum amount of time we will wait before emitting a snapshot, or 0 to disable
+     * time-based snapshotting.
+     */
+    private final long minTimeSinceLastSnapshotNs;
+
+    /**
+     * If non-null, the reason why snapshots have been disabled.
+     */
+    private volatile 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 minBytesSinceLastSnapshot,
+        long minTimeSinceLastSnapshotNs
+    ) {
+        this.nodeId = nodeId;
+        this.time = time;
+        this.emitter = emitter;
+        this.faultHandler = faultHandler;
+        this.minBytesSinceLastSnapshot = minBytesSinceLastSnapshot;
+        this.minTimeSinceLastSnapshotNs = minTimeSinceLastSnapshotNs;
+        LogContext logContext = new LogContext("[SnapshotGenerator " + nodeId + "] ");
+        this.log = logContext.logger(SnapshotGenerator.class);
+        this.disabledReason = null;
+        this.eventQueue = new KafkaEventQueue(time, logContext, "");

Review Comment:
   Perhaps we should add a thread name prefix?



##########
metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.PreVersionChangeManifest;
+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;
+
+
+/**
+ * A metadata publisher that generates snapshots when appropriate.
+ */
+public class SnapshotGenerator implements MetadataPublisher {
+    public static class Builder {
+        private int nodeId = 0;
+        private Time time = Time.SYSTEM;
+        private Emitter emitter = null;
+        private FaultHandler faultHandler = (m, e) -> null;
+        private long minBytesSinceLastSnapshot = 100 * 1024L * 1024L;
+        private long minTimeSinceLastSnapshotNs = TimeUnit.DAYS.toNanos(1);
+
+        public Builder setNodeId(int nodeId) {
+            this.nodeId = nodeId;
+            return this;
+        }
+
+        public Builder setTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder setEmitter(Emitter emitter) {
+            this.emitter = emitter;
+            return this;
+        }
+
+        public Builder setFaultHandler(FaultHandler faultHandler) {
+            this.faultHandler = faultHandler;
+            return this;
+        }
+
+        public Builder setMinBytesSinceLastSnapshot(long minBytesSinceLastSnapshot) {
+            this.minBytesSinceLastSnapshot = minBytesSinceLastSnapshot;
+            return this;
+        }
+
+        public Builder setMinTimeSinceLastSnapshotNs(long minTimeSinceLastSnapshotNs) {
+            this.minTimeSinceLastSnapshotNs = minTimeSinceLastSnapshotNs;
+            return this;
+        }
+
+        public SnapshotGenerator build() {
+            if (emitter == null) throw new RuntimeException("You must set a snapshot emitter.");

Review Comment:
   nit: if this is required, perhaps we should pass it in the constructor



##########
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() {

Review Comment:
   I don't think this is used. Can we remove it?



##########
metadata/src/main/java/org/apache/kafka/image/publisher/SnapshotGenerator.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.PreVersionChangeManifest;
+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;
+
+
+/**
+ * A metadata publisher that generates snapshots when appropriate.
+ */
+public class SnapshotGenerator implements MetadataPublisher {
+    public static class Builder {
+        private int nodeId = 0;
+        private Time time = Time.SYSTEM;
+        private Emitter emitter = null;
+        private FaultHandler faultHandler = (m, e) -> null;
+        private long minBytesSinceLastSnapshot = 100 * 1024L * 1024L;
+        private long minTimeSinceLastSnapshotNs = TimeUnit.DAYS.toNanos(1);
+
+        public Builder setNodeId(int nodeId) {
+            this.nodeId = nodeId;
+            return this;
+        }
+
+        public Builder setTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder setEmitter(Emitter emitter) {
+            this.emitter = emitter;
+            return this;
+        }
+
+        public Builder setFaultHandler(FaultHandler faultHandler) {
+            this.faultHandler = faultHandler;
+            return this;
+        }
+
+        public Builder setMinBytesSinceLastSnapshot(long minBytesSinceLastSnapshot) {
+            this.minBytesSinceLastSnapshot = minBytesSinceLastSnapshot;
+            return this;
+        }
+
+        public Builder setMinTimeSinceLastSnapshotNs(long minTimeSinceLastSnapshotNs) {
+            this.minTimeSinceLastSnapshotNs = minTimeSinceLastSnapshotNs;
+            return this;
+        }
+
+        public SnapshotGenerator build() {
+            if (emitter == null) throw new RuntimeException("You must set a snapshot emitter.");
+            return new SnapshotGenerator(
+                nodeId,
+                time,
+                emitter,
+                faultHandler,
+                minBytesSinceLastSnapshot,
+                minTimeSinceLastSnapshotNs
+            );
+        }
+    }
+
+    /**
+     * The callback which actually generates the snapshot.
+     */
+    public interface Emitter {
+        void emit(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 minimum number of bytes we will wait to see before emitting a snapshot.
+     */
+    private final long minBytesSinceLastSnapshot;
+
+    /**
+     * The minimum amount of time we will wait before emitting a snapshot, or 0 to disable
+     * time-based snapshotting.
+     */
+    private final long minTimeSinceLastSnapshotNs;
+
+    /**
+     * If non-null, the reason why snapshots have been disabled.
+     */
+    private volatile 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 minBytesSinceLastSnapshot,
+        long minTimeSinceLastSnapshotNs
+    ) {
+        this.nodeId = nodeId;
+        this.time = time;
+        this.emitter = emitter;
+        this.faultHandler = faultHandler;
+        this.minBytesSinceLastSnapshot = minBytesSinceLastSnapshot;
+        this.minTimeSinceLastSnapshotNs = minTimeSinceLastSnapshotNs;
+        LogContext logContext = new LogContext("[SnapshotGenerator " + nodeId + "] ");
+        this.log = logContext.logger(SnapshotGenerator.class);
+        this.disabledReason = null;
+        this.eventQueue = new KafkaEventQueue(time, logContext, "");
+        resetSnapshotCounters();
+    }
+
+    @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
+    ) {
+        // Reset the snapshot counters because we just read a snapshot.
+        resetSnapshotCounters();
+    }
+
+    @Override
+    public void publishPreVersionChangeImage(
+        MetadataDelta delta,
+        MetadataImage preVersionChangeImage,
+        PreVersionChangeManifest manifest
+    ) {
+        scheduleEmit("the metadata version changed", preVersionChangeImage);
+    }
+
+    @Override
+    public void publishLogDelta(
+        MetadataDelta delta,
+        MetadataImage newImage,
+        LogDeltaManifest manifest
+    ) {
+        bytesSinceLastSnapshot += manifest.numBytes();
+        if (bytesSinceLastSnapshot >= minBytesSinceLastSnapshot) {
+            if (eventQueue.isEmpty()) {
+                scheduleEmit("we have replayed at least " + minBytesSinceLastSnapshot +
+                    " bytes", newImage);
+            }
+        } else if (minTimeSinceLastSnapshotNs != 0 &&

Review Comment:
   I thought the intent was to disable time-based snapshots with -1.



##########
metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java:
##########
@@ -0,0 +1,504 @@
+/*
+ * 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.MetadataVersionChange;
+import org.apache.kafka.image.MetadataVersionChangeException;
+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.snapshot.SnapshotReader;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+
+/**
+ * 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 = null;
+        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 { }
+        };
+
+        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 MetadataLoader build() {
+            if (logContext == null) {
+                logContext = new LogContext("[MetadataLoader " + nodeId + "] ");
+            }
+            if (faultHandler == null) throw new RuntimeException("You must set a fault handler.");
+            return new MetadataLoader(
+                time,
+                logContext,
+                threadNamePrefix,
+                faultHandler,
+                metrics);
+        }
+    }
+
+    /**
+     * 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;
+
+    /**
+     * The publishers which should receive cluster metadata updates.
+     */
+    private final List<MetadataPublisher> publishers;
+
+    /**
+     * 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
+    ) {
+        this.log = logContext.logger(MetadataLoader.class);
+        this.time = time;
+        this.faultHandler = faultHandler;
+        this.metrics = metrics;
+        this.publishers = new ArrayList<>();
+        this.image = MetadataImage.EMPTY;
+        this.eventQueue = new KafkaEventQueue(time, logContext, threadNamePrefix);
+    }
+
+    @Override
+    public void handleCommit(BatchReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                LogDeltaManifest manifest = loadLogDelta(delta, reader);
+                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;
+                }
+                for (MetadataPublisher publisher : publishers) {
+                    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);
+                    }
+                }
+                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 (MetadataVersionChangeException e) {
+                    handleMetadataVersionChange(delta,
+                        e.change(),
+                        batch.baseOffset() + indexWithinBatch,
+                        lastEpoch,
+                        lastContainedLogTimeMs);
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error loading metadata log record from offset " +
+                            batch.baseOffset() + indexWithinBatch, e);
+                }
+                lastEpoch = batch.epoch();
+                lastContainedLogTimeMs = batch.appendTimestamp();
+                indexWithinBatch++;
+            }
+            metrics.updateBatchSize(batch.records().size());
+            lastOffset = batch.lastOffset();
+            numBytes += batch.sizeInBytes();
+            numBatches++;
+        }
+        MetadataProvenance provenance =
+                new MetadataProvenance(lastOffset, lastEpoch, lastContainedLogTimeMs);
+        long elapsedNs = time.nanoseconds() - startNs;
+        // TODO: this metric should be renamed something like "delta processing time"
+        metrics.updateBatchProcessingTime(elapsedNs);
+        return new LogDeltaManifest(provenance,
+                numBatches,
+                elapsedNs,
+                numBytes);
+    }
+
+    /**
+     * Handle a change in the metadata version.
+     *
+     * @param delta             The metadata delta we're working with.
+     * @param change            The change we're handling.
+     * @param changeOffset      The offset of the change.
+     * @param preChangeEpoch    The log epoch BEFORE the change.
+     */
+    void handleMetadataVersionChange(
+        MetadataDelta delta,
+        MetadataVersionChange change,
+        long changeOffset,
+        int preChangeEpoch,
+        long lastContainedLogTimeMs
+    ) {
+        // First, we materialize the current metadata image and send it to all the publishers that
+        // are interested in preVersionChange images. The most important one is the publisher which
+        // writes snapshots out to disk.
+        MetadataProvenance provenance =
+                new MetadataProvenance(changeOffset - 1, preChangeEpoch, lastContainedLogTimeMs);
+        PreVersionChangeManifest manifest = new PreVersionChangeManifest(provenance, change);
+        MetadataImage preVersionChangeImage = delta.apply(provenance);
+        for (MetadataPublisher publisher : publishers) {
+            try {
+                publisher.publishPreVersionChangeImage(delta, preVersionChangeImage, manifest);
+            } catch (Throwable e) {
+                faultHandler.handleFault("Error publishing pre-version change image at offset " +
+                    provenance.offset() + " with publisher " + publisher.name(), e);
+            }
+        }
+        // Then, we clear the current delta and write out the current image to it in the new format.
+        // If any metadata was lost, we just log it here. We cannot prevent the losses because the
+        // decision to change the metadata version was already taken by the controller.
+        delta.clear();
+        ImageReWriter writer = new ImageReWriter(delta);
+        preVersionChangeImage.write(writer, new ImageWriterOptions.Builder().
+                        setMetadataVersion(change.newVersion()).
+                        setLossHandler(loss -> {
+                            log.warn("{}", loss.getMessage());
+                        }).
+                        build());
+    }
+
+    @Override
+    public void handleSnapshot(SnapshotReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                SnapshotManifest manifest = loadSnapshot(delta, reader);
+                try {
+                    image = delta.apply(manifest.provenance());
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error generating new metadata image from " +
+                            "snapshot at offset " + reader.lastContainedLogOffset(), e);
+                    return;
+                }
+                for (MetadataPublisher publisher : publishers) {
+                    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);
+                    }
+                }
+                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. Additionally, it is an error for a version change to occur unless
+     * it is the very first record. 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;

Review Comment:
   Not sure how useful this index is given the presence of control records in the snapshot. Why not use the offset from `Batch` as we do when processing log records?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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