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 2021/03/08 21:59:26 UTC

[GitHub] [kafka] kowshik opened a new pull request #10280: KIP-405: Log layer refactor

kowshik opened a new pull request #10280:
URL: https://github.com/apache/kafka/pull/10280


   **This PR is a work in progress. Please do not treat this as a finished product yet.**
   
   **TL;DR:**
   
   This PR implements the details of the Log layer refactor, as outlined in this document: https://docs.google.com/document/d/1dQJL4MCwqQJSPmZkVmVzshFZKuFy_bCPtubav4wBfHQ/edit. Few details maybe different from the doc, but it is more or less the same.
   
   **STRATEGY:**
   
   In this PR, I've extracted a new class called `LocalLog` out of `Log`.  Currently `LocalLog` is purely an implementation detail thats not exposed outside `Log` class (except for tests). The object encapsulation is that each `Log` instance  wraps around a `LocalLog` instance.
   
   This new `LocalLog` class attempts to encompass **most** of the responsibilities of local log surrounding the `segments` map, which otherwise were present in `Log` previously. Note that **not** all local log responsibilities have been moved over to this new class (yet). The criteria I used was to preserve (for now) in existing `Log` class, any logic that is mingled  in a complex manner with the `logStartOffset` or the `LeaderEpochCache` or the `ProducerStateManager`.
   
   **WINS:**
   
   The main win is that the new `LocalLog` class is now agnostic of the `logStartOffset`, which continues to be managed mainly by `Log` class. Below is the local log functionality that has successfully moved over from `Log` to `LocalLog`:
   
   1. The `ConcurrentNavigableMap` containing map of of offset -> segment, and the access logic surrounding it.
   2. Read path logic to read records from the log.
   3. Segment file deletion logic.
   4. Segment recovery.
   5. Segment truncation.
   6. Segment roll.
   
   **BLOCKERS:**
   
   The API of `LocalLog` is probably not crisp at this point. Below is the main local log functionality that continues to remain in `Log` due to blockers. The reason is that the below logic is mingled with one or more of the following: `logStartOffset` or `LeaderEpochCache` or `ProducerStateManager`. This makes it hard to separate just the local logic out of it:
   
   1. Segment append.
   2. Last stable offset and logic surrounding it.
   3. High watermark and logic surrounding it.
   4. Logic to `fetchOffsetByTimestamp` and logic to `legacyFetchOffsetsBefore`.
   5. Some of the retention logic thats related with the global view of the log.
   6. All other logic related with handling  `LeaderEpochCache` and `ProducerStateManager`.
   
   **PAINPOINTS:**
   
   1. Producer state manager instance needed to be passed explicitly into the roll API to capture snapshots before roll.
   2. Log locking semantics had to be changed in handful of areas, with lock taken at a coarse level (ex: recovery).
   3. Few APIs needed re-ordering of logic in `Log` class to make migration feasible.
   4. Certain APIs added to `LocalLog` are crude in nature or signature, examples: `def checkIfMemoryMappedBufferClosed`, `def markFlushed`, `def updateRecoveryPoint`, `def replaceSegments` etc.
   5. Certain important APIs (such as `def append` logic) were hard to migrate because logic was mingled with Leader epoch cache, Producer state manager and log start offset.
   
   **TESTS:**
   Note that just the test `LogTest. testProducerSnapshotsRecoveryAfterUncleanShutdown*` is failing, and I'll need to look into this. Otherwise all other tests are expected to pass for this PR. 


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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-808514958


   @junrao Just a heads up on the following. I'm working on the changes for KAFKA-12552 (https://github.com/apache/kafka/pull/10401 to extract segments map) and KAFKA-12553 (refactor recovery logic) in separate PRs. It seems better if we merge those into trunk ahead of the current PR.


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

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



[GitHub] [kafka] junrao commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r600752029



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.

Review comment:
       "each with a base offset denoting the first message in the segment" : This is probably not true for a compacted segment. Perhaps we could just say base offset.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.

Review comment:
       The comment is inaccurate since the method no longer deletes segments.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()

Review comment:
       This seems unused in Log.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {

Review comment:
       This is only used inside localLog and doesn't need package level access.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {

Review comment:
       This is only used inside localLog and doesn't need package level access. Ditto in a few other places.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],

Review comment:
       This is only used inside localLog and doesn't need package level access.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {

Review comment:
       This seems only used in `close()`? If so, could we just fold it into `close()`?

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,

Review comment:
       This is only used inside localLog and doesn't need package level access.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.

Review comment:
       The whole class is not thread-safe.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -836,178 +581,15 @@ class Log(@volatile private var _dir: File,
       updateHighWatermark(offset)
     }
 
-    if (this.recoveryPoint < offset) {
-      this.recoveryPoint = offset
-    }
-  }
-
-  /**
-   * Recover the log segments and return the next offset after recovery.
-   * This method does not need to convert IOException to KafkaStorageException because it is only called before all
-   * logs are loaded.
-   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
-   */
-  private[log] def recoverLog(): Long = {
-    /** return the log end offset if valid */
-    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
-      if (logSegments.nonEmpty) {
-        val logEndOffset = activeSegment.readNextOffset
-        if (logEndOffset >= logStartOffset)
-          Some(logEndOffset)
-        else {
-          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
-            "This could happen if segment files were deleted from the file system.")
-          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
-          leaderEpochCache.foreach(_.clearAndFlush())
-          producerStateManager.truncateFullyAndStartAt(logStartOffset)
-          None
-        }
-      } else None
-    }
-
-    // if we have the clean shutdown marker, skip recovery
-    if (!hadCleanShutdown) {
-      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
-      var truncated = false
-
-      while (unflushed.hasNext && !truncated) {
-        val segment = unflushed.next()
-        info(s"Recovering unflushed segment ${segment.baseOffset}")
-        val truncatedBytes =
-          try {
-            recoverSegment(segment, leaderEpochCache)
-          } catch {
-            case _: InvalidOffsetException =>
-              val startOffset = segment.baseOffset
-              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
-                s"creating an empty one with starting offset $startOffset")
-              segment.truncateTo(startOffset)
-          }
-        if (truncatedBytes > 0) {
-          // we had an invalid message, delete all remaining log
-          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
-          removeAndDeleteSegments(unflushed.toList,
-            asyncDelete = true,
-            reason = LogRecovery)
-          truncated = true
-        }
-      }
-    }
-
-    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
-
-    if (logSegments.isEmpty) {
-      // no existing segments, create a new mutable segment beginning at logStartOffset
-      addSegment(LogSegment.open(dir = dir,
-        baseOffset = logStartOffset,
-        config,
-        time = time,
-        initFileSize = this.initFileSize,
-        preallocate = config.preallocate))
-    }
-
-    // Update the recovery point if there was a clean shutdown and did not perform any changes to
-    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
-    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
-    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
-    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
-    // segment.
-    (hadCleanShutdown, logEndOffsetOption) match {
-      case (true, Some(logEndOffset)) =>
-        recoveryPoint = logEndOffset
-        logEndOffset
-      case _ =>
-        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
-        recoveryPoint = Math.min(recoveryPoint, logEndOffset)
-        logEndOffset
-    }
-  }
-
-  // Rebuild producer state until lastOffset. This method may be called from the recovery code path, and thus must be
-  // free of all side-effects, i.e. it must not update any log-specific state.
-  private def rebuildProducerState(lastOffset: Long,
-                                   reloadFromCleanShutdown: Boolean,
-                                   producerStateManager: ProducerStateManager): Unit = lock synchronized {
-    checkIfMemoryMappedBufferClosed()
-    val segments = logSegments
-    val offsetsToSnapshot =
-      if (segments.nonEmpty) {
-        val nextLatestSegmentBaseOffset = lowerSegment(segments.last.baseOffset).map(_.baseOffset)
-        Seq(nextLatestSegmentBaseOffset, Some(segments.last.baseOffset), Some(lastOffset))
-      } else {
-        Seq(Some(lastOffset))
-      }
-    info(s"Loading producer state till offset $lastOffset with message format version ${recordVersion.value}")
-
-    // We want to avoid unnecessary scanning of the log to build the producer state when the broker is being
-    // upgraded. The basic idea is to use the absence of producer snapshot files to detect the upgrade case,
-    // but we have to be careful not to assume too much in the presence of broker failures. The two most common
-    // upgrade cases in which we expect to find no snapshots are the following:
-    //
-    // 1. The broker has been upgraded, but the topic is still on the old message format.
-    // 2. The broker has been upgraded, the topic is on the new message format, and we had a clean shutdown.
-    //
-    // If we hit either of these cases, we skip producer state loading and write a new snapshot at the log end
-    // offset (see below). The next time the log is reloaded, we will load producer state using this snapshot
-    // (or later snapshots). Otherwise, if there is no snapshot file, then we have to rebuild producer state
-    // from the first segment.
-    if (recordVersion.value < RecordBatch.MAGIC_VALUE_V2 ||
-        (producerStateManager.latestSnapshotOffset.isEmpty && reloadFromCleanShutdown)) {
-      // To avoid an expensive scan through all of the segments, we take empty snapshots from the start of the
-      // last two segments and the last offset. This should avoid the full scan in the case that the log needs
-      // truncation.
-      offsetsToSnapshot.flatten.foreach { offset =>
-        producerStateManager.updateMapEndOffset(offset)
-        producerStateManager.takeSnapshot()
-      }
-    } else {
-      info(s"Reloading from producer snapshot and rebuilding producer state from offset $lastOffset")
-      val isEmptyBeforeTruncation = producerStateManager.isEmpty && producerStateManager.mapEndOffset >= lastOffset
-      val producerStateLoadStart = time.milliseconds()
-      producerStateManager.truncateAndReload(logStartOffset, lastOffset, time.milliseconds())
-      val segmentRecoveryStart = time.milliseconds()
-
-      // Only do the potentially expensive reloading if the last snapshot offset is lower than the log end
-      // offset (which would be the case on first startup) and there were active producers prior to truncation
-      // (which could be the case if truncating after initial loading). If there weren't, then truncating
-      // shouldn't change that fact (although it could cause a producerId to expire earlier than expected),
-      // and we can skip the loading. This is an optimization for users which are not yet using
-      // idempotent/transactional features yet.
-      if (lastOffset > producerStateManager.mapEndOffset && !isEmptyBeforeTruncation) {
-        val segmentOfLastOffset = floorLogSegment(lastOffset)
-
-        logSegments(producerStateManager.mapEndOffset, lastOffset).foreach { segment =>
-          val startOffset = Utils.max(segment.baseOffset, producerStateManager.mapEndOffset, logStartOffset)
-          producerStateManager.updateMapEndOffset(startOffset)
-
-          if (offsetsToSnapshot.contains(Some(segment.baseOffset)))
-            producerStateManager.takeSnapshot()
-
-          val maxPosition = if (segmentOfLastOffset.contains(segment)) {
-            Option(segment.translateOffset(lastOffset))
-              .map(_.position)
-              .getOrElse(segment.size)
-          } else {
-            segment.size
-          }
-
-          val fetchDataInfo = segment.read(startOffset,
-            maxSize = Int.MaxValue,
-            maxPosition = maxPosition,
-            minOneMessage = false)
-          if (fetchDataInfo != null)
-            loadProducersFromRecords(producerStateManager, fetchDataInfo.records)
-        }
-      }
-      producerStateManager.updateMapEndOffset(lastOffset)
-      producerStateManager.takeSnapshot()
-      info(s"Producer state recovery took ${producerStateLoadStart - segmentRecoveryStart}ms for snapshot load " +
-        s"and ${time.milliseconds() - segmentRecoveryStart}ms for segment recovery from offset $lastOffset")
+    if (localLog.recoveryPoint < offset) {
+      localLog.updateRecoveryPoint(offset)
     }
   }
 
   private def loadProducerState(lastOffset: Long, reloadFromCleanShutdown: Boolean): Unit = lock synchronized {
-    rebuildProducerState(lastOffset, reloadFromCleanShutdown, producerStateManager)
+    lock synchronized {

Review comment:
       Hmm, we don't need to synchronize on the lock twice, right?

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.

Review comment:
       This method returns more than the next offset.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because

Review comment:
       This is an existing problem, but the comment is inaccurate since it does convert IOException to KafkaStorageException.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1816,8 +1292,12 @@ class Log(@volatile private var _dir: File,
    */
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
+    def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long): Boolean = {
+      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(logEndOffset) &&

Review comment:
       Hmm, why do we need to wrap `predicate` with an additional condition?

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1794,20 +1269,21 @@ class Log(@volatile private var _dir: File,
     ret.toSeq.sortBy(-_)
   }
 
-  /**
-    * Given a message offset, find its corresponding offset metadata in the log.
-    * If the message offset is out of range, throw an OffsetOutOfRangeException
-    */
-  private def convertToOffsetMetadataOrThrow(offset: Long): LogOffsetMetadata = {
-    val fetchDataInfo = read(offset,
-      maxLength = 1,
-      isolation = FetchLogEnd,
-      minOneMessage = false)
-    fetchDataInfo.fetchOffsetMetadata
+  def convertToOffsetMetadata(offset: Long): Option[LogOffsetMetadata] = {

Review comment:
       Is this method being used?

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -213,32 +208,38 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason {
 }
 
 /**
- * An append-only log for storing messages.
+ * A log which presents a unified view of local and tiered log segments.
  *
- * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * The log consists of tiered and local segments with the tiered portion of the log being optional. There could be an
+ * overlap between the tiered and local segments. The active segment is always guaranteed to be local. If tiered segments
+ * are present, they always appear at the head of the log, followed by an optional region of overlap, followed by the local

Review comment:
       Some people think the end of the log is the head. So, maybe use beginning/end of log instead.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -213,32 +208,38 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason {
 }
 
 /**
- * An append-only log for storing messages.
+ * A log which presents a unified view of local and tiered log segments.
  *
- * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * The log consists of tiered and local segments with the tiered portion of the log being optional. There could be an
+ * overlap between the tiered and local segments. The active segment is always guaranteed to be local. If tiered segments
+ * are present, they always appear at the head of the log, followed by an optional region of overlap, followed by the local
+ * segments including the active segment.
  *
- * 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 handles state and behavior specific to tiered segments as well as any behavior combining both tiered
+ * and local segments. The state and behavior specific to local segments is handled by the encapsulated LocalLog instance.

Review comment:
       is handled => are handled

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator

Review comment:
       No need for `this`. Ditto in a few other places.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {

Review comment:
       This is only used inside localLog and doesn't need package level access.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because
+   * it is either called before all logs are loaded or the caller will catch and handle IOException
+   *
+   * @throws IOException if the segment files can't be renamed and still exists
+   */
+  private[log] def deleteSegmentFiles(segments: Iterable[LogSegment],
+                                      asyncDelete: Boolean): Unit = {
+    segments.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix))
+
+    def deleteSegments(): Unit = {
+      info(s"Deleting segment files ${segments.mkString(",")}")
+      maybeHandleIOException(s"Error while deleting segments for $topicPartition in dir ${dir.getParent}") {
+        segments.foreach { segment =>
+          segment.deleteIfExists()
+        }
+      }
+    }
+
+    if (asyncDelete)
+      scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs)
+    else
+      deleteSegments()
+  }
+
+  /**
+   * This method deletes the given log segments by doing the following for each of them:
+   * <ol>
+   *   <li>It removes the segment from the segment map so that it will no longer be used for reads.
+   *   <li>It renames the index and log files by appending .deleted to the respective file name
+   *   <li>It can either schedule an asynchronous delete operation to occur in the future or perform the deletion synchronously
+   * </ol>
+   * Asynchronous deletion allows reads to happen concurrently without synchronization and without the possibility of
+   * physically deleting a file while it is being read.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the immediate caller will catch and handle IOException
+   *
+   * @param segments The log segments to schedule for deletion
+   * @param asyncDelete Whether the segment files should be deleted asynchronously
+   */
+  private[log] def removeAndDeleteSegments(segments: Iterable[LogSegment],
+                                           asyncDelete: Boolean,
+                                           reason: SegmentDeletionReason): Unit = {
+    if (segments.nonEmpty) {
+      // As 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.
+      val toDelete = segments.toList
+      reason.logReason(this, toDelete)
+      toDelete.foreach { segment =>
+        this.segments.remove(segment.baseOffset)
+      }
+      deleteSegmentFiles(toDelete, asyncDelete)
+    }
+  }
+
+  private[log] def emptyFetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
+                                      includeAbortedTxns: Boolean): FetchDataInfo = {
+    val abortedTransactions =
+      if (includeAbortedTxns) Some(List.empty[FetchResponseData.AbortedTransaction])
+      else None
+    FetchDataInfo(fetchOffsetMetadata,
+      MemoryRecords.EMPTY,
+      abortedTransactions = abortedTransactions)
+  }
+
+  /**
+   * Given a message offset, find its corresponding offset metadata in the log.
+   * If the message offset is out of range, return None to the caller.
+   */
+  private[log] def convertToOffsetMetadata(offset: Long): Option[LogOffsetMetadata] = {
+    try {
+      Some(convertToOffsetMetadataOrThrow(offset))
+    } catch {
+      case _: OffsetOutOfRangeException => None
+    }
+  }
+
+  /**
+   * Given a message offset, find its corresponding offset metadata in the log.
+   * If the message offset is out of range, throw an OffsetOutOfRangeException
+   */
+  private[log] def convertToOffsetMetadataOrThrow(offset: Long): LogOffsetMetadata = {
+    val fetchDataInfo = read(offset,
+      maxLength = 1,
+      minOneMessage = false,
+      maxOffsetMetadata = nextOffsetMetadata,
+      includeAbortedTxns = false)
+    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 this is true, aborted transactions are included in the fetch data information
+   * @throws OffsetOutOfRangeException If startOffset is beyond the log end offset
+   *
+   * @return The fetch data information including fetch starting offset metadata and messages read.
+   */
+  private[log] def read(startOffset: Long,
+                        maxLength: Int,
+                        minOneMessage: Boolean,
+                        maxOffsetMetadata: LogOffsetMetadata,
+                        includeAbortedTxns: Boolean): FetchDataInfo = {
+    maybeHandleIOException(s"Exception while reading from $topicPartition in dir ${dir.getParent}") {
+      trace(s"Reading maximum $maxLength bytes at offset $startOffset from log with " +
+        s"total length $size bytes")
+
+      // Because we don't use the lock for reading, the synchronization is a little bit tricky.
+      // We create the local variables to avoid race conditions with updates to the log.
+      val endOffsetMetadata = nextOffsetMetadata
+      val endOffset = endOffsetMetadata.messageOffset
+      var segmentEntry = segments.floorEntry(startOffset)
+
+      // return error on attempt to read beyond the log end offset or read below log start offset
+      if (startOffset > endOffset || segmentEntry == null)
+        throw new OffsetOutOfRangeException(s"Received request for offset $startOffset for partition $topicPartition, " +
+          s"but we only have log segments upto $endOffset.")
+
+      if (startOffset == maxOffsetMetadata.messageOffset)
+        emptyFetchDataInfo(maxOffsetMetadata, includeAbortedTxns)
+      else if (startOffset > maxOffsetMetadata.messageOffset)
+        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
+        var done = segmentEntry == null
+        var fetchDataInfo: FetchDataInfo = null
+        while (!done) {
+          val segment = segmentEntry.getValue
+
+          val maxPosition =
+          // Use the max offset position if it is on this segment; otherwise, the segment size is the limit.
+            if (maxOffsetMetadata.segmentBaseOffset == segment.baseOffset) maxOffsetMetadata.relativePositionInSegment
+            else segment.size
+
+          fetchDataInfo = segment.read(startOffset, maxLength, maxPosition, minOneMessage)
+          if (fetchDataInfo != null) {
+            if (includeAbortedTxns)
+              fetchDataInfo = addAbortedTransactions(startOffset, segmentEntry, fetchDataInfo)
+          } else segmentEntry = segments.higherEntry(segmentEntry.getKey)
+
+          done = fetchDataInfo != null || segmentEntry == null
+        }
+
+        if (fetchDataInfo != null) 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
+          FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
+        }
+      }
+    }
+  }
+
+  private def addAbortedTransactions(startOffset: Long, segmentEntry: JEntry[JLong, LogSegment],
+                                     fetchInfo: FetchDataInfo): FetchDataInfo = {
+    val fetchSize = fetchInfo.records.sizeInBytes
+    val startOffsetPosition = OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+      fetchInfo.fetchOffsetMetadata.relativePositionInSegment)
+    val upperBoundOffset = segmentEntry.getValue.fetchUpperBoundOffset(startOffsetPosition, fetchSize).getOrElse {
+      val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+      if (nextSegmentEntry != null)
+        nextSegmentEntry.getValue.baseOffset
+      else
+        logEndOffset
+    }
+
+    val abortedTransactions = ListBuffer.empty[FetchResponseData.AbortedTransaction]
+    def accumulator(abortedTxns: List[AbortedTxn]): Unit = abortedTransactions ++= abortedTxns.map(_.asAbortedTransaction)
+    collectAbortedTransactions(startOffset, upperBoundOffset, segmentEntry, accumulator)
+
+    FetchDataInfo(fetchOffsetMetadata = fetchInfo.fetchOffsetMetadata,
+      records = fetchInfo.records,
+      firstEntryIncomplete = fetchInfo.firstEntryIncomplete,
+      abortedTransactions = Some(abortedTransactions.toList))
+  }
+
+  private[log] def collectAbortedTransactions(logStartOffset: Long, baseOffset: Long, upperBoundOffset: Long): List[AbortedTxn] = {
+    val segmentEntry = segments.floorEntry(baseOffset)
+    val allAbortedTxns = ListBuffer.empty[AbortedTxn]
+    def accumulator(abortedTxns: List[AbortedTxn]): Unit = allAbortedTxns ++= abortedTxns
+    collectAbortedTransactions(logStartOffset, upperBoundOffset, segmentEntry, accumulator)
+    allAbortedTxns.toList
+  }
+
+  private def collectAbortedTransactions(startOffset: Long, upperBoundOffset: Long,
+                                         startingSegmentEntry: JEntry[JLong, LogSegment],
+                                         accumulator: List[AbortedTxn] => Unit): Unit = {
+    var segmentEntry = startingSegmentEntry
+    while (segmentEntry != null) {
+      val searchResult = segmentEntry.getValue.collectAbortedTxns(startOffset, upperBoundOffset)
+      accumulator(searchResult.abortedTransactions)
+      if (searchResult.isComplete)
+        return
+      segmentEntry = segments.higherEntry(segmentEntry.getKey)
+    }
+  }
+
+  /**
+   * This function does not acquire Log.lock. The caller has to make sure log segments don't get deleted during

Review comment:
       Log.lock is in a different class now.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because
+   * it is either called before all logs are loaded or the caller will catch and handle IOException
+   *
+   * @throws IOException if the segment files can't be renamed and still exists
+   */
+  private[log] def deleteSegmentFiles(segments: Iterable[LogSegment],
+                                      asyncDelete: Boolean): Unit = {
+    segments.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix))
+
+    def deleteSegments(): Unit = {
+      info(s"Deleting segment files ${segments.mkString(",")}")
+      maybeHandleIOException(s"Error while deleting segments for $topicPartition in dir ${dir.getParent}") {
+        segments.foreach { segment =>
+          segment.deleteIfExists()
+        }
+      }
+    }
+
+    if (asyncDelete)
+      scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs)
+    else
+      deleteSegments()
+  }
+
+  /**
+   * This method deletes the given log segments by doing the following for each of them:
+   * <ol>
+   *   <li>It removes the segment from the segment map so that it will no longer be used for reads.
+   *   <li>It renames the index and log files by appending .deleted to the respective file name
+   *   <li>It can either schedule an asynchronous delete operation to occur in the future or perform the deletion synchronously
+   * </ol>
+   * Asynchronous deletion allows reads to happen concurrently without synchronization and without the possibility of
+   * physically deleting a file while it is being read.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded

Review comment:
       This is an existing problem, but the comment is inaccurate since it does convert IOException to KafkaStorageException.




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

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



[GitHub] [kafka] ijuma commented on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-793927102


   Can we do one PR for renaming `Log` to `LocalLog` and then a separate ones for the rest? It seems like git rename detection failed here and it will make diffs harder to review.


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602052452



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because

Review comment:
       The method only converts `IOException` to `KafkaStorageException` inside the internal `deleteSegments()` helper function, but it doesn't doe it for `changeFileSuffixes` which is what the comment was referring to. I've improved the `@throws` doc to refer to both exceptions now. Please let me know if it needs further improvement.




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-861219607


   I ran the system tests in `kafkatest.tests.client.consumer_test` again:
    * [4564](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4564/) against  `trunk/6de37e536ac76ef13530d49dc7320110332cd1ee`.
    * [4566](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4566/) against 008b701386ce5a4d892d6ac5b90798b981c4fba0 (the latest commit from this PR).
   
   All tests passed. 


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602056301



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -213,32 +208,38 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason {
 }
 
 /**
- * An append-only log for storing messages.
+ * A log which presents a unified view of local and tiered log segments.
  *
- * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * The log consists of tiered and local segments with the tiered portion of the log being optional. There could be an
+ * overlap between the tiered and local segments. The active segment is always guaranteed to be local. If tiered segments
+ * are present, they always appear at the head of the log, followed by an optional region of overlap, followed by the local
+ * segments including the active segment.
  *
- * 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 handles state and behavior specific to tiered segments as well as any behavior combining both tiered
+ * and local segments. The state and behavior specific to local segments is handled by the encapsulated LocalLog instance.

Review comment:
       Done.




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

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



[GitHub] [kafka] junrao commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r646791532



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1578,144 +1409,85 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      val newSegment = localLog.roll(Some(rollOffset))
+      afterRoll(newSegment)
+      newSegment

Review comment:
       Could we replace the above 3 lines with `roll(Some(rollOffset))`?

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1812,37 +1577,36 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
+    localLog.updateLogEndOffset(endOffset)

Review comment:
       We need to preserve the LogOffsetMetadata for endOffset and use it to call updateHighWatermark.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1578,144 +1409,85 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      val newSegment = localLog.roll(Some(rollOffset))
+      afterRoll(newSegment)
+      newSegment
     } else {
       segment
     }
   }
 
   /**
-   * Roll the log over to a new active segment starting with the current logEndOffset.
+   * Roll the local log over to a new active segment starting with the current logEndOffset.

Review comment:
       This comment is not very accurate since we roll to expectedNextOffset or logEndOffset.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1788,14 +1559,8 @@ class Log(@volatile private var _dir: File,
     maybeHandleIOException(s"Error while truncating the entire log for $topicPartition in dir ${dir.getParent}") {
       debug(s"Truncate and start at offset $newOffset")
       lock synchronized {
-        checkIfMemoryMappedBufferClosed()
-        removeAndDeleteSegments(logSegments, asyncDelete = true, LogTruncation)
-        addSegment(LogSegment.open(dir,
-          baseOffset = newOffset,
-          config = config,
-          time = time,
-          initFileSize = config.initFileSize,
-          preallocate = config.preallocate))
+        val deletedSegments = localLog.truncateFullyAndStartAt(newOffset)
+        deleteProducerSnapshots(deletedSegments, asyncDelete = true)

Review comment:
       `producerStateManager.truncateFullyAndStartAt()` removes all producer snapshots. So, this is necessary.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1010 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.nio.file.Files
+import java.text.NumberFormat
+import java.util.concurrent.atomic.AtomicLong
+import java.util.regex.Pattern
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{Logging, Scheduler}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.Seq
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+
+/**
+ * Holds the result of splitting a segment into one or more segments, see LocalLog.splitOverflowedSegment().
+ *
+ * @param deletedSegments segments deleted when splitting a segment
+ * @param newSegments new segments created when splitting a segment
+ */
+case class SplitSegmentResult(deletedSegments: Iterable[LogSegment], newSegments: Iterable[LogSegment])
+
+/**
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @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
+ */
+private[log] class LocalLog(@volatile private var _dir: File,
+                            @volatile var config: LogConfig,
+                            val segments: LogSegments,
+                            @volatile var recoveryPoint: Long,
+                            @volatile private var nextOffsetMetadata: LogOffsetMetadata,
+                            val scheduler: Scheduler,
+                            val time: Time,
+                            val topicPartition: TopicPartition,
+                            val logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[LocalLog partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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.
+  @volatile private[log] var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  // Last time the log was flushed
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  private[log] def dir: File = _dir
+
+  private[log] def name: String = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(LocalLog.FutureDirSuffix)
+
+  private def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    LocalLog.maybeHandleIOException(logDirFailureChannel, parentDir, msg) {
+      fun
+    }
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        segments.updateParentDir(renamedDir)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  /**
+   * Update the existing configuration to the new provided configuration.
+   * @param newConfig the new configuration to be updated to
+   */
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = config
+    config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * Flush local log segments for all offsets up to offset-1.
+   * Does not update the recovery point.
+   *
+   * @param offset The offset to flush up to (non-inclusive)
+   */
+  private[log] def flush(offset: Long): Unit = {
+    val segmentsToFlush = segments.values(recoveryPoint, offset)
+    segmentsToFlush.foreach(_.flush())
+    // If there are any new segments, we need to flush the parent directory for crash consistency.
+    segmentsToFlush.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath))
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and update the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = 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.
+   */
+  private[log] def closeHandlers(): Unit = {
+    segments.closeHandlers()
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the segments of the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      segments.close()
+    }
+  }
+
+  /**
+   * Completely delete this log directory with no delay.
+   */
+  private[log] def deleteEmptyDir(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      if (segments.nonEmpty) {
+        throw new IllegalStateException(s"Can not delete directory when ${segments.numberOfSegments} segments are still present")
+      }
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true

Review comment:
       It seems that we should set isMemoryMappedBufferClosed in deleteAllSegments()?

##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -246,17 +262,17 @@ object LogLoader extends Logging {
         return fn
       } catch {
         case e: LogSegmentOffsetOverflowException =>
-          info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
-          Log.splitOverflowedSegment(
+          info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          val result = Log.splitOverflowedSegment(
             e.segment,
             params.segments,
             params.dir,
             params.topicPartition,
             params.config,
             params.scheduler,
             params.logDirFailureChannel,
-            params.producerStateManager,
             params.logIdentifier)
+          deleteProducerSnapshotsAsync(result.deletedSegments, params)

Review comment:
       Yes, that's fine.




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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-794226594


   @ijuma The purpose of `GlobalLog` class is to serve as a higher layer, stitching together the unified view of both the local and remote portion of the log. Importantly, this class is aware of the global log start offset. 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. It could just be called as `Log` too, but I thought by calling it `GlobalLog` the intention/differentiation is clear.
   
   The above and few more things are explained in the doc attached in the description. I'd suggest having a look at the doc.


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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-854616475


   @junrao Thanks for the review! I've addressed your comments in e201295e03e0ea8a7102983888d1a7afc66d384a, and have also rebased this PR onto most recent commit in `trunk`. [This comment](https://github.com/apache/kafka/pull/10280#discussion_r645441207) is pending and needs discussion. The PR is ready for review again.


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643498093



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1500,50 +1325,67 @@ class Log(@volatile private var _dir: File,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach)
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
       nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
     }
 
-    deleteOldSegments(shouldDelete, StartOffsetBreach)
+    deleteOldSegments(shouldDelete, StartOffsetBreach(this))
   }
 
   def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix)
 
   /**
    * The size of the log in bytes
    */
