satishd commented on code in PR #14034:
URL: https://github.com/apache/kafka/pull/14034#discussion_r1442402484


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java:
##########
@@ -0,0 +1,1146 @@
+/*
+ * 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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.KafkaStorageException;
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.message.FetchResponseData;
+import org.apache.kafka.common.record.FileLogInputStream;
+import org.apache.kafka.common.record.FileRecords;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.RecordVersion;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.util.Scheduler;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * An append-only log for storing messages locally. The log is a sequence of 
LogSegments, each with a base offset.
+ * New log segments are created according to a configurable policy that 
controls the size in bytes or time interval
+ * for a given segment.
+ *
+ * NOTE: this class is not thread-safe.
+ */
+public class LocalLog {
+
+    /**
+     * a file that is scheduled to be deleted
+     */
+    public static final String DELETED_FILE_SUFFIX = 
LogFileUtils.DELETED_FILE_SUFFIX;
+
+    /**
+     * A temporary file that is being used for log cleaning
+     */
+    public static final String CLEANED_FILE_SUFFIX = ".cleaned";
+
+    /**
+     * A temporary file used when swapping files into the log
+     */
+    public static final String SWAP_FILE_SUFFIX = ".swap";
+
+    /**
+     * a directory that is scheduled to be deleted
+     */
+    public static final String DELETE_DIR_SUFFIX = "-delete";
+
+    /**
+     * a directory that is used for future partition
+     */
+    public static final String FUTURE_DIR_SUFFIX = "-future";
+    public static final String STRAY_DIR_SUFFIX = "-stray";
+
+    public static final Pattern DELETE_DIR_PATTERN = 
Pattern.compile("^(\\S+)-(\\S+)\\.(\\S+)" + DELETE_DIR_SUFFIX);
+    public static final Pattern FUTURE_DIR_PATTERN = 
Pattern.compile("^(\\S+)-(\\S+)\\.(\\S+)" + FUTURE_DIR_SUFFIX);
+    public static final Pattern STRAY_DIR_PATTERN = 
Pattern.compile("^(\\S+)-(\\S+)\\.(\\S+)" + STRAY_DIR_SUFFIX);
+
+    public static final long UNKNOWN_OFFSET = -1L;
+
+    private final Logger logger;
+
+    private final LogSegments segments;
+    private final Scheduler scheduler;
+    private final Time time;
+    private final TopicPartition topicPartition;
+    private final LogDirFailureChannel logDirFailureChannel;
+
+    // Last time the log was flushed
+    private final AtomicLong lastFlushedTime;
+
+    private volatile File dir;
+    private volatile LogConfig config;
+    private volatile long recoveryPoint;
+    private volatile LogOffsetMetadata nextOffsetMetadata;
+
+    // Cache value of parent directory to avoid allocations in hot paths like 
ReplicaManager.checkpointHighWatermarks
+    private volatile String parentDir;
+    // The memory mapped buffer for index files of this log will be closed 
with either delete() or closeHandlers()
+    // After memory mapped buffer is closed, no disk IO operation should be 
performed for this log.
+    private volatile boolean isMemoryMappedBufferClosed = false;
+
+    /**
+     * Creates a new LocalLog instance.
+     *
+     * @param dir                  The directory in which log segments are 
created.
+     * @param config               The log configuration settings
+     * @param segments             The non-empty log segments recovered from 
disk
+     * @param recoveryPoint        The offset at which to begin the next 
recovery i.e. the first offset which has not been flushed to disk
+     * @param scheduler            The thread pool scheduler used for 
background actions
+     * @param nextOffsetMetadata   The offset where the next message could be 
appended
+     * @param time                 The time instance used for checking the 
clock
+     * @param topicPartition       The topic partition associated with this log
+     * @param logDirFailureChannel The LogDirFailureChannel instance to 
asynchronously handle Log dir failure
+     */
+    public LocalLog(File dir,
+                    LogConfig config,
+                    LogSegments segments,
+                    long recoveryPoint,
+                    LogOffsetMetadata nextOffsetMetadata,
+                    Scheduler scheduler,
+                    Time time,
+                    TopicPartition topicPartition,
+                    LogDirFailureChannel logDirFailureChannel) {
+        this.dir = dir;
+        this.config = config;
+        this.segments = segments;
+        this.recoveryPoint = recoveryPoint;
+        this.scheduler = scheduler;
+        this.nextOffsetMetadata = nextOffsetMetadata;
+        this.time = time;
+        this.topicPartition = topicPartition;
+        this.logDirFailureChannel = logDirFailureChannel;
+
+        parentDir = dir.getParent();
+        lastFlushedTime = new AtomicLong(time.milliseconds());
+        String logPrefix = "[LocalLog partition=" + topicPartition + ", dir=" 
+ dir + "] ";
+        logger = new LogContext(logPrefix).logger(LocalLog.class);
+    }
+
+    public Logger logger() {
+        return logger;
+    }
+
+    public LogConfig config() {
+        return config;
+    }
+
+    public LogSegments segments() {
+        return segments;
+    }
+
+    public Scheduler scheduler() {
+        return scheduler;
+    }
+
+    public LogOffsetMetadata nextOffsetMetadata() {
+        return nextOffsetMetadata;
+    }
+
+    public TopicPartition topicPartition() {
+        return topicPartition;
+    }
+
+    public LogDirFailureChannel logDirFailureChannel() {
+        return logDirFailureChannel;
+    }
+
+    public long recoveryPoint() {
+        return recoveryPoint;
+    }
+
+    public Time time() {
+        return time;
+    }
+
+    /**
+     * The time this log is last known to have been fully flushed to disk
+     */
+    public long lastFlushTime() {
+        return lastFlushedTime.get();
+    }
+
+    /**
+     * The offset metadata of the next message that will be appended to the log
+     */
+    public LogOffsetMetadata logEndOffsetMetadata() {
+        return nextOffsetMetadata;
+    }
+
+    /**
+     * The offset of the next message that will be appended to the log
+     */
+    public long logEndOffset() {
+        return nextOffsetMetadata.messageOffset;
+    }
+
+    public File dir() {
+        return dir;
+    }
+
+    public String name() {
+        return dir.getName();
+    }
+
+    public String parentDir() {
+        return parentDir;
+    }
+
+    public File parentDirFile() {
+        return new File(parentDir);
+    }
+
+    public boolean isFuture() {
+        return dir.getName().endsWith(LocalLog.FUTURE_DIR_SUFFIX);
+    }
+
+    /**
+     * Rename the directory of the local-log to the given name.
+     *
+     * @param name the new dir name
+     * @throws KafkaStorageException if rename fails
+     */
+    public boolean renameDir(String name) {
+        return maybeHandleIOException("Error while renaming dir for " + 
topicPartition + " in log dir " + dir.getParent(),
+                () -> {
+                    File renamedDir = new File(dir.getParent(), name);
+                    Utils.atomicMoveWithFallback(dir.toPath(), 
renamedDir.toPath());
+                    if (!renamedDir.equals(dir)) {
+                        dir = renamedDir;
+                        parentDir = renamedDir.getParent();
+                        segments.updateParentDir(renamedDir);
+                        return true;
+                    } else {
+                        return false;
+                    }
+                });
+    }
+
+    /**
+     * Update the existing configuration to the new provided configuration.
+     *
+     * @param newConfig the new configuration to be updated to
+     */
+    public void updateConfig(LogConfig newConfig) {
+        LogConfig oldConfig = config;
+        config = newConfig;
+        RecordVersion oldRecordVersion = oldConfig.recordVersion();
+        RecordVersion newRecordVersion = newConfig.recordVersion();
+        if (newRecordVersion.precedes(oldRecordVersion))
+            logger.warn("Record format version has been downgraded from {} to 
{}.", oldRecordVersion, newRecordVersion);
+    }
+
+    public void checkIfMemoryMappedBufferClosed() {
+        if (isMemoryMappedBufferClosed)
+            throw new KafkaStorageException("The memory mapped buffer for log 
of " + topicPartition + " is already closed");
+    }
+
+    public void updateRecoveryPoint(long newRecoveryPoint) {
+        recoveryPoint = newRecoveryPoint;
+    }
+
+    /**
+     * Update recoveryPoint to provided offset and mark the log as flushed, if 
the offset is greater
+     * than the existing recoveryPoint.
+     *
+     * @param offset the offset to be updated
+     */
+    public void markFlushed(long offset) {
+        checkIfMemoryMappedBufferClosed();
+        if (offset > recoveryPoint) {
+            updateRecoveryPoint(offset);
+            lastFlushedTime.set(time.milliseconds());
+        }
+    }
+
+    /**
+     * The number of messages appended to the log since the last flush
+     */
+    public long unflushedMessages() {
+        return logEndOffset() - recoveryPoint;
+    }
+
+    /**
+     * Flush local log segments for all offsets up to offset-1.
+     * Note: It does not update the recovery point.
+     *
+     * @param offset The offset to flush up to (non-inclusive)
+     */
+    public void flush(long offset) throws IOException {
+        long currentRecoveryPoint = recoveryPoint;
+        if (currentRecoveryPoint <= offset) {
+            Collection<LogSegment> segmentsToFlush = 
segments.values(currentRecoveryPoint, offset);
+            for (LogSegment segment : segmentsToFlush) {
+                segment.flush();
+            }
+            // If there are any new segments, we need to flush the parent 
directory for crash consistency.
+            if (segmentsToFlush.stream().anyMatch(x -> x.baseOffset() >= 
currentRecoveryPoint)) {
+                // The directory might be renamed concurrently for topic 
deletion, which may cause NoSuchFileException here.
+                // Since the directory is to be deleted anyways, we just 
swallow NoSuchFileException and let it go.
+                Utils.flushDir(dir.toPath());

Review Comment:
   Good catch!



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java:
##########
@@ -0,0 +1,1146 @@
+/*
+ * 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 org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.KafkaStorageException;
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.message.FetchResponseData;
+import org.apache.kafka.common.record.FileLogInputStream;
+import org.apache.kafka.common.record.FileRecords;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.RecordVersion;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.util.Scheduler;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * An append-only log for storing messages locally. The log is a sequence of 
LogSegments, each with a base offset.
+ * New log segments are created according to a configurable policy that 
controls the size in bytes or time interval
+ * for a given segment.
+ *
+ * NOTE: this class is not thread-safe.
+ */
+public class LocalLog {
+
+    /**
+     * a file that is scheduled to be deleted
+     */
+    public static final String DELETED_FILE_SUFFIX = 
LogFileUtils.DELETED_FILE_SUFFIX;
+
+    /**
+     * A temporary file that is being used for log cleaning
+     */
+    public static final String CLEANED_FILE_SUFFIX = ".cleaned";
+
+    /**
+     * A temporary file used when swapping files into the log
+     */
+    public static final String SWAP_FILE_SUFFIX = ".swap";
+
+    /**
+     * a directory that is scheduled to be deleted
+     */
+    public static final String DELETE_DIR_SUFFIX = "-delete";
+
+    /**
+     * a directory that is used for future partition
+     */
+    public static final String FUTURE_DIR_SUFFIX = "-future";
+    public static final String STRAY_DIR_SUFFIX = "-stray";
+
+    public static final Pattern DELETE_DIR_PATTERN = 
Pattern.compile("^(\\S+)-(\\S+)\\.(\\S+)" + DELETE_DIR_SUFFIX);
+    public static final Pattern FUTURE_DIR_PATTERN = 
Pattern.compile("^(\\S+)-(\\S+)\\.(\\S+)" + FUTURE_DIR_SUFFIX);
+    public static final Pattern STRAY_DIR_PATTERN = 
Pattern.compile("^(\\S+)-(\\S+)\\.(\\S+)" + STRAY_DIR_SUFFIX);
+
+    public static final long UNKNOWN_OFFSET = -1L;
+
+    private final Logger logger;
+
+    private final LogSegments segments;
+    private final Scheduler scheduler;
+    private final Time time;
+    private final TopicPartition topicPartition;
+    private final LogDirFailureChannel logDirFailureChannel;
+
+    // Last time the log was flushed
+    private final AtomicLong lastFlushedTime;
+
+    private volatile File dir;
+    private volatile LogConfig config;
+    private volatile long recoveryPoint;
+    private volatile LogOffsetMetadata nextOffsetMetadata;
+
+    // Cache value of parent directory to avoid allocations in hot paths like 
ReplicaManager.checkpointHighWatermarks
+    private volatile String parentDir;
+    // The memory mapped buffer for index files of this log will be closed 
with either delete() or closeHandlers()
+    // After memory mapped buffer is closed, no disk IO operation should be 
performed for this log.
+    private volatile boolean isMemoryMappedBufferClosed = false;
+
+    /**
+     * Creates a new LocalLog instance.
+     *
+     * @param dir                  The directory in which log segments are 
created.
+     * @param config               The log configuration settings
+     * @param segments             The non-empty log segments recovered from 
disk
+     * @param recoveryPoint        The offset at which to begin the next 
recovery i.e. the first offset which has not been flushed to disk
+     * @param scheduler            The thread pool scheduler used for 
background actions
+     * @param nextOffsetMetadata   The offset where the next message could be 
appended
+     * @param time                 The time instance used for checking the 
clock
+     * @param topicPartition       The topic partition associated with this log
+     * @param logDirFailureChannel The LogDirFailureChannel instance to 
asynchronously handle Log dir failure
+     */
+    public LocalLog(File dir,
+                    LogConfig config,
+                    LogSegments segments,
+                    long recoveryPoint,
+                    LogOffsetMetadata nextOffsetMetadata,
+                    Scheduler scheduler,
+                    Time time,
+                    TopicPartition topicPartition,
+                    LogDirFailureChannel logDirFailureChannel) {
+        this.dir = dir;
+        this.config = config;
+        this.segments = segments;
+        this.recoveryPoint = recoveryPoint;
+        this.scheduler = scheduler;
+        this.nextOffsetMetadata = nextOffsetMetadata;
+        this.time = time;
+        this.topicPartition = topicPartition;
+        this.logDirFailureChannel = logDirFailureChannel;
+
+        parentDir = dir.getParent();
+        lastFlushedTime = new AtomicLong(time.milliseconds());
+        String logPrefix = "[LocalLog partition=" + topicPartition + ", dir=" 
+ dir + "] ";
+        logger = new LogContext(logPrefix).logger(LocalLog.class);
+    }
+
+    public Logger logger() {
+        return logger;
+    }
+
+    public LogConfig config() {
+        return config;
+    }
+
+    public LogSegments segments() {
+        return segments;
+    }
+
+    public Scheduler scheduler() {
+        return scheduler;
+    }
+
+    public LogOffsetMetadata nextOffsetMetadata() {
+        return nextOffsetMetadata;
+    }
+
+    public TopicPartition topicPartition() {
+        return topicPartition;
+    }
+
+    public LogDirFailureChannel logDirFailureChannel() {
+        return logDirFailureChannel;
+    }
+
+    public long recoveryPoint() {
+        return recoveryPoint;
+    }
+
+    public Time time() {
+        return time;
+    }
+
+    /**
+     * The time this log is last known to have been fully flushed to disk
+     */
+    public long lastFlushTime() {
+        return lastFlushedTime.get();
+    }
+
+    /**
+     * The offset metadata of the next message that will be appended to the log
+     */
+    public LogOffsetMetadata logEndOffsetMetadata() {
+        return nextOffsetMetadata;
+    }
+
+    /**
+     * The offset of the next message that will be appended to the log
+     */
+    public long logEndOffset() {
+        return nextOffsetMetadata.messageOffset;
+    }
+
+    public File dir() {
+        return dir;
+    }
+
+    public String name() {
+        return dir.getName();
+    }
+
+    public String parentDir() {
+        return parentDir;
+    }
+
+    public File parentDirFile() {
+        return new File(parentDir);
+    }
+
+    public boolean isFuture() {
+        return dir.getName().endsWith(LocalLog.FUTURE_DIR_SUFFIX);
+    }
+
+    /**
+     * Rename the directory of the local-log to the given name.
+     *
+     * @param name the new dir name
+     * @throws KafkaStorageException if rename fails
+     */
+    public boolean renameDir(String name) {
+        return maybeHandleIOException("Error while renaming dir for " + 
topicPartition + " in log dir " + dir.getParent(),
+                () -> {
+                    File renamedDir = new File(dir.getParent(), name);
+                    Utils.atomicMoveWithFallback(dir.toPath(), 
renamedDir.toPath());
+                    if (!renamedDir.equals(dir)) {
+                        dir = renamedDir;
+                        parentDir = renamedDir.getParent();
+                        segments.updateParentDir(renamedDir);
+                        return true;
+                    } else {
+                        return false;
+                    }
+                });
+    }
+
+    /**
+     * Update the existing configuration to the new provided configuration.
+     *
+     * @param newConfig the new configuration to be updated to
+     */
+    public void updateConfig(LogConfig newConfig) {
+        LogConfig oldConfig = config;
+        config = newConfig;
+        RecordVersion oldRecordVersion = oldConfig.recordVersion();
+        RecordVersion newRecordVersion = newConfig.recordVersion();
+        if (newRecordVersion.precedes(oldRecordVersion))
+            logger.warn("Record format version has been downgraded from {} to 
{}.", oldRecordVersion, newRecordVersion);
+    }
+
+    public void checkIfMemoryMappedBufferClosed() {
+        if (isMemoryMappedBufferClosed)
+            throw new KafkaStorageException("The memory mapped buffer for log 
of " + topicPartition + " is already closed");
+    }
+
+    public void updateRecoveryPoint(long newRecoveryPoint) {
+        recoveryPoint = newRecoveryPoint;
+    }
+
+    /**
+     * Update recoveryPoint to provided offset and mark the log as flushed, if 
the offset is greater
+     * than the existing recoveryPoint.
+     *
+     * @param offset the offset to be updated
+     */
+    public void markFlushed(long offset) {
+        checkIfMemoryMappedBufferClosed();
+        if (offset > recoveryPoint) {
+            updateRecoveryPoint(offset);
+            lastFlushedTime.set(time.milliseconds());
+        }
+    }
+
+    /**
+     * The number of messages appended to the log since the last flush
+     */
+    public long unflushedMessages() {
+        return logEndOffset() - recoveryPoint;
+    }
+
+    /**
+     * Flush local log segments for all offsets up to offset-1.
+     * Note: It does not update the recovery point.
+     *
+     * @param offset The offset to flush up to (non-inclusive)
+     */
+    public void flush(long offset) throws IOException {
+        long currentRecoveryPoint = recoveryPoint;
+        if (currentRecoveryPoint <= offset) {
+            Collection<LogSegment> segmentsToFlush = 
segments.values(currentRecoveryPoint, offset);
+            for (LogSegment segment : segmentsToFlush) {
+                segment.flush();
+            }
+            // If there are any new segments, we need to flush the parent 
directory for crash consistency.
+            if (segmentsToFlush.stream().anyMatch(x -> x.baseOffset() >= 
currentRecoveryPoint)) {
+                // The directory might be renamed concurrently for topic 
deletion, which may cause NoSuchFileException here.
+                // Since the directory is to be deleted anyways, we just 
swallow NoSuchFileException and let it go.
+                Utils.flushDir(dir.toPath());

Review Comment:
   Good catch!



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to