kowshik commented on a change in pull request #10478: URL: https://github.com/apache/kafka/pull/10478#discussion_r613804832
########## 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: Thise comment was trying to convey that in the context of the calling thread, this function doesn't convert the `IOException` to `KafkaStorageException`. In the recovery path, we always asynchronously delete segments. The `IOException` conversion happens only within the background work that's scheduled for segment deletions. https://github.com/apache/kafka/blob/193a9dfc44fa2c0df163a8518a804c4579d37e36/core/src/main/scala/kafka/log/Log.scala#L2374-L2389 Since this is a bit misleading, I've improved the comment now to say: ``` In the context of the calling thread, this function does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded. ``` -- 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