-  def size: Long = Log.sizeInBytes(logSegments)
+  def size: Long = localLog.segments.sizeInBytes
 
   /**
-   * The offset metadata of the next message that will be appended to the log
+   * The offset of the next message that will be appended to the log
    */
-  def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+  def logEndOffset: Long =  localLog.logEndOffset
 
   /**
-   * The offset of the next message that will be appended to the log
+   * The offset metadata of the next message that will be appended to the log
    */
-  def logEndOffset: Long = nextOffsetMetadata.messageOffset
+  def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata
+
+  private val rollAction = RollAction(
+    preRollAction = (newSegment: LogSegment) => {
+      // Take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
+      // offset align with the new segment offset since this ensures we can recover the segment by beginning
+      // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
+      // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
+      // we manually override the state offset here prior to taking the snapshot.
+      producerStateManager.updateMapEndOffset(newSegment.baseOffset)
+      producerStateManager.takeSnapshot()
+    },
+    postRollAction = (newSegment: LogSegment, deletedSegment: Option[LogSegment]) => {
+      deletedSegment.foreach(segment => deleteProducerSnapshotAsync(Seq(segment)))

Review comment:
       I've created a JIRA to track this: https://issues.apache.org/jira/browse/KAFKA-12876.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602056245



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -213,32 +208,38 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason {
 }
 
 /**
- * An append-only log for storing messages.
+ * A log which presents a unified view of local and tiered log segments.
  *
- * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * The log consists of tiered and local segments with the tiered portion of the log being optional. There could be an
+ * overlap between the tiered and local segments. The active segment is always guaranteed to be local. If tiered segments
+ * are present, they always appear at the head of the log, followed by an optional region of overlap, followed by the local

Review comment:
       Done. Good point. I've fixed the doc to refer to `beginning of log`.




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-857548583


   Thanks for the review @junrao! I've addressed the comments in 8ebb39f.


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643464231



##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {
+      log.close()
+    }
+    Utils.delete(tmpDir)
+  }
+
+  case class KeyValue(key: String, value: String) {
+    def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = {
+      new SimpleRecord(timestamp, key.getBytes, value.getBytes)
+    }
+  }
+
+  object KeyValue {
+    def fromRecord(record: Record): KeyValue = {
+      val key =
+        if (record.hasKey)
+          StandardCharsets.UTF_8.decode(record.key()).toString
+        else
+          ""
+      val value =
+        if (record.hasValue)
+          StandardCharsets.UTF_8.decode(record.value()).toString
+        else
+          ""
+      KeyValue(key, value)
+    }
+  }
+
+  private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = {
+    keyValues.map(kv => kv.toRecord())
+  }
+
+  private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = {
+    records.map(r => KeyValue.fromRecord(r))
+  }
+
+  private def appendRecords(records: Iterable[SimpleRecord],
+                            log: LocalLog = log,
+                            initialOffset: Long = 0L): Unit = {
+    log.append(lastOffset = initialOffset + records.size - 1,
+      largestTimestamp = records.head.timestamp,
+      shallowOffsetOfMaxTimestamp = initialOffset,
+      records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
+  }
+
+  private def readRecords(log: LocalLog = log,
+                          startOffset: Long = 0L,
+                          maxLength: => Int = log.segments.activeSegment.size,
+                          minOneMessage: Boolean = false,
+                          maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata,
+                          includeAbortedTxns: Boolean = false): FetchDataInfo = {
+    log.read(startOffset,
+             maxLength,
+             minOneMessage = minOneMessage,
+             maxOffsetMetadata,
+             includeAbortedTxns = includeAbortedTxns)
+  }
+
+  @Test
+  def testLogDeleteSuccess(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    assertFalse(logDir.listFiles.isEmpty)
+    val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values
+    val deletedSegments = log.delete()
+    assertTrue(log.segments.isEmpty)
+    assertEquals(segmentsBeforeDelete, deletedSegments)
+    assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed())
+    assertFalse(logDir.exists)
+  }
+
+  @Test
+  def testLogDeleteFailureAfterCloseHandlers(): Unit = {
+    log.closeHandlers()
+    assertEquals(1, log.segments.numberOfSegments)
+    val segmentsBeforeDelete = log.segments.values
+    assertThrows(classOf[KafkaStorageException], () => log.delete())
+    assertEquals(1, log.segments.numberOfSegments)
+    assertEquals(segmentsBeforeDelete, log.segments.values)
+    assertTrue(logDir.exists)
+  }
+
+  @Test
+  def testUpdateConfig(): Unit = {
+    val oldConfig = log.config
+    assertEquals(oldConfig, log.config)
+
+    val newConfig = createLogConfig()

Review comment:
       That's a good point. While it is still a different reference, we can still create a log config with an override, something like: `createLogConfig(segmentBytes=oldConfig.segmentSize * 2)`

##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {
+      log.close()
+    }
+    Utils.delete(tmpDir)
+  }
+
+  case class KeyValue(key: String, value: String) {
+    def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = {
+      new SimpleRecord(timestamp, key.getBytes, value.getBytes)
+    }
+  }
+
+  object KeyValue {
+    def fromRecord(record: Record): KeyValue = {
+      val key =
+        if (record.hasKey)
+          StandardCharsets.UTF_8.decode(record.key()).toString
+        else
+          ""
+      val value =
+        if (record.hasValue)
+          StandardCharsets.UTF_8.decode(record.value()).toString
+        else
+          ""
+      KeyValue(key, value)
+    }
+  }
+
+  private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = {
+    keyValues.map(kv => kv.toRecord())
+  }
+
+  private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = {
+    records.map(r => KeyValue.fromRecord(r))
+  }
+
+  private def appendRecords(records: Iterable[SimpleRecord],
+                            log: LocalLog = log,
+                            initialOffset: Long = 0L): Unit = {
+    log.append(lastOffset = initialOffset + records.size - 1,
+      largestTimestamp = records.head.timestamp,
+      shallowOffsetOfMaxTimestamp = initialOffset,
+      records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
+  }
+
+  private def readRecords(log: LocalLog = log,
+                          startOffset: Long = 0L,
+                          maxLength: => Int = log.segments.activeSegment.size,
+                          minOneMessage: Boolean = false,
+                          maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata,
+                          includeAbortedTxns: Boolean = false): FetchDataInfo = {
+    log.read(startOffset,
+             maxLength,
+             minOneMessage = minOneMessage,
+             maxOffsetMetadata,
+             includeAbortedTxns = includeAbortedTxns)
+  }
+
+  @Test
+  def testLogDeleteSuccess(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    assertFalse(logDir.listFiles.isEmpty)
+    val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values
+    val deletedSegments = log.delete()
+    assertTrue(log.segments.isEmpty)
+    assertEquals(segmentsBeforeDelete, deletedSegments)
+    assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed())
+    assertFalse(logDir.exists)
+  }
+
+  @Test
+  def testLogDeleteFailureAfterCloseHandlers(): Unit = {
+    log.closeHandlers()
+    assertEquals(1, log.segments.numberOfSegments)
+    val segmentsBeforeDelete = log.segments.values
+    assertThrows(classOf[KafkaStorageException], () => log.delete())
+    assertEquals(1, log.segments.numberOfSegments)
+    assertEquals(segmentsBeforeDelete, log.segments.values)
+    assertTrue(logDir.exists)
+  }
+
+  @Test
+  def testUpdateConfig(): Unit = {
+    val oldConfig = log.config
+    assertEquals(oldConfig, log.config)
+
+    val newConfig = createLogConfig()

Review comment:
       That's a good point. While it is still a different reference, we can still create a new log config with an override, something like: `createLogConfig(segmentBytes=oldConfig.segmentSize * 2)`




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-880044495


   @ijuma Discussed with @satishd. We are not planning to include this in 3.0.


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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643446123



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1572,144 +1414,69 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      localLog.roll(Some(rollOffset), Some(rollAction))
     } else {
       segment
     }
   }
 
   /**
-   * Roll the log over to a new active segment starting with the current logEndOffset.
+   * Roll the local 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.
    *
    * @return The newly rolled segment
    */
   def roll(expectedNextOffset: Option[Long] = None): LogSegment = {
-    maybeHandleIOException(s"Error while rolling log segment for $topicPartition in dir ${dir.getParent}") {
-      val start = time.hiResClockMs()
-      lock synchronized {
-        checkIfMemoryMappedBufferClosed()
-        val newOffset = math.max(expectedNextOffset.getOrElse(0L), logEndOffset)
-        val logFile = Log.logFile(dir, newOffset)
-
-        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).
-            warn(s"Trying to roll a new log segment with start offset $newOffset " +
-                 s"=max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already " +
-                 s"exists and is active with size 0. Size of time index: ${activeSegment.timeIndex.entries}," +
-                 s" size of offset index: ${activeSegment.offsetIndex.entries}.")
-            removeAndDeleteSegments(Seq(activeSegment), asyncDelete = true, LogRoll)
-          } else {
-            throw new KafkaException(s"Trying to roll a new log segment for topic partition $topicPartition with start offset $newOffset" +
-                                     s" =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already exists. Existing " +
-                                     s"segment is ${segments.get(newOffset)}.")
-          }
-        } else if (!segments.isEmpty && newOffset < activeSegment.baseOffset) {
-          throw new KafkaException(
-            s"Trying to roll a new log segment for topic partition $topicPartition with " +
-            s"start offset $newOffset =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) lower than start offset of the active segment $activeSegment")
-        } else {
-          val offsetIdxFile = offsetIndexFile(dir, newOffset)
-          val timeIdxFile = timeIndexFile(dir, newOffset)
-          val txnIdxFile = transactionIndexFile(dir, newOffset)
-
-          for (file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) {
-            warn(s"Newly rolled segment file ${file.getAbsolutePath} already exists; deleting it first")
-            Files.delete(file.toPath)
-          }
-
-          segments.lastSegment.foreach(_.onBecomeInactiveSegment())
-        }
-
-        // take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
-        // offset align with the new segment offset since this ensures we can recover the segment by beginning
-        // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
-        // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
-        // we manually override the state offset here prior to taking the snapshot.
-        producerStateManager.updateMapEndOffset(newOffset)
-        producerStateManager.takeSnapshot()
-
-        val segment = LogSegment.open(dir,
-          baseOffset = newOffset,
-          config,
-          time = time,
-          initFileSize = config.initFileSize,
-          preallocate = config.preallocate)
-        addSegment(segment)
-
-        // 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)
-
-        // schedule an asynchronous flush of the old segment
-        scheduler.schedule("flush-log", () => flush(newOffset), delay = 0L)
-
-        info(s"Rolled new log segment at offset $newOffset in ${time.hiResClockMs() - start} ms.")
-
-        segment
-      }
+    lock synchronized {
+      localLog.roll(expectedNextOffset, Some(rollAction))
     }
   }
 
   /**
    * The number of messages appended to the log since the last flush
    */
-  private def unflushedMessages: Long = this.logEndOffset - this.recoveryPoint
+  private def unflushedMessages: Long = logEndOffset - localLog.recoveryPoint
 
   /**
-   * Flush all log segments
+   * Flush all local log segments
    */
   def flush(): Unit = flush(this.logEndOffset)
 
   /**
-   * Flush log segments for all offsets up to offset-1
+   * Flush local log segments for all offsets up to offset-1
    *
    * @param offset The offset to flush up to (non-inclusive); the new recovery point
    */
   def flush(offset: Long): Unit = {
     maybeHandleIOException(s"Error while flushing log for $topicPartition in dir ${dir.getParent} with offset $offset") {
-      if (offset > this.recoveryPoint) {
+      if (offset > localLog.recoveryPoint) {
         debug(s"Flushing log up to offset $offset, last flushed: $lastFlushTime,  current time: ${time.milliseconds()}, " +
           s"unflushed: $unflushedMessages")
-        val segments = logSegments(this.recoveryPoint, offset)
-        segments.foreach(_.flush())
-        // if there are any new segments, we need to flush the parent directory for crash consistency
-        segments.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath))
-
+        localLog.flush(offset)
         lock synchronized {
-          checkIfMemoryMappedBufferClosed()
-          if (offset > this.recoveryPoint) {
-            this.recoveryPoint = offset
-            lastFlushedTime.set(time.milliseconds)
-          }
+          localLog.markFlushed(offset)
         }
       }
     }
   }
 
   /**
-   * Completely delete this log directory and all contents from the file system with no delay
+   * Completely delete the local log directory and all contents from the file system with no delay
    */
   private[log] def delete(): Unit = {
     maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
       lock synchronized {
-        checkIfMemoryMappedBufferClosed()
         producerExpireCheck.cancel(true)
-        removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
         leaderEpochCache.foreach(_.clear())
-        Utils.delete(dir)
-        // File handlers will be closed if this log is deleted
-        isMemoryMappedBufferClosed = true
+        val deletedSegments = localLog.delete()
+        deleteProducerSnapshotAsync(deletedSegments)

Review comment:
       That's a good point. The best solution I could think of is to split the functionality up into few different APIs in `LocalLog`. For example, this is how the right implementation could look like in `Log.scala`:
   
   ```
   // In `Log.scala`:
   private[log] def delete(): Unit = {
     maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
       lock synchronized {
         producerExpireCheck.cancel(true)
         leaderEpochCache.foreach(_.clear())
         val deletedSegments = localLog.deleteAllSegments()
         deleteProducerSnapshotAsync(deletedSegments)
         localLog.deleteDir()
       }
     }
   }
     
   ```




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602049695



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {

Review comment:
       Done. Good point. I've fixed it now.




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-794226594


   @ijuma The purpose of `GlobalLog` class is to serve as a higher layer, stitching together the unified view of both the local and remote portion of the log. Importantly, this class is aware of the global log start offset. 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. It could just be called as `Log` too, but I thought by calling it `GlobalLog` the intention/differentiation is clear.
   
   The above and few more things are explained in the doc attached in the description. I'd suggest having a look at the doc.


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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-860537313


   @junrao Thanks for the review! I ran the system tests.
   1. [System test run #4560](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4560/) on top of the latest commit 008b701386ce5a4d892d6ac5b90798b981c4fba0 from this PR. The run finished with 12 test failures.
   2. [System test run #4561](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4561/) against AK trunk on top of commit 6de37e536ac76ef13530d49dc7320110332cd1ee which does not contain changes from this PR. The run finished with 13 test failures.
   
   There were 11 overlapping failures in both (1) and (2). For these, I didn't find anything abnormal in the logs so far, the failure reason seems similar in both.
   
   The only new failure in (1) that's not present in (2) was:
   
   ```
   Module: kafkatest.tests.client.consumer_test
   Class:  OffsetValidationTest
   Method: test_broker_failure
   Arguments:
   {
     "clean_shutdown": true,
     "enable_autocommit": false,
     "metadata_quorum": "REMOTE_KRAFT"
   }
   ```
   
   Logs indicate that the test failed [at this line](https://github.com/apache/kafka/blob/b96fc7892f1e885239d3290cf509e1d1bb41e7db/tests/kafkatest/tests/client/consumer_test.py#L388) because one of the worker nodes running the consumer didn't complete within the timeout of 30s. This doesn't seem indicative of a real failure (yet), so I'm rerunning the system tests again in [test run #4562](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4561/) to check if the failure is consistent.
   
   


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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-836254631


   @junrao This PR is ready for another round of review. I've rebased the PR onto latest AK trunk, iterated on the implementation bit more and added new unit tests for `LocalLog` class under `LocalLogTest.scala`.
   
   cc @dhruvilshah3 


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

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



[GitHub] [kafka] junrao commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r638367249



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1500,50 +1325,67 @@ class Log(@volatile private var _dir: File,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach)
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
       nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
     }
 
-    deleteOldSegments(shouldDelete, StartOffsetBreach)
+    deleteOldSegments(shouldDelete, StartOffsetBreach(this))
   }
 
   def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix)
 
   /**
    * The size of the log in bytes
    */
-  def size: Long = Log.sizeInBytes(logSegments)
+  def size: Long = localLog.segments.sizeInBytes
 
   /**
-   * The offset metadata of the next message that will be appended to the log
+   * The offset of the next message that will be appended to the log
    */
-  def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+  def logEndOffset: Long =  localLog.logEndOffset
 
   /**
-   * The offset of the next message that will be appended to the log
+   * The offset metadata of the next message that will be appended to the log
    */
-  def logEndOffset: Long = nextOffsetMetadata.messageOffset
+  def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata
+
+  private val rollAction = RollAction(

Review comment:
       I feel RollAction actually makes the code harder to understand than before. So, it would be useful to think through if we could avoid it. In particular, it seems that anything in postRollAction could just be done in the caller if we return enough context. We are taking a producer snapshot in preRollAction. However, since we are not adding new data here. It seems that we could take producer snapshot in Log.roll() after calling localLog.roll() while holding the Log.lock.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1500,50 +1325,67 @@ class Log(@volatile private var _dir: File,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach)
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
       nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
     }
 
-    deleteOldSegments(shouldDelete, StartOffsetBreach)
+    deleteOldSegments(shouldDelete, StartOffsetBreach(this))
   }
 
   def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix)
 
   /**
    * The size of the log in bytes
    */
-  def size: Long = Log.sizeInBytes(logSegments)
+  def size: Long = localLog.segments.sizeInBytes
 
   /**
-   * The offset metadata of the next message that will be appended to the log
+   * The offset of the next message that will be appended to the log
    */
-  def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+  def logEndOffset: Long =  localLog.logEndOffset
 
   /**
-   * The offset of the next message that will be appended to the log
+   * The offset metadata of the next message that will be appended to the log
    */
-  def logEndOffset: Long = nextOffsetMetadata.messageOffset
+  def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata
+
+  private val rollAction = RollAction(
+    preRollAction = (newSegment: LogSegment) => {
+      // Take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
+      // offset align with the new segment offset since this ensures we can recover the segment by beginning
+      // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
+      // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
+      // we manually override the state offset here prior to taking the snapshot.
+      producerStateManager.updateMapEndOffset(newSegment.baseOffset)
+      producerStateManager.takeSnapshot()
+    },
+    postRollAction = (newSegment: LogSegment, deletedSegment: Option[LogSegment]) => {
+      deletedSegment.foreach(segment => deleteProducerSnapshotAsync(Seq(segment)))

Review comment:
       This seems to have exposed an existing bug. During roll, deletedSegment will be non-empty if there is an existing segment of 0 size with the newOffsetToRoll. However, since we take a producer snapshot on newOffsetToRoll before calling postRollAction, we will be deleting the same snapshot we just created.
   
   In this case, I think we don't need to delete producerSnapshot for deletedSegment.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1572,144 +1414,69 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      localLog.roll(Some(rollOffset), Some(rollAction))
     } else {
       segment
     }
   }
 
   /**
-   * Roll the log over to a new active segment starting with the current logEndOffset.
+   * Roll the local 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.
    *
    * @return The newly rolled segment
    */
   def roll(expectedNextOffset: Option[Long] = None): LogSegment = {
-    maybeHandleIOException(s"Error while rolling log segment for $topicPartition in dir ${dir.getParent}") {
-      val start = time.hiResClockMs()
-      lock synchronized {
-        checkIfMemoryMappedBufferClosed()
-        val newOffset = math.max(expectedNextOffset.getOrElse(0L), logEndOffset)
-        val logFile = Log.logFile(dir, newOffset)
-
-        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).
-            warn(s"Trying to roll a new log segment with start offset $newOffset " +
-                 s"=max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already " +
-                 s"exists and is active with size 0. Size of time index: ${activeSegment.timeIndex.entries}," +
-                 s" size of offset index: ${activeSegment.offsetIndex.entries}.")
-            removeAndDeleteSegments(Seq(activeSegment), asyncDelete = true, LogRoll)
-          } else {
-            throw new KafkaException(s"Trying to roll a new log segment for topic partition $topicPartition with start offset $newOffset" +
-                                     s" =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already exists. Existing " +
-                                     s"segment is ${segments.get(newOffset)}.")
-          }
-        } else if (!segments.isEmpty && newOffset < activeSegment.baseOffset) {
-          throw new KafkaException(
-            s"Trying to roll a new log segment for topic partition $topicPartition with " +
-            s"start offset $newOffset =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) lower than start offset of the active segment $activeSegment")
-        } else {
-          val offsetIdxFile = offsetIndexFile(dir, newOffset)
-          val timeIdxFile = timeIndexFile(dir, newOffset)
-          val txnIdxFile = transactionIndexFile(dir, newOffset)
-
-          for (file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) {
-            warn(s"Newly rolled segment file ${file.getAbsolutePath} already exists; deleting it first")
-            Files.delete(file.toPath)
-          }
-
-          segments.lastSegment.foreach(_.onBecomeInactiveSegment())
-        }
-
-        // take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
-        // offset align with the new segment offset since this ensures we can recover the segment by beginning
-        // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
-        // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
-        // we manually override the state offset here prior to taking the snapshot.
-        producerStateManager.updateMapEndOffset(newOffset)
-        producerStateManager.takeSnapshot()
-
-        val segment = LogSegment.open(dir,
-          baseOffset = newOffset,
-          config,
-          time = time,
-          initFileSize = config.initFileSize,
-          preallocate = config.preallocate)
-        addSegment(segment)
-
-        // 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)
-
-        // schedule an asynchronous flush of the old segment
-        scheduler.schedule("flush-log", () => flush(newOffset), delay = 0L)
-
-        info(s"Rolled new log segment at offset $newOffset in ${time.hiResClockMs() - start} ms.")
-
-        segment
-      }
+    lock synchronized {
+      localLog.roll(expectedNextOffset, Some(rollAction))
     }
   }
 
   /**
    * The number of messages appended to the log since the last flush
    */
