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


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java:
##########
@@ -0,0 +1,1115 @@
+/*
+ * 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.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+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 nextOffsetMetadata   The offset where the next message could be 
appended
+     * @param scheduler            The thread pool scheduler used for 
background actions
+     * @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.flushDirIfExists(dir.toPath());
+            }
+        }
+    }
+
+    /**
+     * Updates end offset of the log, and also updates the recoveryPoint.
+     *
+     * @param endOffset the new end offset of the log
+     */
+    public void updateLogEndOffset(long endOffset) {
+        nextOffsetMetadata = new LogOffsetMetadata(endOffset, 
segments.activeSegment().baseOffset(), segments.activeSegment().size());
+        if (recoveryPoint > endOffset) {
+            updateRecoveryPoint(endOffset);
+        }
+    }
+
+    /**
+     * Close file handlers used by log but don't write to disk.
+     * This is called if the log directory is offline.
+     */
+    public void closeHandlers() {
+        segments.closeHandlers();
+        isMemoryMappedBufferClosed = true;
+    }
+
+    /**
+     * Closes the segments of the log.
+     */
+    public void close() {
+        maybeHandleIOException(
+                () -> "Error while renaming dir for " + topicPartition + " in 
dir " + dir.getParent(),
+                (StorageAction<Void, IOException>) () -> {
+                    checkIfMemoryMappedBufferClosed();
+                    segments.close();
+                    return null;
+                });
+    }
+
+    /**
+     * Completely delete this log directory with no delay.
+     */
+    public void deleteEmptyDir() {
+        maybeHandleIOException(
+                () -> "Error while deleting dir for " + topicPartition + " in 
dir " + dir.getParent(),
+                (StorageAction<Void, IOException>) () -> {
+                    if (segments.nonEmpty()) {
+                        throw new IllegalStateException("Can not delete 
directory when " + segments.numberOfSegments() + " segments are still present");
+                    }
+                    if (!isMemoryMappedBufferClosed) {
+                        throw new IllegalStateException("Can not delete 
directory when memory mapped buffer for log of " + topicPartition + " is still 
open.");
+                    }
+                    Utils.delete(dir);
+                    return null;
+                });
+    }
+
+    /**
+     * Completely delete all segments with no delay.
+     *
+     * @return the deleted segments
+     */
+    public Collection<LogSegment> deleteAllSegments() {
+        return maybeHandleIOException(
+                () -> "Error while deleting all segments for " + 
topicPartition + " in dir " + dir.getParent(),
+                () -> {
+                    Collection<LogSegment> allSegments = segments.values();
+                    List<LogSegment> deletableSegments = new 
ArrayList<>(allSegments);
+                    removeAndDeleteSegments(allSegments, false, new 
LogDeletion(this.logger));
+                    isMemoryMappedBufferClosed = true;
+                    return deletableSegments;
+                });
+    }
+
+    /**
+     * This method deletes the given log segments by doing the following for 
each of them:
+     * - It removes the segment from the segment map so that it will no longer 
be used for reads.
+     * - It renames the index and log files by appending .deleted to the 
respective file name
+     * - It can either schedule an asynchronous delete operation to occur in 
the future or perform the deletion synchronously
+     * <p>
+     * Asynchronous deletion allows reads to happen concurrently without 
synchronization and without the possibility of
+     * physically deleting a file while it is being read.
+     * <p>
+     * This method does not convert IOException to KafkaStorageException, the 
immediate caller
+     * is expected to catch and handle IOException.
+     *
+     * @param segmentsToDelete The log segments to schedule for deletion
+     * @param asyncDelete      Whether the segment files should be deleted 
asynchronously
+     * @param reason           The reason for the segment deletion
+     */
+    public void removeAndDeleteSegments(Collection<LogSegment> 
segmentsToDelete,
+                                        boolean asyncDelete,
+                                        SegmentDeletionReason reason) throws 
IOException {
+        if (!segmentsToDelete.isEmpty()) {
+            // Most callers hold an iterator into the `segments` collection 
and `removeAndDeleteSegment` mutates it by
+            // removing the deleted segment, we should force materialization 
of the iterator here, so that results of the
+            // iteration remain valid and deterministic. We should also pass 
only the materialized view of the
+            // iterator to the logic that actually deletes the segments.
+            List<LogSegment> toDelete = new ArrayList<>(segmentsToDelete);
+            reason.logReason(toDelete);
+            for (LogSegment segment : toDelete) {
+                segments.remove(segment.baseOffset());
+            }
+            deleteSegmentFiles(toDelete, asyncDelete, dir, topicPartition, 
config, scheduler, logDirFailureChannel, logger);
+        }
+    }
+
+    /**
+     * This method deletes the given segment and creates a new segment with 
the given new base offset. It ensures an
+     * active segment exists in the log at all times during this process.
+     * <p>
+     * Asynchronous deletion allows reads to happen concurrently without 
synchronization and without the possibility of
+     * physically deleting a file while it is being read.
+     * <p>
+     * This method does not convert IOException to KafkaStorageException, the 
immediate caller
+     * is expected to catch and handle IOException.
+     *
+     * @param newOffset       The base offset of the new segment
+     * @param segmentToDelete The old active segment to schedule for deletion
+     * @param asyncDelete     Whether the segment files should be deleted 
asynchronously
+     * @param reason          The reason for the segment deletion
+     */
+    public LogSegment createAndDeleteSegment(long newOffset,
+                                             LogSegment segmentToDelete,
+                                             boolean asyncDelete,
+                                             SegmentDeletionReason reason) 
throws IOException {
+        if (newOffset == segmentToDelete.baseOffset()) {
+            segmentToDelete.changeFileSuffixes("", 
LogFileUtils.DELETED_FILE_SUFFIX);
+        }
+
+        LogSegment newSegment = LogSegment.open(dir,
+                newOffset,
+                config,
+                time,
+                false,
+                config.initFileSize(),
+                config.preallocate,
+                 "");
+        segments.add(newSegment);
+
+        reason.logReason(Collections.singletonList(segmentToDelete));
+        if (newOffset != segmentToDelete.baseOffset()) {
+            segments.remove(segmentToDelete.baseOffset());
+        }
+
+        deleteSegmentFiles(Collections.singletonList(segmentToDelete), 
asyncDelete, dir, topicPartition, config, scheduler, logDirFailureChannel, 
logger);
+
+        return newSegment;
+    }
+
+    /**
+     * Given a message offset, find its corresponding offset metadata in the 
log.
+     * If the message offset is out of range, throw an 
OffsetOutOfRangeException
+     */
+    public LogOffsetMetadata convertToOffsetMetadataOrThrow(long offset) {
+        FetchDataInfo fetchDataInfo = read(offset, 1, false, 
nextOffsetMetadata, false);
+        return fetchDataInfo.fetchOffsetMetadata;
+    }
+
+
+    /**
+     * Read messages from the log.
+     *
+     * @param startOffset        The offset to begin reading at
+     * @param maxLength          The maximum number of bytes to read
+     * @param minOneMessage      If this is true, the first message will be 
returned even if it exceeds `maxLength` (if one exists)
+     * @param maxOffsetMetadata  The metadata of the maximum offset to be 
fetched
+     * @param includeAbortedTxns If true, aborted transactions are included
+     * @return The fetch data information including fetch starting offset 
metadata and messages read.
+     * @throws OffsetOutOfRangeException If startOffset is beyond the log end 
offset
+     */
+    public FetchDataInfo read(long startOffset,
+                              int maxLength,
+                              boolean minOneMessage,
+                              LogOffsetMetadata maxOffsetMetadata,
+                              boolean includeAbortedTxns) {
+        return maybeHandleIOException(
+                () -> "Exception while reading for " + topicPartition + " in 
dir " + dir.getParent(),
+                () -> {
+                    logger.trace("Reading maximum {} bytes at offset {} from 
log with total length {} bytes", maxLength, startOffset, 
segments.sizeInBytes());
+
+                    LogOffsetMetadata endOffsetMetadata = nextOffsetMetadata;
+                    long endOffset = endOffsetMetadata.messageOffset;
+                    Optional<LogSegment> segmentOpt = 
segments.floorSegment(startOffset);
+
+                    // return error on attempt to read beyond the log end 
offset
+                    if (startOffset > endOffset || !segmentOpt.isPresent())
+                        throw new OffsetOutOfRangeException("Received request 
for offset " + startOffset + " for partition " + topicPartition + ", but we 
only have log segments upto " + endOffset + ".");
+
+                    if (startOffset == maxOffsetMetadata.messageOffset)
+                        return emptyFetchDataInfo(maxOffsetMetadata, 
includeAbortedTxns);
+                    else if (startOffset > maxOffsetMetadata.messageOffset)
+                        return 
emptyFetchDataInfo(convertToOffsetMetadataOrThrow(startOffset), 
includeAbortedTxns);
+                    else {
+                        // Do the read on the segment with a base offset less 
than the target offset
+                        // but if that segment doesn't contain any messages 
with an offset greater than that
+                        // continue to read from successive segments until we 
get some messages or we reach the end of the log
+                        FetchDataInfo fetchDataInfo = null;
+                        while (fetchDataInfo == null && 
segmentOpt.isPresent()) {
+                            LogSegment segment = segmentOpt.get();
+                            long baseOffset = segment.baseOffset();
+
+                            int maxPosition =
+                                    // Use the max offset position if it is on 
this segment; otherwise, the segment size is the limit.
+                                    maxOffsetMetadata.segmentBaseOffset == 
segment.baseOffset() ? maxOffsetMetadata.relativePositionInSegment : 
segment.size();
+
+                            fetchDataInfo = segment.read(startOffset, 
maxLength, maxPosition, minOneMessage);
+                            if (fetchDataInfo != null) {
+                                if (includeAbortedTxns)
+                                    fetchDataInfo = 
addAbortedTransactions(startOffset, segment, fetchDataInfo);
+                            } else segmentOpt = 
segments.higherSegment(baseOffset);
+                        }
+
+                        if (fetchDataInfo != null) return fetchDataInfo;
+                        else {
+                            // okay we are beyond the end of the last segment 
with no data fetched although the start offset is in range,
+                            // this can happen when all messages with offset 
larger than start offsets have been deleted.
+                            // In this case, we will return the empty set with 
log end offset metadata
+                            return new FetchDataInfo(nextOffsetMetadata, 
MemoryRecords.EMPTY);
+                        }
+                    }
+                });
+    }
+
+    public void append(long lastOffset, long largestTimestamp, long 
shallowOffsetOfMaxTimestamp, MemoryRecords records) throws IOException {
+        segments.activeSegment().append(lastOffset, largestTimestamp, 
shallowOffsetOfMaxTimestamp, records);
+        updateLogEndOffset(lastOffset + 1);
+    }
+
+    FetchDataInfo addAbortedTransactions(long startOffset, LogSegment segment, 
FetchDataInfo fetchInfo) throws IOException {
+        int fetchSize = fetchInfo.records.sizeInBytes();
+        OffsetPosition startOffsetPosition
+                = new 
OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset, 
fetchInfo.fetchOffsetMetadata.relativePositionInSegment);
+        long upperBoundOffset = 
segment.fetchUpperBoundOffset(startOffsetPosition, fetchSize)
+                .orElse(segments.higherSegment(segment.baseOffset())
+                        .map(LogSegment::baseOffset).orElse(logEndOffset()));
+
+        List<FetchResponseData.AbortedTransaction> abortedTransactions = new 
ArrayList<>();
+
+        Consumer<List<AbortedTxn>> accumulator = abortedTxns -> 
abortedTransactions.addAll(abortedTxns.stream()
+                
.map(AbortedTxn::asAbortedTransaction).collect(Collectors.toList()));
+
+        collectAbortedTransactions(startOffset, upperBoundOffset, segment, 
accumulator);
+
+        return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
+                fetchInfo.records,
+                fetchInfo.firstEntryIncomplete,
+                Optional.of(abortedTransactions));
+    }
+
+    void collectAbortedTransactions(long startOffset,
+                                    long upperBoundOffset,
+                                    LogSegment startingSegment,
+                                    Consumer<List<AbortedTxn>> accumulator) 
throws IOException {
+        Iterator<LogSegment> higherSegments = 
segments.higherSegments(startingSegment.baseOffset()).iterator();
+        Optional<LogSegment> segmentEntryOpt = Optional.of(startingSegment);
+        while (segmentEntryOpt.isPresent()) {
+            LogSegment segment = segmentEntryOpt.get();
+            TxnIndexSearchResult searchResult = 
segment.collectAbortedTxns(startOffset, upperBoundOffset);
+            accumulator.accept(searchResult.abortedTransactions);
+            if (searchResult.isComplete) return;
+            segmentEntryOpt = nextItem(higherSegments);
+        }
+    }
+
+    public List<AbortedTxn> collectAbortedTransactions(long logStartOffset,
+                                                       long baseOffset,
+                                                       long upperBoundOffset) 
throws IOException {
+        Optional<LogSegment> segmentEntry = segments.floorSegment(baseOffset);
+        List<AbortedTxn> allAbortedTxns = new ArrayList<>();
+
+        if (segmentEntry.isPresent())
+            collectAbortedTransactions(logStartOffset, upperBoundOffset, 
segmentEntry.get(), allAbortedTxns::addAll);
+
+        return allAbortedTxns;
+    }
+
+    public LogSegment roll() {
+        return roll(OptionalLong.empty());
+    }
+
+    /**
+     * Roll the log over to a new active segment starting with the current 
logEndOffset.
+     * This will trim the index to the exact size of the number of entries it 
currently contains.
+     *
+     * @param expectedNextOffset The expected next offset after the segment is 
rolled
+     * @return The newly rolled segment
+     */
+    public LogSegment roll(OptionalLong expectedNextOffset) {
+        return maybeHandleIOException(
+                () -> "Error while rolling log segment for " + topicPartition 
+ " in dir " + dir.getParent(),
+                () -> {
+                    long start = time.hiResClockMs();
+                    checkIfMemoryMappedBufferClosed();
+                    long newOffset = Math.max(expectedNextOffset.orElse(0L), 
logEndOffset());
+                    File logFile = LogFileUtils.logFile(dir, newOffset, "");
+                    LogSegment activeSegment = segments.activeSegment();
+                    if (segments.contains(newOffset)) {
+                        // segment with the same base offset already exists 
and loaded
+                        if (activeSegment.baseOffset() == newOffset && 
activeSegment.size() == 0) {
+                            // We have seen this happen (see KAFKA-6388) after 
shouldRoll() returns true for an
+                            // active segment of size zero because of one of 
the indexes is "full" (due to _maxEntries == 0).
+                            logger.warn("Trying to roll a new log segment with 
start offset {} =max(provided offset = {}, LEO = {}) while it already exists 
and is active with size 0. Size of time index: {}, size of offset index: {}.",
+                                    newOffset, expectedNextOffset, 
logEndOffset(), activeSegment.timeIndex().entries(), 
activeSegment.offsetIndex().entries());
+                            LogSegment newSegment = 
createAndDeleteSegment(newOffset, activeSegment, true, new 
LogRoll(this.logger));
+                            
updateLogEndOffset(nextOffsetMetadata.messageOffset);
+                            logger.info("Rolled new log segment at offset {} 
in {} ms.", newOffset, time.hiResClockMs() - start);
+                            return newSegment;
+                        } else {
+                            throw new KafkaException("Trying to roll a new log 
segment for topic partition "
+                                    + topicPartition + " with start offset " + 
newOffset + " + =max(provided offset = "
+                                    + expectedNextOffset + ", LEO = " + 
logEndOffset() + " while it already exists. Existing segment is "
+                                    + segments.get(newOffset) + ".");
+                        }
+                    } else if (!segments.isEmpty() && newOffset < 
activeSegment.baseOffset()) {
+                        throw new KafkaException("Trying to roll a new log 
segment for topic partition " + topicPartition
+                                + " with start offset " + newOffset + " 
=max(provided offset = " + expectedNextOffset
+                                + ", LEO = " + logEndOffset() + ") lower than 
start offset of the active segment " + activeSegment);
+                    } else {
+                        File offsetIdxFile = LogFileUtils.offsetIndexFile(dir, 
newOffset);
+                        File timeIdxFile = LogFileUtils.timeIndexFile(dir, 
newOffset);
+                        File txnIdxFile = 
LogFileUtils.transactionIndexFile(dir, newOffset);
+
+                        for (File file : Arrays.asList(logFile, offsetIdxFile, 
timeIdxFile, txnIdxFile)) {
+                            if (file.exists()) {
+                                logger.warn("Newly rolled segment file {} 
already exists; deleting it first", file.getAbsolutePath());
+                                Files.delete(file.toPath());
+                            }
+                        }
+
+                        Optional<LogSegment> logSegmentOption = 
segments.lastSegment();
+                        if (logSegmentOption.isPresent()) 
logSegmentOption.get().onBecomeInactiveSegment();
+                    }
+
+                    LogSegment newSegment = LogSegment.open(dir,
+                            newOffset,
+                            config,
+                            time,
+                            false,
+                            config.initFileSize(),
+                            config.preallocate,
+                            "");
+                    segments.add(newSegment);
+
+                    // We need to update the segment base offset and append 
position data of the metadata when log rolls.
+                    // The next offset should not change.
+                    updateLogEndOffset(nextOffsetMetadata.messageOffset);
+
+                    logger.info("Rolled new log segment at offset {} in {} 
ms.", newOffset, time.hiResClockMs() - start);
+
+                    return newSegment;
+                });
+    }
+
+    /**
+     * Delete all data in the local log and start at the new offset.
+     *
+     * @param newOffset The new offset to start the log with
+     * @return the list of segments that were scheduled for deletion
+     */
+    public Collection<LogSegment> truncateFullyAndStartAt(long newOffset) {
+        return maybeHandleIOException(
+                () -> "Error while truncating the entire log for " + 
topicPartition + " in dir " + dir.getParent(),
+                () -> {
+                    logger.debug("Truncate and start at offset {}", newOffset);
+                    checkIfMemoryMappedBufferClosed();
+                    List<LogSegment> segmentsToDelete = new 
ArrayList<>(segments.values());
+
+                    if (!segmentsToDelete.isEmpty()) {
+                        removeAndDeleteSegments(segmentsToDelete.subList(0, 
segmentsToDelete.size() - 1), true, new LogTruncation(this.logger));
+                        // Use createAndDeleteSegment() to create new segment 
first and then delete the old last segment to prevent missing
+                        // active segment during the deletion process

Review Comment:
   `createAndDeleteSegment` method it self deletes the targeted segment passed 
to this method. Method Javadoc with argument(segmentToDelete) clarifies that 
the the given segment is deleted.



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