mimaison commented on code in PR #18039:
URL: https://github.com/apache/kafka/pull/18039#discussion_r1886615158


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java:
##########
@@ -206,4 +223,119 @@ public static Optional<CompletedTxn> 
updateProducers(ProducerStateManager produc
         }
         return completedTxn;
     }
+
+    public static boolean isRemoteLogEnabled(boolean 
remoteStorageSystemEnable, LogConfig config, String topic) {
+        // Remote log is enabled only for non-compact and non-internal topics
+        return remoteStorageSystemEnable &&
+                !(config.compact || Topic.isInternal(topic)
+                        || 
TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME.equals(topic)
+                        || Topic.CLUSTER_METADATA_TOPIC_NAME.equals(topic)) &&
+                config.remoteStorageEnable();
+    }
+
+    // Visible for benchmarking
+    public static LogValidator.MetricsRecorder 
newValidatorMetricsRecorder(BrokerTopicMetrics allTopicsStats) {
+        return new LogValidator.MetricsRecorder() {
+            public void recordInvalidMagic() {
+                allTopicsStats.invalidMagicNumberRecordsPerSec().mark();
+            }
+
+            public void recordInvalidOffset() {
+                allTopicsStats.invalidOffsetOrSequenceRecordsPerSec().mark();
+            }
+
+            public void recordInvalidSequence() {
+                allTopicsStats.invalidOffsetOrSequenceRecordsPerSec().mark();
+            }
+
+            public void recordInvalidChecksums() {
+                allTopicsStats.invalidMessageCrcRecordsPerSec().mark();
+            }
+
+            public void recordNoKeyCompactedTopic() {
+                allTopicsStats.noKeyCompactedTopicRecordsPerSec().mark();
+            }
+        };
+    }
+
+    /**
+     * If the recordVersion is >= RecordVersion.V2, create a new 
LeaderEpochFileCache instance.
+     * Loading the epoch entries from the backing checkpoint file or the 
provided currentCache if not empty.
+     * Otherwise, the message format is considered incompatible and the 
existing LeaderEpoch file
+     * is deleted.
+     *
+     * @param dir                  The directory in which the log will reside
+     * @param topicPartition       The topic partition
+     * @param logDirFailureChannel The LogDirFailureChannel to asynchronously 
handle log dir failure
+     * @param recordVersion        The record version
+     * @param logPrefix            The logging prefix
+     * @param currentCache         The current LeaderEpochFileCache instance 
(if any)
+     * @param scheduler            The scheduler for executing asynchronous 
tasks
+     * @return The new LeaderEpochFileCache instance (if created), empty 
otherwise
+     */
+    public static Optional<LeaderEpochFileCache> 
maybeCreateLeaderEpochCache(File dir,
+                                                                             
TopicPartition topicPartition,
+                                                                             
LogDirFailureChannel logDirFailureChannel,
+                                                                             
RecordVersion recordVersion,
+                                                                             
String logPrefix,
+                                                                             
Optional<LeaderEpochFileCache> currentCache,
+                                                                             
Scheduler scheduler) throws IOException {
+        File leaderEpochFile = LeaderEpochCheckpointFile.newFile(dir);
+
+        if (recordVersion.precedes(RecordVersion.V2)) {
+            if (leaderEpochFile.exists()) {
+                LOG.warn(logPrefix + "Deleting non-empty leader epoch cache 
due to incompatible message format " + recordVersion);
+            }
+            Files.deleteIfExists(leaderEpochFile.toPath());
+            return Optional.empty();
+        } else {
+            LeaderEpochCheckpointFile checkpointFile = new 
LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel);
+            return Optional.of(currentCache.map(cache -> 
cache.withCheckpoint(checkpointFile))
+                    .orElse(new LeaderEpochFileCache(topicPartition, 
checkpointFile, scheduler)));
+        }
+    }
+
+    public static LogSegment createNewCleanedSegment(File dir, LogConfig 
logConfig, long baseOffset) throws IOException {

Review Comment:
   According to 
https://docs.google.com/document/d/1dQJL4MCwqQJSPmZkVmVzshFZKuFy_bCPtubav4wBfHQ/edit?usp=sharing
 we wanted UnifiedLog to be the public API other components will use:
   ```
   This class is external facing, and acts as the outer shell, meaning that the 
public API of this class will be used by other components such as LogManager, 
LogCleaner etc. and other components outside the kafka.log package.
   ```



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogMetricNames.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.storage.internals.log;
+
+import java.util.List;
+
+public class LogMetricNames {
+
+    public static final String NUM_LOG_SEGMENTS = "NumLogSegments";
+    public static final String LOG_START_OFFSET = "LogStartOffset";
+    public static final String LOG_END_OFFSET = "LogEndOffset";
+    public static final String SIZE = "Size";
+
+    public static List<String> allMetricNames() {

Review Comment:
   Good idea



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to