-  private def unflushedMessages: Long = this.logEndOffset - this.recoveryPoint
+  private def unflushedMessages: Long = logEndOffset - localLog.recoveryPoint
 
   /**
-   * Flush all log segments
+   * Flush all local log segments
    */
   def flush(): Unit = flush(this.logEndOffset)
 
   /**
-   * Flush log segments for all offsets up to offset-1
+   * Flush local log segments for all offsets up to offset-1
    *
    * @param offset The offset to flush up to (non-inclusive); the new recovery point
    */
   def flush(offset: Long): Unit = {
     maybeHandleIOException(s"Error while flushing log for $topicPartition in dir ${dir.getParent} with offset $offset") {
-      if (offset > this.recoveryPoint) {
+      if (offset > localLog.recoveryPoint) {
         debug(s"Flushing log up to offset $offset, last flushed: $lastFlushTime,  current time: ${time.milliseconds()}, " +
           s"unflushed: $unflushedMessages")
-        val segments = logSegments(this.recoveryPoint, offset)
-        segments.foreach(_.flush())
-        // if there are any new segments, we need to flush the parent directory for crash consistency
-        segments.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath))
-
+        localLog.flush(offset)
         lock synchronized {
-          checkIfMemoryMappedBufferClosed()
-          if (offset > this.recoveryPoint) {
-            this.recoveryPoint = offset
-            lastFlushedTime.set(time.milliseconds)
-          }
+          localLog.markFlushed(offset)
         }
       }
     }
   }
 
   /**
-   * Completely delete this log directory and all contents from the file system with no delay
+   * Completely delete the local log directory and all contents from the file system with no delay
    */
   private[log] def delete(): Unit = {
     maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
       lock synchronized {
-        checkIfMemoryMappedBufferClosed()
         producerExpireCheck.cancel(true)
-        removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
         leaderEpochCache.foreach(_.clear())
-        Utils.delete(dir)
-        // File handlers will be closed if this log is deleted
-        isMemoryMappedBufferClosed = true
+        val deletedSegments = localLog.delete()
+        deleteProducerSnapshotAsync(deletedSegments)

Review comment:
       Hmm, the ordering is a bit weird. We delete the directory in localLog.delete(). However, the producer snapshot is in the directory and is deleted later.

##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {
+      log.close()
+    }
+    Utils.delete(tmpDir)
+  }
+
+  case class KeyValue(key: String, value: String) {
+    def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = {
+      new SimpleRecord(timestamp, key.getBytes, value.getBytes)
+    }
+  }
+
+  object KeyValue {
+    def fromRecord(record: Record): KeyValue = {
+      val key =
+        if (record.hasKey)
+          StandardCharsets.UTF_8.decode(record.key()).toString
+        else
+          ""
+      val value =
+        if (record.hasValue)
+          StandardCharsets.UTF_8.decode(record.value()).toString
+        else
+          ""
+      KeyValue(key, value)
+    }
+  }
+
+  private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = {
+    keyValues.map(kv => kv.toRecord())
+  }
+
+  private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = {
+    records.map(r => KeyValue.fromRecord(r))
+  }
+
+  private def appendRecords(records: Iterable[SimpleRecord],
+                            log: LocalLog = log,
+                            initialOffset: Long = 0L): Unit = {
+    log.append(lastOffset = initialOffset + records.size - 1,
+      largestTimestamp = records.head.timestamp,
+      shallowOffsetOfMaxTimestamp = initialOffset,
+      records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
+  }
+
+  private def readRecords(log: LocalLog = log,
+                          startOffset: Long = 0L,
+                          maxLength: => Int = log.segments.activeSegment.size,
+                          minOneMessage: Boolean = false,
+                          maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata,
+                          includeAbortedTxns: Boolean = false): FetchDataInfo = {
+    log.read(startOffset,
+             maxLength,
+             minOneMessage = minOneMessage,
+             maxOffsetMetadata,
+             includeAbortedTxns = includeAbortedTxns)
+  }
+
+  @Test
+  def testLogDeleteSuccess(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    assertFalse(logDir.listFiles.isEmpty)
+    val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values
+    val deletedSegments = log.delete()
+    assertTrue(log.segments.isEmpty)
+    assertEquals(segmentsBeforeDelete, deletedSegments)
+    assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed())
+    assertFalse(logDir.exists)
+  }
+
+  @Test
+  def testLogDeleteFailureAfterCloseHandlers(): Unit = {
+    log.closeHandlers()
+    assertEquals(1, log.segments.numberOfSegments)
+    val segmentsBeforeDelete = log.segments.values
+    assertThrows(classOf[KafkaStorageException], () => log.delete())
+    assertEquals(1, log.segments.numberOfSegments)
+    assertEquals(segmentsBeforeDelete, log.segments.values)
+    assertTrue(logDir.exists)
+  }
+
+  @Test
+  def testUpdateConfig(): Unit = {
+    val oldConfig = log.config
+    assertEquals(oldConfig, log.config)
+
+    val newConfig = createLogConfig()

Review comment:
       It seems newConfig is always the same as oldConfig?

##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {
+      log.close()
+    }
+    Utils.delete(tmpDir)
+  }
+
+  case class KeyValue(key: String, value: String) {
+    def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = {
+      new SimpleRecord(timestamp, key.getBytes, value.getBytes)
+    }
+  }
+
+  object KeyValue {
+    def fromRecord(record: Record): KeyValue = {
+      val key =
+        if (record.hasKey)
+          StandardCharsets.UTF_8.decode(record.key()).toString
+        else
+          ""
+      val value =
+        if (record.hasValue)
+          StandardCharsets.UTF_8.decode(record.value()).toString
+        else
+          ""
+      KeyValue(key, value)
+    }
+  }
+
+  private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = {
+    keyValues.map(kv => kv.toRecord())
+  }
+
+  private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = {
+    records.map(r => KeyValue.fromRecord(r))
+  }
+
+  private def appendRecords(records: Iterable[SimpleRecord],
+                            log: LocalLog = log,
+                            initialOffset: Long = 0L): Unit = {
+    log.append(lastOffset = initialOffset + records.size - 1,
+      largestTimestamp = records.head.timestamp,
+      shallowOffsetOfMaxTimestamp = initialOffset,
+      records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
+  }
+
+  private def readRecords(log: LocalLog = log,
+                          startOffset: Long = 0L,
+                          maxLength: => Int = log.segments.activeSegment.size,
+                          minOneMessage: Boolean = false,
+                          maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata,
+                          includeAbortedTxns: Boolean = false): FetchDataInfo = {
+    log.read(startOffset,
+             maxLength,
+             minOneMessage = minOneMessage,
+             maxOffsetMetadata,
+             includeAbortedTxns = includeAbortedTxns)
+  }
+
+  @Test
+  def testLogDeleteSuccess(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    assertFalse(logDir.listFiles.isEmpty)
+    val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values
+    val deletedSegments = log.delete()
+    assertTrue(log.segments.isEmpty)
+    assertEquals(segmentsBeforeDelete, deletedSegments)
+    assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed())
+    assertFalse(logDir.exists)
+  }
+
+  @Test
+  def testLogDeleteFailureAfterCloseHandlers(): Unit = {
+    log.closeHandlers()
+    assertEquals(1, log.segments.numberOfSegments)
+    val segmentsBeforeDelete = log.segments.values
+    assertThrows(classOf[KafkaStorageException], () => log.delete())
+    assertEquals(1, log.segments.numberOfSegments)
+    assertEquals(segmentsBeforeDelete, log.segments.values)
+    assertTrue(logDir.exists)
+  }
+
+  @Test
+  def testUpdateConfig(): Unit = {
+    val oldConfig = log.config
+    assertEquals(oldConfig, log.config)
+
+    val newConfig = createLogConfig()
+    log.updateConfig(newConfig)
+    assertEquals(newConfig, log.config)
+  }
+
+  @Test
+  def testLogDirRenameToNewDir(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    val newLogDir = TestUtils.randomPartitionLogDir(tmpDir)
+    assertTrue(log.renameDir(newLogDir.getName))
+    assertFalse(logDir.exists())
+    assertTrue(newLogDir.exists())
+    assertEquals(newLogDir, log.dir)
+    assertEquals(newLogDir.getParent, log.parentDir)
+    assertEquals(newLogDir.getParent, log.dir.getParent)
+    log.segments.values.foreach(segment => assertEquals(newLogDir.getPath, segment.log.file().getParentFile.getPath))
+    assertEquals(2, log.segments.numberOfSegments)
+  }
+
+  @Test
+  def testLogDirRenameToExistingDir(): Unit = {
+    assertFalse(log.renameDir(log.dir.getName))
+  }
+
+  @Test
+  def testLogFlush(): Unit = {
+    assertEquals(0L, log.recoveryPoint)
+    assertEquals(mockTime.milliseconds, log.lastFlushTime)
+
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    mockTime.sleep(1)
+    val newSegment = log.roll()
+    log.flush(newSegment.baseOffset)
+    log.markFlushed(newSegment.baseOffset)
+    assertEquals(1L, log.recoveryPoint)
+    assertEquals(mockTime.milliseconds, log.lastFlushTime)
+  }
+
+  @Test
+  def testLogAppend(): Unit = {
+    val fetchDataInfoBeforeAppend = readRecords(maxLength = 1)
+    assertTrue(fetchDataInfoBeforeAppend.records.records.asScala.isEmpty)
+
+    mockTime.sleep(1)
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    assertEquals(2L, log.logEndOffset)
+    assertEquals(0L, log.recoveryPoint)
+    val fetchDataInfo = readRecords()
+    assertEquals(2L, fetchDataInfo.records.records.asScala.size)
+    assertEquals(keyValues, recordsToKvs(fetchDataInfo.records.records.asScala))
+  }
+
+  @Test
+  def testLogCloseSuccess(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.close()
+    assertThrows(classOf[ClosedChannelException], () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L))
+  }
+
+  @Test
+  def testLogCloseIdempotent(): Unit = {
+    log.close()
+    // Check that LocalLog.close() is idempotent
+    log.close()
+  }
+
+  @Test
+  def testLogCloseFailureWhenInMemoryBufferClosed(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.closeHandlers()
+    assertThrows(classOf[KafkaStorageException], () => log.close())
+  }
+
+  @Test
+  def testLogCloseHandlers(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.closeHandlers()
+    assertThrows(classOf[ClosedChannelException],
+                 () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L))
+  }
+
+  @Test
+  def testLogCloseHandlersIdempotent(): Unit = {
+    log.closeHandlers()
+    // Check that LocalLog.closeHandlers() is idempotent
+    log.closeHandlers()
+  }
+
+  private def testRemoveAndDeleteSegments(asyncDelete: Boolean): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    class TestDeletionReason extends SegmentDeletionReason {
+      private var _deletedSegments: Iterable[LogSegment] = List[LogSegment]()
+
+      override def logReason(toDelete: List[LogSegment]): Unit = {
+        _deletedSegments = List[LogSegment]() ++ toDelete
+      }
+
+      def deletedSegments: Iterable[LogSegment] = _deletedSegments
+    }
+    val reason = new TestDeletionReason()
+    val toDelete = List[LogSegment]() ++ log.segments.values
+    log.removeAndDeleteSegments(toDelete, asyncDelete = asyncDelete, reason)
+    if (asyncDelete) {
+      mockTime.sleep(log.config.fileDeleteDelayMs + 1)
+    }
+    assertTrue(log.segments.isEmpty)
+    assertEquals(toDelete, reason.deletedSegments)
+    toDelete.foreach(segment => assertTrue(segment.deleted()))
+  }
+
+  @Test
+  def testRemoveAndDeleteSegmentsSync(): Unit = {
+    testRemoveAndDeleteSegments(asyncDelete = false)
+  }
+
+  @Test
+  def testRemoveAndDeleteSegmentsAsync(): Unit = {
+    testRemoveAndDeleteSegments(asyncDelete = true)
+  }
+
+  private def testDeleteSegmentFiles(asyncDelete: Boolean): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    val toDelete = List[LogSegment]() ++ log.segments.values
+    LocalLog.deleteSegmentFiles(toDelete, asyncDelete = asyncDelete, log.dir, log.topicPartition, log.config, log.scheduler, log.logDirFailureChannel, "")
+    if (asyncDelete) {
+      toDelete.foreach {
+        segment =>
+          assertFalse(segment.deleted())
+          assertTrue(segment.hasSuffix(LocalLog.DeletedFileSuffix))
+      }
+      mockTime.sleep(log.config.fileDeleteDelayMs + 1)
+    }
+    toDelete.foreach(segment => assertTrue(segment.deleted()))
+  }
+
+  @Test
+  def testDeleteSegmentFilesSync(): Unit = {
+    testDeleteSegmentFiles(asyncDelete = false)
+  }
+
+  @Test
+  def testDeleteSegmentFilesAsync(): Unit = {
+    testDeleteSegmentFiles(asyncDelete = true)
+  }
+
+  @Test
+  def testDeletableSegmentsFilter(): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    {
+      val deletable = log.deletableSegments(
+        (segment: LogSegment, _: Option[LogSegment], _: Long) => segment.baseOffset <= 5)
+      val expected = log.segments.nonActiveLogSegmentsFrom(0L).filter(segment => segment.baseOffset <= 5)
+      assertEquals(expected, deletable.toList)
+    }
+
+    {
+      val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true)
+      val expected = log.segments.nonActiveLogSegmentsFrom(0L).toList
+      assertEquals(expected, deletable.toList)
+    }
+
+    {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = 9L)
+      val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true)
+      val expected = log.segments.values.toList
+      assertEquals(expected, deletable.toList)
+    }
+  }
+
+  @Test
+  def testDeletableSegmentsIteration(): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    var offset = 0
+    log.deletableSegments(
+      (segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long) => {
+        assertEquals(offset, segment.baseOffset)
+        val floorSegmentOpt = log.segments.floorSegment(offset)
+        assertTrue(floorSegmentOpt.isDefined)
+        assertEquals(floorSegmentOpt.get, segment)
+        if (offset == log.logEndOffset) {
+          assertFalse(nextSegmentOpt.isDefined)
+        } else {
+          assertTrue(nextSegmentOpt.isDefined)
+          val higherSegmentOpt = log.segments.higherSegment(segment.baseOffset)
+          assertTrue(higherSegmentOpt.isDefined)
+          assertEquals(segment.baseOffset + 1, higherSegmentOpt.get.baseOffset)
+          assertEquals(higherSegmentOpt.get, nextSegmentOpt.get)
+        }
+        assertEquals(log.logEndOffset, logEndOffset)
+        offset += 1
+        true
+      })

Review comment:
       Should we assert sth after the log.deletableSegments() call?

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1852,65 +1612,24 @@ class Log(@volatile private var _dir: File,
     logString.toString
   }
 
