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/12/13 19:54:35 UTC

[GitHub] [kafka] mumrah commented on a diff in pull request #12983: MINOR: ControllerServer should use the new metadata loader and snapshot generator

mumrah commented on code in PR #12983:
URL: https://github.com/apache/kafka/pull/12983#discussion_r1047441883


##########
core/src/main/scala/kafka/server/ControllerServer.scala:
##########
@@ -68,6 +69,7 @@ class ControllerServer(
   val lock = new ReentrantLock()
   val awaitShutdownCond = lock.newCondition()
   var status: ProcessStatus = SHUTDOWN
+  var loader: MetadataLoader = _

Review Comment:
   Is this ever assigned? Shouldn't we just use the loader defined in SharedServer?



##########
metadata/src/main/java/org/apache/kafka/image/publisher/MetadataPublisher.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.image.publisher;
+
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.loader.LogDeltaManifest;
+import org.apache.kafka.image.loader.SnapshotManifest;
+
+
+/**
+ * Publishes metadata deltas which we have loaded from the log and snapshots.
+ *
+ * Publishers receive a stream of callbacks from the metadata loader which keeps them notified
+ * of the latest cluster metadata. This interface abstracts away some of the complications of
+ * following the cluster metadata. For example, if the loader needs to read a snapshot, it will
+ * present the contents of the snapshot in the form of a delta from the previous state.
+ */
+public interface MetadataPublisher extends AutoCloseable {

Review Comment:
   Is "Publisher" the right name here? It makes it sound a bit like instances of this interface will be publishing metadata, when they are actually being published _to_. What about `MetadataListener` or `MetadataConsumer` or something? 



##########
core/src/main/scala/kafka/server/metadata/BrokerServerMetrics.scala:
##########
@@ -17,17 +17,48 @@
 
 package kafka.server.metadata
 
-import java.util.concurrent.atomic.AtomicLong
+import kafka.metrics.KafkaMetricsGroup
+
+import java.util.concurrent.atomic.{AtomicLong, AtomicReference}
 import org.apache.kafka.common.MetricName
 import org.apache.kafka.common.metrics.Gauge
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.metrics.MetricConfig
+import org.apache.kafka.image.MetadataProvenance
+import org.apache.kafka.image.loader.MetadataLoaderMetrics
+import org.apache.kafka.server.metrics.KafkaYammerMetrics
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
 
-final class BrokerServerMetrics private (metrics: Metrics) extends AutoCloseable {
+final class BrokerServerMetrics private (
+  metrics: Metrics
+) extends MetadataLoaderMetrics with KafkaMetricsGroup {

Review Comment:
   Are the changes to BrokerMetadataListener and BrokerServerMetrics needed in this PR, or can we save them for the next one (with the BrokerServer changes)?



##########
metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java:
##########
@@ -0,0 +1,474 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.image.loader;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.MetadataProvenance;
+import org.apache.kafka.image.publisher.MetadataPublisher;
+import org.apache.kafka.image.writer.ImageReWriter;
+import org.apache.kafka.image.writer.ImageWriterOptions;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.raft.Batch;
+import org.apache.kafka.raft.BatchReader;
+import org.apache.kafka.raft.LeaderAndEpoch;
+import org.apache.kafka.raft.RaftClient;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.server.fault.FaultHandler;
+import org.apache.kafka.server.fault.FaultHandlerException;
+import org.apache.kafka.snapshot.SnapshotReader;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+
+/**
+ * The MetadataLoader follows changes provided by a RaftClient, and packages them into metadata
+ * deltas and images that can be consumed by publishers.
+ *
+ * The Loader maintains its own thread, which is used to make all callbacks into publishers. If a
+ * publisher A is installed before B, A will receive all callbacks before B. This is also true if
+ * A and B are installed as part of a list [A, B].
+ *
+ * Publishers should not modify any data structures passed to them.
+ *
+ * It is possible to change the list of publishers dynamically over time. Whenever a new publisher is
+ * added, it receives a catch-up delta which contains the full state. Any publisher installed when the
+ * loader is closed will itself be closed.
+ */
+public class MetadataLoader implements RaftClient.Listener<ApiMessageAndVersion>, AutoCloseable {
+    public static class Builder {
+        private int nodeId = -1;
+        private Time time = Time.SYSTEM;
+        private LogContext logContext = null;
+        private String threadNamePrefix = "";
+        private FaultHandler faultHandler = (m, e) -> new FaultHandlerException(m, e);
+        private MetadataLoaderMetrics metrics = new MetadataLoaderMetrics() {
+            private volatile long lastAppliedOffset = -1L;
+
+            @Override
+            public void updateBatchProcessingTime(long elapsedNs) { }
+
+            @Override
+            public void updateBatchSize(int size) { }
+
+            @Override
+            public void updateLastAppliedImageProvenance(MetadataProvenance provenance) {
+                this.lastAppliedOffset = provenance.offset();
+            }
+
+            @Override
+            public long lastAppliedOffset() {
+                return lastAppliedOffset;
+            }
+
+            @Override
+            public void close() throws Exception { }
+        };
+
+        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 + "] ");
+            }
+            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);
+                if (log.isDebugEnabled()) {
+                    log.debug("Generated a metadata delta between {} and {} from {} batch(es) " +
+                            "in {} us.", image.offset(), manifest.provenance().offset(),
+                            manifest.numBatches(), NANOSECONDS.toMicros(manifest.elapsedNs()));
+                }
+                try {
+                    image = delta.apply(manifest.provenance());
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error generating new metadata image from " +
+                        "metadata delta between offset " + image.offset() +
+                            " and " + manifest.provenance().offset(), e);
+                    return;
+                }
+                log.debug("Publishing new image with provenance {}.", image.provenance());
+                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 (Throwable e) {
+                    faultHandler.handleFault("Error loading metadata log record from offset " +
+                            batch.baseOffset() + indexWithinBatch, e);
+                }
+                indexWithinBatch++;
+            }
+            metrics.updateBatchSize(batch.records().size());
+            lastOffset = batch.lastOffset();
+            lastEpoch = batch.epoch();
+            lastContainedLogTimeMs = batch.appendTimestamp();
+            numBytes += batch.sizeInBytes();
+            numBatches++;
+        }
+        MetadataProvenance provenance =
+                new MetadataProvenance(lastOffset, lastEpoch, lastContainedLogTimeMs);
+        long elapsedNs = time.nanoseconds() - startNs;
+        metrics.updateBatchProcessingTime(elapsedNs);
+        return new LogDeltaManifest(provenance,
+                numBatches,
+                elapsedNs,
+                numBytes);
+    }
+
+    @Override
+    public void handleSnapshot(SnapshotReader<ApiMessageAndVersion> reader) {
+        eventQueue.append(() -> {
+            try {
+                MetadataDelta delta = new MetadataDelta.Builder().
+                        setImage(image).
+                        build();
+                SnapshotManifest manifest = loadSnapshot(delta, reader);
+                if (log.isDebugEnabled()) {
+                    log.debug("Generated a metadata delta from a snapshot at offset {} " +
+                            "in {} us.", manifest.provenance().offset(),
+                            NANOSECONDS.toMicros(manifest.elapsedNs()));
+                }
+                try {
+                    image = delta.apply(manifest.provenance());
+                } catch (Throwable e) {
+                    faultHandler.handleFault("Error generating new metadata image from " +
+                            "snapshot at offset " + reader.lastContainedLogOffset(), e);
+                    return;
+                }
+                log.debug("Publishing new snapshot image with provenance {}.", image.provenance());
+                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. 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());

Review Comment:
   nit: indentation



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