junrao commented on a change in pull request #10478:
URL: https://github.com/apache/kafka/pull/10478#discussion_r613486870



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -0,0 +1,526 @@
+/**
+ * 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, NoSuchFileException}
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.log.Log.{CleanedFileSuffix, DeletedFileSuffix, SwapFileSuffix, 
isIndexFile, isLogFile, offsetFromFile, offsetFromFileName}
+import kafka.server.{LogDirFailureChannel, LogOffsetMetadata}
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.utils.{CoreUtils, Logging, Scheduler}
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.InvalidOffsetException
+import org.apache.kafka.common.utils.Time
+
+import scala.collection.{Seq, Set, mutable}
+
+case class LoadedLogOffsets(logStartOffset: Long,
+                            recoveryPoint: Long,
+                            nextOffsetMetadata: LogOffsetMetadata)
+
+/**
+ * @param dir The directory from which log segments need to be loaded
+ * @param topicPartition The topic partition associated with the log being 
loaded
+ * @param config The configuration settings for the log being loaded
+ * @param scheduler The thread pool scheduler used for background actions
+ * @param time The time instance used for checking the clock
+ * @param logDirFailureChannel The LogDirFailureChannel instance to 
asynchronously handle log
+ *                             directory failure
+ * @param hadCleanShutdown Boolean flag to indicate whether the associated log 
previously had a
+ *                         clean shutdown
+ * @param segments The LogSegments instance into which segments recovered from 
disk will be
+ *                 populated
+ * @param logStartOffsetCheckpoint The checkpoint of the log start offset
+ * @param recoveryPointCheckpoint The checkpoint of the offset at which to 
begin the recovery
+ * @param maxProducerIdExpirationMs The maximum amount of time to wait before 
a producer id is
+ *                                  considered expired
+ * @param leaderEpochCache An optional LeaderEpochFileCache instance to be 
updated during recovery
+ * @param producerStateManager The ProducerStateManager instance to be updated 
during recovery
+ */
+case class LoadLogParams(dir: File,
+                         topicPartition: TopicPartition,
+                         config: LogConfig,
+                         scheduler: Scheduler,
+                         time: Time,
+                         logDirFailureChannel: LogDirFailureChannel,
+                         hadCleanShutdown: Boolean,
+                         segments: LogSegments,
+                         logStartOffsetCheckpoint: Long,
+                         recoveryPointCheckpoint: Long,
+                         maxProducerIdExpirationMs: Int,
+                         leaderEpochCache: Option[LeaderEpochFileCache],
+                         producerStateManager: ProducerStateManager) {
+  val logIdentifier: String = s"[LogLoader partition=$topicPartition, 
dir=${dir.getParent}]"
+}
+
+/**
+ * This object is responsible for all activities related with recovery of log 
segments from disk.
+ */
+object LogLoader extends Logging {
+  /**
+   * Load the log segments from the log files on disk, and return the 
components of the loaded log.
+   * Additionally, it also suitably updates the provided LeaderEpochFileCache 
and ProducerStateManager
+   * to reflect the contents of the loaded log.
+   *
+   * This method does not need to convert IOException to KafkaStorageException 
because it is only
+   * called before all logs are loaded.
+   *
+   * @param params The parameters for the log being loaded from disk
+   *
+   * @return the offsets of the Log successfully loaded from disk
+   *
+   * @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

Review comment:
       It seems that this comment is no longer accurate. The only place that 
can throw LogSegmentOffsetOverflowException is from completeSwapOperations(). 
The rest of the places are all wrapped with retryOnOffsetOverflow().

##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -0,0 +1,526 @@
+/**
+ * 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, NoSuchFileException}
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.log.Log.{CleanedFileSuffix, DeletedFileSuffix, SwapFileSuffix, 
isIndexFile, isLogFile, offsetFromFile, offsetFromFileName}
+import kafka.server.{LogDirFailureChannel, LogOffsetMetadata}
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.utils.{CoreUtils, Logging, Scheduler}
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.InvalidOffsetException
+import org.apache.kafka.common.utils.Time
+
+import scala.collection.{Seq, Set, mutable}
+
+case class LoadedLogOffsets(logStartOffset: Long,
+                            recoveryPoint: Long,
+                            nextOffsetMetadata: LogOffsetMetadata)
+
+/**
+ * @param dir The directory from which log segments need to be loaded
+ * @param topicPartition The topic partition associated with the log being 
loaded
+ * @param config The configuration settings for the log being loaded
+ * @param scheduler The thread pool scheduler used for background actions
+ * @param time The time instance used for checking the clock
+ * @param logDirFailureChannel The LogDirFailureChannel instance to 
asynchronously handle log
+ *                             directory failure
+ * @param hadCleanShutdown Boolean flag to indicate whether the associated log 
previously had a
+ *                         clean shutdown
+ * @param segments The LogSegments instance into which segments recovered from 
disk will be
+ *                 populated
+ * @param logStartOffsetCheckpoint The checkpoint of the log start offset
+ * @param recoveryPointCheckpoint The checkpoint of the offset at which to 
begin the recovery
+ * @param maxProducerIdExpirationMs The maximum amount of time to wait before 
a producer id is
+ *                                  considered expired
+ * @param leaderEpochCache An optional LeaderEpochFileCache instance to be 
updated during recovery
+ * @param producerStateManager The ProducerStateManager instance to be updated 
during recovery
+ */
+case class LoadLogParams(dir: File,
+                         topicPartition: TopicPartition,
+                         config: LogConfig,
+                         scheduler: Scheduler,
+                         time: Time,
+                         logDirFailureChannel: LogDirFailureChannel,
+                         hadCleanShutdown: Boolean,
+                         segments: LogSegments,
+                         logStartOffsetCheckpoint: Long,
+                         recoveryPointCheckpoint: Long,
+                         maxProducerIdExpirationMs: Int,
+                         leaderEpochCache: Option[LeaderEpochFileCache],
+                         producerStateManager: ProducerStateManager) {
+  val logIdentifier: String = s"[LogLoader partition=$topicPartition, 
dir=${dir.getParent}]"
+}
+
+/**
+ * This object is responsible for all activities related with recovery of log 
segments from disk.
+ */
+object LogLoader extends Logging {
+  /**
+   * Load the log segments from the log files on disk, and return the 
components of the loaded log.
+   * Additionally, it also suitably updates the provided LeaderEpochFileCache 
and ProducerStateManager
+   * to reflect the contents of the loaded log.
+   *
+   * This method does not need to convert IOException to KafkaStorageException 
because it is only
+   * called before all logs are loaded.
+   *
+   * @param params The parameters for the log being loaded from disk
+   *
+   * @return the offsets of the Log successfully loaded from disk
+   *
+   * @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
+   */
+  def load(params: LoadLogParams): LoadedLogOffsets = {
+    // 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(params)
+
+    // 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(params, {
+      // 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().
+      params.segments.close()
+      params.segments.clear()
+      loadSegmentFiles(params)
+    })
+
+    completeSwapOperations(swapFiles, params)
+
+    val (newRecoveryPoint: Long, nextOffset: Long) = {
+      if (!params.dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) {
+        val (newRecoveryPoint, nextOffset) = retryOnOffsetOverflow(params, {
+          recoverLog(params)
+        })
+
+        // reset the index size of the currently active log segment to allow 
more entries
+        
params.segments.lastSegment.get.resizeIndexes(params.config.maxIndexSize)
+        (newRecoveryPoint, nextOffset)
+      } else {
+        if (params.segments.isEmpty) {
+          params.segments.add(
+            LogSegment.open(
+              dir = params.dir,
+              baseOffset = 0,
+              params.config,
+              time = params.time,
+              initFileSize = params.config.initFileSize))
+        }
+        (params.recoveryPointCheckpoint, 0L)

Review comment:
       Since the nextOffset is 0, recoveryPoint needs to be 0 too.

##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -0,0 +1,526 @@
+/**
+ * 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, NoSuchFileException}
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.log.Log.{CleanedFileSuffix, DeletedFileSuffix, SwapFileSuffix, 
isIndexFile, isLogFile, offsetFromFile, offsetFromFileName}
+import kafka.server.{LogDirFailureChannel, LogOffsetMetadata}
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.utils.{CoreUtils, Logging, Scheduler}
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.InvalidOffsetException
+import org.apache.kafka.common.utils.Time
+
+import scala.collection.{Seq, Set, mutable}
+
+case class LoadedLogOffsets(logStartOffset: Long,
+                            recoveryPoint: Long,
+                            nextOffsetMetadata: LogOffsetMetadata)
+
+/**
+ * @param dir The directory from which log segments need to be loaded
+ * @param topicPartition The topic partition associated with the log being 
loaded
+ * @param config The configuration settings for the log being loaded
+ * @param scheduler The thread pool scheduler used for background actions
+ * @param time The time instance used for checking the clock
+ * @param logDirFailureChannel The LogDirFailureChannel instance to 
asynchronously handle log

Review comment:
       Currently, if we hit any IOException during loading, we just kill the 
broker. So, it seems that there is no need to pass in logDirFailureChannel 
during log loading.

##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -0,0 +1,526 @@
+/**
+ * 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, NoSuchFileException}
+
+import kafka.common.LogSegmentOffsetOverflowException
+import kafka.log.Log.{CleanedFileSuffix, DeletedFileSuffix, SwapFileSuffix, 
isIndexFile, isLogFile, offsetFromFile, offsetFromFileName}
+import kafka.server.{LogDirFailureChannel, LogOffsetMetadata}
+import kafka.server.epoch.LeaderEpochFileCache
+import kafka.utils.{CoreUtils, Logging, Scheduler}
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.InvalidOffsetException
+import org.apache.kafka.common.utils.Time
+
+import scala.collection.{Seq, Set, mutable}
+
+case class LoadedLogOffsets(logStartOffset: Long,
+                            recoveryPoint: Long,
+                            nextOffsetMetadata: LogOffsetMetadata)
+
+/**
+ * @param dir The directory from which log segments need to be loaded
+ * @param topicPartition The topic partition associated with the log being 
loaded
+ * @param config The configuration settings for the log being loaded
+ * @param scheduler The thread pool scheduler used for background actions
+ * @param time The time instance used for checking the clock
+ * @param logDirFailureChannel The LogDirFailureChannel instance to 
asynchronously handle log
+ *                             directory failure
+ * @param hadCleanShutdown Boolean flag to indicate whether the associated log 
previously had a
+ *                         clean shutdown
+ * @param segments The LogSegments instance into which segments recovered from 
disk will be
+ *                 populated
+ * @param logStartOffsetCheckpoint The checkpoint of the log start offset
+ * @param recoveryPointCheckpoint The checkpoint of the offset at which to 
begin the recovery
+ * @param maxProducerIdExpirationMs The maximum amount of time to wait before 
a producer id is
+ *                                  considered expired
+ * @param leaderEpochCache An optional LeaderEpochFileCache instance to be 
updated during recovery
+ * @param producerStateManager The ProducerStateManager instance to be updated 
during recovery
+ */
+case class LoadLogParams(dir: File,
+                         topicPartition: TopicPartition,
+                         config: LogConfig,
+                         scheduler: Scheduler,
+                         time: Time,
+                         logDirFailureChannel: LogDirFailureChannel,
+                         hadCleanShutdown: Boolean,
+                         segments: LogSegments,
+                         logStartOffsetCheckpoint: Long,
+                         recoveryPointCheckpoint: Long,
+                         maxProducerIdExpirationMs: Int,
+                         leaderEpochCache: Option[LeaderEpochFileCache],
+                         producerStateManager: ProducerStateManager) {
+  val logIdentifier: String = s"[LogLoader partition=$topicPartition, 
dir=${dir.getParent}]"
+}
+
+/**
+ * This object is responsible for all activities related with recovery of log 
segments from disk.
+ */
+object LogLoader extends Logging {
+  /**
+   * Load the log segments from the log files on disk, and return the 
components of the loaded log.
+   * Additionally, it also suitably updates the provided LeaderEpochFileCache 
and ProducerStateManager
+   * to reflect the contents of the loaded log.
+   *
+   * This method does not need to convert IOException to KafkaStorageException 
because it is only
+   * called before all logs are loaded.

Review comment:
       The loading code still converts IOException to KafkaStorageException 
through Log.deleteSegmentFiles().

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -222,25 +222,31 @@ case object SnapshotGenerated extends 
LogStartOffsetIncrementReason {
  *
  * @param _dir The directory in which log segments are created.
  * @param config The log configuration settings
+ * @param segments The log segments, these may be non-empty when recovered 
from disk

Review comment:
       The segments is always non-empty since we initialize it with an empty 
segment if no segments are present.




-- 
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:
us...@infra.apache.org


Reply via email to