-  /**
-   * This method deletes the given log segments by doing the following for each of them:
-   * <ol>
-   *   <li>It removes the segment from the segment map so that it will no longer be used for reads.
-   *   <li>It renames the index and log files by appending .deleted to the respective file name
-   *   <li>It can either schedule an asynchronous delete operation to occur in the future or perform the deletion synchronously
-   * </ol>
-   * Asynchronous deletion allows reads to happen concurrently without synchronization and without the possibility of
-   * physically deleting a file while it is being read.
-   *
-   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
-   * or the immediate caller will catch and handle IOException
-   *
-   * @param segments The log segments to schedule for deletion
-   * @param asyncDelete Whether the segment files should be deleted asynchronously
-   */
-  private def removeAndDeleteSegments(segments: Iterable[LogSegment],
-                                      asyncDelete: Boolean,
-                                      reason: SegmentDeletionReason): Unit = {
-    if (segments.nonEmpty) {
-      lock synchronized {
-        // As 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.
-        val toDelete = segments.toList
-        reason.logReason(this, toDelete)
-        toDelete.foreach { segment =>
-          this.segments.remove(segment.baseOffset)
-        }
-        deleteSegmentFiles(toDelete, asyncDelete)
-      }
-    }
-  }
-
-  private def deleteSegmentFiles(segments: Iterable[LogSegment], asyncDelete: Boolean, deleteProducerStateSnapshots: Boolean = true): Unit = {
-    Log.deleteSegmentFiles(segments, asyncDelete, deleteProducerStateSnapshots, dir, topicPartition,
-      config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent)
-  }
-
   private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = {

Review comment:
       It seems that we could remove isRecoveredSwapFile since it's always false from the caller.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1806,37 +1566,37 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
-    rebuildProducerState(endOffset, producerStateManager)
+    lock synchronized {
+      rebuildProducerState(endOffset, producerStateManager)
+    }

Review comment:
       This change has a couple of issues. 
   (1) updateHighWatermark() now only updates the offset, but not the corresponding offset metadata. The offset metadata is needed in serving fetch requests. Recomputing that requires index lookup and log scan, and can be extensive. So, we need to preserve the offset metadata during truncate() and truncateFully().
   (2) I think updateHighWatermark() needs to be called within the lock. updateHighWatermark() reads local log's logEndOffset. So, we don't want the logEndOffset to change while updateHighWatermark() is called.

##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -246,17 +262,17 @@ object LogLoader extends Logging {
         return fn
       } catch {
         case e: LogSegmentOffsetOverflowException =>
-          info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
-          Log.splitOverflowedSegment(
+          info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          val result = Log.splitOverflowedSegment(
             e.segment,
             params.segments,
             params.dir,
             params.topicPartition,
             params.config,
             params.scheduler,
             params.logDirFailureChannel,
-            params.producerStateManager,
             params.logIdentifier)
+          deleteProducerSnapshotsAsync(result.deletedSegments, params)

Review comment:
       This is unnecessary since during splitting, the old segment is replaced with a new segment with the same base offset. So, result.deletedSegments is always empty.

##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {

Review comment:
       Should we use log.checkIfMemoryMappedBufferClosed()?




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648112486



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -246,17 +262,17 @@ object LogLoader extends Logging {
         return fn
       } catch {
         case e: LogSegmentOffsetOverflowException =>
-          info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
-          Log.splitOverflowedSegment(
+          info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          val result = Log.splitOverflowedSegment(
             e.segment,
             params.segments,
             params.dir,
             params.topicPartition,
             params.config,
             params.scheduler,
             params.logDirFailureChannel,
-            params.producerStateManager,
             params.logIdentifier)
+          deleteProducerSnapshotsAsync(result.deletedSegments, params)

Review comment:
       I have created a jira to track this improvement. https://issues.apache.org/jira/browse/KAFKA-12923




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-878641126


   @junrao Thanks for the review. I ran load tests on the changes from this PR, there weren't any new regressions (i.e. latency regressions or errors) that I noticed, except for an issue that I found which looks unrelated to this PR, its described in this jira: https://issues.apache.org/jira/browse/KAFKA-13070.
   
   The load test was run on a 6-broker cluster with 250GB SSD disks:
    * Produce consume on a test topic 2000 partitions (~1000+ replica count per broker).
    * Per topic # of producers = 6.
    * Produce ingress per broker = ~20.5MBps.
    * Per topic # of consumers = 6.
    * \# of consumer groups = 3.
    * Test duration: ~1h.
   
   Mid-way through the test, I rolled the cluster under load to check how the cluster behaved. Overall things looked OK.


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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602062784



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1816,8 +1292,12 @@ class Log(@volatile private var _dir: File,
    */
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
+    def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long): Boolean = {
+      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(logEndOffset) &&

Review comment:
       This is to accomodate for the hwm check that was previously happening in `Log#deletableSegments` in this [line](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/log/Log.scala#L1872). The `deletableSegments` method has now moved to `LocalLog`, but we can't do the hwm check inside `LocalLog` since hwm is still owned by `Log`. So we piggyback on the predicate here to additionally attach the hwm check.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602041672



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {

Review comment:
       Done. I've moved it into`close()` now. Good point.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648132657



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -246,17 +262,17 @@ object LogLoader extends Logging {
         return fn
       } catch {
         case e: LogSegmentOffsetOverflowException =>
-          info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
-          Log.splitOverflowedSegment(
+          info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          val result = Log.splitOverflowedSegment(
             e.segment,
             params.segments,
             params.dir,
             params.topicPartition,
             params.config,
             params.scheduler,
             params.logDirFailureChannel,
-            params.producerStateManager,
             params.logIdentifier)
+          deleteProducerSnapshotsAsync(result.deletedSegments, params)

Review comment:
       I've created a jira to track this. https://issues.apache.org/jira/browse/KAFKA-12923




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602040784



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.

Review comment:
       Done.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643450237



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -246,17 +262,17 @@ object LogLoader extends Logging {
         return fn
       } catch {
         case e: LogSegmentOffsetOverflowException =>
-          info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
-          Log.splitOverflowedSegment(
+          info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          val result = Log.splitOverflowedSegment(
             e.segment,
             params.segments,
             params.dir,
             params.topicPartition,
             params.config,
             params.scheduler,
             params.logDirFailureChannel,
-            params.producerStateManager,
             params.logIdentifier)
+          deleteProducerSnapshotsAsync(result.deletedSegments, params)

Review comment:
       Sounds good. Great catch. It appears straightforward to just skip deleting the snapshot here, I can leave a comment explaining why.




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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-808514958


   @junrao Just a heads up on the following. I'm working on the changes for the following in separate PRs, these are related with refactoring the recovery logic (KAFKA-12553):
    * KAFKA-12552 (https://github.com/apache/kafka/pull/10401) to extract segments map **[MERGED]**
    * KAFKA-12571: (https://github.com/apache/kafka/pull/10426) to eliminate LeaderEpochFileCache constructor dependency on logEndOffset **[MERGED]**
    * KAFKA-12575: (https://github.com/apache/kafka/pull/10430) to eliminate Log.isLogDirOffline boolean attribute **[MERGED]**
    * KAFKA-12553: (https://github.com/apache/kafka/pull/10478) Refactor recovery logic to introduce LogLoader  **[MERGED]**
   
   It seems better if we merge those into trunk ahead of the current PR.


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602049062



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],

Review comment:
       Done. Good point. I've fixed it now.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {

Review comment:
       Done. Good point. I've fixed it now.




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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-808514958


   @junrao Just a heads up on the following. I'm working on the changes for the following in separate PRs, these are related with refactoring the recovery logic (KAFKA-12553):
    * KAFKA-12552 (https://github.com/apache/kafka/pull/10401) to extract segments map **[MERGED]**
    * KAFKA-12571: (https://github.com/apache/kafka/pull/10426) to eliminate LeaderEpochFileCache constructor dependency on logEndOffset
    * KAFKA-12575: (https://github.com/apache/kafka/pull/10430) to eliminate Log.isLogDirOffline boolean attribute
   
   It seems better if we merge those into trunk ahead of the current PR.


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r645437282



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1500,50 +1325,67 @@ class Log(@volatile private var _dir: File,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach)
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
       nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
     }
 
-    deleteOldSegments(shouldDelete, StartOffsetBreach)
+    deleteOldSegments(shouldDelete, StartOffsetBreach(this))
   }
 
   def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix)
 
   /**
    * The size of the log in bytes
    */
-  def size: Long = Log.sizeInBytes(logSegments)
+  def size: Long = localLog.segments.sizeInBytes
 
   /**
-   * The offset metadata of the next message that will be appended to the log
+   * The offset of the next message that will be appended to the log
    */
-  def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+  def logEndOffset: Long =  localLog.logEndOffset
 
   /**
-   * The offset of the next message that will be appended to the log
+   * The offset metadata of the next message that will be appended to the log
    */
-  def logEndOffset: Long = nextOffsetMetadata.messageOffset
+  def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata
+
+  private val rollAction = RollAction(

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1572,144 +1414,69 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      localLog.roll(Some(rollOffset), Some(rollAction))
     } else {
       segment
     }
   }
 
   /**
-   * Roll the log over to a new active segment starting with the current logEndOffset.
+   * Roll the local 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.
    *
    * @return The newly rolled segment
    */
   def roll(expectedNextOffset: Option[Long] = None): LogSegment = {
-    maybeHandleIOException(s"Error while rolling log segment for $topicPartition in dir ${dir.getParent}") {
-      val start = time.hiResClockMs()
-      lock synchronized {
-        checkIfMemoryMappedBufferClosed()
-        val newOffset = math.max(expectedNextOffset.getOrElse(0L), logEndOffset)
-        val logFile = Log.logFile(dir, newOffset)
-
-        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).
-            warn(s"Trying to roll a new log segment with start offset $newOffset " +
-                 s"=max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already " +
-                 s"exists and is active with size 0. Size of time index: ${activeSegment.timeIndex.entries}," +
-                 s" size of offset index: ${activeSegment.offsetIndex.entries}.")
-            removeAndDeleteSegments(Seq(activeSegment), asyncDelete = true, LogRoll)
-          } else {
-            throw new KafkaException(s"Trying to roll a new log segment for topic partition $topicPartition with start offset $newOffset" +
-                                     s" =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already exists. Existing " +
-                                     s"segment is ${segments.get(newOffset)}.")
-          }
-        } else if (!segments.isEmpty && newOffset < activeSegment.baseOffset) {
-          throw new KafkaException(
-            s"Trying to roll a new log segment for topic partition $topicPartition with " +
-            s"start offset $newOffset =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) lower than start offset of the active segment $activeSegment")
-        } else {
-          val offsetIdxFile = offsetIndexFile(dir, newOffset)
-          val timeIdxFile = timeIndexFile(dir, newOffset)
-          val txnIdxFile = transactionIndexFile(dir, newOffset)
-
-          for (file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) {
-            warn(s"Newly rolled segment file ${file.getAbsolutePath} already exists; deleting it first")
-            Files.delete(file.toPath)
-          }
-
-          segments.lastSegment.foreach(_.onBecomeInactiveSegment())
-        }
-
-        // take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
-        // offset align with the new segment offset since this ensures we can recover the segment by beginning
-        // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
-        // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
-        // we manually override the state offset here prior to taking the snapshot.
-        producerStateManager.updateMapEndOffset(newOffset)
-        producerStateManager.takeSnapshot()
-
-        val segment = LogSegment.open(dir,
-          baseOffset = newOffset,
-          config,
-          time = time,
-          initFileSize = config.initFileSize,
-          preallocate = config.preallocate)
-        addSegment(segment)
-
-        // 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)
-
-        // schedule an asynchronous flush of the old segment
-        scheduler.schedule("flush-log", () => flush(newOffset), delay = 0L)
-
-        info(s"Rolled new log segment at offset $newOffset in ${time.hiResClockMs() - start} ms.")
-
-        segment
-      }
+    lock synchronized {
+      localLog.roll(expectedNextOffset, Some(rollAction))
     }
   }
 
   /**
    * The number of messages appended to the log since the last flush
    */
-  private def unflushedMessages: Long = this.logEndOffset - this.recoveryPoint
+  private def unflushedMessages: Long = logEndOffset - localLog.recoveryPoint
 
   /**
-   * Flush all log segments
+   * Flush all local log segments
    */
   def flush(): Unit = flush(this.logEndOffset)
 
   /**
-   * Flush log segments for all offsets up to offset-1
+   * Flush local log segments for all offsets up to offset-1
    *
    * @param offset The offset to flush up to (non-inclusive); the new recovery point
    */
   def flush(offset: Long): Unit = {
     maybeHandleIOException(s"Error while flushing log for $topicPartition in dir ${dir.getParent} with offset $offset") {
-      if (offset > this.recoveryPoint) {
+      if (offset > localLog.recoveryPoint) {
         debug(s"Flushing log up to offset $offset, last flushed: $lastFlushTime,  current time: ${time.milliseconds()}, " +
           s"unflushed: $unflushedMessages")
-        val segments = logSegments(this.recoveryPoint, offset)
-        segments.foreach(_.flush())
-        // if there are any new segments, we need to flush the parent directory for crash consistency
-        segments.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath))
-
+        localLog.flush(offset)
         lock synchronized {
-          checkIfMemoryMappedBufferClosed()
-          if (offset > this.recoveryPoint) {
-            this.recoveryPoint = offset
-            lastFlushedTime.set(time.milliseconds)
-          }
+          localLog.markFlushed(offset)
         }
       }
     }
   }
 
   /**
-   * Completely delete this log directory and all contents from the file system with no delay
+   * Completely delete the local log directory and all contents from the file system with no delay
    */
   private[log] def delete(): Unit = {
     maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
       lock synchronized {
-        checkIfMemoryMappedBufferClosed()
         producerExpireCheck.cancel(true)
-        removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
         leaderEpochCache.foreach(_.clear())
-        Utils.delete(dir)
-        // File handlers will be closed if this log is deleted
-        isMemoryMappedBufferClosed = true
+        val deletedSegments = localLog.delete()
+        deleteProducerSnapshotAsync(deletedSegments)

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1806,37 +1566,37 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
-    rebuildProducerState(endOffset, producerStateManager)
+    lock synchronized {
+      rebuildProducerState(endOffset, producerStateManager)
+    }

Review comment:
       Done.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1852,65 +1612,24 @@ class Log(@volatile private var _dir: File,
     logString.toString
   }
 
-  /**
-   * This method deletes the given log segments by doing the following for each of them:
-   * <ol>
-   *   <li>It removes the segment from the segment map so that it will no longer be used for reads.
-   *   <li>It renames the index and log files by appending .deleted to the respective file name
-   *   <li>It can either schedule an asynchronous delete operation to occur in the future or perform the deletion synchronously
-   * </ol>
-   * Asynchronous deletion allows reads to happen concurrently without synchronization and without the possibility of
-   * physically deleting a file while it is being read.
-   *
-   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
-   * or the immediate caller will catch and handle IOException
-   *
-   * @param segments The log segments to schedule for deletion
-   * @param asyncDelete Whether the segment files should be deleted asynchronously
-   */
-  private def removeAndDeleteSegments(segments: Iterable[LogSegment],
-                                      asyncDelete: Boolean,
-                                      reason: SegmentDeletionReason): Unit = {
-    if (segments.nonEmpty) {
-      lock synchronized {
-        // As 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.
-        val toDelete = segments.toList
-        reason.logReason(this, toDelete)
-        toDelete.foreach { segment =>
-          this.segments.remove(segment.baseOffset)
-        }
-        deleteSegmentFiles(toDelete, asyncDelete)
-      }
-    }
-  }
-
-  private def deleteSegmentFiles(segments: Iterable[LogSegment], asyncDelete: Boolean, deleteProducerStateSnapshots: Boolean = true): Unit = {
-    Log.deleteSegmentFiles(segments, asyncDelete, deleteProducerStateSnapshots, dir, topicPartition,
-      config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent)
-  }
-
   private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = {

Review comment:
       Done.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r646955278



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1788,14 +1559,8 @@ class Log(@volatile private var _dir: File,
     maybeHandleIOException(s"Error while truncating the entire log for $topicPartition in dir ${dir.getParent}") {
       debug(s"Truncate and start at offset $newOffset")
       lock synchronized {
-        checkIfMemoryMappedBufferClosed()
-        removeAndDeleteSegments(logSegments, asyncDelete = true, LogTruncation)
-        addSegment(LogSegment.open(dir,
-          baseOffset = newOffset,
-          config = config,
-          time = time,
-          initFileSize = config.initFileSize,
-          preallocate = config.preallocate))
+        val deletedSegments = localLog.truncateFullyAndStartAt(newOffset)
+        deleteProducerSnapshots(deletedSegments, asyncDelete = true)

Review comment:
       Sounds good. I'll fix this.




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

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



[GitHub] [kafka] ijuma commented on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-794235118


   @kowshik I can't comment on the doc, that's why I commented here. :) I didn't see any reason there for calling it `Global` btw. `Global` tends to imply something more than what this is doing IMO.


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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-865332690


   @junrao @dhruvilshah3 I ran a perf test against a Broker build with and without this PR. The test involved the following:
   
   1. Created a test topic with 1 partition and replication factor 1 using the command: `$> ./bin/kafka-topics.sh --bootstrap-server localhost:9092 --create --topic kowshik-test-1 --partitions 1 --replication-factor 1`.
   2. Ran `kafka-producer-perf-test.sh` to produce 10M messages each of size 1KB and with max producer throughput 100K to the above topic. Command: `$> ./bin/kafka-producer-perf-test.sh --num-records 10000000 --print-metrics --producer-props bootstrap.servers=localhost:9092 --record-size 1024 --throughput 100000 --topic kowshik-test-1`.
   3. In parallel, ran `kafka-consumer-perf-test.sh` to consume the 10M messages that were produced in (2) using the command: `$> bin/kafka-consumer-perf-test.sh --topic kowshik-test-1 --bootstrap-server localhost:9092 --messages 10000000 --print-metrics --show-detailed-stats`
   
   The tests have similar results, meaning that the performance with and without this PR looks similar. Here are the results:
   
   
   **log.segment.bytes=10MB**
   
   - Without this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/0ea1ae9ac8210f4bba49967727ddb475
   - With this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/c5ec0fb92679bd91613f520455446bf5
   
   **log.segment.bytes=100MB**
   
   - Without this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/2aaa7113fd05e10721c60aaf9bf8c654
   - With this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/b09882c2b13930be2efc69554c31aded
   
   **log.segment.bytes=1GB**
   
   - Without this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/ea36153c9751180c5dbe383b189d50df
   - With this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/133803d10d510df93f1d15858e91035a


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602052452



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because

Review comment:
       The method only converts `IOException` to `KafkaStorageException` inside the internal `deleteSegments()` helper function, but it doesn't do the same for `changeFileSuffixes` which is what the comment was referring to. I've improved the `@throws` doc to refer to both exceptions now. Please let me know if it needs further improvement.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648132657



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -246,17 +262,17 @@ object LogLoader extends Logging {
         return fn
       } catch {
         case e: LogSegmentOffsetOverflowException =>
-          info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
-          Log.splitOverflowedSegment(
+          info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          val result = Log.splitOverflowedSegment(
             e.segment,
             params.segments,
             params.dir,
             params.topicPartition,
             params.config,
             params.scheduler,
             params.logDirFailureChannel,
-            params.producerStateManager,
             params.logIdentifier)
+          deleteProducerSnapshotsAsync(result.deletedSegments, params)

Review comment:
       I've created a jira to track this. https://issues.apache.org/jira/browse/KAFKA-12923




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-836253885


   @junrao This PR is ready for another round of review. I've rebased the PR onto latest AK trunk, iterated on the implementation bit more and added new unit tests for `LocalLog` class under `LocalLogTest.scala`.


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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-858438516


   Thanks for the review @junrao! I've addressed your most recent comments in 28bf22af168ca0db76796b5d3cd67a38ed8ed1c2.


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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-794210868


   @ijuma In this PR the intention was not to rename `Log` to `LocalLog`, but rather extract `LocalLog` class out of `Log`. My current plan is to do the alternative of what you had suggested above i.e. extract `LocalLog` from `Log` in this PR and then rename `Log` to `GlobalLog` in a subsequent PR. The reason is to focus on the more important/major piece in the first PR (which is the Log layer separation), then the renaming the abstractions in a future PR becomes a relatively minor activity. Either way we choose, we will eventually introduce a new abstraction viz. `LocalLog` or `GlobalLog`.
   
   Thoughts?


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602055675



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because
+   * it is either called before all logs are loaded or the caller will catch and handle IOException
+   *
+   * @throws IOException if the segment files can't be renamed and still exists
+   */
+  private[log] def deleteSegmentFiles(segments: Iterable[LogSegment],
+                                      asyncDelete: Boolean): Unit = {
+    segments.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix))
+
+    def deleteSegments(): Unit = {
+      info(s"Deleting segment files ${segments.mkString(",")}")
+      maybeHandleIOException(s"Error while deleting segments for $topicPartition in dir ${dir.getParent}") {
+        segments.foreach { segment =>
+          segment.deleteIfExists()
+        }
+      }
+    }
+
+    if (asyncDelete)
+      scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs)
+    else
+      deleteSegments()
+  }
+
+  /**
+   * This method deletes the given log segments by doing the following for each of them:
+   * <ol>
+   *   <li>It removes the segment from the segment map so that it will no longer be used for reads.
+   *   <li>It renames the index and log files by appending .deleted to the respective file name
+   *   <li>It can either schedule an asynchronous delete operation to occur in the future or perform the deletion synchronously
+   * </ol>
+   * Asynchronous deletion allows reads to happen concurrently without synchronization and without the possibility of
+   * physically deleting a file while it is being read.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the immediate caller will catch and handle IOException
+   *
+   * @param segments The log segments to schedule for deletion
+   * @param asyncDelete Whether the segment files should be deleted asynchronously
+   */
+  private[log] def removeAndDeleteSegments(segments: Iterable[LogSegment],
+                                           asyncDelete: Boolean,
+                                           reason: SegmentDeletionReason): Unit = {
+    if (segments.nonEmpty) {
+      // As 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.
+      val toDelete = segments.toList
+      reason.logReason(this, toDelete)
+      toDelete.foreach { segment =>
+        this.segments.remove(segment.baseOffset)
+      }
+      deleteSegmentFiles(toDelete, asyncDelete)
+    }
+  }
+
+  private[log] def emptyFetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
+                                      includeAbortedTxns: Boolean): FetchDataInfo = {
+    val abortedTransactions =
+      if (includeAbortedTxns) Some(List.empty[FetchResponseData.AbortedTransaction])
+      else None
+    FetchDataInfo(fetchOffsetMetadata,
+      MemoryRecords.EMPTY,
+      abortedTransactions = abortedTransactions)
+  }
+
+  /**
+   * Given a message offset, find its corresponding offset metadata in the log.
+   * If the message offset is out of range, return None to the caller.
+   */
+  private[log] def convertToOffsetMetadata(offset: Long): Option[LogOffsetMetadata] = {
+    try {
+      Some(convertToOffsetMetadataOrThrow(offset))
+    } catch {
+      case _: OffsetOutOfRangeException => None
+    }
+  }
+
+  /**
+   * Given a message offset, find its corresponding offset metadata in the log.
+   * If the message offset is out of range, throw an OffsetOutOfRangeException
+   */
+  private[log] def convertToOffsetMetadataOrThrow(offset: Long): LogOffsetMetadata = {
+    val fetchDataInfo = read(offset,
+      maxLength = 1,
+      minOneMessage = false,
+      maxOffsetMetadata = nextOffsetMetadata,
+      includeAbortedTxns = false)
+    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 this is true, aborted transactions are included in the fetch data information
+   * @throws OffsetOutOfRangeException If startOffset is beyond the log end offset
+   *
+   * @return The fetch data information including fetch starting offset metadata and messages read.
+   */
+  private[log] def read(startOffset: Long,
+                        maxLength: Int,
+                        minOneMessage: Boolean,
+                        maxOffsetMetadata: LogOffsetMetadata,
+                        includeAbortedTxns: Boolean): FetchDataInfo = {
+    maybeHandleIOException(s"Exception while reading from $topicPartition in dir ${dir.getParent}") {
+      trace(s"Reading maximum $maxLength bytes at offset $startOffset from log with " +
+        s"total length $size bytes")
+
+      // Because we don't use the lock for reading, the synchronization is a little bit tricky.
+      // We create the local variables to avoid race conditions with updates to the log.
+      val endOffsetMetadata = nextOffsetMetadata
+      val endOffset = endOffsetMetadata.messageOffset
+      var segmentEntry = segments.floorEntry(startOffset)
+
+      // return error on attempt to read beyond the log end offset or read below log start offset
+      if (startOffset > endOffset || segmentEntry == null)
+        throw new OffsetOutOfRangeException(s"Received request for offset $startOffset for partition $topicPartition, " +
+          s"but we only have log segments upto $endOffset.")
+
+      if (startOffset == maxOffsetMetadata.messageOffset)
+        emptyFetchDataInfo(maxOffsetMetadata, includeAbortedTxns)
+      else if (startOffset > maxOffsetMetadata.messageOffset)
+        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
+        var done = segmentEntry == null
+        var fetchDataInfo: FetchDataInfo = null
+        while (!done) {
+          val segment = segmentEntry.getValue
+
+          val maxPosition =
+          // Use the max offset position if it is on this segment; otherwise, the segment size is the limit.
+            if (maxOffsetMetadata.segmentBaseOffset == segment.baseOffset) maxOffsetMetadata.relativePositionInSegment
+            else segment.size
+
+          fetchDataInfo = segment.read(startOffset, maxLength, maxPosition, minOneMessage)
+          if (fetchDataInfo != null) {
+            if (includeAbortedTxns)
+              fetchDataInfo = addAbortedTransactions(startOffset, segmentEntry, fetchDataInfo)
+          } else segmentEntry = segments.higherEntry(segmentEntry.getKey)
+
+          done = fetchDataInfo != null || segmentEntry == null
+        }
+
+        if (fetchDataInfo != null) 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
+          FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
+        }
+      }
+    }
+  }
+
+  private def addAbortedTransactions(startOffset: Long, segmentEntry: JEntry[JLong, LogSegment],
+                                     fetchInfo: FetchDataInfo): FetchDataInfo = {
+    val fetchSize = fetchInfo.records.sizeInBytes
+    val startOffsetPosition = OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+      fetchInfo.fetchOffsetMetadata.relativePositionInSegment)
+    val upperBoundOffset = segmentEntry.getValue.fetchUpperBoundOffset(startOffsetPosition, fetchSize).getOrElse {
+      val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+      if (nextSegmentEntry != null)
+        nextSegmentEntry.getValue.baseOffset
+      else
+        logEndOffset
+    }
+
+    val abortedTransactions = ListBuffer.empty[FetchResponseData.AbortedTransaction]
+    def accumulator(abortedTxns: List[AbortedTxn]): Unit = abortedTransactions ++= abortedTxns.map(_.asAbortedTransaction)
+    collectAbortedTransactions(startOffset, upperBoundOffset, segmentEntry, accumulator)
+
+    FetchDataInfo(fetchOffsetMetadata = fetchInfo.fetchOffsetMetadata,
+      records = fetchInfo.records,
+      firstEntryIncomplete = fetchInfo.firstEntryIncomplete,
+      abortedTransactions = Some(abortedTransactions.toList))
+  }
+
+  private[log] def collectAbortedTransactions(logStartOffset: Long, baseOffset: Long, upperBoundOffset: Long): List[AbortedTxn] = {
+    val segmentEntry = segments.floorEntry(baseOffset)
+    val allAbortedTxns = ListBuffer.empty[AbortedTxn]
+    def accumulator(abortedTxns: List[AbortedTxn]): Unit = allAbortedTxns ++= abortedTxns
+    collectAbortedTransactions(logStartOffset, upperBoundOffset, segmentEntry, accumulator)
+    allAbortedTxns.toList
+  }
+
+  private def collectAbortedTransactions(startOffset: Long, upperBoundOffset: Long,
+                                         startingSegmentEntry: JEntry[JLong, LogSegment],
+                                         accumulator: List[AbortedTxn] => Unit): Unit = {
+    var segmentEntry = startingSegmentEntry
+    while (segmentEntry != null) {
+      val searchResult = segmentEntry.getValue.collectAbortedTxns(startOffset, upperBoundOffset)
+      accumulator(searchResult.abortedTransactions)
+      if (searchResult.isComplete)
+        return
+      segmentEntry = segments.higherEntry(segmentEntry.getKey)
+    }
+  }
+
+  /**
+   * This function does not acquire Log.lock. The caller has to make sure log segments don't get deleted during

Review comment:
       Done. I've fixed it now.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r645434947



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1500,50 +1325,67 @@ class Log(@volatile private var _dir: File,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach)
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
       nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
     }
 
-    deleteOldSegments(shouldDelete, StartOffsetBreach)
+    deleteOldSegments(shouldDelete, StartOffsetBreach(this))
   }
 
   def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix)
 
   /**
    * The size of the log in bytes
    */
-  def size: Long = Log.sizeInBytes(logSegments)
+  def size: Long = localLog.segments.sizeInBytes
 
   /**
-   * The offset metadata of the next message that will be appended to the log
+   * The offset of the next message that will be appended to the log
    */
-  def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+  def logEndOffset: Long =  localLog.logEndOffset
 
   /**
-   * The offset of the next message that will be appended to the log
+   * The offset metadata of the next message that will be appended to the log
    */
-  def logEndOffset: Long = nextOffsetMetadata.messageOffset
+  def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata
+
+  private val rollAction = RollAction(
+    preRollAction = (newSegment: LogSegment) => {
+      // Take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
+      // offset align with the new segment offset since this ensures we can recover the segment by beginning
+      // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
+      // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
+      // we manually override the state offset here prior to taking the snapshot.
+      producerStateManager.updateMapEndOffset(newSegment.baseOffset)
+      producerStateManager.takeSnapshot()
+    },
+    postRollAction = (newSegment: LogSegment, deletedSegment: Option[LogSegment]) => {
+      deletedSegment.foreach(segment => deleteProducerSnapshotAsync(Seq(segment)))

Review comment:
       Done.




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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-860537313


   @junrao Thanks for the review! I ran the system tests.
   1. [System test run #4560](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4560/) on top of the latest commit 008b701386ce5a4d892d6ac5b90798b981c4fba0 from this PR. The run finished with 12 test failures.
   2. [System test run #4561](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4561/) against AK trunk on top of commit 6de37e536ac76ef13530d49dc7320110332cd1ee which does not contain changes from this PR. The run finished with 13 test failures.
   
   There were 11 overlapping failures in both (1) and (2). For these, I didn't find anything abnormal in the logs so far, the failure reason seems similar in both.
   
   The only new failure in (1) that's not present in (2) was:
   
   ```
   Module: kafkatest.tests.client.consumer_test
   Class:  OffsetValidationTest
   Method: test_broker_failure
   Arguments:
   {
     "clean_shutdown": true,
     "enable_autocommit": false,
     "metadata_quorum": "REMOTE_KRAFT"
   }
   ```
   
   Logs indicate that the test failed [at this line](https://github.com/apache/kafka/blob/b96fc7892f1e885239d3290cf509e1d1bb41e7db/tests/kafkatest/tests/client/consumer_test.py#L388) because one of the worker nodes running the consumer didn't complete within the timeout of 30s. This doesn't seem indicative of a real failure (yet), so I'm rerunning the system tests again to check if the failure is consistent. I'll keep you posted on the outcome of this second run.
   
   


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602062784



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1816,8 +1292,12 @@ class Log(@volatile private var _dir: File,
    */
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
+    def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long): Boolean = {
+      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(logEndOffset) &&

Review comment:
       this is to accomodate for the hwm check that was previously happening in `Log#deletableSegments` in this [line](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/log/Log.scala#L1872). The `deletableSegments` method has now moved to `LocalLog`, so we piggyback on the predicate to additionally attach the hwm check.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1816,8 +1292,12 @@ class Log(@volatile private var _dir: File,
    */
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
+    def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long): Boolean = {
+      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(logEndOffset) &&

Review comment:
       This is to accomodate for the hwm check that was previously happening in `Log#deletableSegments` in this [line](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/log/Log.scala#L1872). The `deletableSegments` method has now moved to `LocalLog`, so we piggyback on the predicate to additionally attach the hwm check.




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-794673540


   @ijuma I've opened up the doc for comments. I've also updated it to use the name `UnifiedLog` instead of `GlobalLog`. Hopefully the intent is better communicated now in the naming.


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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-857548583


   Thanks for the review @junrao! I've addressed the comments in 8f14879.


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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-794210868


   @ijuma In this PR the intention was not to rename `Log` to `LocalLog`, but rather extract `LocalLog` class out of `Log`. My current plan is to do the alternative of what you had suggested i.e. extract `LocalLog` from `Log` in this PR and then rename `Log` to `GlobalLog` in a subsequent PR. The reason is to focus on the more important/major piece in the first PR (which is the Log layer separation), then the renaming the abstractions in a future PR becomes a relatively minor activity. Either way we choose, we will eventually introduce a new abstraction viz. `LocalLog` or `GlobalLog`.
   
   Thoughts?


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648133097



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1578,144 +1409,85 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      val newSegment = localLog.roll(Some(rollOffset))
+      afterRoll(newSegment)
+      newSegment

Review comment:
       Done in 8ebb39f.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643465477



##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {
+      log.close()
+    }
+    Utils.delete(tmpDir)
+  }
+
+  case class KeyValue(key: String, value: String) {
+    def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = {
+      new SimpleRecord(timestamp, key.getBytes, value.getBytes)
+    }
+  }
+
+  object KeyValue {
+    def fromRecord(record: Record): KeyValue = {
+      val key =
+        if (record.hasKey)
+          StandardCharsets.UTF_8.decode(record.key()).toString
+        else
+          ""
+      val value =
+        if (record.hasValue)
+          StandardCharsets.UTF_8.decode(record.value()).toString
+        else
+          ""
+      KeyValue(key, value)
+    }
+  }
+
+  private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = {
+    keyValues.map(kv => kv.toRecord())
+  }
+
+  private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = {
+    records.map(r => KeyValue.fromRecord(r))
+  }
+
+  private def appendRecords(records: Iterable[SimpleRecord],
+                            log: LocalLog = log,
+                            initialOffset: Long = 0L): Unit = {
+    log.append(lastOffset = initialOffset + records.size - 1,
+      largestTimestamp = records.head.timestamp,
+      shallowOffsetOfMaxTimestamp = initialOffset,
+      records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
+  }
+
+  private def readRecords(log: LocalLog = log,
+                          startOffset: Long = 0L,
+                          maxLength: => Int = log.segments.activeSegment.size,
+                          minOneMessage: Boolean = false,
+                          maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata,
+                          includeAbortedTxns: Boolean = false): FetchDataInfo = {
+    log.read(startOffset,
+             maxLength,
+             minOneMessage = minOneMessage,
+             maxOffsetMetadata,
+             includeAbortedTxns = includeAbortedTxns)
+  }
+
+  @Test
+  def testLogDeleteSuccess(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    assertFalse(logDir.listFiles.isEmpty)
+    val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values
+    val deletedSegments = log.delete()
+    assertTrue(log.segments.isEmpty)
+    assertEquals(segmentsBeforeDelete, deletedSegments)
+    assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed())
+    assertFalse(logDir.exists)
+  }
+
+  @Test
+  def testLogDeleteFailureAfterCloseHandlers(): Unit = {
+    log.closeHandlers()
+    assertEquals(1, log.segments.numberOfSegments)
+    val segmentsBeforeDelete = log.segments.values
+    assertThrows(classOf[KafkaStorageException], () => log.delete())
+    assertEquals(1, log.segments.numberOfSegments)
+    assertEquals(segmentsBeforeDelete, log.segments.values)
+    assertTrue(logDir.exists)
+  }
+
+  @Test
+  def testUpdateConfig(): Unit = {
+    val oldConfig = log.config
+    assertEquals(oldConfig, log.config)
+
+    val newConfig = createLogConfig()
+    log.updateConfig(newConfig)
+    assertEquals(newConfig, log.config)
+  }
+
+  @Test
+  def testLogDirRenameToNewDir(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    val newLogDir = TestUtils.randomPartitionLogDir(tmpDir)
+    assertTrue(log.renameDir(newLogDir.getName))
+    assertFalse(logDir.exists())
+    assertTrue(newLogDir.exists())
+    assertEquals(newLogDir, log.dir)
+    assertEquals(newLogDir.getParent, log.parentDir)
+    assertEquals(newLogDir.getParent, log.dir.getParent)
+    log.segments.values.foreach(segment => assertEquals(newLogDir.getPath, segment.log.file().getParentFile.getPath))
+    assertEquals(2, log.segments.numberOfSegments)
+  }
+
+  @Test
+  def testLogDirRenameToExistingDir(): Unit = {
+    assertFalse(log.renameDir(log.dir.getName))
+  }
+
+  @Test
+  def testLogFlush(): Unit = {
+    assertEquals(0L, log.recoveryPoint)
+    assertEquals(mockTime.milliseconds, log.lastFlushTime)
+
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    mockTime.sleep(1)
+    val newSegment = log.roll()
+    log.flush(newSegment.baseOffset)
+    log.markFlushed(newSegment.baseOffset)
+    assertEquals(1L, log.recoveryPoint)
+    assertEquals(mockTime.milliseconds, log.lastFlushTime)
+  }
+
+  @Test
+  def testLogAppend(): Unit = {
+    val fetchDataInfoBeforeAppend = readRecords(maxLength = 1)
+    assertTrue(fetchDataInfoBeforeAppend.records.records.asScala.isEmpty)
+
+    mockTime.sleep(1)
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    assertEquals(2L, log.logEndOffset)
+    assertEquals(0L, log.recoveryPoint)
+    val fetchDataInfo = readRecords()
+    assertEquals(2L, fetchDataInfo.records.records.asScala.size)
+    assertEquals(keyValues, recordsToKvs(fetchDataInfo.records.records.asScala))
+  }
+
+  @Test
+  def testLogCloseSuccess(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.close()
+    assertThrows(classOf[ClosedChannelException], () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L))
+  }
+
+  @Test
+  def testLogCloseIdempotent(): Unit = {
+    log.close()
+    // Check that LocalLog.close() is idempotent
+    log.close()
+  }
+
+  @Test
+  def testLogCloseFailureWhenInMemoryBufferClosed(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.closeHandlers()
+    assertThrows(classOf[KafkaStorageException], () => log.close())
+  }
+
+  @Test
+  def testLogCloseHandlers(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.closeHandlers()
+    assertThrows(classOf[ClosedChannelException],
+                 () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L))
+  }
+
+  @Test
+  def testLogCloseHandlersIdempotent(): Unit = {
+    log.closeHandlers()
+    // Check that LocalLog.closeHandlers() is idempotent
+    log.closeHandlers()
+  }
+
+  private def testRemoveAndDeleteSegments(asyncDelete: Boolean): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    class TestDeletionReason extends SegmentDeletionReason {
+      private var _deletedSegments: Iterable[LogSegment] = List[LogSegment]()
+
+      override def logReason(toDelete: List[LogSegment]): Unit = {
+        _deletedSegments = List[LogSegment]() ++ toDelete
+      }
+
+      def deletedSegments: Iterable[LogSegment] = _deletedSegments
+    }
+    val reason = new TestDeletionReason()
+    val toDelete = List[LogSegment]() ++ log.segments.values
+    log.removeAndDeleteSegments(toDelete, asyncDelete = asyncDelete, reason)
+    if (asyncDelete) {
+      mockTime.sleep(log.config.fileDeleteDelayMs + 1)
+    }
+    assertTrue(log.segments.isEmpty)
+    assertEquals(toDelete, reason.deletedSegments)
+    toDelete.foreach(segment => assertTrue(segment.deleted()))
+  }
+
+  @Test
+  def testRemoveAndDeleteSegmentsSync(): Unit = {
+    testRemoveAndDeleteSegments(asyncDelete = false)
+  }
+
+  @Test
+  def testRemoveAndDeleteSegmentsAsync(): Unit = {
+    testRemoveAndDeleteSegments(asyncDelete = true)
+  }
+
+  private def testDeleteSegmentFiles(asyncDelete: Boolean): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    val toDelete = List[LogSegment]() ++ log.segments.values
+    LocalLog.deleteSegmentFiles(toDelete, asyncDelete = asyncDelete, log.dir, log.topicPartition, log.config, log.scheduler, log.logDirFailureChannel, "")
+    if (asyncDelete) {
+      toDelete.foreach {
+        segment =>
+          assertFalse(segment.deleted())
+          assertTrue(segment.hasSuffix(LocalLog.DeletedFileSuffix))
+      }
+      mockTime.sleep(log.config.fileDeleteDelayMs + 1)
+    }
+    toDelete.foreach(segment => assertTrue(segment.deleted()))
+  }
+
+  @Test
+  def testDeleteSegmentFilesSync(): Unit = {
+    testDeleteSegmentFiles(asyncDelete = false)
+  }
+
+  @Test
+  def testDeleteSegmentFilesAsync(): Unit = {
+    testDeleteSegmentFiles(asyncDelete = true)
+  }
+
+  @Test
+  def testDeletableSegmentsFilter(): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    {
+      val deletable = log.deletableSegments(
+        (segment: LogSegment, _: Option[LogSegment], _: Long) => segment.baseOffset <= 5)
+      val expected = log.segments.nonActiveLogSegmentsFrom(0L).filter(segment => segment.baseOffset <= 5)
+      assertEquals(expected, deletable.toList)
+    }
+
+    {
+      val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true)
+      val expected = log.segments.nonActiveLogSegmentsFrom(0L).toList
+      assertEquals(expected, deletable.toList)
+    }
+
+    {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = 9L)
+      val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true)
+      val expected = log.segments.values.toList
+      assertEquals(expected, deletable.toList)
+    }
+  }
+
+  @Test
+  def testDeletableSegmentsIteration(): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    var offset = 0
+    log.deletableSegments(
+      (segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long) => {
+        assertEquals(offset, segment.baseOffset)
+        val floorSegmentOpt = log.segments.floorSegment(offset)
+        assertTrue(floorSegmentOpt.isDefined)
+        assertEquals(floorSegmentOpt.get, segment)
+        if (offset == log.logEndOffset) {
+          assertFalse(nextSegmentOpt.isDefined)
+        } else {
+          assertTrue(nextSegmentOpt.isDefined)
+          val higherSegmentOpt = log.segments.higherSegment(segment.baseOffset)
+          assertTrue(higherSegmentOpt.isDefined)
+          assertEquals(segment.baseOffset + 1, higherSegmentOpt.get.baseOffset)
+          assertEquals(higherSegmentOpt.get, nextSegmentOpt.get)
+        }
+        assertEquals(log.logEndOffset, logEndOffset)
+        offset += 1
+        true
+      })

Review comment:
       Good question. We could assert that `log.deletableSegments()` does not make any mutations to the `LocalLog` object.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602046050



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.

Review comment:
       Done. Updated the doc now.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602049230



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {

Review comment:
       Done. Good point. I've fixed it now.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602052393



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.

Review comment:
       Done. Fixed the comment now.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because

Review comment:
       Done. Fixed the comment now.




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-854616475


   @junrao Thanks for the review! I've addressed the comments in e201295e03e0ea8a7102983888d1a7afc66d384a, and have also rebased this PR onto most recent commit in `trunk`. It is ready for review again.


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

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



[GitHub] [kafka] junrao merged pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
junrao merged pull request #10280:
URL: https://github.com/apache/kafka/pull/10280


   


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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-860537313


   @junrao Thanks for the review! I ran the system tests.
   1. [System test run #4560](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4560/) on top of the latest commit 008b701386ce5a4d892d6ac5b90798b981c4fba0 from this PR. The run finished with 12 test failures.
   2. [System test run #4561](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4561/) against AK trunk on top of commit 6de37e536ac76ef13530d49dc7320110332cd1ee which does not contain changes from this PR. The run finished with 13 test failures.
   
   There were 11 overlapping failures in both (1) and (2). For these, I didn't find anything abnormal in the logs so far, the failure reason seems similar in both.
   
   The only new failure in (1) that's not present in (2) was:
   
   ```
   Module: kafkatest.tests.client.consumer_test
   Class:  OffsetValidationTest
   Method: test_broker_failure
   Arguments:
   {
     "clean_shutdown": true,
     "enable_autocommit": false,
     "metadata_quorum": "REMOTE_KRAFT"
   }
   ```
   
   Logs indicate that the test failed [at this line](https://github.com/apache/kafka/blob/b96fc7892f1e885239d3290cf509e1d1bb41e7db/tests/kafkatest/tests/client/consumer_test.py#L388) because one of the worker nodes running the consumer didn't complete within the timeout of 30s. This doesn't seem indicative of a real failure (yet), so I'm rerunning the system tests again in [test run #4562](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4561/) to check if the failure is consistent. I'll keep you posted on the outcome of this second run.
   
   


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

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



[GitHub] [kafka] ijuma commented on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-794218123


   OK, thanks for the explanation. Btw, why do we call one of the logs `GlobalLog`? In what sense is it `Global`?


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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-808514958


   @junrao Just a heads up on the following. I'm working on the changes for the following in separate PRs, these are related with refactoring the recovery logic (KAFKA-12553):
    * KAFKA-12552 (https://github.com/apache/kafka/pull/10401 to extract segments map)
    * KAFKA-12571: (https://github.com/apache/kafka/pull/10426 to eliminate LeaderEpochFileCache constructor dependency on logEndOffset)
   
   It seems better if we merge those into trunk ahead of the current PR.


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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-878641126


   @junrao Thanks for the review. I ran load tests on the changes from this PR, there weren't any new regressions (i.e. latency regressions or errors) that I noticed, except for an issue that I found which looks unrelated to this PR, its described in this jira: https://issues.apache.org/jira/browse/KAFKA-13070.
   
   The load test was run on a 6-broker cluster with 250GB SSD disks:
    * Produce consume on a test topic 2000 partitions (~1000+ replica count per broker).
    * Per topic # of producers = 6.
    * Produce ingress per broker = ~20.5MBps.
    * Per topic # of consumers = 6.
    * \# of consumer groups = 3.
    * Test duration: ~1h.
   
   Mid-way through the test, I rolled the cluster under load to check how the cluster behaved. Overall things looked OK.
   
   There weren't any additional tests that I was planning to do.


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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602046133



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,

Review comment:
       Done. Good point. I've made it private now.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643456815



##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {

Review comment:
       Is the concern here that we are accessing the private attribute?
   In certain tests, `log.checkIfMemoryMappedBufferClosed()` will raise an exception if the log has already been closed. If the private attribute access is a concern, I can change this to just call `log.checkIfMemoryMappedBufferClosed()` but ignore the `KafkaStorageException` thats raised.




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

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



[GitHub] [kafka] satishd commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
satishd commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-879773476


   Thanks @junrao for merging into trunk. Can we also push this to [3.0 branch](https://github.com/apache/kafka/tree/3.0) as we discussed earlier?
   cc @kowshik 


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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-860537313


   @junrao Thanks for the review! I ran the system tests.
   1. [System test run #4560](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4560/) on top of the latest commit 008b701386ce5a4d892d6ac5b90798b981c4fba0 from this PR. The run finished with 12 test failures.
   2. [System test run #4561](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4561/) against AK trunk on top of commit 6de37e536ac76ef13530d49dc7320110332cd1ee which does not contain changes from this PR. The run finished with 13 test failures.
   
   There were 11 overlapping failures in both (1) and (2). For these, I didn't find anything abnormal in the logs so far, the failure reason seems similar in both.
   
   The only new failure in (1) that's not present in (2) was:
   
   ```
   Module: kafkatest.tests.client.consumer_test
   Class:  OffsetValidationTest
   Method: test_broker_failure
   Arguments:
   {
     "clean_shutdown": true,
     "enable_autocommit": false,
     "metadata_quorum": "REMOTE_KRAFT"
   }
   ```
   
   Logs indicate that the test failed [at this line](https://github.com/apache/kafka/blob/b96fc7892f1e885239d3290cf509e1d1bb41e7db/tests/kafkatest/tests/client/consumer_test.py#L388) because one of the worker nodes running the consumer didn't complete within the timeout of 30s. This doesn't seem indicative of a real failure (yet), so I'm rerunning the system tests again in [test run #4562](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4562/) to check if the failure is consistent. I'll keep you posted on the outcome of this second run.
   
   


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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-794210868


   @ijuma In this PR the intention was not to rename `Log` to `LocalLog`, but rather extract `LocalLog` class out of `Log`. My current plan is to do the alternative of what you had suggested above i.e. extract `LocalLog` from `Log` in this PR and then rename `Log` to `GlobalLog` in a subsequent PR. The reason is to focus on the more important/major piece in the first PR (which is the Log layer separation), then renaming the abstractions in a future PR becomes a relatively minor activity. Either way we choose, we will eventually introduce a new abstraction viz. `LocalLog` or `GlobalLog`.
   
   Thoughts?


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648133473



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1578,144 +1409,85 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      val newSegment = localLog.roll(Some(rollOffset))
+      afterRoll(newSegment)
+      newSegment
     } else {
       segment
     }
   }
 
   /**
-   * Roll the log over to a new active segment starting with the current logEndOffset.
+   * Roll the local log over to a new active segment starting with the current logEndOffset.

Review comment:
       Done in 8ebb39f.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r645441207



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -246,17 +262,17 @@ object LogLoader extends Logging {
         return fn
       } catch {
         case e: LogSegmentOffsetOverflowException =>
-          info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
-          Log.splitOverflowedSegment(
+          info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          val result = Log.splitOverflowedSegment(
             e.segment,
             params.segments,
             params.dir,
             params.topicPartition,
             params.config,
             params.scheduler,
             params.logDirFailureChannel,
-            params.producerStateManager,
             params.logIdentifier)
+          deleteProducerSnapshotsAsync(result.deletedSegments, params)

Review comment:
       @junrao I thought about this again. Correct me if I'm wrong, but it appears we may be altering existing behavior if we go down this route. Should we do it in a separate PR to isolate the change?




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602041366



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()

Review comment:
       Done. Removed.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.

Review comment:
       Done. Removed.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643450237



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -246,17 +262,17 @@ object LogLoader extends Logging {
         return fn
       } catch {
         case e: LogSegmentOffsetOverflowException =>
-          info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
-          Log.splitOverflowedSegment(
+          info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          val result = Log.splitOverflowedSegment(
             e.segment,
             params.segments,
             params.dir,
             params.topicPartition,
             params.config,
             params.scheduler,
             params.logDirFailureChannel,
-            params.producerStateManager,
             params.logIdentifier)
+          deleteProducerSnapshotsAsync(result.deletedSegments, params)

Review comment:
       Sounds good. It appears straightforward to just skip deleting the snapshot here, I can leave a comment explaining why.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r646958227



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1812,37 +1577,36 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
+    localLog.updateLogEndOffset(endOffset)

Review comment:
       Sounds good. This can be updated to `updateHighWatermark(localLog.logEndOffsetMetadata)`.




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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-836254631


   @junrao This PR is ready for another round of review. I've rebased the PR onto latest AK trunk, iterated on the implementation bit more and added new unit tests for `LocalLog` class under `LocalLogTest.scala`.
   
   cc @dhruvilshah3


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r645437580



##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {

Review comment:
       Done.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r645439534



##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {

Review comment:
       Done.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602055217



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because
+   * it is either called before all logs are loaded or the caller will catch and handle IOException
+   *
+   * @throws IOException if the segment files can't be renamed and still exists
+   */
+  private[log] def deleteSegmentFiles(segments: Iterable[LogSegment],
+                                      asyncDelete: Boolean): Unit = {
+    segments.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix))
+
+    def deleteSegments(): Unit = {
+      info(s"Deleting segment files ${segments.mkString(",")}")
+      maybeHandleIOException(s"Error while deleting segments for $topicPartition in dir ${dir.getParent}") {
+        segments.foreach { segment =>
+          segment.deleteIfExists()
+        }
+      }
+    }
+
+    if (asyncDelete)
+      scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs)
+    else
+      deleteSegments()
+  }
+
+  /**
+   * This method deletes the given log segments by doing the following for each of them:
+   * <ol>
+   *   <li>It removes the segment from the segment map so that it will no longer be used for reads.
+   *   <li>It renames the index and log files by appending .deleted to the respective file name
+   *   <li>It can either schedule an asynchronous delete operation to occur in the future or perform the deletion synchronously
+   * </ol>
+   * Asynchronous deletion allows reads to happen concurrently without synchronization and without the possibility of
+   * physically deleting a file while it is being read.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded

Review comment:
       I've updated the doc now. The reason is the same the one I mentioned above: https://github.com/apache/kafka/pull/10280/files#r602052452.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r646945592



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1578,144 +1409,85 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      val newSegment = localLog.roll(Some(rollOffset))
+      afterRoll(newSegment)
+      newSegment

Review comment:
       Sure.




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

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



[GitHub] [kafka] junrao commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643443877



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1500,50 +1325,67 @@ class Log(@volatile private var _dir: File,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach)
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
       nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
     }
 
-    deleteOldSegments(shouldDelete, StartOffsetBreach)
+    deleteOldSegments(shouldDelete, StartOffsetBreach(this))
   }
 
   def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix)
 
   /**
    * The size of the log in bytes
    */
-  def size: Long = Log.sizeInBytes(logSegments)
+  def size: Long = localLog.segments.sizeInBytes
 
   /**
-   * The offset metadata of the next message that will be appended to the log
+   * The offset of the next message that will be appended to the log
    */
-  def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+  def logEndOffset: Long =  localLog.logEndOffset
 
   /**
-   * The offset of the next message that will be appended to the log
+   * The offset metadata of the next message that will be appended to the log
    */
-  def logEndOffset: Long = nextOffsetMetadata.messageOffset
+  def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata
+
+  private val rollAction = RollAction(
+    preRollAction = (newSegment: LogSegment) => {
+      // Take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
+      // offset align with the new segment offset since this ensures we can recover the segment by beginning
+      // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
+      // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
+      // we manually override the state offset here prior to taking the snapshot.
+      producerStateManager.updateMapEndOffset(newSegment.baseOffset)
+      producerStateManager.takeSnapshot()
+    },
+    postRollAction = (newSegment: LogSegment, deletedSegment: Option[LogSegment]) => {
+      deletedSegment.foreach(segment => deleteProducerSnapshotAsync(Seq(segment)))

Review comment:
       We could fix this in a separate jira too.




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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-808514958


   @junrao Just a heads up on the following. I'm working on the changes for the following in separate PRs, these are related with refactoring the recovery logic (KAFKA-12553):
    * KAFKA-12552 (https://github.com/apache/kafka/pull/10401) to extract segments map **[MERGED]**
    * KAFKA-12571: (https://github.com/apache/kafka/pull/10426) to eliminate LeaderEpochFileCache constructor dependency on logEndOffset **[MERGED]**
    * KAFKA-12575: (https://github.com/apache/kafka/pull/10430) to eliminate Log.isLogDirOffline boolean attribute
   
   It seems better if we merge those into trunk ahead of the current PR.


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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-861219607


   I ran the system tests in `kafkatest.tests.client.consumer_test` again:
    * [System test run #4564](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4564/) against  `trunk/6de37e536ac76ef13530d49dc7320110332cd1ee`.
    * [System test run #4566](https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4566/) against 008b701386ce5a4d892d6ac5b90798b981c4fba0 (the latest commit from this PR).
   
   All tests passed. 


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

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



[GitHub] [kafka] kowshik removed a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik removed a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-836253885


   @junrao This PR is ready for another round of review. I've rebased the PR onto latest AK trunk, iterated on the implementation bit more and added new unit tests for `LocalLog` class under `LocalLogTest.scala`.


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648134244



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1788,14 +1559,8 @@ class Log(@volatile private var _dir: File,
     maybeHandleIOException(s"Error while truncating the entire log for $topicPartition in dir ${dir.getParent}") {
       debug(s"Truncate and start at offset $newOffset")
       lock synchronized {
-        checkIfMemoryMappedBufferClosed()
-        removeAndDeleteSegments(logSegments, asyncDelete = true, LogTruncation)
-        addSegment(LogSegment.open(dir,
-          baseOffset = newOffset,
-          config = config,
-          time = time,
-          initFileSize = config.initFileSize,
-          preallocate = config.preallocate))
+        val deletedSegments = localLog.truncateFullyAndStartAt(newOffset)
+        deleteProducerSnapshots(deletedSegments, asyncDelete = true)

Review comment:
       Done in 8ebb39f.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648978604



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1788,17 +1554,9 @@ class Log(@volatile private var _dir: File,
     maybeHandleIOException(s"Error while truncating the entire log for $topicPartition in dir ${dir.getParent}") {
       debug(s"Truncate and start at offset $newOffset")
       lock synchronized {
-        checkIfMemoryMappedBufferClosed()
-        removeAndDeleteSegments(logSegments, asyncDelete = true, LogTruncation)
-        addSegment(LogSegment.open(dir,
-          baseOffset = newOffset,
-          config = config,
-          time = time,
-          initFileSize = config.initFileSize,
-          preallocate = config.preallocate))
+        localLog.truncateFullyAndStartAt(newOffset)

Review comment:
       Done in 28bf22af168ca0db76796b5d3cd67a38ed8ed1c2.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1812,37 +1570,39 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
+    localLog.updateLogEndOffset(endOffset)
     rebuildProducerState(endOffset, producerStateManager)
-    updateHighWatermark(math.min(highWatermark, endOffset))
+    if (highWatermark < localLog.logEndOffset)

Review comment:
       Done in 28bf22af168ca0db76796b5d3cd67a38ed8ed1c2.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643465477



##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {
+      log.close()
+    }
+    Utils.delete(tmpDir)
+  }
+
+  case class KeyValue(key: String, value: String) {
+    def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = {
+      new SimpleRecord(timestamp, key.getBytes, value.getBytes)
+    }
+  }
+
+  object KeyValue {
+    def fromRecord(record: Record): KeyValue = {
+      val key =
+        if (record.hasKey)
+          StandardCharsets.UTF_8.decode(record.key()).toString
+        else
+          ""
+      val value =
+        if (record.hasValue)
+          StandardCharsets.UTF_8.decode(record.value()).toString
+        else
+          ""
+      KeyValue(key, value)
+    }
+  }
+
+  private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = {
+    keyValues.map(kv => kv.toRecord())
+  }
+
+  private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = {
+    records.map(r => KeyValue.fromRecord(r))
+  }
+
+  private def appendRecords(records: Iterable[SimpleRecord],
+                            log: LocalLog = log,
+                            initialOffset: Long = 0L): Unit = {
+    log.append(lastOffset = initialOffset + records.size - 1,
+      largestTimestamp = records.head.timestamp,
+      shallowOffsetOfMaxTimestamp = initialOffset,
+      records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
+  }
+
+  private def readRecords(log: LocalLog = log,
+                          startOffset: Long = 0L,
+                          maxLength: => Int = log.segments.activeSegment.size,
+                          minOneMessage: Boolean = false,
+                          maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata,
+                          includeAbortedTxns: Boolean = false): FetchDataInfo = {
+    log.read(startOffset,
+             maxLength,
+             minOneMessage = minOneMessage,
+             maxOffsetMetadata,
+             includeAbortedTxns = includeAbortedTxns)
+  }
+
+  @Test
+  def testLogDeleteSuccess(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    assertFalse(logDir.listFiles.isEmpty)
+    val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values
+    val deletedSegments = log.delete()
+    assertTrue(log.segments.isEmpty)
+    assertEquals(segmentsBeforeDelete, deletedSegments)
+    assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed())
+    assertFalse(logDir.exists)
+  }
+
+  @Test
+  def testLogDeleteFailureAfterCloseHandlers(): Unit = {
+    log.closeHandlers()
+    assertEquals(1, log.segments.numberOfSegments)
+    val segmentsBeforeDelete = log.segments.values
+    assertThrows(classOf[KafkaStorageException], () => log.delete())
+    assertEquals(1, log.segments.numberOfSegments)
+    assertEquals(segmentsBeforeDelete, log.segments.values)
+    assertTrue(logDir.exists)
+  }
+
+  @Test
+  def testUpdateConfig(): Unit = {
+    val oldConfig = log.config
+    assertEquals(oldConfig, log.config)
+
+    val newConfig = createLogConfig()
+    log.updateConfig(newConfig)
+    assertEquals(newConfig, log.config)
+  }
+
+  @Test
+  def testLogDirRenameToNewDir(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    val newLogDir = TestUtils.randomPartitionLogDir(tmpDir)
+    assertTrue(log.renameDir(newLogDir.getName))
+    assertFalse(logDir.exists())
+    assertTrue(newLogDir.exists())
+    assertEquals(newLogDir, log.dir)
+    assertEquals(newLogDir.getParent, log.parentDir)
+    assertEquals(newLogDir.getParent, log.dir.getParent)
+    log.segments.values.foreach(segment => assertEquals(newLogDir.getPath, segment.log.file().getParentFile.getPath))
+    assertEquals(2, log.segments.numberOfSegments)
+  }
+
+  @Test
+  def testLogDirRenameToExistingDir(): Unit = {
+    assertFalse(log.renameDir(log.dir.getName))
+  }
+
+  @Test
+  def testLogFlush(): Unit = {
+    assertEquals(0L, log.recoveryPoint)
+    assertEquals(mockTime.milliseconds, log.lastFlushTime)
+
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    mockTime.sleep(1)
+    val newSegment = log.roll()
+    log.flush(newSegment.baseOffset)
+    log.markFlushed(newSegment.baseOffset)
+    assertEquals(1L, log.recoveryPoint)
+    assertEquals(mockTime.milliseconds, log.lastFlushTime)
+  }
+
+  @Test
+  def testLogAppend(): Unit = {
+    val fetchDataInfoBeforeAppend = readRecords(maxLength = 1)
+    assertTrue(fetchDataInfoBeforeAppend.records.records.asScala.isEmpty)
+
+    mockTime.sleep(1)
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    assertEquals(2L, log.logEndOffset)
+    assertEquals(0L, log.recoveryPoint)
+    val fetchDataInfo = readRecords()
+    assertEquals(2L, fetchDataInfo.records.records.asScala.size)
+    assertEquals(keyValues, recordsToKvs(fetchDataInfo.records.records.asScala))
+  }
+
+  @Test
+  def testLogCloseSuccess(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.close()
+    assertThrows(classOf[ClosedChannelException], () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L))
+  }
+
+  @Test
+  def testLogCloseIdempotent(): Unit = {
+    log.close()
+    // Check that LocalLog.close() is idempotent
+    log.close()
+  }
+
+  @Test
+  def testLogCloseFailureWhenInMemoryBufferClosed(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.closeHandlers()
+    assertThrows(classOf[KafkaStorageException], () => log.close())
+  }
+
+  @Test
+  def testLogCloseHandlers(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.closeHandlers()
+    assertThrows(classOf[ClosedChannelException],
+                 () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L))
+  }
+
+  @Test
+  def testLogCloseHandlersIdempotent(): Unit = {
+    log.closeHandlers()
+    // Check that LocalLog.closeHandlers() is idempotent
+    log.closeHandlers()
+  }
+
+  private def testRemoveAndDeleteSegments(asyncDelete: Boolean): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    class TestDeletionReason extends SegmentDeletionReason {
+      private var _deletedSegments: Iterable[LogSegment] = List[LogSegment]()
+
+      override def logReason(toDelete: List[LogSegment]): Unit = {
+        _deletedSegments = List[LogSegment]() ++ toDelete
+      }
+
+      def deletedSegments: Iterable[LogSegment] = _deletedSegments
+    }
+    val reason = new TestDeletionReason()
+    val toDelete = List[LogSegment]() ++ log.segments.values
+    log.removeAndDeleteSegments(toDelete, asyncDelete = asyncDelete, reason)
+    if (asyncDelete) {
+      mockTime.sleep(log.config.fileDeleteDelayMs + 1)
+    }
+    assertTrue(log.segments.isEmpty)
+    assertEquals(toDelete, reason.deletedSegments)
+    toDelete.foreach(segment => assertTrue(segment.deleted()))
+  }
+
+  @Test
+  def testRemoveAndDeleteSegmentsSync(): Unit = {
+    testRemoveAndDeleteSegments(asyncDelete = false)
+  }
+
+  @Test
+  def testRemoveAndDeleteSegmentsAsync(): Unit = {
+    testRemoveAndDeleteSegments(asyncDelete = true)
+  }
+
+  private def testDeleteSegmentFiles(asyncDelete: Boolean): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    val toDelete = List[LogSegment]() ++ log.segments.values
+    LocalLog.deleteSegmentFiles(toDelete, asyncDelete = asyncDelete, log.dir, log.topicPartition, log.config, log.scheduler, log.logDirFailureChannel, "")
+    if (asyncDelete) {
+      toDelete.foreach {
+        segment =>
+          assertFalse(segment.deleted())
+          assertTrue(segment.hasSuffix(LocalLog.DeletedFileSuffix))
+      }
+      mockTime.sleep(log.config.fileDeleteDelayMs + 1)
+    }
+    toDelete.foreach(segment => assertTrue(segment.deleted()))
+  }
+
+  @Test
+  def testDeleteSegmentFilesSync(): Unit = {
+    testDeleteSegmentFiles(asyncDelete = false)
+  }
+
+  @Test
+  def testDeleteSegmentFilesAsync(): Unit = {
+    testDeleteSegmentFiles(asyncDelete = true)
+  }
+
+  @Test
+  def testDeletableSegmentsFilter(): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    {
+      val deletable = log.deletableSegments(
+        (segment: LogSegment, _: Option[LogSegment], _: Long) => segment.baseOffset <= 5)
+      val expected = log.segments.nonActiveLogSegmentsFrom(0L).filter(segment => segment.baseOffset <= 5)
+      assertEquals(expected, deletable.toList)
+    }
+
+    {
+      val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true)
+      val expected = log.segments.nonActiveLogSegmentsFrom(0L).toList
+      assertEquals(expected, deletable.toList)
+    }
+
+    {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = 9L)
+      val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true)
+      val expected = log.segments.values.toList
+      assertEquals(expected, deletable.toList)
+    }
+  }
+
+  @Test
+  def testDeletableSegmentsIteration(): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    var offset = 0
+    log.deletableSegments(
+      (segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long) => {
+        assertEquals(offset, segment.baseOffset)
+        val floorSegmentOpt = log.segments.floorSegment(offset)
+        assertTrue(floorSegmentOpt.isDefined)
+        assertEquals(floorSegmentOpt.get, segment)
+        if (offset == log.logEndOffset) {
+          assertFalse(nextSegmentOpt.isDefined)
+        } else {
+          assertTrue(nextSegmentOpt.isDefined)
+          val higherSegmentOpt = log.segments.higherSegment(segment.baseOffset)
+          assertTrue(higherSegmentOpt.isDefined)
+          assertEquals(segment.baseOffset + 1, higherSegmentOpt.get.baseOffset)
+          assertEquals(higherSegmentOpt.get, nextSegmentOpt.get)
+        }
+        assertEquals(log.logEndOffset, logEndOffset)
+        offset += 1
+        true
+      })

Review comment:
       Good question. We could assert that the returned value matches our expectations. Also we could assert that `log.deletableSegments()` does not make any mutations to the `LocalLog` object.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602055217



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because
+   * it is either called before all logs are loaded or the caller will catch and handle IOException
+   *
+   * @throws IOException if the segment files can't be renamed and still exists
+   */
+  private[log] def deleteSegmentFiles(segments: Iterable[LogSegment],
+                                      asyncDelete: Boolean): Unit = {
+    segments.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix))
+
+    def deleteSegments(): Unit = {
+      info(s"Deleting segment files ${segments.mkString(",")}")
+      maybeHandleIOException(s"Error while deleting segments for $topicPartition in dir ${dir.getParent}") {
+        segments.foreach { segment =>
+          segment.deleteIfExists()
+        }
+      }
+    }
+
+    if (asyncDelete)
+      scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs)
+    else
+      deleteSegments()
+  }
+
+  /**
+   * This method deletes the given log segments by doing the following for each of them:
+   * <ol>
+   *   <li>It removes the segment from the segment map so that it will no longer be used for reads.
+   *   <li>It renames the index and log files by appending .deleted to the respective file name
+   *   <li>It can either schedule an asynchronous delete operation to occur in the future or perform the deletion synchronously
+   * </ol>
+   * Asynchronous deletion allows reads to happen concurrently without synchronization and without the possibility of
+   * physically deleting a file while it is being read.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded

Review comment:
       I've updated the doc now. The reason is the same the one I mentioned in the [above response](https://github.com/apache/kafka/pull/10280/files#r602052452).




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602056487



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1794,20 +1269,21 @@ class Log(@volatile private var _dir: File,
     ret.toSeq.sortBy(-_)
   }
 
-  /**
-    * Given a message offset, find its corresponding offset metadata in the log.
-    * If the message offset is out of range, throw an OffsetOutOfRangeException
-    */
-  private def convertToOffsetMetadataOrThrow(offset: Long): LogOffsetMetadata = {
-    val fetchDataInfo = read(offset,
-      maxLength = 1,
-      isolation = FetchLogEnd,
-      minOneMessage = false)
-    fetchDataInfo.fetchOffsetMetadata
+  def convertToOffsetMetadata(offset: Long): Option[LogOffsetMetadata] = {

Review comment:
       Done. Removed now. 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.

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



[GitHub] [kafka] junrao commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643445208



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1500,50 +1325,67 @@ class Log(@volatile private var _dir: File,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach)
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
       nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
     }
 
-    deleteOldSegments(shouldDelete, StartOffsetBreach)
+    deleteOldSegments(shouldDelete, StartOffsetBreach(this))
   }
 
   def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix)
 
   /**
    * The size of the log in bytes
    */
-  def size: Long = Log.sizeInBytes(logSegments)
+  def size: Long = localLog.segments.sizeInBytes
 
   /**
-   * The offset metadata of the next message that will be appended to the log
+   * The offset of the next message that will be appended to the log
    */
-  def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+  def logEndOffset: Long =  localLog.logEndOffset
 
   /**
-   * The offset of the next message that will be appended to the log
+   * The offset metadata of the next message that will be appended to the log
    */
-  def logEndOffset: Long = nextOffsetMetadata.messageOffset
+  def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata
+
+  private val rollAction = RollAction(

Review comment:
       Right, I don't think there is a particular reason that we have to take a snapshot before adding a new empty segment.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643449279



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1852,65 +1612,24 @@ class Log(@volatile private var _dir: File,
     logString.toString
   }
 
-  /**
-   * This method deletes the given log segments by doing the following for each of them:
-   * <ol>
-   *   <li>It removes the segment from the segment map so that it will no longer be used for reads.
-   *   <li>It renames the index and log files by appending .deleted to the respective file name
-   *   <li>It can either schedule an asynchronous delete operation to occur in the future or perform the deletion synchronously
-   * </ol>
-   * Asynchronous deletion allows reads to happen concurrently without synchronization and without the possibility of
-   * physically deleting a file while it is being read.
-   *
-   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
-   * or the immediate caller will catch and handle IOException
-   *
-   * @param segments The log segments to schedule for deletion
-   * @param asyncDelete Whether the segment files should be deleted asynchronously
-   */
-  private def removeAndDeleteSegments(segments: Iterable[LogSegment],
-                                      asyncDelete: Boolean,
-                                      reason: SegmentDeletionReason): Unit = {
-    if (segments.nonEmpty) {
-      lock synchronized {
-        // As 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.
-        val toDelete = segments.toList
-        reason.logReason(this, toDelete)
-        toDelete.foreach { segment =>
-          this.segments.remove(segment.baseOffset)
-        }
-        deleteSegmentFiles(toDelete, asyncDelete)
-      }
-    }
-  }
-
-  private def deleteSegmentFiles(segments: Iterable[LogSegment], asyncDelete: Boolean, deleteProducerStateSnapshots: Boolean = true): Unit = {
-    Log.deleteSegmentFiles(segments, asyncDelete, deleteProducerStateSnapshots, dir, topicPartition,
-      config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent)
-  }
-
   private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = {

Review comment:
       Great catch, I'll fix this.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648955649



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1812,37 +1570,39 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
+    localLog.updateLogEndOffset(endOffset)
     rebuildProducerState(endOffset, producerStateManager)
-    updateHighWatermark(math.min(highWatermark, endOffset))
+    if (highWatermark < localLog.logEndOffset)

Review comment:
       Done.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643436284



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1500,50 +1325,67 @@ class Log(@volatile private var _dir: File,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach)
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
       nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
     }
 
-    deleteOldSegments(shouldDelete, StartOffsetBreach)
+    deleteOldSegments(shouldDelete, StartOffsetBreach(this))
   }
 
   def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix)
 
   /**
    * The size of the log in bytes
    */
-  def size: Long = Log.sizeInBytes(logSegments)
+  def size: Long = localLog.segments.sizeInBytes
 
   /**
-   * The offset metadata of the next message that will be appended to the log
+   * The offset of the next message that will be appended to the log
    */
-  def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+  def logEndOffset: Long =  localLog.logEndOffset
 
   /**
-   * The offset of the next message that will be appended to the log
+   * The offset metadata of the next message that will be appended to the log
    */
-  def logEndOffset: Long = nextOffsetMetadata.messageOffset
+  def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata
+
+  private val rollAction = RollAction(

Review comment:
       Great point. I agree with you that `RollAction` is avoidable and it complicates the code a bit. The main part I wasn't sure about was why previously in the code we chose to take the producer state snapshot before the segment was rolled, instead of doing it after. Am I right that taking the snapshot prior to the roll is not a strict requirement for crash safety? (I'm not able to think of a case myself, but wanted to double check)




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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-854616475


   @junrao Thanks for the review! I've addressed your comments in e201295e03e0ea8a7102983888d1a7afc66d384a, and have also rebased this PR onto most recent commit in `trunk`. The only pending comment that needs a discussion is [this one](https://github.com/apache/kafka/pull/10280#discussion_r645441207). The PR is ready for review again.


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

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



[GitHub] [kafka] junrao commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648679167



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1812,37 +1570,39 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
+    localLog.updateLogEndOffset(endOffset)
     rebuildProducerState(endOffset, producerStateManager)
-    updateHighWatermark(math.min(highWatermark, endOffset))
+    if (highWatermark < localLog.logEndOffset)

Review comment:
       If highWatermark is smaller, there is no need to update high watermark.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1788,17 +1554,9 @@ class Log(@volatile private var _dir: File,
     maybeHandleIOException(s"Error while truncating the entire log for $topicPartition in dir ${dir.getParent}") {
       debug(s"Truncate and start at offset $newOffset")
       lock synchronized {
-        checkIfMemoryMappedBufferClosed()
-        removeAndDeleteSegments(logSegments, asyncDelete = true, LogTruncation)
-        addSegment(LogSegment.open(dir,
-          baseOffset = newOffset,
-          config = config,
-          time = time,
-          initFileSize = config.initFileSize,
-          preallocate = config.preallocate))
+        localLog.truncateFullyAndStartAt(newOffset)

Review comment:
       This is an existing issue. In this case, it seems that we should always update high watermark in completeTruncation() with localLog.logEndOffsetMetadata.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r646945793



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1578,144 +1409,85 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      val newSegment = localLog.roll(Some(rollOffset))
+      afterRoll(newSegment)
+      newSegment
     } else {
       segment
     }
   }
 
   /**
-   * Roll the log over to a new active segment starting with the current logEndOffset.
+   * Roll the local log over to a new active segment starting with the current logEndOffset.

Review comment:
       Sure, I'll fix it. 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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643514883



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1806,37 +1566,37 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
-    rebuildProducerState(endOffset, producerStateManager)
+    lock synchronized {
+      rebuildProducerState(endOffset, producerStateManager)
+    }

Review comment:
       Sounds good. I'll fix this.




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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-857548583


   Thanks for the review @junrao! I've addressed the comments in 8f14879. I've also triggered a system test run on the most recent commit, I'll review it once it completes. The link is attached to the PR description.


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r646945351



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1010 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.nio.file.Files
+import java.text.NumberFormat
+import java.util.concurrent.atomic.AtomicLong
+import java.util.regex.Pattern
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{Logging, Scheduler}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.Seq
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+
+/**
+ * Holds the result of splitting a segment into one or more segments, see LocalLog.splitOverflowedSegment().
+ *
+ * @param deletedSegments segments deleted when splitting a segment
+ * @param newSegments new segments created when splitting a segment
+ */
+case class SplitSegmentResult(deletedSegments: Iterable[LogSegment], newSegments: Iterable[LogSegment])
+
+/**
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @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
+ */
+private[log] class LocalLog(@volatile private var _dir: File,
+                            @volatile var config: LogConfig,
+                            val segments: LogSegments,
+                            @volatile var recoveryPoint: Long,
+                            @volatile private var nextOffsetMetadata: LogOffsetMetadata,
+                            val scheduler: Scheduler,
+                            val time: Time,
+                            val topicPartition: TopicPartition,
+                            val logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[LocalLog partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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.
+  @volatile private[log] var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  // Last time the log was flushed
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  private[log] def dir: File = _dir
+
+  private[log] def name: String = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(LocalLog.FutureDirSuffix)
+
+  private def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    LocalLog.maybeHandleIOException(logDirFailureChannel, parentDir, msg) {
+      fun
+    }
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        segments.updateParentDir(renamedDir)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  /**
+   * Update the existing configuration to the new provided configuration.
+   * @param newConfig the new configuration to be updated to
+   */
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = config
+    config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * Flush local log segments for all offsets up to offset-1.
+   * Does not update the recovery point.
+   *
+   * @param offset The offset to flush up to (non-inclusive)
+   */
+  private[log] def flush(offset: Long): Unit = {
+    val segmentsToFlush = segments.values(recoveryPoint, offset)
+    segmentsToFlush.foreach(_.flush())
+    // If there are any new segments, we need to flush the parent directory for crash consistency.
+    segmentsToFlush.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath))
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and update the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = 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.
+   */
+  private[log] def closeHandlers(): Unit = {
+    segments.closeHandlers()
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the segments of the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      segments.close()
+    }
+  }
+
+  /**
+   * Completely delete this log directory with no delay.
+   */
+  private[log] def deleteEmptyDir(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      if (segments.nonEmpty) {
+        throw new IllegalStateException(s"Can not delete directory when ${segments.numberOfSegments} segments are still present")
+      }
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true

Review comment:
       That's a good point. I'll move it there.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r645439645



##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {
+      log.close()
+    }
+    Utils.delete(tmpDir)
+  }
+
+  case class KeyValue(key: String, value: String) {
+    def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = {
+      new SimpleRecord(timestamp, key.getBytes, value.getBytes)
+    }
+  }
+
+  object KeyValue {
+    def fromRecord(record: Record): KeyValue = {
+      val key =
+        if (record.hasKey)
+          StandardCharsets.UTF_8.decode(record.key()).toString
+        else
+          ""
+      val value =
+        if (record.hasValue)
+          StandardCharsets.UTF_8.decode(record.value()).toString
+        else
+          ""
+      KeyValue(key, value)
+    }
+  }
+
+  private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = {
+    keyValues.map(kv => kv.toRecord())
+  }
+
+  private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = {
+    records.map(r => KeyValue.fromRecord(r))
+  }
+
+  private def appendRecords(records: Iterable[SimpleRecord],
+                            log: LocalLog = log,
+                            initialOffset: Long = 0L): Unit = {
+    log.append(lastOffset = initialOffset + records.size - 1,
+      largestTimestamp = records.head.timestamp,
+      shallowOffsetOfMaxTimestamp = initialOffset,
+      records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
+  }
+
+  private def readRecords(log: LocalLog = log,
+                          startOffset: Long = 0L,
+                          maxLength: => Int = log.segments.activeSegment.size,
+                          minOneMessage: Boolean = false,
+                          maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata,
+                          includeAbortedTxns: Boolean = false): FetchDataInfo = {
+    log.read(startOffset,
+             maxLength,
+             minOneMessage = minOneMessage,
+             maxOffsetMetadata,
+             includeAbortedTxns = includeAbortedTxns)
+  }
+
+  @Test
+  def testLogDeleteSuccess(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    assertFalse(logDir.listFiles.isEmpty)
+    val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values
+    val deletedSegments = log.delete()
+    assertTrue(log.segments.isEmpty)
+    assertEquals(segmentsBeforeDelete, deletedSegments)
+    assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed())
+    assertFalse(logDir.exists)
+  }
+
+  @Test
+  def testLogDeleteFailureAfterCloseHandlers(): Unit = {
+    log.closeHandlers()
+    assertEquals(1, log.segments.numberOfSegments)
+    val segmentsBeforeDelete = log.segments.values
+    assertThrows(classOf[KafkaStorageException], () => log.delete())
+    assertEquals(1, log.segments.numberOfSegments)
+    assertEquals(segmentsBeforeDelete, log.segments.values)
+    assertTrue(logDir.exists)
+  }
+
+  @Test
+  def testUpdateConfig(): Unit = {
+    val oldConfig = log.config
+    assertEquals(oldConfig, log.config)
+
+    val newConfig = createLogConfig()

Review comment:
       Done.

##########
File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.log
+
+import java.io.File
+import java.nio.channels.ClosedChannelException
+import java.nio.charset.StandardCharsets
+import java.util.regex.Pattern
+import java.util.{Collections, Properties}
+
+import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{MockTime, Scheduler, TestUtils}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.KafkaStorageException
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertFalse, _}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+
+import scala.jdk.CollectionConverters._
+
+class LocalLogTest {
+
+  import kafka.log.LocalLogTest._
+
+  var config: KafkaConfig = null
+  val tmpDir: File = TestUtils.tempDir()
+  val logDir: File = TestUtils.randomPartitionLogDir(tmpDir)
+  val topicPartition = new TopicPartition("test_topic", 1)
+  val logDirFailureChannel = new LogDirFailureChannel(10)
+  val mockTime = new MockTime()
+  val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig())
+
+  @BeforeEach
+  def setUp(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1)
+    config = KafkaConfig.fromProps(props)
+  }
+
+  @AfterEach
+  def tearDown(): Unit = {
+    if (!log.isMemoryMappedBufferClosed) {
+      log.close()
+    }
+    Utils.delete(tmpDir)
+  }
+
+  case class KeyValue(key: String, value: String) {
+    def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = {
+      new SimpleRecord(timestamp, key.getBytes, value.getBytes)
+    }
+  }
+
+  object KeyValue {
+    def fromRecord(record: Record): KeyValue = {
+      val key =
+        if (record.hasKey)
+          StandardCharsets.UTF_8.decode(record.key()).toString
+        else
+          ""
+      val value =
+        if (record.hasValue)
+          StandardCharsets.UTF_8.decode(record.value()).toString
+        else
+          ""
+      KeyValue(key, value)
+    }
+  }
+
+  private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = {
+    keyValues.map(kv => kv.toRecord())
+  }
+
+  private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = {
+    records.map(r => KeyValue.fromRecord(r))
+  }
+
+  private def appendRecords(records: Iterable[SimpleRecord],
+                            log: LocalLog = log,
+                            initialOffset: Long = 0L): Unit = {
+    log.append(lastOffset = initialOffset + records.size - 1,
+      largestTimestamp = records.head.timestamp,
+      shallowOffsetOfMaxTimestamp = initialOffset,
+      records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
+  }
+
+  private def readRecords(log: LocalLog = log,
+                          startOffset: Long = 0L,
+                          maxLength: => Int = log.segments.activeSegment.size,
+                          minOneMessage: Boolean = false,
+                          maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata,
+                          includeAbortedTxns: Boolean = false): FetchDataInfo = {
+    log.read(startOffset,
+             maxLength,
+             minOneMessage = minOneMessage,
+             maxOffsetMetadata,
+             includeAbortedTxns = includeAbortedTxns)
+  }
+
+  @Test
+  def testLogDeleteSuccess(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    assertFalse(logDir.listFiles.isEmpty)
+    val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values
+    val deletedSegments = log.delete()
+    assertTrue(log.segments.isEmpty)
+    assertEquals(segmentsBeforeDelete, deletedSegments)
+    assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed())
+    assertFalse(logDir.exists)
+  }
+
+  @Test
+  def testLogDeleteFailureAfterCloseHandlers(): Unit = {
+    log.closeHandlers()
+    assertEquals(1, log.segments.numberOfSegments)
+    val segmentsBeforeDelete = log.segments.values
+    assertThrows(classOf[KafkaStorageException], () => log.delete())
+    assertEquals(1, log.segments.numberOfSegments)
+    assertEquals(segmentsBeforeDelete, log.segments.values)
+    assertTrue(logDir.exists)
+  }
+
+  @Test
+  def testUpdateConfig(): Unit = {
+    val oldConfig = log.config
+    assertEquals(oldConfig, log.config)
+
+    val newConfig = createLogConfig()
+    log.updateConfig(newConfig)
+    assertEquals(newConfig, log.config)
+  }
+
+  @Test
+  def testLogDirRenameToNewDir(): Unit = {
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    log.roll()
+    assertEquals(2, log.segments.numberOfSegments)
+    val newLogDir = TestUtils.randomPartitionLogDir(tmpDir)
+    assertTrue(log.renameDir(newLogDir.getName))
+    assertFalse(logDir.exists())
+    assertTrue(newLogDir.exists())
+    assertEquals(newLogDir, log.dir)
+    assertEquals(newLogDir.getParent, log.parentDir)
+    assertEquals(newLogDir.getParent, log.dir.getParent)
+    log.segments.values.foreach(segment => assertEquals(newLogDir.getPath, segment.log.file().getParentFile.getPath))
+    assertEquals(2, log.segments.numberOfSegments)
+  }
+
+  @Test
+  def testLogDirRenameToExistingDir(): Unit = {
+    assertFalse(log.renameDir(log.dir.getName))
+  }
+
+  @Test
+  def testLogFlush(): Unit = {
+    assertEquals(0L, log.recoveryPoint)
+    assertEquals(mockTime.milliseconds, log.lastFlushTime)
+
+    val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+    appendRecords(List(record))
+    mockTime.sleep(1)
+    val newSegment = log.roll()
+    log.flush(newSegment.baseOffset)
+    log.markFlushed(newSegment.baseOffset)
+    assertEquals(1L, log.recoveryPoint)
+    assertEquals(mockTime.milliseconds, log.lastFlushTime)
+  }
+
+  @Test
+  def testLogAppend(): Unit = {
+    val fetchDataInfoBeforeAppend = readRecords(maxLength = 1)
+    assertTrue(fetchDataInfoBeforeAppend.records.records.asScala.isEmpty)
+
+    mockTime.sleep(1)
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    assertEquals(2L, log.logEndOffset)
+    assertEquals(0L, log.recoveryPoint)
+    val fetchDataInfo = readRecords()
+    assertEquals(2L, fetchDataInfo.records.records.asScala.size)
+    assertEquals(keyValues, recordsToKvs(fetchDataInfo.records.records.asScala))
+  }
+
+  @Test
+  def testLogCloseSuccess(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.close()
+    assertThrows(classOf[ClosedChannelException], () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L))
+  }
+
+  @Test
+  def testLogCloseIdempotent(): Unit = {
+    log.close()
+    // Check that LocalLog.close() is idempotent
+    log.close()
+  }
+
+  @Test
+  def testLogCloseFailureWhenInMemoryBufferClosed(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.closeHandlers()
+    assertThrows(classOf[KafkaStorageException], () => log.close())
+  }
+
+  @Test
+  def testLogCloseHandlers(): Unit = {
+    val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE"))
+    appendRecords(kvsToRecords(keyValues))
+    log.closeHandlers()
+    assertThrows(classOf[ClosedChannelException],
+                 () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L))
+  }
+
+  @Test
+  def testLogCloseHandlersIdempotent(): Unit = {
+    log.closeHandlers()
+    // Check that LocalLog.closeHandlers() is idempotent
+    log.closeHandlers()
+  }
+
+  private def testRemoveAndDeleteSegments(asyncDelete: Boolean): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    class TestDeletionReason extends SegmentDeletionReason {
+      private var _deletedSegments: Iterable[LogSegment] = List[LogSegment]()
+
+      override def logReason(toDelete: List[LogSegment]): Unit = {
+        _deletedSegments = List[LogSegment]() ++ toDelete
+      }
+
+      def deletedSegments: Iterable[LogSegment] = _deletedSegments
+    }
+    val reason = new TestDeletionReason()
+    val toDelete = List[LogSegment]() ++ log.segments.values
+    log.removeAndDeleteSegments(toDelete, asyncDelete = asyncDelete, reason)
+    if (asyncDelete) {
+      mockTime.sleep(log.config.fileDeleteDelayMs + 1)
+    }
+    assertTrue(log.segments.isEmpty)
+    assertEquals(toDelete, reason.deletedSegments)
+    toDelete.foreach(segment => assertTrue(segment.deleted()))
+  }
+
+  @Test
+  def testRemoveAndDeleteSegmentsSync(): Unit = {
+    testRemoveAndDeleteSegments(asyncDelete = false)
+  }
+
+  @Test
+  def testRemoveAndDeleteSegmentsAsync(): Unit = {
+    testRemoveAndDeleteSegments(asyncDelete = true)
+  }
+
+  private def testDeleteSegmentFiles(asyncDelete: Boolean): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    val toDelete = List[LogSegment]() ++ log.segments.values
+    LocalLog.deleteSegmentFiles(toDelete, asyncDelete = asyncDelete, log.dir, log.topicPartition, log.config, log.scheduler, log.logDirFailureChannel, "")
+    if (asyncDelete) {
+      toDelete.foreach {
+        segment =>
+          assertFalse(segment.deleted())
+          assertTrue(segment.hasSuffix(LocalLog.DeletedFileSuffix))
+      }
+      mockTime.sleep(log.config.fileDeleteDelayMs + 1)
+    }
+    toDelete.foreach(segment => assertTrue(segment.deleted()))
+  }
+
+  @Test
+  def testDeleteSegmentFilesSync(): Unit = {
+    testDeleteSegmentFiles(asyncDelete = false)
+  }
+
+  @Test
+  def testDeleteSegmentFilesAsync(): Unit = {
+    testDeleteSegmentFiles(asyncDelete = true)
+  }
+
+  @Test
+  def testDeletableSegmentsFilter(): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    {
+      val deletable = log.deletableSegments(
+        (segment: LogSegment, _: Option[LogSegment], _: Long) => segment.baseOffset <= 5)
+      val expected = log.segments.nonActiveLogSegmentsFrom(0L).filter(segment => segment.baseOffset <= 5)
+      assertEquals(expected, deletable.toList)
+    }
+
+    {
+      val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true)
+      val expected = log.segments.nonActiveLogSegmentsFrom(0L).toList
+      assertEquals(expected, deletable.toList)
+    }
+
+    {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = 9L)
+      val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true)
+      val expected = log.segments.values.toList
+      assertEquals(expected, deletable.toList)
+    }
+  }
+
+  @Test
+  def testDeletableSegmentsIteration(): Unit = {
+    for (offset <- 0 to 8) {
+      val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes)
+      appendRecords(List(record), initialOffset = offset)
+      log.roll()
+    }
+
+    assertEquals(10L, log.segments.numberOfSegments)
+
+    var offset = 0
+    log.deletableSegments(
+      (segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long) => {
+        assertEquals(offset, segment.baseOffset)
+        val floorSegmentOpt = log.segments.floorSegment(offset)
+        assertTrue(floorSegmentOpt.isDefined)
+        assertEquals(floorSegmentOpt.get, segment)
+        if (offset == log.logEndOffset) {
+          assertFalse(nextSegmentOpt.isDefined)
+        } else {
+          assertTrue(nextSegmentOpt.isDefined)
+          val higherSegmentOpt = log.segments.higherSegment(segment.baseOffset)
+          assertTrue(higherSegmentOpt.isDefined)
+          assertEquals(segment.baseOffset + 1, higherSegmentOpt.get.baseOffset)
+          assertEquals(higherSegmentOpt.get, nextSegmentOpt.get)
+        }
+        assertEquals(log.logEndOffset, logEndOffset)
+        offset += 1
+        true
+      })

Review comment:
       Done.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602048606



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator

Review comment:
       Done. Removed now.




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

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



[GitHub] [kafka] ijuma commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-879991378


   What is the reason for including a refactoring in 3.0 after the feature freeze?


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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602041263



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.

Review comment:
       Done. Removed now.




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-865332690


   @junrao @dhruvilshah3 I ran a perf test against a Broker build with and without this PR. The test involved the following:
   
   1. Created a test topic with 1 partition and replication factor 1 using the command: `$> ./bin/kafka-topics.sh --bootstrap-server localhost:9092 --create --topic kowshik-test-1 --partitions 1 --replication-factor 1`.
   2. Ran `kafka-producer-perf-test.sh` to produce 10M messages each of size 1KB and with max producer throughput 100K to the above topic. Command: `$> ./bin/kafka-producer-perf-test.sh --num-records 10000000 --print-metrics --producer-props bootstrap.servers=localhost:9092 --record-size 1024 --throughput 100000 --topic kowshik-test-1`.
   3. In parallel, ran `kafka-consumer-perf-test.sh` to consume the 10M messages that were produced in (2) using the command: `$> bin/kafka-consumer-perf-test.sh --topic kowshik-test-1 --bootstrap-server localhost:9092 --messages 10000000 --print-metrics --show-detailed-stats`
   
   The tests have similar results, meaning that the performance with and without this PR looks similar. Here are the results:
   
   
   **log.segment.bytes=10MB**
   
   - Without this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/0ea1ae9ac8210f4bba49967727ddb475
   - With this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/c5ec0fb92679bd91613f520455446bf5
   
   **log.segment.bytes=100MB**
   
   - Without this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/2aaa7113fd05e10721c60aaf9bf8c654
   - With this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/b09882c2b13930be2efc69554c31aded
   
   **log.segment.bytes=1GB**
   
   - Without this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/ea36153c9751180c5dbe383b189d50df
   - With this PR on top of c333bfd41766bf33f41b5d32d8959ebbeff240b4: https://gist.github.com/kowshik/133803d10d510df93f1d15858e91035a


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

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



[GitHub] [kafka] kowshik edited a comment on pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik edited a comment on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-808514958


   @junrao Just a heads up on the following. I'm working on the changes for the following in separate PRs, these are related with refactoring the recovery logic (KAFKA-12553):
    * KAFKA-12552 (https://github.com/apache/kafka/pull/10401) to extract segments map
    * KAFKA-12571: (https://github.com/apache/kafka/pull/10426) to eliminate LeaderEpochFileCache constructor dependency on logEndOffset
    * KAFKA-12575: (https://github.com/apache/kafka/pull/10430) to eliminate Log.isLogDirOffline boolean attribute
   
   It seems better if we merge those into trunk ahead of the current PR.


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602062784



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1816,8 +1292,12 @@ class Log(@volatile private var _dir: File,
    */
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
+    def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long): Boolean = {
+      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(logEndOffset) &&

Review comment:
       This is to accomodate for the hwm check that was previously happening in `Log#deletableSegments` in [this line](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/log/Log.scala#L1872). The `deletableSegments` method has now moved to `LocalLog`, but we can't do the hwm check inside `LocalLog` since hwm is still owned by `Log`. So we piggyback on the predicate here to additionally attach the hwm check.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648134244



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1788,14 +1559,8 @@ class Log(@volatile private var _dir: File,
     maybeHandleIOException(s"Error while truncating the entire log for $topicPartition in dir ${dir.getParent}") {
       debug(s"Truncate and start at offset $newOffset")
       lock synchronized {
-        checkIfMemoryMappedBufferClosed()
-        removeAndDeleteSegments(logSegments, asyncDelete = true, LogTruncation)
-        addSegment(LogSegment.open(dir,
-          baseOffset = newOffset,
-          config = config,
-          time = time,
-          initFileSize = config.initFileSize,
-          preallocate = config.preallocate))
+        val deletedSegments = localLog.truncateFullyAndStartAt(newOffset)
+        deleteProducerSnapshots(deletedSegments, asyncDelete = true)

Review comment:
       Done in 8f14879.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1812,37 +1577,36 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
+    localLog.updateLogEndOffset(endOffset)

Review comment:
       Done in 8f14879.

##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1010 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.nio.file.Files
+import java.text.NumberFormat
+import java.util.concurrent.atomic.AtomicLong
+import java.util.regex.Pattern
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{Logging, Scheduler}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.Seq
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+
+/**
+ * Holds the result of splitting a segment into one or more segments, see LocalLog.splitOverflowedSegment().
+ *
+ * @param deletedSegments segments deleted when splitting a segment
+ * @param newSegments new segments created when splitting a segment
+ */
+case class SplitSegmentResult(deletedSegments: Iterable[LogSegment], newSegments: Iterable[LogSegment])
+
+/**
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @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
+ */
+private[log] class LocalLog(@volatile private var _dir: File,
+                            @volatile var config: LogConfig,
+                            val segments: LogSegments,
+                            @volatile var recoveryPoint: Long,
+                            @volatile private var nextOffsetMetadata: LogOffsetMetadata,
+                            val scheduler: Scheduler,
+                            val time: Time,
+                            val topicPartition: TopicPartition,
+                            val logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[LocalLog partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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.
+  @volatile private[log] var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  // Last time the log was flushed
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  private[log] def dir: File = _dir
+
+  private[log] def name: String = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(LocalLog.FutureDirSuffix)
+
+  private def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    LocalLog.maybeHandleIOException(logDirFailureChannel, parentDir, msg) {
+      fun
+    }
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        segments.updateParentDir(renamedDir)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  /**
+   * Update the existing configuration to the new provided configuration.
+   * @param newConfig the new configuration to be updated to
+   */
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = config
+    config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * Flush local log segments for all offsets up to offset-1.
+   * Does not update the recovery point.
+   *
+   * @param offset The offset to flush up to (non-inclusive)
+   */
+  private[log] def flush(offset: Long): Unit = {
+    val segmentsToFlush = segments.values(recoveryPoint, offset)
+    segmentsToFlush.foreach(_.flush())
+    // If there are any new segments, we need to flush the parent directory for crash consistency.
+    segmentsToFlush.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath))
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and update the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = 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.
+   */
+  private[log] def closeHandlers(): Unit = {
+    segments.closeHandlers()
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the segments of the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      segments.close()
+    }
+  }
+
+  /**
+   * Completely delete this log directory with no delay.
+   */
+  private[log] def deleteEmptyDir(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      if (segments.nonEmpty) {
+        throw new IllegalStateException(s"Can not delete directory when ${segments.numberOfSegments} segments are still present")
+      }
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true

Review comment:
       Done in 8f14879.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1578,144 +1409,85 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      val newSegment = localLog.roll(Some(rollOffset))
+      afterRoll(newSegment)
+      newSegment
     } else {
       segment
     }
   }
 
   /**
-   * Roll the log over to a new active segment starting with the current logEndOffset.
+   * Roll the local log over to a new active segment starting with the current logEndOffset.

Review comment:
       Done in 8f14879.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1578,144 +1409,85 @@ class Log(@volatile private var _dir: File,
         .map(_.messageOffset)
         .getOrElse(maxOffsetInMessages - Integer.MAX_VALUE)
 
-      roll(Some(rollOffset))
+      val newSegment = localLog.roll(Some(rollOffset))
+      afterRoll(newSegment)
+      newSegment

Review comment:
       Done in 8f14879.




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r643433770



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1500,50 +1325,67 @@ class Log(@volatile private var _dir: File,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach)
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
       nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
     }
 
-    deleteOldSegments(shouldDelete, StartOffsetBreach)
+    deleteOldSegments(shouldDelete, StartOffsetBreach(this))
   }
 
   def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix)
 
   /**
    * The size of the log in bytes
    */
-  def size: Long = Log.sizeInBytes(logSegments)
+  def size: Long = localLog.segments.sizeInBytes
 
   /**
-   * The offset metadata of the next message that will be appended to the log
+   * The offset of the next message that will be appended to the log
    */
-  def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+  def logEndOffset: Long =  localLog.logEndOffset
 
   /**
-   * The offset of the next message that will be appended to the log
+   * The offset metadata of the next message that will be appended to the log
    */
-  def logEndOffset: Long = nextOffsetMetadata.messageOffset
+  def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata
+
+  private val rollAction = RollAction(
+    preRollAction = (newSegment: LogSegment) => {
+      // Take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
+      // offset align with the new segment offset since this ensures we can recover the segment by beginning
+      // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
+      // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
+      // we manually override the state offset here prior to taking the snapshot.
+      producerStateManager.updateMapEndOffset(newSegment.baseOffset)
+      producerStateManager.takeSnapshot()
+    },
+    postRollAction = (newSegment: LogSegment, deletedSegment: Option[LogSegment]) => {
+      deletedSegment.foreach(segment => deleteProducerSnapshotAsync(Seq(segment)))

Review comment:
       This is a great catch. I agree with you. While I can address it in this PR, should we create a separate JIRA for it?




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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KAFKA-12554: Refactor Log layer

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r648133704



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1010 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.nio.file.Files
+import java.text.NumberFormat
+import java.util.concurrent.atomic.AtomicLong
+import java.util.regex.Pattern
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{Logging, Scheduler}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.Seq
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+
+/**
+ * Holds the result of splitting a segment into one or more segments, see LocalLog.splitOverflowedSegment().
+ *
+ * @param deletedSegments segments deleted when splitting a segment
+ * @param newSegments new segments created when splitting a segment
+ */
+case class SplitSegmentResult(deletedSegments: Iterable[LogSegment], newSegments: Iterable[LogSegment])
+
+/**
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @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
+ */
+private[log] class LocalLog(@volatile private var _dir: File,
+                            @volatile var config: LogConfig,
+                            val segments: LogSegments,
+                            @volatile var recoveryPoint: Long,
+                            @volatile private var nextOffsetMetadata: LogOffsetMetadata,
+                            val scheduler: Scheduler,
+                            val time: Time,
+                            val topicPartition: TopicPartition,
+                            val logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[LocalLog partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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.
+  @volatile private[log] var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  // Last time the log was flushed
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  private[log] def dir: File = _dir
+
+  private[log] def name: String = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(LocalLog.FutureDirSuffix)
+
+  private def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    LocalLog.maybeHandleIOException(logDirFailureChannel, parentDir, msg) {
+      fun
+    }
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        segments.updateParentDir(renamedDir)
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  /**
+   * Update the existing configuration to the new provided configuration.
+   * @param newConfig the new configuration to be updated to
+   */
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = config
+    config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * Flush local log segments for all offsets up to offset-1.
+   * Does not update the recovery point.
+   *
+   * @param offset The offset to flush up to (non-inclusive)
+   */
+  private[log] def flush(offset: Long): Unit = {
+    val segmentsToFlush = segments.values(recoveryPoint, offset)
+    segmentsToFlush.foreach(_.flush())
+    // If there are any new segments, we need to flush the parent directory for crash consistency.
+    segmentsToFlush.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath))
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and update the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = 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.
+   */
+  private[log] def closeHandlers(): Unit = {
+    segments.closeHandlers()
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the segments of the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      segments.close()
+    }
+  }
+
+  /**
+   * Completely delete this log directory with no delay.
+   */
+  private[log] def deleteEmptyDir(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      if (segments.nonEmpty) {
+        throw new IllegalStateException(s"Can not delete directory when ${segments.numberOfSegments} segments are still present")
+      }
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true

Review comment:
       Done in 8ebb39f.

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -1812,37 +1577,36 @@ class Log(@volatile private var _dir: File,
     endOffset: Long
   ): Unit = {
     logStartOffset = startOffset
-    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
-    recoveryPoint = math.min(recoveryPoint, endOffset)
+    localLog.updateLogEndOffset(endOffset)

Review comment:
       Done in 8ebb39f.




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

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



[GitHub] [kafka] kowshik commented on pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#issuecomment-808012495


   @junrao Thanks a lot for the review! I've addressed your comments in 63be325b8b3ff5514e22137bb02dfc70e10f7a72.


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

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602071574



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -836,178 +581,15 @@ class Log(@volatile private var _dir: File,
       updateHighWatermark(offset)
     }
 
-    if (this.recoveryPoint < offset) {
-      this.recoveryPoint = offset
-    }
-  }
-
-  /**
-   * Recover the log segments and return the next offset after recovery.
-   * This method does not need to convert IOException to KafkaStorageException because it is only called before all
-   * logs are loaded.
-   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
-   */
-  private[log] def recoverLog(): Long = {
-    /** return the log end offset if valid */
-    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
-      if (logSegments.nonEmpty) {
-        val logEndOffset = activeSegment.readNextOffset
-        if (logEndOffset >= logStartOffset)
-          Some(logEndOffset)
-        else {
-          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
-            "This could happen if segment files were deleted from the file system.")
-          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
-          leaderEpochCache.foreach(_.clearAndFlush())
-          producerStateManager.truncateFullyAndStartAt(logStartOffset)
-          None
-        }
-      } else None
-    }
-
-    // if we have the clean shutdown marker, skip recovery
-    if (!hadCleanShutdown) {
-      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
-      var truncated = false
-
-      while (unflushed.hasNext && !truncated) {
-        val segment = unflushed.next()
-        info(s"Recovering unflushed segment ${segment.baseOffset}")
-        val truncatedBytes =
-          try {
-            recoverSegment(segment, leaderEpochCache)
-          } catch {
-            case _: InvalidOffsetException =>
-              val startOffset = segment.baseOffset
-              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
-                s"creating an empty one with starting offset $startOffset")
-              segment.truncateTo(startOffset)
-          }
-        if (truncatedBytes > 0) {
-          // we had an invalid message, delete all remaining log
-          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
-          removeAndDeleteSegments(unflushed.toList,
-            asyncDelete = true,
-            reason = LogRecovery)
-          truncated = true
-        }
-      }
-    }
-
-    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
-
-    if (logSegments.isEmpty) {
-      // no existing segments, create a new mutable segment beginning at logStartOffset
-      addSegment(LogSegment.open(dir = dir,
-        baseOffset = logStartOffset,
-        config,
-        time = time,
-        initFileSize = this.initFileSize,
-        preallocate = config.preallocate))
-    }
-
-    // Update the recovery point if there was a clean shutdown and did not perform any changes to
-    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
-    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
-    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
-    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
-    // segment.
-    (hadCleanShutdown, logEndOffsetOption) match {
-      case (true, Some(logEndOffset)) =>
-        recoveryPoint = logEndOffset
-        logEndOffset
-      case _ =>
-        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
-        recoveryPoint = Math.min(recoveryPoint, logEndOffset)
-        logEndOffset
-    }
-  }
-
-  // Rebuild producer state until lastOffset. This method may be called from the recovery code path, and thus must be
-  // free of all side-effects, i.e. it must not update any log-specific state.
-  private def rebuildProducerState(lastOffset: Long,
-                                   reloadFromCleanShutdown: Boolean,
-                                   producerStateManager: ProducerStateManager): Unit = lock synchronized {
-    checkIfMemoryMappedBufferClosed()
-    val segments = logSegments
-    val offsetsToSnapshot =
-      if (segments.nonEmpty) {
-        val nextLatestSegmentBaseOffset = lowerSegment(segments.last.baseOffset).map(_.baseOffset)
-        Seq(nextLatestSegmentBaseOffset, Some(segments.last.baseOffset), Some(lastOffset))
-      } else {
-        Seq(Some(lastOffset))
-      }
-    info(s"Loading producer state till offset $lastOffset with message format version ${recordVersion.value}")
-
-    // We want to avoid unnecessary scanning of the log to build the producer state when the broker is being
-    // upgraded. The basic idea is to use the absence of producer snapshot files to detect the upgrade case,
-    // but we have to be careful not to assume too much in the presence of broker failures. The two most common
-    // upgrade cases in which we expect to find no snapshots are the following:
-    //
-    // 1. The broker has been upgraded, but the topic is still on the old message format.
-    // 2. The broker has been upgraded, the topic is on the new message format, and we had a clean shutdown.
-    //
-    // If we hit either of these cases, we skip producer state loading and write a new snapshot at the log end
-    // offset (see below). The next time the log is reloaded, we will load producer state using this snapshot
-    // (or later snapshots). Otherwise, if there is no snapshot file, then we have to rebuild producer state
-    // from the first segment.
-    if (recordVersion.value < RecordBatch.MAGIC_VALUE_V2 ||
-        (producerStateManager.latestSnapshotOffset.isEmpty && reloadFromCleanShutdown)) {
-      // To avoid an expensive scan through all of the segments, we take empty snapshots from the start of the
-      // last two segments and the last offset. This should avoid the full scan in the case that the log needs
-      // truncation.
-      offsetsToSnapshot.flatten.foreach { offset =>
-        producerStateManager.updateMapEndOffset(offset)
-        producerStateManager.takeSnapshot()
-      }
-    } else {
-      info(s"Reloading from producer snapshot and rebuilding producer state from offset $lastOffset")
-      val isEmptyBeforeTruncation = producerStateManager.isEmpty && producerStateManager.mapEndOffset >= lastOffset
-      val producerStateLoadStart = time.milliseconds()
-      producerStateManager.truncateAndReload(logStartOffset, lastOffset, time.milliseconds())
-      val segmentRecoveryStart = time.milliseconds()
-
-      // Only do the potentially expensive reloading if the last snapshot offset is lower than the log end
-      // offset (which would be the case on first startup) and there were active producers prior to truncation
-      // (which could be the case if truncating after initial loading). If there weren't, then truncating
-      // shouldn't change that fact (although it could cause a producerId to expire earlier than expected),
-      // and we can skip the loading. This is an optimization for users which are not yet using
-      // idempotent/transactional features yet.
-      if (lastOffset > producerStateManager.mapEndOffset && !isEmptyBeforeTruncation) {
-        val segmentOfLastOffset = floorLogSegment(lastOffset)
-
-        logSegments(producerStateManager.mapEndOffset, lastOffset).foreach { segment =>
-          val startOffset = Utils.max(segment.baseOffset, producerStateManager.mapEndOffset, logStartOffset)
-          producerStateManager.updateMapEndOffset(startOffset)
-
-          if (offsetsToSnapshot.contains(Some(segment.baseOffset)))
-            producerStateManager.takeSnapshot()
-
-          val maxPosition = if (segmentOfLastOffset.contains(segment)) {
-            Option(segment.translateOffset(lastOffset))
-              .map(_.position)
-              .getOrElse(segment.size)
-          } else {
-            segment.size
-          }
-
-          val fetchDataInfo = segment.read(startOffset,
-            maxSize = Int.MaxValue,
-            maxPosition = maxPosition,
-            minOneMessage = false)
-          if (fetchDataInfo != null)
-            loadProducersFromRecords(producerStateManager, fetchDataInfo.records)
-        }
-      }
-      producerStateManager.updateMapEndOffset(lastOffset)
-      producerStateManager.takeSnapshot()
-      info(s"Producer state recovery took ${producerStateLoadStart - segmentRecoveryStart}ms for snapshot load " +
-        s"and ${time.milliseconds() - segmentRecoveryStart}ms for segment recovery from offset $lastOffset")
+    if (localLog.recoveryPoint < offset) {
+      localLog.updateRecoveryPoint(offset)
     }
   }
 
   private def loadProducerState(lastOffset: Long, reloadFromCleanShutdown: Boolean): Unit = lock synchronized {
-    rebuildProducerState(lastOffset, reloadFromCleanShutdown, producerStateManager)
+    lock synchronized {

Review comment:
       Done. 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.

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



[GitHub] [kafka] kowshik commented on a change in pull request #10280: KIP-405: Log layer refactor

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10280:
URL: https://github.com/apache/kafka/pull/10280#discussion_r602052452



##########
File path: core/src/main/scala/kafka/log/LocalLog.scala
##########
@@ -0,0 +1,1561 @@
+/**
+ * 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.log
+
+import java.io.{File, IOException}
+import java.lang.{Long => JLong}
+import java.nio.file.{Files, NoSuchFileException}
+import java.text.NumberFormat
+import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
+import java.util.concurrent.atomic.AtomicLong
+import java.util.Map.{Entry => JEntry}
+import java.util.regex.Pattern
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
+import kafka.utils.{CoreUtils, Logging, Scheduler, threadsafe}
+import org.apache.kafka.common.{KafkaException, TopicPartition}
+import org.apache.kafka.common.errors.{InvalidOffsetException, KafkaStorageException, OffsetOutOfRangeException}
+import org.apache.kafka.common.message.FetchResponseData
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion, Records}
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.jdk.CollectionConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.{Seq, Set, mutable}
+
+// Used to define pre/post roll actions to be performed.
+case class RollAction(preRollAction: Long => Unit, postRollAction: (LogSegment, Option[LogSegment]) => Unit)
+
+// Used to hold the result of splitting a segment into one or more segments, see LocalLog#splitOverflowedSegment
+case class SplitSegmentResult(deletedSegments: Seq[LogSegment], newSegments: Seq[LogSegment])
+
+/**
+ * An append-only log for storing messages locally.
+ * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment.
+ * 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, and it relies on the thread safety provided by the Log class.
+ *
+ * @param _dir The directory in which log segments are created.
+ * @param config The log configuration settings
+ * @param recoveryPoint The offset at which to begin recovery i.e. the first offset which has not been flushed to disk
+ * @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
+ * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means
+ *                         clean shutdown whereas false means a crash.
+ */
+class LocalLog(@volatile private var _dir: File,
+               @volatile var config: LogConfig,
+               @volatile var recoveryPoint: Long,
+               scheduler: Scheduler,
+               val time: Time,
+               val topicPartition: TopicPartition,
+               logDirFailureChannel: LogDirFailureChannel,
+               private val hadCleanShutdown: Boolean = true) extends Logging with KafkaMetricsGroup {
+
+  import kafka.log.LocalLog._
+
+  this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] "
+
+  // 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
+  @volatile private var isMemoryMappedBufferClosed = false
+
+  // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks
+  @volatile private var _parentDir: String = dir.getParent
+
+  /* last time the log was flushed */
+  private val lastFlushedTime = new AtomicLong(time.milliseconds)
+
+  // The offset where the next message could be appended
+  @volatile private var nextOffsetMetadata: LogOffsetMetadata = _
+
+  // Log dir failure is handled asynchronously we need to prevent threads
+  // from reading inconsistent state caused by a failure in another thread
+  @volatile private[log] var logDirOffline = false
+
+  // The actual segments of the log
+  private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
+
+  locally {
+    // Create the log directory if it doesn't exist
+    Files.createDirectories(dir.toPath)
+  }
+
+  private[log] def dir: File = _dir
+
+  private[log] def name = dir.getName()
+
+  private[log] def parentDir: String = _parentDir
+
+  private[log] def parentDirFile: File = new File(_parentDir)
+
+  private[log] def isFuture: Boolean = dir.getName.endsWith(FutureDirSuffix)
+
+  private[log] def initFileSize: Int = {
+    if (config.preallocate)
+      config.segmentSize
+    else
+      0
+  }
+
+  /**
+   * Rename the directory of the log
+   * @param name the new dir name
+   * @throws KafkaStorageException if rename fails
+   */
+  private[log] def renameDir(name: String): Boolean = {
+    maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") {
+      val renamedDir = new File(dir.getParent, name)
+      Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath)
+      if (renamedDir != dir) {
+        _dir = renamedDir
+        _parentDir = renamedDir.getParent
+        logSegments.foreach(_.updateParentDir(renamedDir))
+        true
+      } else {
+        false
+      }
+    }
+  }
+
+  private[log] def updateConfig(newConfig: LogConfig): Unit = {
+    val oldConfig = this.config
+    this.config = newConfig
+    val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion
+    val newRecordVersion = newConfig.messageFormatVersion.recordVersion
+    if (newRecordVersion.precedes(oldRecordVersion))
+      warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.")
+  }
+
+  private[log] def checkIfMemoryMappedBufferClosed(): Unit = {
+    if (isMemoryMappedBufferClosed)
+      throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed")
+  }
+
+  private[log] def checkForLogDirFailure(): Unit = {
+    if (logDirOffline) {
+      throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.")
+    }
+  }
+
+  private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = {
+    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
+   */
+  private[log] def markFlushed(offset: Long): Unit = {
+    checkIfMemoryMappedBufferClosed()
+    if (offset > recoveryPoint) {
+      updateRecoveryPoint(offset)
+      lastFlushedTime.set(time.milliseconds)
+    }
+  }
+
+  /**
+   * The time this log is last known to have been fully flushed to disk
+   */
+  private[log] def lastFlushTime: Long = lastFlushedTime.get
+
+  /**
+   * The offset metadata of the next message that will be appended to the log
+   */
+  private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
+
+  /**
+   * The offset of the next message that will be appended to the log
+   */
+  private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset
+
+  /**
+   * Update end offset of the log, and the recoveryPoint.
+   *
+   * @param endOffset the new end offset of the log
+   */
+  private[log] def updateLogEndOffset(endOffset: Long): Unit = {
+    nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size)
+    if (recoveryPoint > endOffset) {
+      updateRecoveryPoint(endOffset)
+    }
+  }
+
+  /**
+   * @return the base offset of the first local segment, if it exists
+   */
+  private[log] def firstSegmentBaseOffset: Option[Long] = Option(segments.firstEntry).map(_.getValue.baseOffset)
+
+  /**
+   * The active segment that is currently taking appends
+   */
+  private[log] def activeSegment = segments.lastEntry.getValue
+
+  /**
+   * The number of segments in the log.
+   * Take care! this is an O(n) operation.
+   */
+  private[log] def numberOfSegments: Int = segments.size
+
+  /**
+   * The size of the log in bytes
+   */
+  private[log] def size: Long = LocalLog.sizeInBytes(logSegments)
+
+  /**
+   * All the log segments in this log ordered from oldest to newest
+   */
+  private[log] def logSegments: Iterable[LogSegment] = segments.values.asScala
+
+  /**
+   * Get all segments beginning with the segment that includes "from" and ending with the segment
+   * that includes up to "to-1" or the end of the log (if to > logEndOffset).
+   */
+  private[log] def logSegments(from: Long, to: Long): Iterable[LogSegment] = {
+    if (from == to) {
+      // Handle non-segment-aligned empty sets
+      List.empty[LogSegment]
+    } else if (to < from) {
+      throw new IllegalArgumentException(s"Invalid log segment range: requested segments in $topicPartition " +
+        s"from offset $from which is greater than limit offset $to")
+    } else {
+      val view = Option(segments.floorKey(from)).map { floor =>
+        segments.subMap(floor, to)
+      }.getOrElse(segments.headMap(to))
+      view.values.asScala
+    }
+  }
+
+  /**
+   * Return all non-active log segments beginning with the segment that includes "from".
+   *
+   * @param from the from offset
+   */
+  private[log] def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = {
+    if (from > activeSegment.baseOffset)
+      Seq.empty
+    else
+      logSegments(from, activeSegment.baseOffset)
+  }
+
+  private[log]  def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion
+
+  private[log]  def lowerSegment(offset: Long): Option[LogSegment] =
+    Option(segments.lowerEntry(offset)).map(_.getValue)
+
+  /**
+   * Get the largest log segment with a base offset less than or equal to the given offset, if one exists.
+   * @return the optional log segment
+   */
+  private[log]  def floorLogSegment(offset: Long): Option[LogSegment] = {
+    Option(segments.floorEntry(offset)).map(_.getValue)
+  }
+
+  /**
+   * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it.
+   * @param segment The segment to add
+   */
+  @threadsafe
+  private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.put(segment.baseOffset, segment)
+
+  /**
+   * Clears all segments
+   */
+  private[log] def clearSegments(): Unit = segments.clear()
+
+  /**
+   * Closes all segments
+   */
+  private[log] def closeSegments(): Unit = {
+    logSegments.foreach(_.close())
+  }
+
+  /**
+   * Close file handlers used by log but don't write to disk. This is called if the log directory is offline
+   */
+  private[log] def closeHandlers(): Unit = {
+    logSegments.foreach(_.closeHandlers())
+    isMemoryMappedBufferClosed = true
+  }
+
+  /**
+   * Closes the log.
+   */
+  private[log] def close(): Unit = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      closeSegments()
+    }
+  }
+
+  /**
+   * Completely delete this log directory and all contents from the file system with no delay
+   */
+  private[log] def delete(): Seq[LogSegment] = {
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      checkIfMemoryMappedBufferClosed()
+      val deleted = logSegments.toSeq
+      removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion)
+      Utils.delete(dir)
+      // File handlers will be closed if this log is deleted
+      isMemoryMappedBufferClosed = true
+      deleted
+    }
+  }
+
+  /**
+   * Load the log segments from the log files on disk and update the next offset.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all logs
+   * are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments
+   *
+   * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when
+   *                                           we find an unexpected number of .log files with overflow
+   */
+  private[log] def loadSegments(logStartOffset: Long,
+                                maxProducerIdExpirationMs: Int,
+                                producerStateManager: ProducerStateManager,
+                                leaderEpochCache: Option[LeaderEpochFileCache]): Seq[LogSegment] = {
+    // first do a pass through the files in the log directory and remove any temporary files
+    // and find any interrupted swap operations
+    val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+    // Now do a second pass and load all the log and index files.
+    // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When
+    // this happens, restart loading segment files from scratch.
+    retryOnOffsetOverflow({
+      // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry
+      // loading of segments. In that case, we also need to close all segments that could have been left open in previous
+      // call to loadSegmentFiles().
+      logSegments.foreach(_.close())
+      segments.clear()
+      loadSegmentFiles(logStartOffset, maxProducerIdExpirationMs)
+    })
+
+    val deletedSegments = ListBuffer[LogSegment]()
+
+    // Finally, complete any interrupted swap operations. To be crash-safe,
+    // log files that are replaced by the swap segment should be renamed to .deleted
+    // before the swap file is restored as the new segment file.
+    deletedSegments ++= completeSwapOperations(swapFiles, logStartOffset, maxProducerIdExpirationMs)
+
+    if (!dir.getAbsolutePath.endsWith(DeleteDirSuffix)) {
+      val (deleted, nextOffset) = retryOnOffsetOverflow(
+        {
+          recoverLog(logStartOffset,
+                     maxProducerIdExpirationMs,
+                     producerStateManager,
+                     leaderEpochCache)
+        })
+      deletedSegments ++= deleted
+
+      // reset the index size of the currently active log segment to allow more entries
+      activeSegment.resizeIndexes(config.maxIndexSize)
+      updateLogEndOffset(nextOffset)
+    } else {
+      if (logSegments.isEmpty) {
+        addSegment(LogSegment.open(dir = dir,
+          baseOffset = 0,
+          config,
+          time = time,
+          initFileSize = this.initFileSize))
+      }
+      updateLogEndOffset(0)
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Recover the log segments and return the next offset after recovery.
+   * This method does not need to convert IOException to KafkaStorageException because it is usually called before all
+   * logs are loaded.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param producerStateManager The ProducerStateManager instance
+   * @param leaderEpochCache The LeaderEpochFileCache instance
+   *
+   * @return the list of deleted segments and the next offset
+   *
+   * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+   */
+  private[log] def recoverLog(logStartOffset: Long,
+                              maxProducerIdExpirationMs: Int,
+                              producerStateManager: ProducerStateManager,
+                              leaderEpochCache: Option[LeaderEpochFileCache]): (List[LogSegment], Long) = {
+    val deleted = scala.collection.mutable.ListBuffer[LogSegment]()
+    /** return the log end offset if valid */
+    def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+      if (logSegments.nonEmpty) {
+        val logEndOffset = activeSegment.readNextOffset
+        if (logEndOffset >= logStartOffset)
+          Some(logEndOffset)
+        else {
+          warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " +
+            "This could happen if segment files were deleted from the file system.")
+          val toDelete = logSegments.toList
+          removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery)
+          deleted ++= toDelete
+          leaderEpochCache.foreach(_.clearAndFlush())
+          producerStateManager.truncateFullyAndStartAt(logStartOffset)
+          None
+        }
+      } else None
+    }
+
+    // if we have the clean shutdown marker, skip recovery
+    if (!hadCleanShutdown) {
+      val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
+      var truncated = false
+
+      while (unflushed.hasNext && !truncated) {
+        val segment = unflushed.next()
+        info(s"Recovering unflushed segment ${segment.baseOffset}")
+        val truncatedBytes =
+          try {
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs, leaderEpochCache)
+          } catch {
+            case _: InvalidOffsetException =>
+              val startOffset = segment.baseOffset
+              warn("Found invalid offset during recovery. Deleting the corrupt segment and " +
+                s"creating an empty one with starting offset $startOffset")
+              segment.truncateTo(startOffset)
+          }
+        if (truncatedBytes > 0) {
+          // we had an invalid message, delete all remaining log
+          warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
+          val toDelete = unflushed.toList
+          removeAndDeleteSegments(toDelete,
+                                  asyncDelete = true,
+                                  reason = LogRecovery)
+          deleted ++= toDelete
+          truncated = true
+        }
+      }
+    }
+
+    val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+    if (logSegments.isEmpty) {
+      // no existing segments, create a new mutable segment beginning at logStartOffset
+      addSegment(LogSegment.open(dir = dir,
+        baseOffset = logStartOffset,
+        config,
+        time = time,
+        initFileSize = this.initFileSize,
+        preallocate = config.preallocate))
+    }
+
+    // Update the recovery point if there was a clean shutdown and did not perform any changes to
+    // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end
+    // offset. To ensure correctness and to make it easier to reason about, it's best to only advance
+    // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for
+    // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the
+    // segment.
+    (hadCleanShutdown, logEndOffsetOption) match {
+      case (true, Some(logEndOffset)) =>
+        updateRecoveryPoint(logEndOffset)
+        (deleted.toList, logEndOffset)
+      case _ =>
+        val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset)
+        updateRecoveryPoint(Math.min(recoveryPoint, logEndOffset))
+        (deleted.toList, logEndOffset)
+    }
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
+   * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException
+   * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the
+   * caller is responsible for closing them appropriately, if needed.
+   *
+   * @param logStartOffset the log start offset
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   *
+   * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+   */
+  private[log] def loadSegmentFiles(logStartOffset: Long, maxProducerIdExpirationMs: Int): Unit = {
+    // load segments in ascending order because transactional data from one segment may depend on the
+    // segments that come before it
+    for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+      if (isIndexFile(file)) {
+        // if it is an index file, make sure it has a corresponding .log file
+        val offset = offsetFromFile(file)
+        val logFile = LocalLog.logFile(dir, offset)
+        if (!logFile.exists) {
+          warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+          Files.deleteIfExists(file.toPath)
+        }
+      } else if (isLogFile(file)) {
+        // if it's a log file, load the corresponding log segment
+        val baseOffset = offsetFromFile(file)
+        val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists()
+        val segment = LogSegment.open(dir = dir,
+          baseOffset = baseOffset,
+          config,
+          time = time,
+          fileAlreadyExists = true)
+
+        try segment.sanityCheck(timeIndexFileNewlyCreated)
+        catch {
+          case _: NoSuchFileException =>
+            error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
+              "recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+          case e: CorruptIndexException =>
+            warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " +
+              s"to ${e.getMessage}}, recovering segment and rebuilding index files...")
+            recoverSegment(logStartOffset, segment, maxProducerIdExpirationMs)
+        }
+        addSegment(segment)
+      }
+    }
+  }
+
+  /**
+   * Recover the given segment.
+   *
+   * @param logStartOffset the log start offset
+   * @param segment Segment to recover
+   * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired
+   * @param leaderEpochCache Optional cache for updating the leader epoch during recovery
+   *
+   * @return The number of bytes truncated from the segment
+   *
+   * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow
+   */
+  private[log] def recoverSegment(logStartOffset: Long,
+                                  segment: LogSegment,
+                                  maxProducerIdExpirationMs: Int,
+                                  leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = {
+    val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+    rebuildProducerState(logStartOffset, segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager)
+    val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache)
+    // once we have recovered the segment's data, take a snapshot to ensure that we won't
+    // need to reload the same segment again while recovering another segment.
+    producerStateManager.takeSnapshot()
+    bytesTruncated
+  }
+
+  /**
+   * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs
+   * are loaded.
+   * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to
+   *                                           overflow. Note that this is currently a fatal exception as we do not have
+   *                                           a way to deal with it. The exception is propagated all the way up to
+   *                                           KafkaServer#startup which will cause the broker to shut down if we are in
+   *                                           this situation. This is expected to be an extremely rare scenario in practice,
+   *                                           and manual intervention might be required to get out of it.
+   */
+  private[log] def completeSwapOperations(swapFiles: Set[File],
+                                          logStartOffset: Long,
+                                          maxProducerIdExpirationMs: Int): Seq[LogSegment] = {
+    val deletedSegments = ListBuffer[LogSegment]()
+    for (swapFile <- swapFiles) {
+      val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
+      val baseOffset = offsetFromFile(logFile)
+      val swapSegment = LogSegment.open(swapFile.getParentFile,
+        baseOffset = baseOffset,
+        config,
+        time = time,
+        fileSuffix = SwapFileSuffix)
+      info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+      recoverSegment(logStartOffset, swapSegment, maxProducerIdExpirationMs)
+
+      // We create swap files for two cases:
+      // (1) Log cleaning where multiple segments are merged into one, and
+      // (2) Log splitting where one segment is split into multiple.
+      //
+      // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment
+      // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion
+      // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to
+      // do a replace with an existing segment.
+      val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+        segment.readNextOffset > swapSegment.baseOffset
+      }
+      val deleted = replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true)
+      deletedSegments ++= deleted
+    }
+    deletedSegments.toSeq
+  }
+
+  /**
+   * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped
+   * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than
+   * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted
+   * by this method.
+   *
+   * @return Set of .swap files that are valid to be swapped in as segment files
+   */
+  private[log] def removeTempFilesAndCollectSwapFiles(): Set[File] = {
+
+    def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
+      info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+      val offset = offsetFromFile(baseFile)
+      Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath)
+      Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath)
+    }
+
+    val swapFiles = mutable.Set[File]()
+    val cleanFiles = mutable.Set[File]()
+    var minCleanedFileOffset = Long.MaxValue
+
+    for (file <- dir.listFiles if file.isFile) {
+      if (!file.canRead)
+        throw new IOException(s"Could not read file $file")
+      val filename = file.getName
+      if (filename.endsWith(DeletedFileSuffix)) {
+        debug(s"Deleting stray temporary file ${file.getAbsolutePath}")
+        Files.deleteIfExists(file.toPath)
+      } else if (filename.endsWith(CleanedFileSuffix)) {
+        minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset)
+        cleanFiles += file
+      } else if (filename.endsWith(SwapFileSuffix)) {
+        // we crashed in the middle of a swap operation, to recover:
+        // if a log, delete the index files, complete the swap operation later
+        // if an index just delete the index files, they will be rebuilt
+        val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+        info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.")
+        if (isIndexFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+        } else if (isLogFile(baseFile)) {
+          deleteIndicesIfExist(baseFile)
+          swapFiles += file
+        }
+      }
+    }
+
+    // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap
+    // files could be part of an incomplete split operation that could not complete. See LocalLog#splitOverflowedSegment
+    // for more details about the split operation.
+    val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset)
+    invalidSwapFiles.foreach { file =>
+      debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset")
+      val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+      deleteIndicesIfExist(baseFile, SwapFileSuffix)
+      Files.deleteIfExists(file.toPath)
+    }
+
+    // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files
+    cleanFiles.foreach { file =>
+      debug(s"Deleting stray .clean file ${file.getAbsolutePath}")
+      Files.deleteIfExists(file.toPath)
+    }
+
+    validSwapFiles
+  }
+
+  private[log] def retryOnOffsetOverflow[T](fn: => T): T = {
+    while (true) {
+      try {
+        return fn
+      } catch {
+        case e: LogSegmentOffsetOverflowException =>
+          info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+          splitOverflowedSegment(e.segment)
+      }
+    }
+    throw new IllegalStateException()
+  }
+
+  private[log] def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      checkForLogDirFailure()
+      fun
+    } catch {
+      case e: IOException =>
+        logDirOffline = true
+        logDirFailureChannel.maybeAddOfflineLogDir(dir.getParent, msg, e)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
+  /**
+   * Split a segment into one or more segments such that there is no offset overflow in any of them. The
+   * resulting segments will contain the exact same messages that are present in the input segment. On successful
+   * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments.
+   * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation.
+   * <p>Note that this method assumes we have already determined that the segment passed in contains records that cause
+   * offset overflow.</p>
+   * <p>The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing
+   * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments
+   * and completeSwapOperations for the implementation to make this operation recoverable on crashes.</p>
+   *
+   * @param segment Segment to split
+   *
+   * @return a result instance containing list of new segments that replace the input segment and deleted segments (if any)
+   */
+  private[log] def splitOverflowedSegment(segment: LogSegment): SplitSegmentResult = {
+    require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}")
+    require(segment.hasOverflow, "Split operation is only permitted for segments with overflow")
+
+    info(s"Splitting overflowed segment $segment")
+
+    val newSegments = ListBuffer[LogSegment]()
+    try {
+      var position = 0
+      val sourceRecords = segment.log
+
+      while (position < sourceRecords.sizeInBytes) {
+        val firstBatch = sourceRecords.batchesFrom(position).asScala.head
+        val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset)
+        newSegments += newSegment
+
+        val bytesAppended = newSegment.appendFromFile(sourceRecords, position)
+        if (bytesAppended == 0)
+          throw new IllegalStateException(s"Failed to append records from position $position in $segment")
+
+        position += bytesAppended
+      }
+
+      // prepare new segments
+      var totalSizeOfNewSegments = 0
+      newSegments.foreach { splitSegment =>
+        splitSegment.onBecomeInactiveSegment()
+        splitSegment.flush()
+        splitSegment.lastModified = segment.lastModified
+        totalSizeOfNewSegments += splitSegment.log.sizeInBytes
+      }
+      // size of all the new segments combined must equal size of the original segment
+      if (totalSizeOfNewSegments != segment.log.sizeInBytes)
+        throw new IllegalStateException("Inconsistent segment sizes after split" +
+          s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments")
+
+      // replace old segment with new ones
+      info(s"Replacing overflowed segment $segment with split segments $newSegments")
+      val toAdd = newSegments.toSeq
+      val deletedSegments = replaceSegments(newSegments.toSeq, List(segment))
+      SplitSegmentResult(deletedSegments.toSeq, toAdd)
+    } catch {
+      case e: Exception =>
+        newSegments.foreach { splitSegment =>
+          splitSegment.close()
+          splitSegment.deleteIfExists()
+        }
+        throw e
+    }
+  }
+
+  /**
+   * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old
+   * segments will be asynchronously deleted.
+   *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
+   * The sequence of operations is:
+   * <ol>
+   *   <li> Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments().
+   *        If broker crashes at this point, the clean-and-swap operation is aborted and
+   *        the .cleaned files are deleted on recovery in loadSegments().
+   *   <li> New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the
+   *        clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in
+   *        loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from
+   *        .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files
+   *        whose offset is greater than the minimum-offset .clean file are deleted.
+   *   <li> If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap
+   *        operation is resumed on recovery as described in the next step.
+   *   <li> Old segment files are renamed to .deleted and asynchronous delete is scheduled.
+   *        If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments().
+   *        replaceSegments() is then invoked to complete the swap with newSegment recreated from
+   *        the .swap file and oldSegments containing segments which were not renamed before the crash.
+   *   <li> Swap segment(s) are renamed to replace the existing segments, completing this operation.
+   *        If the broker crashes, any .deleted files which may be left behind are deleted
+   *        on recovery in loadSegments().
+   * </ol>
+   *
+   * @param newSegments The new log segment to add to the log
+   * @param oldSegments The old log segments to delete from the log
+   * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash
+   *
+   * @return segments which were deleted but not replaced
+   */
+  private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Seq[LogSegment] = {
+    val sortedNewSegments = newSegments.sortBy(_.baseOffset)
+    // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments
+    // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment()
+    // multiple times for the same segment.
+    val sortedOldSegments = oldSegments.filter(seg => segments.containsKey(seg.baseOffset)).sortBy(_.baseOffset)
+
+    checkIfMemoryMappedBufferClosed()
+    // need to do this in two phases to be crash safe AND do the delete asynchronously
+    // if we crash in the middle of this we complete the swap in loadSegments()
+    if (!isRecoveredSwapFile)
+      sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix))
+    sortedNewSegments.reverse.foreach(addSegment(_))
+    val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet
+
+    // delete the old files
+    val deletedNotReplaced = sortedOldSegments.map { seg =>
+      // remove the index entry
+      if (seg.baseOffset != sortedNewSegments.head.baseOffset)
+        segments.remove(seg.baseOffset)
+      deleteSegmentFiles(List(seg), asyncDelete = true)
+      if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg)
+    }.filter(item => item.isDefined).map(item => item.get)
+    // okay we are safe now, remove the swap suffix
+    sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, ""))
+    deletedNotReplaced
+  }
+
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate is false or the segment
+   * containing the current high watermark is reached. We do not delete segments with offsets at or beyond
+   * the high watermark to ensure that the log start offset can never exceed it. If the high watermark
+   * has not yet been initialized, no segments are eligible for deletion.
+   *
+   * A final segment that is empty will never be returned (since we would just end up re-creating it).
+   *
+   * @param predicate A function that takes in a candidate log segment and the next higher segment
+   *                  (if there is one), logEndOffset and returns true iff it is deletable
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment], Long) => Boolean): Iterable[LogSegment] = {
+    if (segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      var segmentEntry = segments.firstEntry
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+        val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+        val (nextSegment, isLastSegmentAndEmpty) = if (nextSegmentEntry != null)
+          (nextSegmentEntry.getValue, false)
+        else
+          (null, segment.size == 0)
+
+        if (predicate(segment, Option(nextSegment), logEndOffset) && !isLastSegmentAndEmpty) {
+          deletable += segment
+          segmentEntry = nextSegmentEntry
+        } else {
+          segmentEntry = null
+        }
+      }
+      deletable
+    }
+  }
+
+  /**
+   * Perform physical deletion for the given segments. Allows the segments to be deleted asynchronously or synchronously.
+   *
+   * This method assumes that the segment exists and the method is not thread-safe.
+   *
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because

Review comment:
       The method only converts `IOException` to `KafkaStorageException` inside the internal `deleteSegments()` helper function. The comment is referring to only `changeFileSuffixes`. I've improved the `@throws` doc to refer to both exceptions now. Please let me know if it needs further improvement.




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

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