kowshik commented on a change in pull request #10478: URL: https://github.com/apache/kafka/pull/10478#discussion_r613787269
########## 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: Great catch. 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: us...@infra.apache.org