diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 1111d98bae1cf..edbabb88feef6 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -19,7 +19,7 @@ package kafka.log import java.io.{File, IOException} import java.lang.{Long => JLong} -import java.nio.file.{Files, NoSuchFileException} +import java.nio.file.Files import java.text.NumberFormat import java.util.Map.{Entry => JEntry} import java.util.Optional @@ -28,7 +28,7 @@ import java.util.concurrent.TimeUnit import java.util.regex.Pattern import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0} -import kafka.common.{LogSegmentOffsetOverflowException, LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} +import kafka.common.{LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.AppendOrigin.RaftLeader import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec} import kafka.metrics.KafkaMetricsGroup @@ -48,7 +48,7 @@ import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPar import scala.jdk.CollectionConverters._ import scala.collection.mutable.{ArrayBuffer, ListBuffer} -import scala.collection.{Seq, Set, mutable} +import scala.collection.{Seq, mutable} object LogAppendInfo { val UnknownLogAppendInfo = LogAppendInfo(None, -1, None, RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP, -1L, @@ -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 non-empty log segments recovered from disk * @param logStartOffset The earliest offset allowed to be exposed to kafka client. * The logStartOffset can be updated by : * - user's DeleteRecordsRequest * - broker's log retention * - broker's log truncation + * - broker's log recovery * The logStartOffset is used to decide the following: * - Log deletion. LogSegment whose nextOffset <= log's logStartOffset can be deleted. * It may trigger log rolling if the active segment is deleted. * - Earliest offset of the log in response to ListOffsetRequest. To avoid OffsetOutOfRange exception after user seeks to earliest offset, * we make sure that logStartOffset <= log's highWatermark * Other activities such as log cleaning are not affected by logStartOffset. - * @param recoveryPoint The offset at which to begin recovery--i.e. the first offset which has not been flushed to 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 brokerTopicStats Container for Broker Topic Yammer Metrics * @param time The time instance used for checking the clock * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired * @param producerIdExpirationCheckIntervalMs How often to check for producer ids which need to be expired - * @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. + * @param topicPartition The topic partition associated with this Log instance + * @param leaderEpochCache The LeaderEpochFileCache instance (if any) containing state associated + * with the provided logStartOffset and nextOffsetMetadata + * @param producerStateManager The ProducerStateManager instance containing state associated with the provided segments + * @param logDirFailureChannel The LogDirFailureChannel instance to asynchronously handle log directory failure * @param topicId optional Uuid to specify the topic ID for the topic if it exists. Should only be specified when * first creating the log through Partition.makeLeader or Partition.makeFollower. When reloading a log, * this field will be populated by reading the topic ID value from partition.metadata if it exists. @@ -255,17 +261,19 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason { @threadsafe class Log(@volatile private var _dir: File, @volatile var config: LogConfig, + val segments: LogSegments, @volatile var logStartOffset: Long, @volatile var recoveryPoint: Long, + @volatile var nextOffsetMetadata: LogOffsetMetadata, scheduler: Scheduler, brokerTopicStats: BrokerTopicStats, val time: Time, val maxProducerIdExpirationMs: Int, val producerIdExpirationCheckIntervalMs: Int, val topicPartition: TopicPartition, + @volatile var leaderEpochCache: Option[LeaderEpochFileCache], val producerStateManager: ProducerStateManager, logDirFailureChannel: LogDirFailureChannel, - private val hadCleanShutdown: Boolean = true, @volatile var topicId: Option[Uuid], val keepPartitionMetadataFile: Boolean) extends Logging with KafkaMetricsGroup { @@ -286,8 +294,6 @@ class Log(@volatile private var _dir: File, /* last time it was flushed */ private val lastFlushedTime = new AtomicLong(time.milliseconds) - @volatile private var nextOffsetMetadata: LogOffsetMetadata = _ - /* The earliest offset which is part of an incomplete transaction. This is used to compute the * last stable offset (LSO) in ReplicaManager. Note that it is possible that the "true" first unstable offset * gets removed from the log (through record or segment deletion). In this case, the first unstable offset @@ -308,44 +314,12 @@ class Log(@volatile private var _dir: File, */ @volatile private var highWatermarkMetadata: LogOffsetMetadata = LogOffsetMetadata(logStartOffset) - /* the actual segments of the log */ - private val segments: LogSegments = new LogSegments(topicPartition) - - // Visible for testing - @volatile var leaderEpochCache: Option[LeaderEpochFileCache] = None - @volatile var partitionMetadataFile : PartitionMetadataFile = null locally { - // create the log directory if it doesn't exist - Files.createDirectories(dir.toPath) - - initializeLeaderEpochCache() initializePartitionMetadata() - - val nextOffset = loadSegments() - - /* Calculate the offset of the next message */ - nextOffsetMetadata = LogOffsetMetadata(nextOffset, activeSegment.baseOffset, activeSegment.size) - - leaderEpochCache.foreach(_.truncateFromEnd(nextOffsetMetadata.messageOffset)) - - updateLogStartOffset(math.max(logStartOffset, segments.firstSegment.get.baseOffset)) - - // The earliest leader epoch may not be flushed during a hard failure. Recover it here. - leaderEpochCache.foreach(_.truncateFromStart(logStartOffset)) - - // Any segment loading or recovery code must not use producerStateManager, so that we can build the full state here - // from scratch. - if (!producerStateManager.isEmpty) - throw new IllegalStateException("Producer state must be empty during log initialization") - - // Reload all snapshots into the ProducerStateManager cache, the intermediate ProducerStateManager used - // during log recovery may have deleted some files without the Log.producerStateManager instance witnessing the - // deletion. - producerStateManager.removeStraySnapshots(segments.baseOffsets.toSeq) - loadProducerState(logEndOffset, reloadFromCleanShutdown = hadCleanShutdown) - + updateLogStartOffset(logStartOffset) + maybeIncrementFirstUnstableOffset() // Delete partition metadata file if the version does not support topic IDs. // Recover topic ID if present and topic IDs are supported // If we were provided a topic ID when creating the log, partition metadata files are supported, and one does not yet exist @@ -372,13 +346,6 @@ class Log(@volatile private var _dir: File, def parentDirFile: File = new File(_parentDir) - def initFileSize: Int = { - if (config.preallocate) - config.segmentSize - else - 0 - } - def updateConfig(newConfig: LogConfig): Unit = { val oldConfig = this.config this.config = newConfig @@ -592,243 +559,7 @@ class Log(@volatile private var _dir: File, } private def initializeLeaderEpochCache(): Unit = lock synchronized { - val leaderEpochFile = LeaderEpochCheckpointFile.newFile(dir) - - def newLeaderEpochFileCache(): LeaderEpochFileCache = { - val checkpointFile = new LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel) - new LeaderEpochFileCache(topicPartition, checkpointFile) - } - - if (recordVersion.precedes(RecordVersion.V2)) { - val currentCache = if (leaderEpochFile.exists()) - Some(newLeaderEpochFileCache()) - else - None - - if (currentCache.exists(_.nonEmpty)) - warn(s"Deleting non-empty leader epoch cache due to incompatible message format $recordVersion") - - Files.deleteIfExists(leaderEpochFile.toPath) - leaderEpochCache = None - } else { - leaderEpochCache = Some(newLeaderEpochFileCache()) - } - } - - /** - * 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 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 Log#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 - } - - /** - * 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. - * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset - */ - private def loadSegmentFiles(): 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 = Log.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, - needsRecovery = !hadCleanShutdown) - - 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(segment) - 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(segment) - } - addSegment(segment) - } - } - } - - /** - * Recover the given segment. - * @param segment Segment to recover - * @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 def recoverSegment(segment: LogSegment, - leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = lock synchronized { - val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) - rebuildProducerState(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 def completeSwapOperations(swapFiles: Set[File]): Unit = { - 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(swapSegment) - - // 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 - } - replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true) - } - } - - /** - * Load the log segments from the log files on disk and return the next offset. - * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs - * are loaded. - * @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 def loadSegments(): Long = { - // 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(). - segments.close() - segments.clear() - loadSegmentFiles() - } - - // 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. - completeSwapOperations(swapFiles) - - if (!dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) { - val nextOffset = retryOnOffsetOverflow { - recoverLog() - } - - // reset the index size of the currently active log segment to allow more entries - activeSegment.resizeIndexes(config.maxIndexSize) - nextOffset - } else { - if (logSegments.isEmpty) { - addSegment(LogSegment.open(dir = dir, - baseOffset = 0, - config, - time = time, - initFileSize = this.initFileSize)) - } - 0 - } + leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion) } private def updateLogEndOffset(offset: Long): Unit = { @@ -857,174 +588,13 @@ class Log(@volatile private var _dir: File, } } - /** - * 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 (segments.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 allSegments = logSegments - val offsetsToSnapshot = - if (allSegments.nonEmpty) { - val nextLatestSegmentBaseOffset = segments.lowerSegment(allSegments.last.baseOffset).map(_.baseOffset) - Seq(nextLatestSegmentBaseOffset, Some(allSegments.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 = segments.floorSegment(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 ${segmentRecoveryStart - producerStateLoadStart}ms for snapshot load " + - s"and ${time.milliseconds() - segmentRecoveryStart}ms for segment recovery from offset $lastOffset") - } - } - - private def loadProducerState(lastOffset: Long, reloadFromCleanShutdown: Boolean): Unit = lock synchronized { - rebuildProducerState(lastOffset, reloadFromCleanShutdown, producerStateManager) - maybeIncrementFirstUnstableOffset() + Log.rebuildProducerState(producerStateManager, segments, logStartOffset, lastOffset, recordVersion, time, + reloadFromCleanShutdown = false) } def activeProducers: Seq[DescribeProducersResponseData.ProducerState] = { @@ -2068,7 +1638,7 @@ class Log(@volatile private var _dir: File, baseOffset = newOffset, config, time = time, - initFileSize = initFileSize, + initFileSize = config.initFileSize, preallocate = config.preallocate) addSegment(segment) @@ -2217,7 +1787,7 @@ class Log(@volatile private var _dir: File, baseOffset = newOffset, config = config, time = time, - initFileSize = initFileSize, + initFileSize = config.initFileSize, preallocate = config.preallocate)) leaderEpochCache.foreach(_.clearAndFlush()) producerStateManager.truncateFullyAndStartAt(newOffset) @@ -2237,7 +1807,7 @@ class Log(@volatile private var _dir: File, logStartOffset = startOffset nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size) recoveryPoint = math.min(recoveryPoint, endOffset) - rebuildProducerState(endOffset, reloadFromCleanShutdown = false, producerStateManager) + rebuildProducerState(endOffset, producerStateManager) updateHighWatermark(math.min(highWatermark, endOffset)) } @@ -2315,94 +1885,16 @@ class Log(@volatile private var _dir: File, } } - /** - * Perform physical deletion for the given file. Allows the file to be deleted asynchronously or synchronously. - * - * This method assumes that the file 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 file can't be renamed and still exists - */ private def deleteSegmentFiles(segments: Iterable[LogSegment], asyncDelete: Boolean, deleteProducerStateSnapshots: Boolean = true): Unit = { - segments.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix, false)) - - 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 (deleteProducerStateSnapshots) - producerStateManager.removeAndDeleteSnapshot(segment.baseOffset) - } - } - } - - if (asyncDelete) - scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs) - else - deleteSegments() + Log.deleteSegmentFiles(segments, asyncDelete, deleteProducerStateSnapshots, dir, topicPartition, + config, scheduler, logDirFailureChannel, producerStateManager) } - /** - * 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: - *
    - *
  1. 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(). - *
  2. 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. - *
  3. 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. - *
  4. 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. - *
  5. 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(). - *
- * - * @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 - */ private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = { lock synchronized { - 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.contains(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, false)) - sortedNewSegments.reverse.foreach(addSegment(_)) - val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet - - // delete the old files - sortedOldSegments.foreach { seg => - // remove the index entry - if (seg.baseOffset != sortedNewSegments.head.baseOffset) - segments.remove(seg.baseOffset) - // delete segment files, but do not delete producer state for segment objects which are being replaced. - deleteSegmentFiles(List(seg), asyncDelete = true, deleteProducerStateSnapshots = !newSegmentBaseOffsets.contains(seg.baseOffset)) - } - // okay we are safe now, remove the swap suffix - sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, "")) + Log.replaceSegments(segments, newSegments, oldSegments, isRecoveredSwapFile, dir, topicPartition, + config, scheduler, logDirFailureChannel, producerStateManager) } } @@ -2438,102 +1930,23 @@ class Log(@volatile private var _dir: File, private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.add(segment) private def maybeHandleIOException[T](msg: => String)(fun: => T): T = { - if (logDirFailureChannel.hasOfflineLogDir(parentDir)) { - throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.") - } - try { + Log.maybeHandleIOException(logDirFailureChannel, parentDir, msg) { fun - } catch { - case e: IOException => - logDirFailureChannel.maybeAddOfflineLogDir(parentDir, msg, e) - throw new KafkaStorageException(msg, e) } } - 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 splitOverflowedSegment(segment: LogSegment): List[LogSegment] = lock synchronized { + Log.splitOverflowedSegment(segment, segments, dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager) } - /** - * 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. - *

Note that this method assumes we have already determined that the segment passed in contains records that cause - * offset overflow.

- *

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.

- * @param segment Segment to split - * @return List of new segments that replace the input segment - */ - private[log] def splitOverflowedSegment(segment: LogSegment): List[LogSegment] = { - 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(this, 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") - replaceSegments(newSegments.toList, List(segment)) - newSegments.toList - } catch { - case e: Exception => - newSegments.foreach { splitSegment => - splitSegment.close() - splitSegment.deleteIfExists() - } - throw e - } - } -} - -/** - * Helper functions for logs - */ -object Log { - - /** a log file */ - val LogFileSuffix = ".log" +/** + * Helper functions for logs + */ +object Log extends Logging { + /** a log file */ + val LogFileSuffix = ".log" /** an index file */ val IndexFileSuffix = ".index" @@ -2587,10 +2000,29 @@ object Log { lastShutdownClean: Boolean = true, topicId: Option[Uuid], keepPartitionMetadataFile: Boolean): Log = { + // create the log directory if it doesn't exist + Files.createDirectories(dir.toPath) val topicPartition = Log.parseTopicPartitionName(dir) + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) - new Log(dir, config, logStartOffset, recoveryPoint, scheduler, brokerTopicStats, time, maxProducerIdExpirationMs, - producerIdExpirationCheckIntervalMs, topicPartition, producerStateManager, logDirFailureChannel, lastShutdownClean, topicId, keepPartitionMetadataFile) + val offsets = LogLoader.load(LoadLogParams( + dir, + topicPartition, + config, + scheduler, + time, + logDirFailureChannel, + lastShutdownClean, + segments, + logStartOffset, + recoveryPoint, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + new Log(dir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, offsets.nextOffsetMetadata, scheduler, + brokerTopicStats, time, maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs, topicPartition, + leaderEpochCache, producerStateManager, logDirFailureChannel, topicId, keepPartitionMetadataFile) } /** @@ -2748,12 +2180,12 @@ object Log { new TopicPartition(topic, partition) } - private def isIndexFile(file: File): Boolean = { + private[log] def isIndexFile(file: File): Boolean = { val filename = file.getName filename.endsWith(IndexFileSuffix) || filename.endsWith(TimeIndexFileSuffix) || filename.endsWith(TxnIndexFileSuffix) } - private def isLogFile(file: File): Boolean = + private[log] def isLogFile(file: File): Boolean = file.getPath.endsWith(LogFileSuffix) private def loadProducersFromRecords(producerStateManager: ProducerStateManager, records: Records): Unit = { @@ -2784,6 +2216,382 @@ object Log { appendInfo.append(batch, firstOffsetMetadata) } + /** + * If the recordVersion is >= RecordVersion.V2, then create and return a LeaderEpochFileCache. + * Otherwise, the message format is considered incompatible and the existing LeaderEpoch file + * is deleted. + * + * @param dir The directory in which the log will reside + * @param topicPartition The topic partition + * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure + * @param recordVersion The record version + * @return The new LeaderEpochFileCache instance (if created), none otherwise + */ + def maybeCreateLeaderEpochCache(dir: File, + topicPartition: TopicPartition, + logDirFailureChannel: LogDirFailureChannel, + recordVersion: RecordVersion): Option[LeaderEpochFileCache] = { + val leaderEpochFile = LeaderEpochCheckpointFile.newFile(dir) + + def newLeaderEpochFileCache(): LeaderEpochFileCache = { + val checkpointFile = new LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel) + new LeaderEpochFileCache(topicPartition, checkpointFile) + } + + if (recordVersion.precedes(RecordVersion.V2)) { + val currentCache = if (leaderEpochFile.exists()) + Some(newLeaderEpochFileCache()) + else + None + + if (currentCache.exists(_.nonEmpty)) + warn(s"Deleting non-empty leader epoch cache due to incompatible message format $recordVersion") + + Files.deleteIfExists(leaderEpochFile.toPath) + None + } else { + Some(newLeaderEpochFileCache()) + } + } + + /** + * 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: + * + * - Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments() on + * the Log instance. If broker crashes at this point, the clean-and-swap operation is aborted and + * the .cleaned files are deleted on recovery in LogLoader. + * - 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 + * in LogLoader. 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. + * - 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. + * - 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 LogLoader. + * 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. + * - 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 LogLoader. + * + * @param existingSegments The existing segments of the log + * @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 + * @param dir The directory in which the log will reside + * @param topicPartition The topic + * @param config The log configuration settings + * @param scheduler The thread pool scheduler used for background actions + * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure + * @param producerStateManager The ProducerStateManager instance (if any) containing state associated + * with the existingSegments + */ + private[log] def replaceSegments(existingSegments: LogSegments, + newSegments: Seq[LogSegment], + oldSegments: Seq[LogSegment], + isRecoveredSwapFile: Boolean = false, + dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + logDirFailureChannel: LogDirFailureChannel, + producerStateManager: ProducerStateManager): Unit = { + 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 => existingSegments.contains(seg.baseOffset)).sortBy(_.baseOffset) + + // 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, false)) + sortedNewSegments.reverse.foreach(existingSegments.add(_)) + val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet + + // delete the old files + sortedOldSegments.foreach { seg => + // remove the index entry + if (seg.baseOffset != sortedNewSegments.head.baseOffset) + existingSegments.remove(seg.baseOffset) + // delete segment files, but do not delete producer state for segment objects which are being replaced. + deleteSegmentFiles( + List(seg), + asyncDelete = true, + deleteProducerStateSnapshots = !newSegmentBaseOffsets.contains(seg.baseOffset), + dir, + topicPartition, + config, + scheduler, + logDirFailureChannel, + producerStateManager) + } + // okay we are safe now, remove the swap suffix + sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, "")) + } + + /** + * Perform physical deletion of the index, log and producer snapshot files for the given segment. + * Prior to the deletion, the index and log files are renamed by appending .deleted to the + * respective file name. Allows these files to be optionally deleted asynchronously. + * + * This method assumes that the file exists. It 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. + * + * @param segmentsToDelete The segments to be deleted + * @param asyncDelete If true, the deletion of the segments is done asynchronously + * @param deleteProducerStateSnapshots If true, the producer state snapshot associated with a + * segment will be deleted after the segment is deleted + * @param dir The directory in which the log will reside + * @param topicPartition The topic + * @param config The log configuration settings + * @param scheduler The thread pool scheduler used for background actions + * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure + * @param producerStateManager The ProducerStateManager instance (if any) containing state associated + * with the existingSegments + * + * @throws IOException if the file can't be renamed and still exists + */ + private[log] def deleteSegmentFiles(segmentsToDelete: Iterable[LogSegment], + asyncDelete: Boolean, + deleteProducerStateSnapshots: Boolean = true, + dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + logDirFailureChannel: LogDirFailureChannel, + producerStateManager: ProducerStateManager): Unit = { + segmentsToDelete.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix, false)) + + def deleteSegments(): Unit = { + info(s"Deleting segment files ${segmentsToDelete.mkString(",")}") + val parentDir = dir.getParent + maybeHandleIOException(logDirFailureChannel, parentDir, s"Error while deleting segments for $topicPartition in dir $parentDir") { + segmentsToDelete.foreach { segment => + segment.deleteIfExists() + if (deleteProducerStateSnapshots) + producerStateManager.removeAndDeleteSnapshot(segment.baseOffset) + } + } + } + + if (asyncDelete) + scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs) + else + deleteSegments() + } + + /** + * Invokes the provided function and handles any IOException raised by the function by marking the + * provided directory offline. + * + * @param logDirFailureChannel Used to asynchronously handle log directory failure. + * @param logDir The log directory to be marked offline during an IOException. + * @param errorMsg The error message to be used when marking the log directory offline. + * @param fun The function to be executed. + * @return The value returned by the function after a successful invocation + */ + private def maybeHandleIOException[T](logDirFailureChannel: LogDirFailureChannel, + logDir: String, + errorMsg: => String)(fun: => T): T = { + if (logDirFailureChannel.hasOfflineLogDir(logDir)) { + throw new KafkaStorageException(s"The log dir $logDir is already offline due to a previous IO exception.") + } + try { + fun + } catch { + case e: IOException => + logDirFailureChannel.maybeAddOfflineLogDir(logDir, errorMsg, e) + throw new KafkaStorageException(errorMsg, e) + } + } + + /** + * Rebuilds producer state until the provided lastOffset. This function 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. + * + * @param producerStateManager The ProducerStateManager instance to be rebuilt. + * @param segments The segments of the log whose producer state is being rebuilt + * @param logStartOffset The log start offset + * @param lastOffset The last offset upto which the producer state needs to be rebuilt + * @param recordVersion The record version + * @param time The time instance used for checking the clock + * @param reloadFromCleanShutdown True if the producer state is being built after a clean shutdown, + * false otherwise. + */ + private[log] def rebuildProducerState(producerStateManager: ProducerStateManager, + segments: LogSegments, + logStartOffset: Long, + lastOffset: Long, + recordVersion: RecordVersion, + time: Time, + reloadFromCleanShutdown: Boolean): Unit = { + val allSegments = segments.values + val offsetsToSnapshot = + if (allSegments.nonEmpty) { + val nextLatestSegmentBaseOffset = segments.lowerSegment(allSegments.last.baseOffset).map(_.baseOffset) + Seq(nextLatestSegmentBaseOffset, Some(allSegments.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 = segments.floorSegment(lastOffset) + + segments.values(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) + if (fetchDataInfo != null) + loadProducersFromRecords(producerStateManager, fetchDataInfo.records) + } + } + producerStateManager.updateMapEndOffset(lastOffset) + producerStateManager.takeSnapshot() + info(s"Producer state recovery took ${segmentRecoveryStart - producerStateLoadStart}ms for snapshot load " + + s"and ${time.milliseconds() - segmentRecoveryStart}ms for segment recovery from offset $lastOffset") + } + } + + /** + * 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. + * + * Note that this method assumes we have already determined that the segment passed in contains records that cause + * offset overflow. + * + * 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.

+ * + * @param segment Segment to split + * @param existingSegments The existing segments of the log + * @param dir The directory in which the log will reside + * @param topicPartition The topic + * @param config The log configuration settings + * @param scheduler The thread pool scheduler used for background actions + * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure + * @param producerStateManager The ProducerStateManager instance (if any) containing state associated + * with the existingSegments + * @return List of new segments that replace the input segment + */ + private[log] def splitOverflowedSegment(segment: LogSegment, + existingSegments: LogSegments, + dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + logDirFailureChannel: LogDirFailureChannel, + producerStateManager: ProducerStateManager): List[LogSegment] = { + require(Log.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") + replaceSegments(existingSegments, newSegments.toList, List(segment), isRecoveredSwapFile = false, + dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager) + newSegments.toList + } catch { + case e: Exception => + newSegments.foreach { splitSegment => + splitSegment.close() + splitSegment.deleteIfExists() + } + throw e + } + } } object LogMetricNames { @@ -2834,12 +2642,6 @@ case object StartOffsetBreach extends SegmentDeletionReason { } } -case object LogRecovery extends SegmentDeletionReason { - override def logReason(log: Log, toDelete: List[LogSegment]): Unit = { - log.info(s"Deleting segments as part of log recovery: ${toDelete.mkString(",")}") - } -} - case object LogTruncation extends SegmentDeletionReason { override def logReason(log: Log, toDelete: List[LogSegment]): Unit = { log.info(s"Deleting segments as part of log truncation: ${toDelete.mkString(",")}") diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index df9722c6ee2e0..80916cddaa8ba 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -450,12 +450,11 @@ object LogCleaner { } - def createNewCleanedSegment(log: Log, baseOffset: Long): LogSegment = { - LogSegment.deleteIfExists(log.dir, baseOffset, fileSuffix = Log.CleanedFileSuffix) - LogSegment.open(log.dir, baseOffset, log.config, Time.SYSTEM, - fileSuffix = Log.CleanedFileSuffix, initFileSize = log.initFileSize, preallocate = log.config.preallocate) + def createNewCleanedSegment(dir: File, logConfig: LogConfig, baseOffset: Long): LogSegment = { + LogSegment.deleteIfExists(dir, baseOffset, fileSuffix = Log.CleanedFileSuffix) + LogSegment.open(dir, baseOffset, logConfig, Time.SYSTEM, + fileSuffix = Log.CleanedFileSuffix, initFileSize = logConfig.initFileSize, preallocate = logConfig.preallocate) } - } /** @@ -563,7 +562,7 @@ private[log] class Cleaner(val id: Int, stats: CleanerStats, transactionMetadata: CleanedTransactionMetadata): Unit = { // create a new segment with a suffix appended to the name of the log and indexes - val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) + val cleaned = LogCleaner.createNewCleanedSegment(log.dir, log.config, segments.head.baseOffset) transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) try { diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 4a184423c7fa3..874dc13ea0af6 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -103,6 +103,13 @@ case class LogConfig(props: java.util.Map[_, _], overriddenConfigs: Set[String] if (compact && maxCompactionLagMs > 0) math.min(maxCompactionLagMs, segmentMs) else segmentMs } + + def initFileSize: Int = { + if (preallocate) + segmentSize + else + 0 + } } object LogConfig { diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala new file mode 100644 index 0000000000000..6b28ec51bb91d --- /dev/null +++ b/core/src/main/scala/kafka/log/LogLoader.scala @@ -0,0 +1,525 @@ +/** + * 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. + * + * 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. + * + * @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 + */ + 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)) + } + (0L, 0L) + } + } + + params.leaderEpochCache.foreach(_.truncateFromEnd(nextOffset)) + val newLogStartOffset = math.max(params.logStartOffsetCheckpoint, params.segments.firstSegment.get.baseOffset) + // The earliest leader epoch may not be flushed during a hard failure. Recover it here. + params.leaderEpochCache.foreach(_.truncateFromStart(params.logStartOffsetCheckpoint)) + + // Any segment loading or recovery code must not use producerStateManager, so that we can build the full state here + // from scratch. + if (!params.producerStateManager.isEmpty) + throw new IllegalStateException("Producer state must be empty during log initialization") + + // Reload all snapshots into the ProducerStateManager cache, the intermediate ProducerStateManager used + // during log recovery may have deleted some files without the LogLoader.producerStateManager instance witnessing the + // deletion. + params.producerStateManager.removeStraySnapshots(params.segments.baseOffsets.toSeq) + Log.rebuildProducerState( + params.producerStateManager, + params.segments, + newLogStartOffset, + nextOffset, + params.config.messageFormatVersion.recordVersion, + params.time, + reloadFromCleanShutdown = params.hadCleanShutdown) + + val activeSegment = params.segments.lastSegment.get + LoadedLogOffsets( + newLogStartOffset, + newRecoveryPoint, + LogOffsetMetadata(nextOffset, activeSegment.baseOffset, activeSegment.size)) + } + + /** + * 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. + * @param params The parameters for the log being loaded from disk + * @return Set of .swap files that are valid to be swapped in as segment files + */ + private def removeTempFilesAndCollectSwapFiles(params: LoadLogParams): Set[File] = { + + def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = { + info(s"${params.logIdentifier} Deleting index files with suffix $suffix for baseFile $baseFile") + val offset = offsetFromFile(baseFile) + Files.deleteIfExists(Log.offsetIndexFile(params.dir, offset, suffix).toPath) + Files.deleteIfExists(Log.timeIndexFile(params.dir, offset, suffix).toPath) + Files.deleteIfExists(Log.transactionIndexFile(params.dir, offset, suffix).toPath) + } + + val swapFiles = mutable.Set[File]() + val cleanFiles = mutable.Set[File]() + var minCleanedFileOffset = Long.MaxValue + + for (file <- params.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"${params.logIdentifier} 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"${params.logIdentifier} Found file ${file.getAbsolutePath} from interrupted swap operation.") + if (Log.isIndexFile(baseFile)) { + deleteIndicesIfExist(baseFile) + } else if (Log.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 Log#splitOverflowedSegment + // for more details about the split operation. + val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset) + invalidSwapFiles.foreach { file => + debug(s"${params.logIdentifier} 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"${params.logIdentifier} Deleting stray .clean file ${file.getAbsolutePath}") + Files.deleteIfExists(file.toPath) + } + + validSwapFiles + } + + /** + * Retries the provided function only whenever an LogSegmentOffsetOverflowException is raised by + * it during execution. Before every retry, the overflowed segment is split into one or more segments + * such that there is no offset overflow in any of them. + * + * @param params The parameters for the log being loaded from disk + * @param fn The function to be executed + * @return The value returned by the function, if successful + * @throws Exception whenever the executed function throws any exception other than + * LogSegmentOffsetOverflowException, the same exception is raised to the caller + */ + private def retryOnOffsetOverflow[T](params: LoadLogParams, fn: => T): T = { + while (true) { + try { + return fn + } catch { + case e: LogSegmentOffsetOverflowException => + info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.") + Log.splitOverflowedSegment( + e.segment, + params.segments, + params.dir, + params.topicPartition, + params.config, + params.scheduler, + params.logDirFailureChannel, + params.producerStateManager) + } + } + throw new IllegalStateException() + } + + /** + * Loads segments from disk into the provided params.segments. + * + * 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 params The parameters for the log being loaded from disk + * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset + */ + private def loadSegmentFiles(params: LoadLogParams): Unit = { + // load segments in ascending order because transactional data from one segment may depend on the + // segments that come before it + for (file <- params.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 = Log.logFile(params.dir, offset) + if (!logFile.exists) { + warn(s"${params.logIdentifier} 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(params.dir, baseOffset).exists() + val segment = LogSegment.open( + dir = params.dir, + baseOffset = baseOffset, + params.config, + time = params.time, + fileAlreadyExists = true) + + try segment.sanityCheck(timeIndexFileNewlyCreated) + catch { + case _: NoSuchFileException => + error(s"${params.logIdentifier} Could not find offset index file corresponding to log file" + + s" ${segment.log.file.getAbsolutePath}, recovering segment and rebuilding index files...") + recoverSegment(segment, params) + case e: CorruptIndexException => + warn(s"${params.logIdentifier} Found a corrupted index file corresponding to log file" + + s" ${segment.log.file.getAbsolutePath} due to ${e.getMessage}}, recovering segment and" + + " rebuilding index files...") + recoverSegment(segment, params) + } + params.segments.add(segment) + } + } + } + + /** + * Just recovers the given segment, without adding it to the provided params.segments. + * + * @param segment Segment to recover + * @param params The parameters for the log being loaded from disk + * + * @return The number of bytes truncated from the segment + * + * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow + */ + private def recoverSegment(segment: LogSegment, params: LoadLogParams): Int = { + val producerStateManager = new ProducerStateManager(params.topicPartition, params.dir, params.maxProducerIdExpirationMs) + Log.rebuildProducerState( + producerStateManager, + params.segments, + params.logStartOffsetCheckpoint, + segment.baseOffset, + params.config.messageFormatVersion.recordVersion, + params.time, + reloadFromCleanShutdown = false) + val bytesTruncated = segment.recover(producerStateManager, params.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 completes any interrupted swap operations. In order to be crash-safe, the 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. + * + * This method does not need to convert IOException to KafkaStorageException because it is only + * called before all logs are loaded. + * + * @param swapFiles the set of swap + * @param params The parameters for the log being loaded from disk + * + * @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 def completeSwapOperations(swapFiles: Set[File], + params: LoadLogParams): Unit = { + for (swapFile <- swapFiles) { + val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, Log.SwapFileSuffix, "")) + val baseOffset = Log.offsetFromFile(logFile) + val swapSegment = LogSegment.open(swapFile.getParentFile, + baseOffset = baseOffset, + params.config, + time = params.time, + fileSuffix = Log.SwapFileSuffix) + info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.") + recoverSegment(swapSegment, params) + + // 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 = params.segments.values(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment => + segment.readNextOffset > swapSegment.baseOffset + } + Log.replaceSegments( + params.segments, + Seq(swapSegment), + oldSegments.toSeq, + isRecoveredSwapFile = true, + params.dir, + params.topicPartition, + params.config, + params.scheduler, + params.logDirFailureChannel, + params.producerStateManager) + } + } + + /** + * Recover the log segments (if there was an unclean shutdown). Ensures there is at least one + * active segment, and returns the updated recovery point and next offset after recovery. Along + * the way, the method suitably updates the LeaderEpochFileCache or ProducerStateManager inside + * the provided LogComponents. + * + * 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 a tuple containing (newRecoveryPoint, nextOffset). + * + * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow + */ + private[log] def recoverLog(params: LoadLogParams): (Long, Long) = { + /** return the log end offset if valid */ + def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = { + if (params.segments.nonEmpty) { + val logEndOffset = params.segments.lastSegment.get.readNextOffset + if (logEndOffset >= params.logStartOffsetCheckpoint) + Some(logEndOffset) + else { + warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ${params.logStartOffsetCheckpoint}. " + + "This could happen if segment files were deleted from the file system.") + removeAndDeleteSegmentsAsync(params.segments.values, params) + params.leaderEpochCache.foreach(_.clearAndFlush()) + params.producerStateManager.truncateFullyAndStartAt(params.logStartOffsetCheckpoint) + None + } + } else None + } + + // If we have the clean shutdown marker, skip recovery. + if (!params.hadCleanShutdown) { + val unflushed = params.segments.values(params.recoveryPointCheckpoint, Long.MaxValue).iterator + var truncated = false + + while (unflushed.hasNext && !truncated) { + val segment = unflushed.next() + info(s"${params.logIdentifier} Recovering unflushed segment ${segment.baseOffset}") + val truncatedBytes = + try { + recoverSegment(segment, params) + } catch { + case _: InvalidOffsetException => + val startOffset = segment.baseOffset + warn(s"${params.logIdentifier} Found invalid offset during recovery. Deleting the" + + s" corrupt segment and 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"${params.logIdentifier} Corruption found in segment ${segment.baseOffset}," + + s" truncating to offset ${segment.readNextOffset}") + removeAndDeleteSegmentsAsync(unflushed.toList, params) + truncated = true + } + } + } + + val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd() + + if (params.segments.isEmpty) { + // no existing segments, create a new mutable segment beginning at logStartOffset + params.segments.add( + LogSegment.open( + dir = params.dir, + baseOffset = params.logStartOffsetCheckpoint, + params.config, + time = params.time, + initFileSize = params.config.initFileSize, + preallocate = params.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 when the log is flushed. 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. + (params.hadCleanShutdown, logEndOffsetOption) match { + case (true, Some(logEndOffset)) => + (logEndOffset, logEndOffset) + case _ => + val logEndOffset = logEndOffsetOption.getOrElse(params.segments.lastSegment.get.readNextOffset) + (Math.min(params.recoveryPointCheckpoint, logEndOffset), logEndOffset) + } + } + + /** + * This method deletes the given log segments and the associated producer snapshots, by doing the + * following for each of them: + * - It removes the segment from the segment map so that it will no longer be used for reads. + * - It schedules asynchronous deletion of the segments that 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 segmentsToDelete The log segments to schedule for deletion + * @param params The parameters for the log being loaded from disk + */ + private def removeAndDeleteSegmentsAsync(segmentsToDelete: Iterable[LogSegment], + params: LoadLogParams): Unit = { + if (segmentsToDelete.nonEmpty) { + // As most callers hold an iterator into the `params.segments` collection and + // `removeAndDeleteSegmentAsync` 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 = segmentsToDelete.toList + info(s"Deleting segments as part of log recovery: ${toDelete.mkString(",")}") + toDelete.foreach { segment => + params.segments.remove(segment.baseOffset) + } + Log.deleteSegmentFiles( + segmentsToDelete, + asyncDelete = true, + deleteProducerStateSnapshots = true, + params.dir, + params.topicPartition, + params.config, + params.scheduler, + params.logDirFailureChannel, + params.producerStateManager) + } + } +} diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index a8198c9c02e09..8b7b18b03693b 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -25,6 +25,7 @@ import kafka.api.ApiVersion import kafka.log._ import kafka.server._ import kafka.server.checkpoints.OffsetCheckpoints +import kafka.server.epoch.LeaderEpochFileCache import kafka.utils._ import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.{TopicPartition, Uuid} @@ -280,7 +281,25 @@ class PartitionLockTest extends Logging { override def createLog(isNew: Boolean, isFutureReplica: Boolean, offsetCheckpoints: OffsetCheckpoints, topicId: Option[Uuid]): Log = { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None) - new SlowLog(log, mockTime, appendSemaphore) + val logDirFailureChannel = new LogDirFailureChannel(1) + val segments = new LogSegments(log.topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(log.topicPartition, log.dir, log.maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + log.dir, + log.topicPartition, + log.config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + log.maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + new SlowLog(log, segments, offsets, leaderEpochCache, producerStateManager, mockTime, logDirFailureChannel, appendSemaphore) } } when(offsetCheckpoints.fetch(ArgumentMatchers.anyString, ArgumentMatchers.eq(topicPartition))) @@ -341,19 +360,31 @@ class PartitionLockTest extends Logging { } } - private class SlowLog(log: Log, mockTime: MockTime, appendSemaphore: Semaphore) extends Log( + private class SlowLog( + log: Log, + segments: LogSegments, + offsets: LoadedLogOffsets, + leaderEpochCache: Option[LeaderEpochFileCache], + producerStateManager: ProducerStateManager, + mockTime: MockTime, + logDirFailureChannel: LogDirFailureChannel, + appendSemaphore: Semaphore + ) extends Log( log.dir, log.config, - log.logStartOffset, - log.recoveryPoint, + segments, + offsets.logStartOffset, + offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, new BrokerTopicStats, - log.time, + mockTime, log.maxProducerIdExpirationMs, log.producerIdExpirationCheckIntervalMs, log.topicPartition, - log.producerStateManager, - new LogDirFailureChannel(1), + leaderEpochCache, + producerStateManager, + logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) { @@ -362,5 +393,5 @@ class PartitionLockTest extends Logging { appendSemaphore.acquire() appendInfo } - } + } } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index dd671e543e29c..784f70d1678b2 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -41,10 +41,12 @@ import org.mockito.ArgumentMatchers import org.mockito.ArgumentMatchers.{any, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock - import java.nio.ByteBuffer import java.util.Optional import java.util.concurrent.{CountDownLatch, Semaphore} + +import kafka.server.epoch.LeaderEpochFileCache + import scala.jdk.CollectionConverters._ class PartitionTest extends AbstractPartitionTest { @@ -232,7 +234,25 @@ class PartitionTest extends AbstractPartitionTest { override def createLog(isNew: Boolean, isFutureReplica: Boolean, offsetCheckpoints: OffsetCheckpoints, topicId: Option[Uuid]): Log = { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None) - new SlowLog(log, mockTime, appendSemaphore) + val logDirFailureChannel = new LogDirFailureChannel(1) + val segments = new LogSegments(log.topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(log.topicPartition, log.dir, log.maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + log.dir, + log.topicPartition, + log.config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + log.maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + new SlowLog(log, segments, offsets, leaderEpochCache, producerStateManager, mockTime, logDirFailureChannel, appendSemaphore) } } @@ -1934,19 +1954,31 @@ class PartitionTest extends AbstractPartitionTest { } } - private class SlowLog(log: Log, mockTime: MockTime, appendSemaphore: Semaphore) extends Log( + private class SlowLog( + log: Log, + segments: LogSegments, + offsets: LoadedLogOffsets, + leaderEpochCache: Option[LeaderEpochFileCache], + producerStateManager: ProducerStateManager, + mockTime: MockTime, + logDirFailureChannel: LogDirFailureChannel, + appendSemaphore: Semaphore + ) extends Log( log.dir, log.config, - log.logStartOffset, - log.recoveryPoint, + segments, + offsets.logStartOffset, + offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, new BrokerTopicStats, - log.time, + mockTime, log.maxProducerIdExpirationMs, log.producerIdExpirationCheckIntervalMs, log.topicPartition, - log.producerStateManager, - new LogDirFailureChannel(1), + leaderEpochCache, + producerStateManager, + logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 16aac94dc5df6..cdaa89de9e34e 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -18,6 +18,7 @@ package kafka.log import java.io.File +import java.nio.file.Files import java.util.Properties import kafka.server.{BrokerTopicStats, LogDirFailureChannel} @@ -94,19 +95,39 @@ class LogCleanerManagerTest extends Logging { val logSegmentSize = TestUtils.singletonRecords("test".getBytes).sizeInBytes * 10 val logSegmentsCount = 2 val tpDir = new File(logDir, "A-1") - - // the exception should be catched and the partition that caused it marked as uncleanable - class LogMock(dir: File, config: LogConfig) extends Log(dir, config, 0L, 0L, - time.scheduler, new BrokerTopicStats, time, 60 * 60 * 1000, LogManager.ProducerIdExpirationCheckIntervalMs, - topicPartition, new ProducerStateManager(tp, tpDir, 60 * 60 * 1000), - new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true) { - + Files.createDirectories(tpDir.toPath) + val logDirFailureChannel = new LogDirFailureChannel(10) + val config = createLowRetentionLogConfig(logSegmentSize, LogConfig.Compact) + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val segments = new LogSegments(tp) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(tpDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(topicPartition, tpDir, maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + tpDir, + tp, + config, + time.scheduler, + time, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + // the exception should be caught and the partition that caused it marked as uncleanable + class LogMock(dir: File, config: LogConfig, offsets: LoadedLogOffsets) + extends Log(dir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, + offsets.nextOffsetMetadata, time.scheduler, new BrokerTopicStats, time, maxProducerIdExpirationMs, + LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, + producerStateManager, logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) { // Throw an error in getFirstBatchTimestampForSegments since it is called in grabFilthiestLog() override def getFirstBatchTimestampForSegments(segments: Iterable[LogSegment]): Iterable[Long] = throw new IllegalStateException("Error!") } - val log: Log = new LogMock(tpDir, createLowRetentionLogConfig(logSegmentSize, LogConfig.Compact)) + val log: Log = new LogMock(tpDir, config, offsets) writeRecords(log = log, numBatches = logSegmentsCount * 2, recordsPerBatch = 10, diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 155bd6a776546..de55724d6ba71 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -100,19 +100,43 @@ class LogCleanerTest { val logProps = new Properties() logProps.put(LogConfig.SegmentBytesProp, 1024 : java.lang.Integer) logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact + "," + LogConfig.Delete) + val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = Log.parseTopicPartitionName(dir) - val producerStateManager = new ProducerStateManager(topicPartition, dir) + val logDirFailureChannel = new LogDirFailureChannel(10) + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val logSegments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + dir, + topicPartition, + config, + time.scheduler, + time, + logDirFailureChannel, + hadCleanShutdown = true, + logSegments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + val log = new Log(dir, - config = LogConfig.fromProps(logConfig.originals, logProps), - logStartOffset = 0L, - recoveryPoint = 0L, + config = config, + segments = logSegments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, scheduler = time.scheduler, - brokerTopicStats = new BrokerTopicStats, time, - maxProducerIdExpirationMs = 60 * 60 * 1000, + brokerTopicStats = new BrokerTopicStats, + time, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, producerStateManager = producerStateManager, - logDirFailureChannel = new LogDirFailureChannel(10), + logDirFailureChannel = logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) { override def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = { @@ -1755,7 +1779,7 @@ class LogCleanerTest { private def tombstoneRecord(key: Int): MemoryRecords = record(key, null) private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): Log = { - LogTest.recoverAndCheck(dir, config, expectedKeys, new BrokerTopicStats(), time, time.scheduler) + LogTestUtils.recoverAndCheck(dir, config, expectedKeys, new BrokerTopicStats(), time, time.scheduler) } } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala new file mode 100644 index 0000000000000..0d6e7e3dfc9d4 --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -0,0 +1,1501 @@ +/** + * 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.{BufferedWriter, File, FileWriter} +import java.nio.ByteBuffer +import java.nio.file.{Files, Paths} +import java.util.Properties + +import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0} +import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} +import kafka.server.{BrokerTopicStats, FetchDataInfo, KafkaConfig, LogDirFailureChannel} +import kafka.server.metadata.CachedConfigRepository +import kafka.utils.{CoreUtils, MockTime, Scheduler, TestUtils} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.record.{CompressionType, ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType} +import org.apache.kafka.common.utils.{Time, Utils} +import org.easymock.EasyMock +import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} + +import scala.collection.mutable.ListBuffer +import scala.collection.{Iterable, Map, mutable} +import scala.jdk.CollectionConverters._ + +class LogLoaderTest { + var config: KafkaConfig = null + val brokerTopicStats = new BrokerTopicStats + val tmpDir = TestUtils.tempDir() + val logDir = TestUtils.randomPartitionLogDir(tmpDir) + val mockTime = new MockTime() + + @BeforeEach + def setUp(): Unit = { + val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1) + config = KafkaConfig.fromProps(props) + } + + @AfterEach + def tearDown(): Unit = { + brokerTopicStats.close() + Utils.delete(tmpDir) + } + + @Test + def testLogRecoveryIsCalledUponBrokerCrash(): Unit = { + // LogManager must realize correctly if the last shutdown was not clean and the logs need + // to run recovery while loading upon subsequent broker boot up. + val logDir: File = TestUtils.tempDir() + val logProps = new Properties() + val logConfig = LogConfig(logProps) + val logDirs = Seq(logDir) + val topicPartition = new TopicPartition("foo", 0) + var log: Log = null + val time = new MockTime() + var cleanShutdownInterceptedValue = false + case class SimulateError(var hasError: Boolean = false) + val simulateError = SimulateError() + + // Create a LogManager with some overridden methods to facilitate interception of clean shutdown + // flag and to inject a runtime error + def interceptedLogManager(logConfig: LogConfig, logDirs: Seq[File], simulateError: SimulateError): LogManager = { + new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], new CachedConfigRepository(), + initialDefaultConfig = logConfig, cleanerConfig = CleanerConfig(enableCleaner = false), recoveryThreadsPerDataDir = 4, + flushCheckMs = 1000L, flushRecoveryOffsetCheckpointMs = 10000L, flushStartOffsetCheckpointMs = 10000L, + retentionCheckMs = 1000L, maxPidExpirationMs = 60 * 60 * 1000, scheduler = time.scheduler, time = time, + brokerTopicStats = new BrokerTopicStats, logDirFailureChannel = new LogDirFailureChannel(logDirs.size), keepPartitionMetadataFile = config.usesTopicId) { + + override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long], + logStartOffsets: Map[TopicPartition, Long], topicConfigs: Map[String, LogConfig]): Log = { + if (simulateError.hasError) { + throw new RuntimeException("Simulated error") + } + cleanShutdownInterceptedValue = hadCleanShutdown + val topicPartition = Log.parseTopicPartitionName(logDir) + val config = topicConfigs.getOrElse(topicPartition.topic, currentDefaultConfig) + val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) + val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) + val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) + val loadLogParams = LoadLogParams(logDir, topicPartition, config, time.scheduler, time, + logDirFailureChannel, hadCleanShutdown, segments, logStartOffset, logRecoveryPoint, + maxProducerIdExpirationMs, leaderEpochCache, producerStateManager) + val offsets = LogLoader.load(loadLogParams) + new Log(logDir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, + offsets.nextOffsetMetadata, time.scheduler, brokerTopicStats, time, maxPidExpirationMs, + LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, + producerStateManager, logDirFailureChannel, None, true) + } + } + } + + val cleanShutdownFile = new File(logDir, Log.CleanShutdownFile) + locally { + val logManager: LogManager = interceptedLogManager(logConfig, logDirs, simulateError) + log = logManager.getOrCreateLog(topicPartition, isNew = true, topicId = None) + + // Load logs after a clean shutdown + Files.createFile(cleanShutdownFile.toPath) + cleanShutdownInterceptedValue = false + logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) + assertTrue(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") + assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") + // Load logs without clean shutdown file + cleanShutdownInterceptedValue = true + logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) + assertFalse(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") + assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") + // Create clean shutdown file and then simulate error while loading logs such that log loading does not complete. + Files.createFile(cleanShutdownFile.toPath) + logManager.shutdown() + } + + locally { + simulateError.hasError = true + val logManager: LogManager = interceptedLogManager(logConfig, logDirs, simulateError) + log = logManager.getOrCreateLog(topicPartition, isNew = true, topicId = None) + + // Simulate error + assertThrows(classOf[RuntimeException], () => logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty))) + assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not have existed") + // Do not simulate error on next call to LogManager#loadLogs. LogManager must understand that log had unclean shutdown the last time. + simulateError.hasError = false + cleanShutdownInterceptedValue = true + logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) + assertFalse(cleanShutdownInterceptedValue, "Unexpected value for clean shutdown flag") + } + } + + @Test + def testProducerSnapshotsRecoveryAfterUncleanShutdownV1(): Unit = { + testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.minSupportedFor(RecordVersion.V1).version) + } + + @Test + def testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat(): Unit = { + testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.latestVersion.version) + } + + private def createLog(dir: File, + config: LogConfig, + brokerTopicStats: BrokerTopicStats = brokerTopicStats, + logStartOffset: Long = 0L, + recoveryPoint: Long = 0L, + scheduler: Scheduler = mockTime.scheduler, + time: Time = mockTime, + maxProducerIdExpirationMs: Int = 60 * 60 * 1000, + producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs, + lastShutdownClean: Boolean = true): Log = { + LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint, + maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs, lastShutdownClean) + } + + private def createLogWithOffsetOverflow(logConfig: LogConfig): (Log, LogSegment) = { + LogTestUtils.initializeLogDirWithOverflowedSegment(logDir) + + val log = createLog(logDir, logConfig, recoveryPoint = Long.MaxValue) + val segmentWithOverflow = LogTestUtils.firstOverflowSegment(log).getOrElse { + throw new AssertionError("Failed to create log with a segment which has overflowed offsets") + } + + (log, segmentWithOverflow) + } + + private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): Log = { + // method is called only in case of recovery from hard reset + LogTestUtils.recoverAndCheck(logDir, config, expectedKeys, brokerTopicStats, mockTime, mockTime.scheduler) + } + + /** + * Wrap a single record log buffer with leader epoch. + */ + private def singletonRecordsWithLeaderEpoch(value: Array[Byte], + key: Array[Byte] = null, + leaderEpoch: Int, + offset: Long, + codec: CompressionType = CompressionType.NONE, + timestamp: Long = RecordBatch.NO_TIMESTAMP, + magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = { + val records = Seq(new SimpleRecord(timestamp, key, value)) + + val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) + val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, offset, + mockTime.milliseconds, leaderEpoch) + records.foreach(builder.append) + builder.build() + } + + private def testProducerSnapshotsRecoveryAfterUncleanShutdown(messageFormatVersion: String): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 64 * 10, messageFormatVersion = messageFormatVersion) + var log = createLog(logDir, logConfig) + assertEquals(None, log.oldestProducerSnapshotOffset) + + for (i <- 0 to 100) { + val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes) + log.appendAsLeader(TestUtils.records(List(record)), leaderEpoch = 0) + } + + assertTrue(log.logSegments.size >= 5) + val segmentOffsets = log.logSegments.toVector.map(_.baseOffset) + val activeSegmentOffset = segmentOffsets.last + + // We want the recovery point to be past the segment offset and before the last 2 segments including a gap of + // 1 segment. We collect the data before closing the log. + val offsetForSegmentAfterRecoveryPoint = segmentOffsets(segmentOffsets.size - 3) + val offsetForRecoveryPointSegment = segmentOffsets(segmentOffsets.size - 4) + val (segOffsetsBeforeRecovery, segOffsetsAfterRecovery) = segmentOffsets.toSet.partition(_ < offsetForRecoveryPointSegment) + val recoveryPoint = offsetForRecoveryPointSegment + 1 + assertTrue(recoveryPoint < offsetForSegmentAfterRecoveryPoint) + log.close() + + val segmentsWithReads = mutable.Set[LogSegment]() + val recoveredSegments = mutable.Set[LogSegment]() + val expectedSegmentsWithReads = mutable.Set[Long]() + val expectedSnapshotOffsets = mutable.Set[Long]() + + if (logConfig.messageFormatVersion < KAFKA_0_11_0_IV0) { + expectedSegmentsWithReads += activeSegmentOffset + expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(2) :+ log.logEndOffset + } else { + expectedSegmentsWithReads ++= segOffsetsBeforeRecovery ++ Set(activeSegmentOffset) + expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(4) :+ log.logEndOffset + } + + def createLogWithInterceptedReads(recoveryPoint: Long) = { + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val topicPartition = Log.parseTopicPartitionName(logDir) + val logDirFailureChannel = new LogDirFailureChannel(10) + // Intercept all segment read calls + val interceptedLogSegments = new LogSegments(topicPartition) { + override def add(segment: LogSegment): LogSegment = { + val wrapper = new LogSegment(segment.log, segment.lazyOffsetIndex, segment.lazyTimeIndex, segment.txnIndex, segment.baseOffset, + segment.indexIntervalBytes, segment.rollJitterMs, mockTime) { + + override def read(startOffset: Long, maxSize: Int, maxPosition: Long, minOneMessage: Boolean): FetchDataInfo = { + segmentsWithReads += this + super.read(startOffset, maxSize, maxPosition, minOneMessage) + } + + override def recover(producerStateManager: ProducerStateManager, + leaderEpochCache: Option[LeaderEpochFileCache]): Int = { + recoveredSegments += this + super.recover(producerStateManager, leaderEpochCache) + } + } + super.add(wrapper) + } + } + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) + val loadLogParams = LoadLogParams( + logDir, + topicPartition, + logConfig, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = false, + interceptedLogSegments, + 0L, + recoveryPoint, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager) + val offsets = LogLoader.load(loadLogParams) + new Log(logDir, logConfig, interceptedLogSegments, offsets.logStartOffset, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, brokerTopicStats, mockTime, + maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, + leaderEpochCache, producerStateManager, logDirFailureChannel, topicId = None, + keepPartitionMetadataFile = true) + } + + // Retain snapshots for the last 2 segments + log.producerStateManager.deleteSnapshotsBefore(segmentOffsets(segmentOffsets.size - 2)) + log = createLogWithInterceptedReads(offsetForRecoveryPointSegment) + // We will reload all segments because the recovery point is behind the producer snapshot files (pre KAFKA-5829 behaviour) + assertEquals(expectedSegmentsWithReads, segmentsWithReads.map(_.baseOffset)) + assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir).toSet) + log.close() + segmentsWithReads.clear() + recoveredSegments.clear() + + // Only delete snapshots before the base offset of the recovery point segment (post KAFKA-5829 behaviour) to + // avoid reading all segments + log.producerStateManager.deleteSnapshotsBefore(offsetForRecoveryPointSegment) + log = createLogWithInterceptedReads(recoveryPoint = recoveryPoint) + assertEquals(Set(activeSegmentOffset), segmentsWithReads.map(_.baseOffset)) + assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir).toSet) + + log.close() + } + + @Test + def testSkipLoadingIfEmptyProducerStateBeforeTruncation(): Unit = { + val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) + EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() + // Load the log + EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) + + stateManager.updateMapEndOffset(0L) + EasyMock.expectLastCall().anyTimes() + + EasyMock.expect(stateManager.mapEndOffset).andStubReturn(0L) + EasyMock.expect(stateManager.isEmpty).andStubReturn(true) + + stateManager.takeSnapshot() + EasyMock.expectLastCall().anyTimes() + + stateManager.truncateAndReload(EasyMock.eq(0L), EasyMock.eq(0L), EasyMock.anyLong) + EasyMock.expectLastCall() + + EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) + + EasyMock.replay(stateManager) + + val topicPartition = Log.parseTopicPartitionName(logDir) + val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) + val config = LogConfig(new Properties()) + val maxProducerIdExpirationMs = 300000 + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = false, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + stateManager)) + val log = new Log(logDir, + config, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, + scheduler = mockTime.scheduler, + brokerTopicStats = brokerTopicStats, + time = mockTime, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = 30000, + topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, + producerStateManager = stateManager, + logDirFailureChannel = logDirFailureChannel, + topicId = None, + keepPartitionMetadataFile = true) + + EasyMock.verify(stateManager) + + // Append some messages + EasyMock.reset(stateManager) + EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) + + stateManager.updateMapEndOffset(1L) + EasyMock.expectLastCall() + stateManager.updateMapEndOffset(2L) + EasyMock.expectLastCall() + + EasyMock.replay(stateManager) + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes))), leaderEpoch = 0) + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes))), leaderEpoch = 0) + + EasyMock.verify(stateManager) + + // Now truncate + EasyMock.reset(stateManager) + EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) + EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) + EasyMock.expect(stateManager.isEmpty).andStubReturn(true) + EasyMock.expect(stateManager.mapEndOffset).andReturn(2L) + stateManager.truncateAndReload(EasyMock.eq(0L), EasyMock.eq(1L), EasyMock.anyLong) + EasyMock.expectLastCall() + // Truncation causes the map end offset to reset to 0 + EasyMock.expect(stateManager.mapEndOffset).andReturn(0L) + // We skip directly to updating the map end offset + EasyMock.expect(stateManager.updateMapEndOffset(1L)) + EasyMock.expect(stateManager.onHighWatermarkUpdated(0L)) + + // Finally, we take a snapshot + stateManager.takeSnapshot() + EasyMock.expectLastCall().once() + + EasyMock.replay(stateManager) + + log.truncateTo(1L) + + EasyMock.verify(stateManager) + } + + @Test + def testRecoverAfterNonMonotonicCoordinatorEpochWrite(): Unit = { + // Due to KAFKA-9144, we may encounter a coordinator epoch which goes backwards. + // This test case verifies that recovery logic relaxes validation in this case and + // just takes the latest write. + + val producerId = 1L + val coordinatorEpoch = 5 + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) + var log = createLog(logDir, logConfig) + val epoch = 0.toShort + + val firstAppendTimestamp = mockTime.milliseconds() + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, + firstAppendTimestamp, coordinatorEpoch = coordinatorEpoch) + assertEquals(firstAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) + + mockTime.sleep(log.maxProducerIdExpirationMs) + assertEquals(None, log.producerStateManager.lastEntry(producerId)) + + val secondAppendTimestamp = mockTime.milliseconds() + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, + secondAppendTimestamp, coordinatorEpoch = coordinatorEpoch - 1) + + log.close() + + // Force recovery by setting the recoveryPoint to the log start + log = createLog(logDir, logConfig, recoveryPoint = 0L, lastShutdownClean = false) + assertEquals(secondAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) + log.close() + } + + @Test + def testSkipTruncateAndReloadIfOldMessageFormatAndNoCleanShutdown(): Unit = { + val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) + EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() + + stateManager.updateMapEndOffset(0L) + EasyMock.expectLastCall().anyTimes() + + stateManager.takeSnapshot() + EasyMock.expectLastCall().anyTimes() + + EasyMock.expect(stateManager.isEmpty).andReturn(true) + EasyMock.expectLastCall().once() + + EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) + EasyMock.expectLastCall().once() + + EasyMock.replay(stateManager) + + val topicPartition = Log.parseTopicPartitionName(logDir) + val logProps = new Properties() + logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") + val config = LogConfig(logProps) + val maxProducerIdExpirationMs = 300000 + val logDirFailureChannel = null + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = false, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + stateManager)) + new Log(logDir, + config, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, + scheduler = mockTime.scheduler, + brokerTopicStats = brokerTopicStats, + time = mockTime, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = 30000, + topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, + producerStateManager = stateManager, + logDirFailureChannel = logDirFailureChannel, + topicId = None, + keepPartitionMetadataFile = true) + + EasyMock.verify(stateManager) + } + + @Test + def testSkipTruncateAndReloadIfOldMessageFormatAndCleanShutdown(): Unit = { + val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) + EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() + + stateManager.updateMapEndOffset(0L) + EasyMock.expectLastCall().anyTimes() + + stateManager.takeSnapshot() + EasyMock.expectLastCall().anyTimes() + + EasyMock.expect(stateManager.isEmpty).andReturn(true) + EasyMock.expectLastCall().once() + + EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) + EasyMock.expectLastCall().once() + + EasyMock.replay(stateManager) + + val topicPartition = Log.parseTopicPartitionName(logDir) + val logProps = new Properties() + logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") + val config = LogConfig(logProps) + val maxProducerIdExpirationMs = 300000 + val logDirFailureChannel = null + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + stateManager)) + new Log(logDir, + config, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, + scheduler = mockTime.scheduler, + brokerTopicStats = brokerTopicStats, + time = mockTime, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = 30000, + topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, + producerStateManager = stateManager, + logDirFailureChannel = logDirFailureChannel, + topicId = None, + keepPartitionMetadataFile = true) + + EasyMock.verify(stateManager) + } + + @Test + def testSkipTruncateAndReloadIfNewMessageFormatAndCleanShutdown(): Unit = { + val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) + EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() + + EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) + + stateManager.updateMapEndOffset(0L) + EasyMock.expectLastCall().anyTimes() + + stateManager.takeSnapshot() + EasyMock.expectLastCall().anyTimes() + + EasyMock.expect(stateManager.isEmpty).andReturn(true) + EasyMock.expectLastCall().once() + + EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) + EasyMock.expectLastCall().once() + + EasyMock.replay(stateManager) + + val topicPartition = Log.parseTopicPartitionName(logDir) + val logProps = new Properties() + logProps.put(LogConfig.MessageFormatVersionProp, "0.11.0") + val config = LogConfig(logProps) + val maxProducerIdExpirationMs = 300000 + val logDirFailureChannel = null + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + stateManager)) + new Log(logDir, + config, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, + scheduler = mockTime.scheduler, + brokerTopicStats = brokerTopicStats, + time = mockTime, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = 30000, + topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, + producerStateManager = stateManager, + logDirFailureChannel = logDirFailureChannel, + topicId = None, + keepPartitionMetadataFile = true) + + EasyMock.verify(stateManager) + } + + @Test + def testLoadProducersAfterDeleteRecordsMidSegment(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) + val log = createLog(logDir, logConfig) + val pid1 = 1L + val pid2 = 2L + val epoch = 0.toShort + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1, + producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2, + producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + assertEquals(2, log.activeProducersWithLastSequence.size) + + log.updateHighWatermark(log.logEndOffset) + log.maybeIncrementLogStartOffset(1L, ClientRecordDeletion) + + // Deleting records should not remove producer state + assertEquals(2, log.activeProducersWithLastSequence.size) + val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) + assertTrue(retainedLastSeqOpt.isDefined) + assertEquals(0, retainedLastSeqOpt.get) + + log.close() + + // Because the log start offset did not advance, producer snapshots will still be present and the state will be rebuilt + val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false) + assertEquals(2, reloadedLog.activeProducersWithLastSequence.size) + val reloadedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) + assertEquals(retainedLastSeqOpt, reloadedLastSeqOpt) + } + + @Test + def testLoadingLogKeepsLargestStrayProducerStateSnapshot(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = 0, retentionMs = 1000 * 60, fileDeleteDelayMs = 0) + val log = createLog(logDir, logConfig) + val pid1 = 1L + val epoch = 0.toShort + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + log.roll() + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 1), leaderEpoch = 0) + log.roll() + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("c".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 2), leaderEpoch = 0) + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("d".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 3), leaderEpoch = 0) + + // Close the log, we should now have 3 segments + log.close() + assertEquals(log.logSegments.size, 3) + // We expect 3 snapshot files, two of which are for the first two segments, the last was written out during log closing. + assertEquals(Seq(1, 2, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) + // Inject a stray snapshot file within the bounds of the log at offset 3, it should be cleaned up after loading the log + val straySnapshotFile = Log.producerSnapshotFile(logDir, 3).toPath + Files.createFile(straySnapshotFile) + assertEquals(Seq(1, 2, 3, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) + + createLog(logDir, logConfig, lastShutdownClean = false) + // We should clean up the stray producer state snapshot file, but keep the largest snapshot file (4) + assertEquals(Seq(1, 2, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) + } + + @Test + def testLoadProducersAfterDeleteRecordsOnSegment(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) + val log = createLog(logDir, logConfig) + val pid1 = 1L + val pid2 = 2L + val epoch = 0.toShort + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1, + producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + log.roll() + log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2, + producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + + assertEquals(2, log.logSegments.size) + assertEquals(2, log.activeProducersWithLastSequence.size) + + log.updateHighWatermark(log.logEndOffset) + log.maybeIncrementLogStartOffset(1L, ClientRecordDeletion) + log.deleteOldSegments() + + // Deleting records should not remove producer state + assertEquals(1, log.logSegments.size) + assertEquals(2, log.activeProducersWithLastSequence.size) + val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) + assertTrue(retainedLastSeqOpt.isDefined) + assertEquals(0, retainedLastSeqOpt.get) + + log.close() + + // After reloading log, producer state should not be regenerated + val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false) + assertEquals(1, reloadedLog.activeProducersWithLastSequence.size) + val reloadedEntryOpt = log.activeProducersWithLastSequence.get(pid2) + assertEquals(retainedLastSeqOpt, reloadedEntryOpt) + } + + /** + * Append a bunch of messages to a log and then re-open it both with and without recovery and check that the log re-initializes correctly. + */ + @Test + def testLogRecoversToCorrectOffset(): Unit = { + val numMessages = 100 + val messageSize = 100 + val segmentSize = 7 * messageSize + val indexInterval = 3 * messageSize + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = indexInterval, segmentIndexBytes = 4096) + var log = createLog(logDir, logConfig) + for(i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize), + timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) + assertEquals(numMessages, log.logEndOffset, + "After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages)) + val lastIndexOffset = log.activeSegment.offsetIndex.lastOffset + val numIndexEntries = log.activeSegment.offsetIndex.entries + val lastOffset = log.logEndOffset + // After segment is closed, the last entry in the time index should be (largest timestamp -> last offset). + val lastTimeIndexOffset = log.logEndOffset - 1 + val lastTimeIndexTimestamp = log.activeSegment.largestTimestamp + // Depending on when the last time index entry is inserted, an entry may or may not be inserted into the time index. + val numTimeIndexEntries = log.activeSegment.timeIndex.entries + { + if (log.activeSegment.timeIndex.lastEntry.offset == log.logEndOffset - 1) 0 else 1 + } + log.close() + + def verifyRecoveredLog(log: Log, expectedRecoveryPoint: Long): Unit = { + assertEquals(expectedRecoveryPoint, log.recoveryPoint, s"Unexpected recovery point") + assertEquals(numMessages, log.logEndOffset, s"Should have $numMessages messages when log is reopened w/o recovery") + assertEquals(lastIndexOffset, log.activeSegment.offsetIndex.lastOffset, "Should have same last index offset as before.") + assertEquals(numIndexEntries, log.activeSegment.offsetIndex.entries, "Should have same number of index entries as before.") + assertEquals(lastTimeIndexTimestamp, log.activeSegment.timeIndex.lastEntry.timestamp, "Should have same last time index timestamp") + assertEquals(lastTimeIndexOffset, log.activeSegment.timeIndex.lastEntry.offset, "Should have same last time index offset") + assertEquals(numTimeIndexEntries, log.activeSegment.timeIndex.entries, "Should have same number of time index entries as before.") + } + + log = createLog(logDir, logConfig, recoveryPoint = lastOffset, lastShutdownClean = false) + verifyRecoveredLog(log, lastOffset) + log.close() + + // test recovery case + val recoveryPoint = 10 + log = createLog(logDir, logConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) + // the recovery point should not be updated after unclean shutdown until the log is flushed + verifyRecoveredLog(log, recoveryPoint) + log.flush() + verifyRecoveredLog(log, lastOffset) + log.close() + } + + /** + * Test that if we manually delete an index segment it is rebuilt when the log is re-opened + */ + @Test + def testIndexRebuild(): Unit = { + // publish the messages and close the log + val numMessages = 200 + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) + var log = createLog(logDir, logConfig) + for(i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) + val indexFiles = log.logSegments.map(_.lazyOffsetIndex.file) + val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) + log.close() + + // delete all the index files + indexFiles.foreach(_.delete()) + timeIndexFiles.foreach(_.delete()) + + // reopen the log + log = createLog(logDir, logConfig, lastShutdownClean = false) + assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages)) + assertTrue(log.logSegments.head.offsetIndex.entries > 0, "The index should have been rebuilt") + assertTrue(log.logSegments.head.timeIndex.entries > 0, "The time index should have been rebuilt") + for(i <- 0 until numMessages) { + assertEquals(i, LogTestUtils.readLog(log, i, 100).records.batches.iterator.next().lastOffset) + if (i == 0) + assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) + else + assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) + } + log.close() + } + + /** + * Test that if messages format version of the messages in a segment is before 0.10.0, the time index should be empty. + */ + @Test + def testRebuildTimeIndexForOldMessages(): Unit = { + val numMessages = 200 + val segmentSize = 200 + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = 1, messageFormatVersion = "0.9.0") + var log = createLog(logDir, logConfig) + for (i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), + timestamp = mockTime.milliseconds + i * 10, magicValue = RecordBatch.MAGIC_VALUE_V1), leaderEpoch = 0) + val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) + log.close() + + // Delete the time index. + timeIndexFiles.foreach(file => Files.delete(file.toPath)) + + // The rebuilt time index should be empty + log = createLog(logDir, logConfig, recoveryPoint = numMessages + 1, lastShutdownClean = false) + for (segment <- log.logSegments.init) { + assertEquals(0, segment.timeIndex.entries, "The time index should be empty") + assertEquals(0, segment.lazyTimeIndex.file.length, "The time index file size should be 0") + } + } + + + /** + * Test that if we have corrupted an index segment it is rebuilt when the log is re-opened + */ + @Test + def testCorruptIndexRebuild(): Unit = { + // publish the messages and close the log + val numMessages = 200 + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) + var log = createLog(logDir, logConfig) + for(i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) + val indexFiles = log.logSegments.map(_.lazyOffsetIndex.file) + val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) + log.close() + + // corrupt all the index files + for( file <- indexFiles) { + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(" ") + bw.close() + } + + // corrupt all the index files + for( file <- timeIndexFiles) { + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(" ") + bw.close() + } + + // reopen the log with recovery point=0 so that the segment recovery can be triggered + log = createLog(logDir, logConfig, lastShutdownClean = false) + assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages)) + for(i <- 0 until numMessages) { + assertEquals(i, LogTestUtils.readLog(log, i, 100).records.batches.iterator.next().lastOffset) + if (i == 0) + assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) + else + assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) + } + log.close() + } + + /** + * When we open a log any index segments without an associated log segment should be deleted. + */ + @Test + def testBogusIndexSegmentsAreRemoved(): Unit = { + val bogusIndex1 = Log.offsetIndexFile(logDir, 0) + val bogusTimeIndex1 = Log.timeIndexFile(logDir, 0) + val bogusIndex2 = Log.offsetIndexFile(logDir, 5) + val bogusTimeIndex2 = Log.timeIndexFile(logDir, 5) + + // The files remain absent until we first access it because we are doing lazy loading for time index and offset index + // files but in this test case we need to create these files in order to test we will remove them. + bogusIndex2.createNewFile() + bogusTimeIndex2.createNewFile() + + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 1) + val log = createLog(logDir, logConfig) + + // Force the segment to access the index files because we are doing index lazy loading. + log.logSegments.toSeq.head.offsetIndex + log.logSegments.toSeq.head.timeIndex + + assertTrue(bogusIndex1.length > 0, + "The first index file should have been replaced with a larger file") + assertTrue(bogusTimeIndex1.length > 0, + "The first time index file should have been replaced with a larger file") + assertFalse(bogusIndex2.exists, + "The second index file should have been deleted.") + assertFalse(bogusTimeIndex2.exists, + "The second time index file should have been deleted.") + + // check that we can append to the log + for (_ <- 0 until 10) + log.appendAsLeader(createRecords, leaderEpoch = 0) + + log.delete() + } + + /** + * Verify that truncation works correctly after re-opening the log + */ + @Test + def testReopenThenTruncate(): Unit = { + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) + // create a log + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000) + var log = createLog(logDir, logConfig) + + // add enough messages to roll over several segments then close and re-open and attempt to truncate + for (_ <- 0 until 100) + log.appendAsLeader(createRecords, leaderEpoch = 0) + log.close() + log = createLog(logDir, logConfig, lastShutdownClean = false) + log.truncateTo(3) + assertEquals(1, log.numberOfSegments, "All but one segment should be deleted.") + assertEquals(3, log.logEndOffset, "Log end offset should be 3.") + } + + /** + * Any files ending in .deleted should be removed when the log is re-opened. + */ + @Test + def testOpenDeletesObsoleteFiles(): Unit = { + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) + var log = createLog(logDir, logConfig) + + // append some messages to create some segments + for (_ <- 0 until 100) + log.appendAsLeader(createRecords, leaderEpoch = 0) + + // expire all segments + log.updateHighWatermark(log.logEndOffset) + log.deleteOldSegments() + log.close() + log = createLog(logDir, logConfig, lastShutdownClean = false) + assertEquals(1, log.numberOfSegments, "The deleted segments should be gone.") + } + + @Test + def testCorruptLog(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) + val recoveryPoint = 50L + for (_ <- 0 until 10) { + // create a log and write some messages to it + logDir.mkdirs() + var log = createLog(logDir, logConfig) + val numMessages = 50 + TestUtils.random.nextInt(50) + for (_ <- 0 until numMessages) + log.appendAsLeader(createRecords, leaderEpoch = 0) + val records = log.logSegments.flatMap(_.log.records.asScala.toList).toList + log.close() + + // corrupt index and log by appending random bytes + TestUtils.appendNonsenseToFile(log.activeSegment.lazyOffsetIndex.file, TestUtils.random.nextInt(1024) + 1) + TestUtils.appendNonsenseToFile(log.activeSegment.log.file, TestUtils.random.nextInt(1024) + 1) + + // attempt recovery + log = createLog(logDir, logConfig, brokerTopicStats, 0L, recoveryPoint, lastShutdownClean = false) + assertEquals(numMessages, log.logEndOffset) + + val recovered = log.logSegments.flatMap(_.log.records.asScala.toList).toList + assertEquals(records.size, recovered.size) + + for (i <- records.indices) { + val expected = records(i) + val actual = recovered(i) + assertEquals(expected.key, actual.key, s"Keys not equal") + assertEquals(expected.value, actual.value, s"Values not equal") + assertEquals(expected.timestamp, actual.timestamp, s"Timestamps not equal") + } + + Utils.delete(logDir) + } + } + + @Test + def testOverCompactedLogRecovery(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val log = createLog(logDir, logConfig) + val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) + val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.NONE, 0, new SimpleRecord("v3".getBytes(), "k3".getBytes())) + val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, CompressionType.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".getBytes())) + val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.NONE, 0, new SimpleRecord("v5".getBytes(), "k5".getBytes())) + //Writes into an empty log with baseOffset 0 + log.appendAsFollower(set1) + assertEquals(0L, log.activeSegment.baseOffset) + //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 + log.appendAsFollower(set2) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) + //This will go into the existing log + log.appendAsFollower(set3) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + //This will go into the existing log + log.appendAsFollower(set4) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + log.close() + val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) + assertEquals(2, indexFiles.length) + for (file <- indexFiles) { + val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) + assertTrue(offsetIndex.lastOffset >= 0) + offsetIndex.close() + } + Utils.delete(logDir) + } + + @Test + def testLeaderEpochCacheClearedAfterStaticMessageFormatDowngrade(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val log = createLog(logDir, logConfig) + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) + assertEquals(Some(5), log.latestEpoch) + log.close() + + // reopen the log with an older message format version and check the cache + val downgradedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, + maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_10_2_IV0.shortVersion) + val reopened = createLog(logDir, downgradedLogConfig, lastShutdownClean = false) + LogTestUtils.assertLeaderEpochCacheEmpty(reopened) + + reopened.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), + magicValue = RecordVersion.V1.value), leaderEpoch = 5) + LogTestUtils.assertLeaderEpochCacheEmpty(reopened) + } + + @Test + def testOverCompactedLogRecoveryMultiRecord(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val log = createLog(logDir, logConfig) + val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) + val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, 0, + new SimpleRecord("v3".getBytes(), "k3".getBytes()), + new SimpleRecord("v4".getBytes(), "k4".getBytes())) + val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, 0, + new SimpleRecord("v5".getBytes(), "k5".getBytes()), + new SimpleRecord("v6".getBytes(), "k6".getBytes())) + val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, 0, + new SimpleRecord("v7".getBytes(), "k7".getBytes()), + new SimpleRecord("v8".getBytes(), "k8".getBytes())) + //Writes into an empty log with baseOffset 0 + log.appendAsFollower(set1) + assertEquals(0L, log.activeSegment.baseOffset) + //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 + log.appendAsFollower(set2) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) + //This will go into the existing log + log.appendAsFollower(set3) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + //This will go into the existing log + log.appendAsFollower(set4) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + log.close() + val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) + assertEquals(2, indexFiles.length) + for (file <- indexFiles) { + val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) + assertTrue(offsetIndex.lastOffset >= 0) + offsetIndex.close() + } + Utils.delete(logDir) + } + + @Test + def testOverCompactedLogRecoveryMultiRecordV1(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val log = createLog(logDir, logConfig) + val set1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0, CompressionType.NONE, + new SimpleRecord("v1".getBytes(), "k1".getBytes())) + val set2 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, + new SimpleRecord("v3".getBytes(), "k3".getBytes()), + new SimpleRecord("v4".getBytes(), "k4".getBytes())) + val set3 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, + new SimpleRecord("v5".getBytes(), "k5".getBytes()), + new SimpleRecord("v6".getBytes(), "k6".getBytes())) + val set4 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, + new SimpleRecord("v7".getBytes(), "k7".getBytes()), + new SimpleRecord("v8".getBytes(), "k8".getBytes())) + //Writes into an empty log with baseOffset 0 + log.appendAsFollower(set1) + assertEquals(0L, log.activeSegment.baseOffset) + //This write will roll the segment, yielding a new segment with base offset = max(1, 3) = 3 + log.appendAsFollower(set2) + assertEquals(3, log.activeSegment.baseOffset) + assertTrue(Log.producerSnapshotFile(logDir, 3).exists) + //This will also roll the segment, yielding a new segment with base offset = max(5, Integer.MAX_VALUE+4) = Integer.MAX_VALUE+4 + log.appendAsFollower(set3) + assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) + assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 4).exists) + //This will go into the existing log + log.appendAsFollower(set4) + assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) + log.close() + val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) + assertEquals(3, indexFiles.length) + for (file <- indexFiles) { + val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) + assertTrue(offsetIndex.lastOffset >= 0) + offsetIndex.close() + } + Utils.delete(logDir) + } + + @Test + def testRecoveryOfSegmentWithOffsetOverflow(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, _) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + + // Run recovery on the log. This should split the segment underneath. Ignore .deleted files as we could have still + // have them lying around after the split. + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + + // Running split again would throw an error + + for (segment <- recoveredLog.logSegments) { + assertThrows(classOf[IllegalArgumentException], () => log.splitOverflowedSegment(segment)) + } + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase1(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery just after .cleaned file is created, before rename to .swap. On recovery, existing split + // operation is aborted but the recovery process itself kicks off split which should complete. + newSegments.reverse.foreach(segment => { + segment.changeFileSuffixes("", Log.CleanedFileSuffix) + segment.truncateTo(0) + }) + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase2(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery just after one of the new segments has been renamed to .swap. On recovery, existing split + // operation is aborted but the recovery process itself kicks off split which should complete. + newSegments.reverse.foreach { segment => + if (segment != newSegments.last) + segment.changeFileSuffixes("", Log.CleanedFileSuffix) + else + segment.changeFileSuffixes("", Log.SwapFileSuffix) + segment.truncateTo(0) + } + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase3(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery right after all new segments have been renamed to .swap. On recovery, existing split operation + // is completed and the old segment must be deleted. + newSegments.reverse.foreach(segment => { + segment.changeFileSuffixes("", Log.SwapFileSuffix) + }) + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + + // Truncate the old segment + segmentWithOverflow.truncateTo(0) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + log.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase4(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery right after all new segments have been renamed to .swap and old segment has been deleted. On + // recovery, existing split operation is completed. + newSegments.reverse.foreach(_.changeFileSuffixes("", Log.SwapFileSuffix)) + + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.delete(file) + + // Truncate the old segment + segmentWithOverflow.truncateTo(0) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase5(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery right after one of the new segment has been renamed to .swap and the other to .log. On + // recovery, existing split operation is completed. + newSegments.last.changeFileSuffixes("", Log.SwapFileSuffix) + + // Truncate the old segment + segmentWithOverflow.truncateTo(0) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testCleanShutdownFile(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) + + var recoveryPoint = 0L + // create a log and write some messages to it + var log = createLog(logDir, logConfig) + for (_ <- 0 until 100) + log.appendAsLeader(createRecords, leaderEpoch = 0) + log.close() + + // check if recovery was attempted. Even if the recovery point is 0L, recovery should not be attempted as the + // clean shutdown file exists. Note: Earlier, Log layer relied on the presence of clean shutdown file to determine the status + // of last shutdown. Now, LogManager checks for the presence of this file and immediately deletes the same. It passes + // down a clean shutdown flag to the Log layer as log is loaded. Recovery is attempted based on this flag. + recoveryPoint = log.logEndOffset + log = createLog(logDir, logConfig) + assertEquals(recoveryPoint, log.logEndOffset) + } + + /** + * Append a bunch of messages to a log and then re-open it with recovery and check that the leader epochs are recovered properly. + */ + @Test + def testLogRecoversForLeaderEpoch(): Unit = { + val log = createLog(logDir, LogConfig()) + val leaderEpochCache = log.leaderEpochCache.get + val firstBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 1, offset = 0) + log.appendAsFollower(records = firstBatch) + + val secondBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 1) + log.appendAsFollower(records = secondBatch) + + val thirdBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 2) + log.appendAsFollower(records = thirdBatch) + + val fourthBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 3, offset = 3) + log.appendAsFollower(records = fourthBatch) + + assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) + + // deliberately remove some of the epoch entries + leaderEpochCache.truncateFromEnd(2) + assertNotEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) + log.close() + + // reopen the log and recover from the beginning + val recoveredLog = createLog(logDir, LogConfig(), lastShutdownClean = false) + val recoveredLeaderEpochCache = recoveredLog.leaderEpochCache.get + + // epoch entries should be recovered + assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), recoveredLeaderEpochCache.epochEntries) + recoveredLog.close() + } + + @Test + def testFullTransactionIndexRecovery(): Unit = { + val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) + val log = createLog(logDir, logConfig) + val epoch = 0.toShort + + val pid1 = 1L + val pid2 = 2L + val pid3 = 3L + val pid4 = 4L + + val appendPid1 = LogTestUtils.appendTransactionalAsLeader(log, pid1, epoch, mockTime) + val appendPid2 = LogTestUtils.appendTransactionalAsLeader(log, pid2, epoch, mockTime) + val appendPid3 = LogTestUtils.appendTransactionalAsLeader(log, pid3, epoch, mockTime) + val appendPid4 = LogTestUtils.appendTransactionalAsLeader(log, pid4, epoch, mockTime) + + // mix transactional and non-transactional data + appendPid1(5) // nextOffset: 5 + LogTestUtils.appendNonTransactionalAsLeader(log, 3) // 8 + appendPid2(2) // 10 + appendPid1(4) // 14 + appendPid3(3) // 17 + LogTestUtils.appendNonTransactionalAsLeader(log, 2) // 19 + appendPid1(10) // 29 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 30 + appendPid2(6) // 36 + appendPid4(3) // 39 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 49 + appendPid3(9) // 58 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 59 + appendPid4(8) // 67 + appendPid2(7) // 74 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 75 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 85 + appendPid4(4) // 89 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 90 + + // delete all the offset and transaction index files to force recovery + log.logSegments.foreach { segment => + segment.offsetIndex.deleteIfExists() + segment.txnIndex.deleteIfExists() + } + + log.close() + + val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) + val reloadedLog = createLog(logDir, reloadedLogConfig, lastShutdownClean = false) + val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog) + assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) + } + + @Test + def testRecoverOnlyLastSegment(): Unit = { + val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) + val log = createLog(logDir, logConfig) + val epoch = 0.toShort + + val pid1 = 1L + val pid2 = 2L + val pid3 = 3L + val pid4 = 4L + + val appendPid1 = LogTestUtils.appendTransactionalAsLeader(log, pid1, epoch, mockTime) + val appendPid2 = LogTestUtils.appendTransactionalAsLeader(log, pid2, epoch, mockTime) + val appendPid3 = LogTestUtils.appendTransactionalAsLeader(log, pid3, epoch, mockTime) + val appendPid4 = LogTestUtils.appendTransactionalAsLeader(log, pid4, epoch, mockTime) + + // mix transactional and non-transactional data + appendPid1(5) // nextOffset: 5 + LogTestUtils.appendNonTransactionalAsLeader(log, 3) // 8 + appendPid2(2) // 10 + appendPid1(4) // 14 + appendPid3(3) // 17 + LogTestUtils.appendNonTransactionalAsLeader(log, 2) // 19 + appendPid1(10) // 29 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 30 + appendPid2(6) // 36 + appendPid4(3) // 39 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 49 + appendPid3(9) // 58 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 59 + appendPid4(8) // 67 + appendPid2(7) // 74 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 75 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 85 + appendPid4(4) // 89 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 90 + + // delete the last offset and transaction index files to force recovery + val lastSegment = log.logSegments.last + val recoveryPoint = lastSegment.baseOffset + lastSegment.offsetIndex.deleteIfExists() + lastSegment.txnIndex.deleteIfExists() + + log.close() + + val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) + val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) + val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog) + assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) + } + + @Test + def testRecoverLastSegmentWithNoSnapshots(): Unit = { + val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) + val log = createLog(logDir, logConfig) + val epoch = 0.toShort + + val pid1 = 1L + val pid2 = 2L + val pid3 = 3L + val pid4 = 4L + + val appendPid1 = LogTestUtils.appendTransactionalAsLeader(log, pid1, epoch, mockTime) + val appendPid2 = LogTestUtils.appendTransactionalAsLeader(log, pid2, epoch, mockTime) + val appendPid3 = LogTestUtils.appendTransactionalAsLeader(log, pid3, epoch, mockTime) + val appendPid4 = LogTestUtils.appendTransactionalAsLeader(log, pid4, epoch, mockTime) + + // mix transactional and non-transactional data + appendPid1(5) // nextOffset: 5 + LogTestUtils.appendNonTransactionalAsLeader(log, 3) // 8 + appendPid2(2) // 10 + appendPid1(4) // 14 + appendPid3(3) // 17 + LogTestUtils.appendNonTransactionalAsLeader(log, 2) // 19 + appendPid1(10) // 29 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 30 + appendPid2(6) // 36 + appendPid4(3) // 39 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 49 + appendPid3(9) // 58 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 59 + appendPid4(8) // 67 + appendPid2(7) // 74 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 75 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 85 + appendPid4(4) // 89 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 90 + + LogTestUtils.deleteProducerSnapshotFiles(logDir) + + // delete the last offset and transaction index files to force recovery. this should force us to rebuild + // the producer state from the start of the log + val lastSegment = log.logSegments.last + val recoveryPoint = lastSegment.baseOffset + lastSegment.offsetIndex.deleteIfExists() + lastSegment.txnIndex.deleteIfExists() + + log.close() + + val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) + val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) + val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog) + assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) + } +} diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 20ca8e6b761bd..c8891ebd58bc2 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -19,18 +19,16 @@ package kafka.log import java.io._ import java.nio.ByteBuffer -import java.nio.file.{Files, Paths} +import java.nio.file.Files import java.util.concurrent.{Callable, Executors} import java.util.regex.Pattern import java.util.{Collections, Optional, Properties} -import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0} import kafka.common.{OffsetsOutOfOrderException, RecordValidationException, UnexpectedAppendOffsetException} import kafka.log.Log.DeleteDirSuffix import kafka.metrics.KafkaYammerMetrics import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} -import kafka.server.metadata.CachedConfigRepository -import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata, PartitionMetadataFile} +import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata, PartitionMetadataFile} import kafka.utils._ import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.errors._ @@ -45,7 +43,7 @@ import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} -import scala.collection.{Iterable, Map, mutable} +import scala.collection.{Iterable, Map} import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer @@ -76,82 +74,6 @@ class LogTest { } } - @Test - def testLogRecoveryIsCalledUponBrokerCrash(): Unit = { - // LogManager must realize correctly if the last shutdown was not clean and the logs need - // to run recovery while loading upon subsequent broker boot up. - val logDir: File = TestUtils.tempDir() - val logProps = new Properties() - val logConfig = LogConfig(logProps) - val logDirs = Seq(logDir) - val topicPartition = new TopicPartition("foo", 0) - var log: Log = null - val time = new MockTime() - var cleanShutdownInterceptedValue = false - var simulateError = false - - // Create a LogManager with some overridden methods to facilitate interception of clean shutdown - // flag and to inject a runtime error - def interceptedLogManager(logConfig: LogConfig, logDirs: Seq[File]): LogManager = { - new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], new CachedConfigRepository(), - initialDefaultConfig = logConfig, cleanerConfig = CleanerConfig(enableCleaner = false), recoveryThreadsPerDataDir = 4, - flushCheckMs = 1000L, flushRecoveryOffsetCheckpointMs = 10000L, flushStartOffsetCheckpointMs = 10000L, - retentionCheckMs = 1000L, maxPidExpirationMs = 60 * 60 * 1000, scheduler = time.scheduler, time = time, - brokerTopicStats = new BrokerTopicStats, logDirFailureChannel = new LogDirFailureChannel(logDirs.size), keepPartitionMetadataFile = config.usesTopicId) { - - override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long], - logStartOffsets: Map[TopicPartition, Long], topicConfigs: Map[String, LogConfig]): Log = { - - val topicPartition = Log.parseTopicPartitionName(logDir) - val config = topicConfigs.getOrElse(topicPartition.topic, currentDefaultConfig) - val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) - val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) - val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) - - val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxPidExpirationMs) - val log = new Log(logDir, config, logStartOffset, logRecoveryPoint, time.scheduler, brokerTopicStats, time, maxPidExpirationMs, - LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, producerStateManager, logDirFailureChannel, hadCleanShutdown, None, true) { - override def recoverLog(): Long = { - if (simulateError) - throw new RuntimeException - cleanShutdownInterceptedValue = hadCleanShutdown - super.recoverLog() - } - } - log - - } - - } - } - - val cleanShutdownFile = new File(logDir, Log.CleanShutdownFile) - val logManager: LogManager = interceptedLogManager(logConfig, logDirs) - log = logManager.getOrCreateLog(topicPartition, isNew = true, topicId = None) - - // Load logs after a clean shutdown - Files.createFile(cleanShutdownFile.toPath) - cleanShutdownInterceptedValue = false - logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) - assertTrue(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") - assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") - // Load logs without clean shutdown file - cleanShutdownInterceptedValue = true - logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) - assertFalse(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") - assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") - // Create clean shutdown file and then simulate error while loading logs such that log loading does not complete. - Files.createFile(cleanShutdownFile.toPath) - simulateError = true - assertThrows(classOf[RuntimeException], () => logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty))) - assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not have existed") - // Do not simulate error on next call to LogManager#loadLogs. LogManager must understand that log had unclean shutdown the last time. - simulateError = false - cleanShutdownInterceptedValue = true - logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) - assertFalse(cleanShutdownInterceptedValue, "Unexpected value for clean shutdown flag") - } - @Test def testHighWatermarkMetadataUpdatedAfterSegmentRoll(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) @@ -458,7 +380,7 @@ class LogTest { // Test transactional producer state (open transaction) val producer1Epoch = 5.toShort val producerId1 = 1L - appendTransactionalAsLeader(log, producerId1, producer1Epoch)(5) + LogTestUtils.appendTransactionalAsLeader(log, producerId1, producer1Epoch, mockTime)(5) assertProducerState( producerId1, producer1Epoch, @@ -469,7 +391,7 @@ class LogTest { // Test transactional producer state (closed transaction) val coordinatorEpoch = 15 - appendEndTxnMarkerAsLeader(log, producerId1, producer1Epoch, ControlRecordType.COMMIT, coordinatorEpoch) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId1, producer1Epoch, ControlRecordType.COMMIT, mockTime.milliseconds(), coordinatorEpoch) assertProducerState( producerId1, producer1Epoch, @@ -481,7 +403,7 @@ class LogTest { // Test idempotent producer state val producer2Epoch = 5.toShort val producerId2 = 2L - appendIdempotentAsLeader(log, producerId2, producer2Epoch)(3) + LogTestUtils.appendIdempotentAsLeader(log, producerId2, producer2Epoch, mockTime)(3) assertProducerState( producerId2, producer2Epoch, @@ -500,14 +422,14 @@ class LogTest { val producerId1 = 1L val producerId2 = 2L - val appendProducer1 = appendTransactionalAsLeader(log, producerId1, epoch) - val appendProducer2 = appendTransactionalAsLeader(log, producerId2, epoch) + val appendProducer1 = LogTestUtils.appendTransactionalAsLeader(log, producerId1, epoch, mockTime) + val appendProducer2 = LogTestUtils.appendTransactionalAsLeader(log, producerId2, epoch, mockTime) appendProducer1(5) - appendNonTransactionalAsLeader(log, 3) + LogTestUtils.appendNonTransactionalAsLeader(log, 3) appendProducer2(2) appendProducer1(4) - appendNonTransactionalAsLeader(log, 2) + LogTestUtils.appendNonTransactionalAsLeader(log, 2) appendProducer1(10) def assertLsoBoundedFetches(): Unit = { @@ -525,14 +447,14 @@ class LogTest { log.updateHighWatermark(log.logEndOffset) assertLsoBoundedFetches() - appendEndTxnMarkerAsLeader(log, producerId1, epoch, ControlRecordType.COMMIT) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId1, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) assertEquals(0L, log.lastStableOffset) log.updateHighWatermark(log.logEndOffset) assertEquals(8L, log.lastStableOffset) assertLsoBoundedFetches() - appendEndTxnMarkerAsLeader(log, producerId2, epoch, ControlRecordType.ABORT) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) assertEquals(8L, log.lastStableOffset) log.updateHighWatermark(log.logEndOffset) @@ -651,8 +573,8 @@ class LogTest { log.appendAsFollower(records2) assertEquals(2, log.logEndOffset, "Expect two records in the log") - assertEquals(0, readLog(log, 0, 1).records.batches.iterator.next().lastOffset) - assertEquals(1, readLog(log, 1, 1).records.batches.iterator.next().lastOffset) + assertEquals(0, LogTestUtils.readLog(log, 0, 1).records.batches.iterator.next().lastOffset) + assertEquals(1, LogTestUtils.readLog(log, 1, 1).records.batches.iterator.next().lastOffset) // roll so that active segment is empty log.roll() @@ -666,7 +588,7 @@ class LogTest { baseOffset = 2L, partitionLeaderEpoch = 0) log.appendAsFollower(records3) assertTrue(log.activeSegment.offsetIndex.maxEntries > 1) - assertEquals(2, readLog(log, 2, 1).records.batches.iterator.next().lastOffset) + assertEquals(2, LogTestUtils.readLog(log, 2, 1).records.batches.iterator.next().lastOffset) assertEquals(2, log.numberOfSegments, "Expect two segments.") } @@ -791,74 +713,31 @@ class LogTest { val logEndOffset = log.logEndOffset log.close() - deleteProducerSnapshotFiles() + LogTestUtils.deleteProducerSnapshotFiles(logDir) // Reload after clean shutdown log = createLog(logDir, logConfig, recoveryPoint = logEndOffset) var expectedSnapshotOffsets = log.logSegments.map(_.baseOffset).takeRight(2).toVector :+ log.logEndOffset - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir)) log.close() - deleteProducerSnapshotFiles() + LogTestUtils.deleteProducerSnapshotFiles(logDir) // Reload after unclean shutdown with recoveryPoint set to log end offset log = createLog(logDir, logConfig, recoveryPoint = logEndOffset, lastShutdownClean = false) - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir)) log.close() - deleteProducerSnapshotFiles() + LogTestUtils.deleteProducerSnapshotFiles(logDir) // Reload after unclean shutdown with recoveryPoint set to 0 log = createLog(logDir, logConfig, recoveryPoint = 0L, lastShutdownClean = false) // We progressively create a snapshot for each segment after the recovery point expectedSnapshotOffsets = log.logSegments.map(_.baseOffset).tail.toVector :+ log.logEndOffset - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) - log.close() - } - - - @Test - def testRecoverAfterNonMonotonicCoordinatorEpochWrite(): Unit = { - // Due to KAFKA-9144, we may encounter a coordinator epoch which goes backwards. - // This test case verifies that recovery logic relaxes validation in this case and - // just takes the latest write. - - val producerId = 1L - val coordinatorEpoch = 5 - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) - var log = createLog(logDir, logConfig) - val epoch = 0.toShort - - val firstAppendTimestamp = mockTime.milliseconds() - appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, - timestamp = firstAppendTimestamp, coordinatorEpoch = coordinatorEpoch) - assertEquals(firstAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) - - mockTime.sleep(log.maxProducerIdExpirationMs) - assertEquals(None, log.producerStateManager.lastEntry(producerId)) - - val secondAppendTimestamp = mockTime.milliseconds() - appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, - timestamp = secondAppendTimestamp, coordinatorEpoch = coordinatorEpoch - 1) - - log.close() - - // Force recovery by setting the recoveryPoint to the log start - log = createLog(logDir, logConfig, recoveryPoint = 0L, lastShutdownClean = false) - assertEquals(secondAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir)) log.close() } - @Test - def testProducerSnapshotsRecoveryAfterUncleanShutdownV1(): Unit = { - testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.minSupportedFor(RecordVersion.V1).version) - } - - @Test - def testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat(): Unit = { - testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.latestVersion.version) - } - @Test def testLogReinitializeAfterManualDelete(): Unit = { val logConfig = LogTest.createLogConfig() @@ -981,94 +860,6 @@ class LogTest { } } - private def testProducerSnapshotsRecoveryAfterUncleanShutdown(messageFormatVersion: String): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 64 * 10, messageFormatVersion = messageFormatVersion) - var log = createLog(logDir, logConfig) - assertEquals(None, log.oldestProducerSnapshotOffset) - - for (i <- 0 to 100) { - val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes) - log.appendAsLeader(TestUtils.records(List(record)), leaderEpoch = 0) - } - - assertTrue(log.logSegments.size >= 5) - val segmentOffsets = log.logSegments.toVector.map(_.baseOffset) - val activeSegmentOffset = segmentOffsets.last - - // We want the recovery point to be past the segment offset and before the last 2 segments including a gap of - // 1 segment. We collect the data before closing the log. - val offsetForSegmentAfterRecoveryPoint = segmentOffsets(segmentOffsets.size - 3) - val offsetForRecoveryPointSegment = segmentOffsets(segmentOffsets.size - 4) - val (segOffsetsBeforeRecovery, segOffsetsAfterRecovery) = segmentOffsets.toSet.partition(_ < offsetForRecoveryPointSegment) - val recoveryPoint = offsetForRecoveryPointSegment + 1 - assertTrue(recoveryPoint < offsetForSegmentAfterRecoveryPoint) - log.close() - - val segmentsWithReads = mutable.Set[LogSegment]() - val recoveredSegments = mutable.Set[LogSegment]() - val expectedSegmentsWithReads = mutable.Set[Long]() - val expectedSnapshotOffsets = mutable.Set[Long]() - - if (logConfig.messageFormatVersion < KAFKA_0_11_0_IV0) { - expectedSegmentsWithReads += activeSegmentOffset - expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(2) :+ log.logEndOffset - } else { - expectedSegmentsWithReads ++= segOffsetsBeforeRecovery ++ Set(activeSegmentOffset) - expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(4) :+ log.logEndOffset - } - - def createLogWithInterceptedReads(recoveryPoint: Long) = { - val maxProducerIdExpirationMs = 60 * 60 * 1000 - val topicPartition = Log.parseTopicPartitionName(logDir) - val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) - - // Intercept all segment read calls - new Log(logDir, logConfig, logStartOffset = 0, recoveryPoint = recoveryPoint, mockTime.scheduler, - brokerTopicStats, mockTime, maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs, - topicPartition, producerStateManager, new LogDirFailureChannel(10), hadCleanShutdown = false, topicId = None, keepPartitionMetadataFile = true) { - - override def addSegment(segment: LogSegment): LogSegment = { - val wrapper = new LogSegment(segment.log, segment.lazyOffsetIndex, segment.lazyTimeIndex, segment.txnIndex, segment.baseOffset, - segment.indexIntervalBytes, segment.rollJitterMs, mockTime) { - - override def read(startOffset: Long, maxSize: Int, maxPosition: Long, minOneMessage: Boolean): FetchDataInfo = { - segmentsWithReads += this - super.read(startOffset, maxSize, maxPosition, minOneMessage) - } - - override def recover(producerStateManager: ProducerStateManager, - leaderEpochCache: Option[LeaderEpochFileCache]): Int = { - recoveredSegments += this - super.recover(producerStateManager, leaderEpochCache) - } - } - super.addSegment(wrapper) - } - } - } - - // Retain snapshots for the last 2 segments - log.producerStateManager.deleteSnapshotsBefore(segmentOffsets(segmentOffsets.size - 2)) - log = createLogWithInterceptedReads(offsetForRecoveryPointSegment) - // We will reload all segments because the recovery point is behind the producer snapshot files (pre KAFKA-5829 behaviour) - assertEquals(expectedSegmentsWithReads, segmentsWithReads.map(_.baseOffset)) - assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets.toSet) - log.close() - segmentsWithReads.clear() - recoveredSegments.clear() - - // Only delete snapshots before the base offset of the recovery point segment (post KAFKA-5829 behaviour) to - // avoid reading all segments - log.producerStateManager.deleteSnapshotsBefore(offsetForRecoveryPointSegment) - log = createLogWithInterceptedReads(recoveryPoint = recoveryPoint) - assertEquals(Set(activeSegmentOffset), segmentsWithReads.map(_.baseOffset)) - assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets.toSet) - - log.close() - } - @Test def testSizeForLargeLogs(): Unit = { val largeSize = Int.MaxValue.toLong * 2 @@ -1092,211 +883,6 @@ class LogTest { assertEquals(Some(1), log.latestProducerSnapshotOffset) } - @Test - def testSkipLoadingIfEmptyProducerStateBeforeTruncation(): Unit = { - val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() - // Load the log - EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) - - stateManager.updateMapEndOffset(0L) - EasyMock.expectLastCall().anyTimes() - - EasyMock.expect(stateManager.mapEndOffset).andStubReturn(0L) - EasyMock.expect(stateManager.isEmpty).andStubReturn(true) - - stateManager.takeSnapshot() - EasyMock.expectLastCall().anyTimes() - - stateManager.truncateAndReload(EasyMock.eq(0L), EasyMock.eq(0L), EasyMock.anyLong) - EasyMock.expectLastCall() - - EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) - - EasyMock.replay(stateManager) - - val config = LogConfig(new Properties()) - val log = new Log(logDir, - config, - logStartOffset = 0L, - recoveryPoint = 0L, - scheduler = mockTime.scheduler, - brokerTopicStats = brokerTopicStats, - time = mockTime, - maxProducerIdExpirationMs = 300000, - producerIdExpirationCheckIntervalMs = 30000, - topicPartition = Log.parseTopicPartitionName(logDir), - producerStateManager = stateManager, - logDirFailureChannel = new LogDirFailureChannel(1), - hadCleanShutdown = false, - topicId = None, - keepPartitionMetadataFile = true) - - EasyMock.verify(stateManager) - - // Append some messages - EasyMock.reset(stateManager) - EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) - - stateManager.updateMapEndOffset(1L) - EasyMock.expectLastCall() - stateManager.updateMapEndOffset(2L) - EasyMock.expectLastCall() - - EasyMock.replay(stateManager) - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes))), leaderEpoch = 0) - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes))), leaderEpoch = 0) - - EasyMock.verify(stateManager) - - // Now truncate - EasyMock.reset(stateManager) - EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) - EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) - EasyMock.expect(stateManager.isEmpty).andStubReturn(true) - EasyMock.expect(stateManager.mapEndOffset).andReturn(2L) - stateManager.truncateAndReload(EasyMock.eq(0L), EasyMock.eq(1L), EasyMock.anyLong) - EasyMock.expectLastCall() - // Truncation causes the map end offset to reset to 0 - EasyMock.expect(stateManager.mapEndOffset).andReturn(0L) - // We skip directly to updating the map end offset - EasyMock.expect(stateManager.updateMapEndOffset(1L)) - EasyMock.expect(stateManager.onHighWatermarkUpdated(0L)) - - // Finally, we take a snapshot - stateManager.takeSnapshot() - EasyMock.expectLastCall().once() - - EasyMock.replay(stateManager) - - log.truncateTo(1L) - - EasyMock.verify(stateManager) - } - - @Test - def testSkipTruncateAndReloadIfOldMessageFormatAndNoCleanShutdown(): Unit = { - val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() - - stateManager.updateMapEndOffset(0L) - EasyMock.expectLastCall().anyTimes() - - stateManager.takeSnapshot() - EasyMock.expectLastCall().anyTimes() - - EasyMock.expect(stateManager.isEmpty).andReturn(true) - EasyMock.expectLastCall().once() - - EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) - EasyMock.expectLastCall().once() - - EasyMock.replay(stateManager) - - val logProps = new Properties() - logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") - val config = LogConfig(logProps) - new Log(logDir, - config, - logStartOffset = 0L, - recoveryPoint = 0L, - scheduler = mockTime.scheduler, - brokerTopicStats = brokerTopicStats, - time = mockTime, - maxProducerIdExpirationMs = 300000, - producerIdExpirationCheckIntervalMs = 30000, - topicPartition = Log.parseTopicPartitionName(logDir), - producerStateManager = stateManager, - logDirFailureChannel = null, - topicId = None, - keepPartitionMetadataFile = true) - - EasyMock.verify(stateManager) - } - - @Test - def testSkipTruncateAndReloadIfOldMessageFormatAndCleanShutdown(): Unit = { - val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() - - stateManager.updateMapEndOffset(0L) - EasyMock.expectLastCall().anyTimes() - - stateManager.takeSnapshot() - EasyMock.expectLastCall().anyTimes() - - EasyMock.expect(stateManager.isEmpty).andReturn(true) - EasyMock.expectLastCall().once() - - EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) - EasyMock.expectLastCall().once() - - EasyMock.replay(stateManager) - - val logProps = new Properties() - logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") - val config = LogConfig(logProps) - new Log(logDir, - config, - logStartOffset = 0L, - recoveryPoint = 0L, - scheduler = mockTime.scheduler, - brokerTopicStats = brokerTopicStats, - time = mockTime, - maxProducerIdExpirationMs = 300000, - producerIdExpirationCheckIntervalMs = 30000, - topicPartition = Log.parseTopicPartitionName(logDir), - producerStateManager = stateManager, - logDirFailureChannel = null, - topicId = None, - keepPartitionMetadataFile = true) - - EasyMock.verify(stateManager) - } - - @Test - def testSkipTruncateAndReloadIfNewMessageFormatAndCleanShutdown(): Unit = { - val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() - - EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) - - stateManager.updateMapEndOffset(0L) - EasyMock.expectLastCall().anyTimes() - - stateManager.takeSnapshot() - EasyMock.expectLastCall().anyTimes() - - EasyMock.expect(stateManager.isEmpty).andReturn(true) - EasyMock.expectLastCall().once() - - EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) - EasyMock.expectLastCall().once() - - EasyMock.replay(stateManager) - - val logProps = new Properties() - logProps.put(LogConfig.MessageFormatVersionProp, "0.11.0") - val config = LogConfig(logProps) - new Log(logDir, - config, - logStartOffset = 0L, - recoveryPoint = 0L, - scheduler = mockTime.scheduler, - brokerTopicStats = brokerTopicStats, - time = mockTime, - maxProducerIdExpirationMs = 300000, - producerIdExpirationCheckIntervalMs = 30000, - topicPartition = Log.parseTopicPartitionName(logDir), - producerStateManager = stateManager, - logDirFailureChannel = null, - topicId = None, - keepPartitionMetadataFile = true) - - EasyMock.verify(stateManager) - } - @Test def testRebuildProducerIdMapWithCompactedData(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) @@ -1451,7 +1037,7 @@ class LogTest { log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes)), producerId = pid, producerEpoch = epoch, sequence = 1), leaderEpoch = 0) - deleteProducerSnapshotFiles() + LogTestUtils.deleteProducerSnapshotFiles(logDir) log.truncateTo(1L) assertEquals(1, log.activeProducersWithLastSequence.size) @@ -1463,38 +1049,6 @@ class LogTest { assertEquals(0, lastSeq) } - @Test - def testLoadProducersAfterDeleteRecordsMidSegment(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) - val log = createLog(logDir, logConfig) - val pid1 = 1L - val pid2 = 2L - val epoch = 0.toShort - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1, - producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2, - producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - assertEquals(2, log.activeProducersWithLastSequence.size) - - log.updateHighWatermark(log.logEndOffset) - log.maybeIncrementLogStartOffset(1L, ClientRecordDeletion) - - // Deleting records should not remove producer state - assertEquals(2, log.activeProducersWithLastSequence.size) - val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) - assertTrue(retainedLastSeqOpt.isDefined) - assertEquals(0, retainedLastSeqOpt.get) - - log.close() - - // Because the log start offset did not advance, producer snapshots will still be present and the state will be rebuilt - val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false) - assertEquals(2, reloadedLog.activeProducersWithLastSequence.size) - val reloadedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) - assertEquals(retainedLastSeqOpt, reloadedLastSeqOpt) - } - @Test def testRetentionDeletesProducerStateSnapshots(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = 0, retentionMs = 1000 * 60, fileDeleteDelayMs = 0) @@ -1623,73 +1177,6 @@ class LogTest { "expected producer state snapshots greater than the log end offset to be cleaned up") } - @Test - def testLoadingLogKeepsLargestStrayProducerStateSnapshot(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = 0, retentionMs = 1000 * 60, fileDeleteDelayMs = 0) - val log = createLog(logDir, logConfig) - val pid1 = 1L - val epoch = 0.toShort - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - log.roll() - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 1), leaderEpoch = 0) - log.roll() - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("c".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 2), leaderEpoch = 0) - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("d".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 3), leaderEpoch = 0) - - // Close the log, we should now have 3 segments - log.close() - assertEquals(log.logSegments.size, 3) - // We expect 3 snapshot files, two of which are for the first two segments, the last was written out during log closing. - assertEquals(Seq(1, 2, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) - // Inject a stray snapshot file within the bounds of the log at offset 3, it should be cleaned up after loading the log - val straySnapshotFile = Log.producerSnapshotFile(logDir, 3).toPath - Files.createFile(straySnapshotFile) - assertEquals(Seq(1, 2, 3, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) - - createLog(logDir, logConfig, lastShutdownClean = false) - // We should clean up the stray producer state snapshot file, but keep the largest snapshot file (4) - assertEquals(Seq(1, 2, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) - } - - @Test - def testLoadProducersAfterDeleteRecordsOnSegment(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) - val log = createLog(logDir, logConfig) - val pid1 = 1L - val pid2 = 2L - val epoch = 0.toShort - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1, - producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - log.roll() - log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2, - producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - - assertEquals(2, log.logSegments.size) - assertEquals(2, log.activeProducersWithLastSequence.size) - - log.updateHighWatermark(log.logEndOffset) - log.maybeIncrementLogStartOffset(1L, ClientRecordDeletion) - log.deleteOldSegments() - - // Deleting records should not remove producer state - assertEquals(1, log.logSegments.size) - assertEquals(2, log.activeProducersWithLastSequence.size) - val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) - assertTrue(retainedLastSeqOpt.isDefined) - assertEquals(0, retainedLastSeqOpt.get) - - log.close() - - // After reloading log, producer state should not be regenerated - val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false) - assertEquals(1, reloadedLog.activeProducersWithLastSequence.size) - val reloadedEntryOpt = log.activeProducersWithLastSequence.get(pid2) - assertEquals(retainedLastSeqOpt, reloadedEntryOpt) - } - @Test def testProducerIdMapTruncateFullyAndStartAt(): Unit = { val records = TestUtils.singletonRecords("foo".getBytes) @@ -1814,7 +1301,7 @@ class LogTest { new SimpleRecord("bar".getBytes), new SimpleRecord("baz".getBytes)) log.appendAsLeader(records, leaderEpoch = 0) - val abortAppendInfo = appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT) + val abortAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) log.updateHighWatermark(abortAppendInfo.lastOffset + 1) // now there should be no first unstable offset @@ -1827,17 +1314,6 @@ class LogTest { assertEquals(None, reopenedLog.firstUnstableOffset) } - private def endTxnRecords(controlRecordType: ControlRecordType, - producerId: Long, - epoch: Short, - offset: Long = 0L, - coordinatorEpoch: Int, - partitionLeaderEpoch: Int = 0, - timestamp: Long): MemoryRecords = { - val marker = new EndTransactionMarker(controlRecordType, coordinatorEpoch) - MemoryRecords.withEndTransactionMarker(offset, timestamp, partitionLeaderEpoch, producerId, epoch, marker) - } - @Test def testPeriodicProducerIdExpiration(): Unit = { val maxProducerIdExpirationMs = 200 @@ -1969,7 +1445,7 @@ class LogTest { log.appendAsFollower(memoryRecords) log.flush() - val fetchedData = readLog(log, 0, Int.MaxValue) + val fetchedData = LogTestUtils.readLog(log, 0, Int.MaxValue) val origIterator = memoryRecords.batches.iterator() for (batch <- fetchedData.records.batches.asScala) { @@ -2170,13 +1646,13 @@ class LogTest { log.appendAsLeader(TestUtils.singletonRecords(value = value), leaderEpoch = 0) for(i <- values.indices) { - val read = readLog(log, i, 1).records.batches.iterator.next() + val read = LogTestUtils.readLog(log, i, 1).records.batches.iterator.next() assertEquals(i, read.lastOffset, "Offset read should match order appended.") val actual = read.iterator.next() assertNull(actual.key, "Key should be null") assertEquals(ByteBuffer.wrap(values(i)), actual.value, "Values not equal") } - assertEquals(0, readLog(log, values.length, 100).records.batches.asScala.size, + assertEquals(0, LogTestUtils.readLog(log, values.length, 100).records.batches.asScala.size, "Reading beyond the last message returns nothing.") } @@ -2196,7 +1672,7 @@ class LogTest { log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i))) for(i <- 50 until messageIds.max) { val idx = messageIds.indexWhere(_ >= i) - val read = readLog(log, i, 100).records.records.iterator.next() + val read = LogTestUtils.readLog(log, i, 100).records.records.iterator.next() assertEquals(messageIds(idx), read.offset, "Offset read should match message id.") assertEquals(records(idx), new SimpleRecord(read), "Message should match appended.") } @@ -2220,7 +1696,7 @@ class LogTest { // now manually truncate off all but one message from the first segment to create a gap in the messages log.logSegments.head.truncateTo(1) - assertEquals(log.logEndOffset - 1, readLog(log, 1, 200).records.batches.iterator.next().lastOffset, + assertEquals(log.logEndOffset - 1, LogTestUtils.readLog(log, 1, 200).records.batches.iterator.next().lastOffset, "A read should now return the last message in the log") } @@ -2246,9 +1722,9 @@ class LogTest { for (i <- 50 until messageIds.max) { val idx = messageIds.indexWhere(_ >= i) val reads = Seq( - readLog(log, i, 1), - readLog(log, i, 100000), - readLog(log, i, 100) + LogTestUtils.readLog(log, i, 1), + LogTestUtils.readLog(log, i, 100000), + LogTestUtils.readLog(log, i, 100) ).map(_.records.records.iterator.next()) reads.foreach { read => assertEquals(messageIds(idx), read.offset, "Offset read should match message id.") @@ -2269,14 +1745,14 @@ class LogTest { log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i))) for (i <- 50 until messageIds.max) { - assertEquals(MemoryRecords.EMPTY, readLog(log, i, maxLength = 0, minOneMessage = false).records) + assertEquals(MemoryRecords.EMPTY, LogTestUtils.readLog(log, i, maxLength = 0, minOneMessage = false).records) // we return an incomplete message instead of an empty one for the case below // we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is // larger than the fetch size // in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty // partition - val fetchInfo = readLog(log, i, maxLength = 1, minOneMessage = false) + val fetchInfo = LogTestUtils.readLog(log, i, maxLength = 1, minOneMessage = false) assertTrue(fetchInfo.firstEntryIncomplete) assertTrue(fetchInfo.records.isInstanceOf[FileRecords]) assertEquals(1, fetchInfo.records.sizeInBytes) @@ -2297,11 +1773,11 @@ class LogTest { val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.singletonRecords(value = "42".getBytes), leaderEpoch = 0) - assertEquals(0, readLog(log, 1025, 1000).records.sizeInBytes, + assertEquals(0, LogTestUtils.readLog(log, 1025, 1000).records.sizeInBytes, "Reading at the log end offset should produce 0 byte read.") - assertThrows(classOf[OffsetOutOfRangeException], () => readLog(log, 0, 1000)) - assertThrows(classOf[OffsetOutOfRangeException], () => readLog(log, 1026, 1000)) + assertThrows(classOf[OffsetOutOfRangeException], () => LogTestUtils.readLog(log, 0, 1000)) + assertThrows(classOf[OffsetOutOfRangeException], () => LogTestUtils.readLog(log, 1026, 1000)) } /** @@ -2322,7 +1798,7 @@ class LogTest { /* do successive reads to ensure all our messages are there */ var offset = 0L for(i <- 0 until numMessages) { - val messages = readLog(log, offset, 1024*1024).records.batches + val messages = LogTestUtils.readLog(log, offset, 1024*1024).records.batches val head = messages.iterator.next() assertEquals(offset, head.lastOffset, "Offsets not equal") @@ -2333,7 +1809,7 @@ class LogTest { assertEquals(expected.timestamp, actual.timestamp, s"Timestamps not equal at offset $offset") offset = head.lastOffset + 1 } - val lastRead = readLog(log, startOffset = numMessages, maxLength = 1024*1024).records + val lastRead = LogTestUtils.readLog(log, startOffset = numMessages, maxLength = 1024*1024).records assertEquals(0, lastRead.records.asScala.size, "Should be no more messages") // check that rolling the log forced a flushed, the flush is async so retry in case of failure @@ -2355,7 +1831,7 @@ class LogTest { log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0) log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("alpha".getBytes), new SimpleRecord("beta".getBytes)), leaderEpoch = 0) - def read(offset: Int) = readLog(log, offset, 4096).records.records + def read(offset: Int) = LogTestUtils.readLog(log, offset, 4096).records.records /* we should always get the first message in the compressed set when reading any offset in the set */ assertEquals(0, read(0).iterator.next().offset, "Read at offset 0 should produce 0") @@ -2389,7 +1865,7 @@ class LogTest { assertEquals(0, log.deleteOldSegments(), "Further collection shouldn't delete anything") assertEquals(currOffset, log.logEndOffset, "Still no change in the logEndOffset") assertEquals( - currOffset, + currOffset, log.appendAsLeader( TestUtils.singletonRecords(value = "hello".getBytes, timestamp = mockTime.milliseconds), leaderEpoch = 0 @@ -2504,58 +1980,6 @@ class LogTest { log.appendAsFollower(second) } - /** - * Append a bunch of messages to a log and then re-open it both with and without recovery and check that the log re-initializes correctly. - */ - @Test - def testLogRecoversToCorrectOffset(): Unit = { - val numMessages = 100 - val messageSize = 100 - val segmentSize = 7 * messageSize - val indexInterval = 3 * messageSize - val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = indexInterval, segmentIndexBytes = 4096) - var log = createLog(logDir, logConfig) - for(i <- 0 until numMessages) - log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize), - timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) - assertEquals(numMessages, log.logEndOffset, - "After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages)) - val lastIndexOffset = log.activeSegment.offsetIndex.lastOffset - val numIndexEntries = log.activeSegment.offsetIndex.entries - val lastOffset = log.logEndOffset - // After segment is closed, the last entry in the time index should be (largest timestamp -> last offset). - val lastTimeIndexOffset = log.logEndOffset - 1 - val lastTimeIndexTimestamp = log.activeSegment.largestTimestamp - // Depending on when the last time index entry is inserted, an entry may or may not be inserted into the time index. - val numTimeIndexEntries = log.activeSegment.timeIndex.entries + { - if (log.activeSegment.timeIndex.lastEntry.offset == log.logEndOffset - 1) 0 else 1 - } - log.close() - - def verifyRecoveredLog(log: Log, expectedRecoveryPoint: Long): Unit = { - assertEquals(expectedRecoveryPoint, log.recoveryPoint, s"Unexpected recovery point") - assertEquals(numMessages, log.logEndOffset, s"Should have $numMessages messages when log is reopened w/o recovery") - assertEquals(lastIndexOffset, log.activeSegment.offsetIndex.lastOffset, "Should have same last index offset as before.") - assertEquals(numIndexEntries, log.activeSegment.offsetIndex.entries, "Should have same number of index entries as before.") - assertEquals(lastTimeIndexTimestamp, log.activeSegment.timeIndex.lastEntry.timestamp, "Should have same last time index timestamp") - assertEquals(lastTimeIndexOffset, log.activeSegment.timeIndex.lastEntry.offset, "Should have same last time index offset") - assertEquals(numTimeIndexEntries, log.activeSegment.timeIndex.entries, "Should have same number of time index entries as before.") - } - - log = createLog(logDir, logConfig, recoveryPoint = lastOffset, lastShutdownClean = false) - verifyRecoveredLog(log, lastOffset) - log.close() - - // test recovery case - val recoveryPoint = 10 - log = createLog(logDir, logConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) - // the recovery point should not be updated after unclean shutdown until the log is flushed - verifyRecoveredLog(log, recoveryPoint) - log.flush() - verifyRecoveredLog(log, lastOffset) - log.close() - } - @Test def testLogRecoversTopicId(): Unit = { val logConfig = LogTest.createLogConfig() @@ -2609,40 +2033,6 @@ class LogTest { s"The last time index entry should have timestamp ${mockTime.milliseconds + numMessages - 1}") } - /** - * Test that if we manually delete an index segment it is rebuilt when the log is re-opened - */ - @Test - def testIndexRebuild(): Unit = { - // publish the messages and close the log - val numMessages = 200 - val logConfig = LogTest.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) - var log = createLog(logDir, logConfig) - for(i <- 0 until numMessages) - log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) - val indexFiles = log.logSegments.map(_.lazyOffsetIndex.file) - val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) - log.close() - - // delete all the index files - indexFiles.foreach(_.delete()) - timeIndexFiles.foreach(_.delete()) - - // reopen the log - log = createLog(logDir, logConfig, lastShutdownClean = false) - assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages)) - assertTrue(log.logSegments.head.offsetIndex.entries > 0, "The index should have been rebuilt") - assertTrue(log.logSegments.head.timeIndex.entries > 0, "The time index should have been rebuilt") - for(i <- 0 until numMessages) { - assertEquals(i, readLog(log, i, 100).records.batches.iterator.next().lastOffset) - if (i == 0) - assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) - else - assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) - } - log.close() - } - @Test def testFetchOffsetByTimestampIncludesLeaderEpoch(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) @@ -2682,74 +2072,6 @@ class LogTest { log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)) } - /** - * Test that if messages format version of the messages in a segment is before 0.10.0, the time index should be empty. - */ - @Test - def testRebuildTimeIndexForOldMessages(): Unit = { - val numMessages = 200 - val segmentSize = 200 - val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = 1, messageFormatVersion = "0.9.0") - var log = createLog(logDir, logConfig) - for (i <- 0 until numMessages) - log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), - timestamp = mockTime.milliseconds + i * 10, magicValue = RecordBatch.MAGIC_VALUE_V1), leaderEpoch = 0) - val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) - log.close() - - // Delete the time index. - timeIndexFiles.foreach(file => Files.delete(file.toPath)) - - // The rebuilt time index should be empty - log = createLog(logDir, logConfig, recoveryPoint = numMessages + 1, lastShutdownClean = false) - for (segment <- log.logSegments.init) { - assertEquals(0, segment.timeIndex.entries, "The time index should be empty") - assertEquals(0, segment.lazyTimeIndex.file.length, "The time index file size should be 0") - } - } - - /** - * Test that if we have corrupted an index segment it is rebuilt when the log is re-opened - */ - @Test - def testCorruptIndexRebuild(): Unit = { - // publish the messages and close the log - val numMessages = 200 - val logConfig = LogTest.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) - var log = createLog(logDir, logConfig) - for(i <- 0 until numMessages) - log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) - val indexFiles = log.logSegments.map(_.lazyOffsetIndex.file) - val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) - log.close() - - // corrupt all the index files - for( file <- indexFiles) { - val bw = new BufferedWriter(new FileWriter(file)) - bw.write(" ") - bw.close() - } - - // corrupt all the index files - for( file <- timeIndexFiles) { - val bw = new BufferedWriter(new FileWriter(file)) - bw.write(" ") - bw.close() - } - - // reopen the log with recovery point=0 so that the segment recovery can be triggered - log = createLog(logDir, logConfig, lastShutdownClean = false) - assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages)) - for(i <- 0 until numMessages) { - assertEquals(i, readLog(log, i, 100).records.batches.iterator.next().lastOffset) - if (i == 0) - assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) - else - assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) - } - log.close() - } - /** * Test the Log truncate operations */ @@ -2847,66 +2169,7 @@ class LogTest { } /** - * When we open a log any index segments without an associated log segment should be deleted. - */ - @Test - def testBogusIndexSegmentsAreRemoved(): Unit = { - val bogusIndex1 = Log.offsetIndexFile(logDir, 0) - val bogusTimeIndex1 = Log.timeIndexFile(logDir, 0) - val bogusIndex2 = Log.offsetIndexFile(logDir, 5) - val bogusTimeIndex2 = Log.timeIndexFile(logDir, 5) - - // The files remain absent until we first access it because we are doing lazy loading for time index and offset index - // files but in this test case we need to create these files in order to test we will remove them. - bogusIndex2.createNewFile() - bogusTimeIndex2.createNewFile() - - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 1) - val log = createLog(logDir, logConfig) - - // Force the segment to access the index files because we are doing index lazy loading. - log.logSegments.toSeq.head.offsetIndex - log.logSegments.toSeq.head.timeIndex - - assertTrue(bogusIndex1.length > 0, - "The first index file should have been replaced with a larger file") - assertTrue(bogusTimeIndex1.length > 0, - "The first time index file should have been replaced with a larger file") - assertFalse(bogusIndex2.exists, - "The second index file should have been deleted.") - assertFalse(bogusTimeIndex2.exists, - "The second time index file should have been deleted.") - - // check that we can append to the log - for (_ <- 0 until 10) - log.appendAsLeader(createRecords, leaderEpoch = 0) - - log.delete() - } - - /** - * Verify that truncation works correctly after re-opening the log - */ - @Test - def testReopenThenTruncate(): Unit = { - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - // create a log - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000) - var log = createLog(logDir, logConfig) - - // add enough messages to roll over several segments then close and re-open and attempt to truncate - for (_ <- 0 until 100) - log.appendAsLeader(createRecords, leaderEpoch = 0) - log.close() - log = createLog(logDir, logConfig, lastShutdownClean = false) - log.truncateTo(3) - assertEquals(1, log.numberOfSegments, "All but one segment should be deleted.") - assertEquals(3, log.logEndOffset, "Log end offset should be 3.") - } - - /** - * Test that deleted files are deleted after the appropriate time. + * Test that deleted files are deleted after the appropriate time. */ @Test def testAsyncDelete(): Unit = { @@ -2941,32 +2204,11 @@ class LogTest { assertTrue(deletedFiles.forall(!_.exists), "Files should all be gone.") } - /** - * Any files ending in .deleted should be removed when the log is re-opened. - */ - @Test - def testOpenDeletesObsoleteFiles(): Unit = { - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) - var log = createLog(logDir, logConfig) - - // append some messages to create some segments - for (_ <- 0 until 100) - log.appendAsLeader(createRecords, leaderEpoch = 0) - - // expire all segments - log.updateHighWatermark(log.logEndOffset) - log.deleteOldSegments() - log.close() - log = createLog(logDir, logConfig, lastShutdownClean = false) - assertEquals(1, log.numberOfSegments, "The deleted segments should be gone.") - } - @Test def testAppendMessageWithNullPayload(): Unit = { val log = createLog(logDir, LogConfig()) log.appendAsLeader(TestUtils.singletonRecords(value = null), leaderEpoch = 0) - val head = readLog(log, 0, 4096).records.records.iterator.next() + val head = LogTestUtils.readLog(log, 0, 4096).records.records.iterator.next() assertEquals(0, head.offset) assertFalse(head.hasValue, "Message payload should be null.") } @@ -3044,87 +2286,15 @@ class LogTest { val epoch = 0.toShort val log = createLog(logDir, LogConfig()) log.appendAsLeader(TestUtils.singletonRecords(value = null), leaderEpoch = 0) - assertEquals(0, readLog(log, 0, 4096).records.records.iterator.next().offset) - val append = appendTransactionalAsLeader(log, pid, epoch) + assertEquals(0, LogTestUtils.readLog(log, 0, 4096).records.records.iterator.next().offset) + val append = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) append(10) // Kind of a hack, but renaming the index to a directory ensures that the append // to the index will fail. log.activeSegment.txnIndex.renameTo(log.dir) - assertThrows(classOf[KafkaStorageException], () => appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1)) + assertThrows(classOf[KafkaStorageException], () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) assertThrows(classOf[KafkaStorageException], () => log.appendAsLeader(TestUtils.singletonRecords(value = null), leaderEpoch = 0)) - assertThrows(classOf[KafkaStorageException], () => readLog(log, 0, 4096).records.records.iterator.next().offset) - } - - @Test - def testCorruptLog(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - val recoveryPoint = 50L - for (_ <- 0 until 10) { - // create a log and write some messages to it - logDir.mkdirs() - var log = createLog(logDir, logConfig) - val numMessages = 50 + TestUtils.random.nextInt(50) - for (_ <- 0 until numMessages) - log.appendAsLeader(createRecords, leaderEpoch = 0) - val records = log.logSegments.flatMap(_.log.records.asScala.toList).toList - log.close() - - // corrupt index and log by appending random bytes - TestUtils.appendNonsenseToFile(log.activeSegment.lazyOffsetIndex.file, TestUtils.random.nextInt(1024) + 1) - TestUtils.appendNonsenseToFile(log.activeSegment.log.file, TestUtils.random.nextInt(1024) + 1) - - // attempt recovery - log = createLog(logDir, logConfig, brokerTopicStats, 0L, recoveryPoint, lastShutdownClean = false) - assertEquals(numMessages, log.logEndOffset) - - val recovered = log.logSegments.flatMap(_.log.records.asScala.toList).toList - assertEquals(records.size, recovered.size) - - for (i <- records.indices) { - val expected = records(i) - val actual = recovered(i) - assertEquals(expected.key, actual.key, s"Keys not equal") - assertEquals(expected.value, actual.value, s"Values not equal") - assertEquals(expected.timestamp, actual.timestamp, s"Timestamps not equal") - } - - Utils.delete(logDir) - } - } - - @Test - def testOverCompactedLogRecovery(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - val log = createLog(logDir, logConfig) - val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) - val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.NONE, 0, new SimpleRecord("v3".getBytes(), "k3".getBytes())) - val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, CompressionType.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".getBytes())) - val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.NONE, 0, new SimpleRecord("v5".getBytes(), "k5".getBytes())) - //Writes into an empty log with baseOffset 0 - log.appendAsFollower(set1) - assertEquals(0L, log.activeSegment.baseOffset) - //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 - log.appendAsFollower(set2) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) - //This will go into the existing log - log.appendAsFollower(set3) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - //This will go into the existing log - log.appendAsFollower(set4) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - log.close() - val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) - assertEquals(2, indexFiles.length) - for (file <- indexFiles) { - val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) - assertTrue(offsetIndex.lastOffset >= 0) - offsetIndex.close() - } - Utils.delete(logDir) + assertThrows(classOf[KafkaStorageException], () => LogTestUtils.readLog(log, 0, 4096).records.records.iterator.next().offset) } @Test @@ -3183,25 +2353,6 @@ class LogTest { assertEquals(None, log.leaderEpochCache.flatMap(_.latestEpoch)) } - @Test - def testLeaderEpochCacheClearedAfterStaticMessageFormatDowngrade(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - val log = createLog(logDir, logConfig) - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) - assertEquals(Some(5), log.latestEpoch) - log.close() - - // reopen the log with an older message format version and check the cache - val downgradedLogConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, - maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_10_2_IV0.shortVersion) - val reopened = createLog(logDir, downgradedLogConfig, lastShutdownClean = false) - assertLeaderEpochCacheEmpty(reopened) - - reopened.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), - magicValue = RecordVersion.V1.value), leaderEpoch = 5) - assertLeaderEpochCacheEmpty(reopened) - } - @Test def testLeaderEpochCacheClearedAfterDynamicMessageFormatDowngrade(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) @@ -3212,11 +2363,11 @@ class LogTest { val downgradedLogConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_10_2_IV0.shortVersion) log.updateConfig(downgradedLogConfig) - assertLeaderEpochCacheEmpty(log) + LogTestUtils.assertLeaderEpochCacheEmpty(log) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), magicValue = RecordVersion.V1.value), leaderEpoch = 5) - assertLeaderEpochCacheEmpty(log) + LogTestUtils.assertLeaderEpochCacheEmpty(log) } @Test @@ -3226,7 +2377,7 @@ class LogTest { val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), magicValue = RecordVersion.V1.value), leaderEpoch = 5) - assertLeaderEpochCacheEmpty(log) + LogTestUtils.assertLeaderEpochCacheEmpty(log) val upgradedLogConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_11_0_IV0.shortVersion) @@ -3235,91 +2386,7 @@ class LogTest { assertEquals(Some(5), log.latestEpoch) } - private def assertLeaderEpochCacheEmpty(log: Log): Unit = { - assertEquals(None, log.leaderEpochCache) - assertEquals(None, log.latestEpoch) - assertFalse(LeaderEpochCheckpointFile.newFile(log.dir).exists()) - } - @Test - def testOverCompactedLogRecoveryMultiRecord(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - val log = createLog(logDir, logConfig) - val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) - val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, 0, - new SimpleRecord("v3".getBytes(), "k3".getBytes()), - new SimpleRecord("v4".getBytes(), "k4".getBytes())) - val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, 0, - new SimpleRecord("v5".getBytes(), "k5".getBytes()), - new SimpleRecord("v6".getBytes(), "k6".getBytes())) - val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, 0, - new SimpleRecord("v7".getBytes(), "k7".getBytes()), - new SimpleRecord("v8".getBytes(), "k8".getBytes())) - //Writes into an empty log with baseOffset 0 - log.appendAsFollower(set1) - assertEquals(0L, log.activeSegment.baseOffset) - //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 - log.appendAsFollower(set2) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) - //This will go into the existing log - log.appendAsFollower(set3) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - //This will go into the existing log - log.appendAsFollower(set4) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - log.close() - val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) - assertEquals(2, indexFiles.length) - for (file <- indexFiles) { - val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) - assertTrue(offsetIndex.lastOffset >= 0) - offsetIndex.close() - } - Utils.delete(logDir) - } - - @Test - def testOverCompactedLogRecoveryMultiRecordV1(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - val log = createLog(logDir, logConfig) - val set1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0, CompressionType.NONE, - new SimpleRecord("v1".getBytes(), "k1".getBytes())) - val set2 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, - new SimpleRecord("v3".getBytes(), "k3".getBytes()), - new SimpleRecord("v4".getBytes(), "k4".getBytes())) - val set3 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, - new SimpleRecord("v5".getBytes(), "k5".getBytes()), - new SimpleRecord("v6".getBytes(), "k6".getBytes())) - val set4 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, - new SimpleRecord("v7".getBytes(), "k7".getBytes()), - new SimpleRecord("v8".getBytes(), "k8".getBytes())) - //Writes into an empty log with baseOffset 0 - log.appendAsFollower(set1) - assertEquals(0L, log.activeSegment.baseOffset) - //This write will roll the segment, yielding a new segment with base offset = max(1, 3) = 3 - log.appendAsFollower(set2) - assertEquals(3, log.activeSegment.baseOffset) - assertTrue(Log.producerSnapshotFile(logDir, 3).exists) - //This will also roll the segment, yielding a new segment with base offset = max(5, Integer.MAX_VALUE+4) = Integer.MAX_VALUE+4 - log.appendAsFollower(set3) - assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) - assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 4).exists) - //This will go into the existing log - log.appendAsFollower(set4) - assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) - log.close() - val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) - assertEquals(3, indexFiles.length) - for (file <- indexFiles) { - val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) - assertTrue(offsetIndex.lastOffset >= 0) - offsetIndex.close() - } - Utils.delete(logDir) - } @Test def testSplitOnOffsetOverflow(): Unit = { @@ -3396,175 +2463,6 @@ class LogTest { assertFalse(LogTest.hasOffsetOverflow(log)) } - @Test - def testRecoveryOfSegmentWithOffsetOverflow(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, _) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - - // Run recovery on the log. This should split the segment underneath. Ignore .deleted files as we could have still - // have them lying around after the split. - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - - // Running split again would throw an error - - for (segment <- recoveredLog.logSegments) { - assertThrows(classOf[IllegalArgumentException], () => log.splitOverflowedSegment(segment)) - } - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase1(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery just after .cleaned file is created, before rename to .swap. On recovery, existing split - // operation is aborted but the recovery process itself kicks off split which should complete. - newSegments.reverse.foreach(segment => { - segment.changeFileSuffixes("", Log.CleanedFileSuffix) - segment.truncateTo(0) - }) - for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) - Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")), false) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - recoveredLog.close() - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase2(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery just after one of the new segments has been renamed to .swap. On recovery, existing split - // operation is aborted but the recovery process itself kicks off split which should complete. - newSegments.reverse.foreach { segment => - if (segment != newSegments.last) - segment.changeFileSuffixes("", Log.CleanedFileSuffix) - else - segment.changeFileSuffixes("", Log.SwapFileSuffix) - segment.truncateTo(0) - } - for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) - Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")), false) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - recoveredLog.close() - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase3(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery right after all new segments have been renamed to .swap. On recovery, existing split operation - // is completed and the old segment must be deleted. - newSegments.reverse.foreach(segment => { - segment.changeFileSuffixes("", Log.SwapFileSuffix) - }) - for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) - Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")), false) - - // Truncate the old segment - segmentWithOverflow.truncateTo(0) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - log.close() - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase4(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery right after all new segments have been renamed to .swap and old segment has been deleted. On - // recovery, existing split operation is completed. - newSegments.reverse.foreach(_.changeFileSuffixes("", Log.SwapFileSuffix)) - - for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) - Utils.delete(file) - - // Truncate the old segment - segmentWithOverflow.truncateTo(0) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - recoveredLog.close() - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase5(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery right after one of the new segment has been renamed to .swap and the other to .log. On - // recovery, existing split operation is completed. - newSegments.last.changeFileSuffixes("", Log.SwapFileSuffix) - - // Truncate the old segment - segmentWithOverflow.truncateTo(0) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - recoveredLog.close() - } - - @Test - def testCleanShutdownFile(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - - var recoveryPoint = 0L - // create a log and write some messages to it - var log = createLog(logDir, logConfig) - for (_ <- 0 until 100) - log.appendAsLeader(createRecords, leaderEpoch = 0) - log.close() - - // check if recovery was attempted. Even if the recovery point is 0L, recovery should not be attempted as the - // clean shutdown file exists. Note: Earlier, Log layer relied on the presence of clean shutdown file to determine the status - // of last shutdown. Now, LogManager checks for the presence of this file and immediately deletes the same. It passes - // down a clean shutdown flag to the Log layer as log is loaded. Recovery is attempted based on this flag. - recoveryPoint = log.logEndOffset - log = createLog(logDir, logConfig) - assertEquals(recoveryPoint, log.logEndOffset) - } - @Test def testParseTopicPartitionName(): Unit = { val topic = "test_topic" @@ -3910,7 +2808,7 @@ class LogTest { //Then leader epoch should be set on messages for (i <- records.indices) { - val read = readLog(log, i, 1).records.batches.iterator.next() + val read = LogTestUtils.readLog(log, i, 1).records.batches.iterator.next() assertEquals(72, read.partitionLeaderEpoch, "Should have set leader epoch") } } @@ -4038,60 +2936,6 @@ class LogTest { assertEquals(0, cache.epochEntries.size) } - /** - * Append a bunch of messages to a log and then re-open it with recovery and check that the leader epochs are recovered properly. - */ - @Test - def testLogRecoversForLeaderEpoch(): Unit = { - val log = createLog(logDir, LogConfig()) - val leaderEpochCache = epochCache(log) - val firstBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 1, offset = 0) - log.appendAsFollower(records = firstBatch) - - val secondBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 1) - log.appendAsFollower(records = secondBatch) - - val thirdBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 2) - log.appendAsFollower(records = thirdBatch) - - val fourthBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 3, offset = 3) - log.appendAsFollower(records = fourthBatch) - - assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) - - // deliberately remove some of the epoch entries - leaderEpochCache.truncateFromEnd(2) - assertNotEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) - log.close() - - // reopen the log and recover from the beginning - val recoveredLog = createLog(logDir, LogConfig(), lastShutdownClean = false) - val recoveredLeaderEpochCache = epochCache(recoveredLog) - - // epoch entries should be recovered - assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), recoveredLeaderEpochCache.epochEntries) - recoveredLog.close() - } - - /** - * Wrap a single record log buffer with leader epoch. - */ - private def singletonRecordsWithLeaderEpoch(value: Array[Byte], - key: Array[Byte] = null, - leaderEpoch: Int, - offset: Long, - codec: CompressionType = CompressionType.NONE, - timestamp: Long = RecordBatch.NO_TIMESTAMP, - magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = { - val records = Seq(new SimpleRecord(timestamp, key, value)) - - val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) - val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, offset, - mockTime.milliseconds, leaderEpoch) - records.foreach(builder.append) - builder.build() - } - @Test def testFirstUnstableOffsetNoTransactionalData(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) @@ -4133,7 +2977,7 @@ class LogTest { assertEquals(firstAppendInfo.firstOffset.map(_.messageOffset), log.firstUnstableOffset) // now transaction is committed - val commitAppendInfo = appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT) + val commitAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // first unstable offset is not updated until the high watermark is advanced assertEquals(firstAppendInfo.firstOffset.map(_.messageOffset), log.firstUnstableOffset) @@ -4151,7 +2995,7 @@ class LogTest { val producerEpoch = 0.toShort val producerId = 15L - val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch) + val appendProducer = LogTestUtils.appendTransactionalAsLeader(log, producerId, producerEpoch, mockTime) // Thread 1 writes single-record transactions and attempts to read them // before they have been aborted, and then aborts them @@ -4171,7 +3015,7 @@ class LogTest { if (readInfo.records.sizeInBytes() > 0) nonEmptyReads += 1 - appendEndTxnMarkerAsLeader(log, producerId, producerEpoch, ControlRecordType.ABORT) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, producerEpoch, ControlRecordType.ABORT, mockTime.milliseconds()) } nonEmptyReads } @@ -4207,33 +3051,33 @@ class LogTest { val pid3 = 3L val pid4 = 4L - val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch) - val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch) - val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch) - val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch) + val appendPid1 = LogTestUtils.appendTransactionalAsLeader(log, pid1, epoch, mockTime) + val appendPid2 = LogTestUtils.appendTransactionalAsLeader(log, pid2, epoch, mockTime) + val appendPid3 = LogTestUtils.appendTransactionalAsLeader(log, pid3, epoch, mockTime) + val appendPid4 = LogTestUtils.appendTransactionalAsLeader(log, pid4, epoch, mockTime) // mix transactional and non-transactional data appendPid1(5) // nextOffset: 5 - appendNonTransactionalAsLeader(log, 3) // 8 + LogTestUtils.appendNonTransactionalAsLeader(log, 3) // 8 appendPid2(2) // 10 appendPid1(4) // 14 appendPid3(3) // 17 - appendNonTransactionalAsLeader(log, 2) // 19 + LogTestUtils.appendNonTransactionalAsLeader(log, 2) // 19 appendPid1(10) // 29 - appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 30 appendPid2(6) // 36 appendPid4(3) // 39 - appendNonTransactionalAsLeader(log, 10) // 49 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 49 appendPid3(9) // 58 - appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 59 appendPid4(8) // 67 appendPid2(7) // 74 - appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75 - appendNonTransactionalAsLeader(log, 10) // 85 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 75 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 85 appendPid4(4) // 89 - appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 90 - val abortedTransactions = allAbortedTransactions(log) + val abortedTransactions = LogTestUtils.allAbortedTransactions(log) val expectedTransactions = List( new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L) @@ -4251,162 +3095,6 @@ class LogTest { assertEquals(None, log.firstUnstableOffset) } - @Test - def testFullTransactionIndexRecovery(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) - val log = createLog(logDir, logConfig) - val epoch = 0.toShort - - val pid1 = 1L - val pid2 = 2L - val pid3 = 3L - val pid4 = 4L - - val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch) - val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch) - val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch) - val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch) - - // mix transactional and non-transactional data - appendPid1(5) // nextOffset: 5 - appendNonTransactionalAsLeader(log, 3) // 8 - appendPid2(2) // 10 - appendPid1(4) // 14 - appendPid3(3) // 17 - appendNonTransactionalAsLeader(log, 2) // 19 - appendPid1(10) // 29 - appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30 - appendPid2(6) // 36 - appendPid4(3) // 39 - appendNonTransactionalAsLeader(log, 10) // 49 - appendPid3(9) // 58 - appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59 - appendPid4(8) // 67 - appendPid2(7) // 74 - appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75 - appendNonTransactionalAsLeader(log, 10) // 85 - appendPid4(4) // 89 - appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90 - - // delete all the offset and transaction index files to force recovery - log.logSegments.foreach { segment => - segment.offsetIndex.deleteIfExists() - segment.txnIndex.deleteIfExists() - } - - log.close() - - val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) - val reloadedLog = createLog(logDir, reloadedLogConfig, lastShutdownClean = false) - val abortedTransactions = allAbortedTransactions(reloadedLog) - assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) - } - - @Test - def testRecoverOnlyLastSegment(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) - val log = createLog(logDir, logConfig) - val epoch = 0.toShort - - val pid1 = 1L - val pid2 = 2L - val pid3 = 3L - val pid4 = 4L - - val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch) - val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch) - val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch) - val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch) - - // mix transactional and non-transactional data - appendPid1(5) // nextOffset: 5 - appendNonTransactionalAsLeader(log, 3) // 8 - appendPid2(2) // 10 - appendPid1(4) // 14 - appendPid3(3) // 17 - appendNonTransactionalAsLeader(log, 2) // 19 - appendPid1(10) // 29 - appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30 - appendPid2(6) // 36 - appendPid4(3) // 39 - appendNonTransactionalAsLeader(log, 10) // 49 - appendPid3(9) // 58 - appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59 - appendPid4(8) // 67 - appendPid2(7) // 74 - appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75 - appendNonTransactionalAsLeader(log, 10) // 85 - appendPid4(4) // 89 - appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90 - - // delete the last offset and transaction index files to force recovery - val lastSegment = log.logSegments.last - val recoveryPoint = lastSegment.baseOffset - lastSegment.offsetIndex.deleteIfExists() - lastSegment.txnIndex.deleteIfExists() - - log.close() - - val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) - val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) - val abortedTransactions = allAbortedTransactions(reloadedLog) - assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) - } - - @Test - def testRecoverLastSegmentWithNoSnapshots(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) - val log = createLog(logDir, logConfig) - val epoch = 0.toShort - - val pid1 = 1L - val pid2 = 2L - val pid3 = 3L - val pid4 = 4L - - val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch) - val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch) - val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch) - val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch) - - // mix transactional and non-transactional data - appendPid1(5) // nextOffset: 5 - appendNonTransactionalAsLeader(log, 3) // 8 - appendPid2(2) // 10 - appendPid1(4) // 14 - appendPid3(3) // 17 - appendNonTransactionalAsLeader(log, 2) // 19 - appendPid1(10) // 29 - appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30 - appendPid2(6) // 36 - appendPid4(3) // 39 - appendNonTransactionalAsLeader(log, 10) // 49 - appendPid3(9) // 58 - appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59 - appendPid4(8) // 67 - appendPid2(7) // 74 - appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75 - appendNonTransactionalAsLeader(log, 10) // 85 - appendPid4(4) // 89 - appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90 - - deleteProducerSnapshotFiles() - - // delete the last offset and transaction index files to force recovery. this should force us to rebuild - // the producer state from the start of the log - val lastSegment = log.logSegments.last - val recoveryPoint = lastSegment.baseOffset - lastSegment.offsetIndex.deleteIfExists() - lastSegment.txnIndex.deleteIfExists() - - log.close() - - val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) - val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) - val abortedTransactions = allAbortedTransactions(reloadedLog) - assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) - } - @Test def testTransactionIndexUpdatedThroughReplication(): Unit = { val epoch = 0.toShort @@ -4448,7 +3136,7 @@ class LogTest { appendAsFollower(log, MemoryRecords.readableRecords(buffer)) - val abortedTransactions = allAbortedTransactions(log) + val abortedTransactions = LogTestUtils.allAbortedTransactions(log) val expectedTransactions = List( new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L) @@ -4504,16 +3192,17 @@ class LogTest { val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) - val append = appendTransactionalAsLeader(log, pid, epoch) + val append = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) append(10) - appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1) + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1) append(5) - appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT, coordinatorEpoch = 2) + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT, mockTime.milliseconds(), coordinatorEpoch = 2) - assertThrows(classOf[TransactionCoordinatorFencedException], () => appendEndTxnMarkerAsLeader(log, pid, epoch, - ControlRecordType.ABORT, coordinatorEpoch = 1)) + assertThrows( + classOf[TransactionCoordinatorFencedException], + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) } @Test @@ -4526,16 +3215,15 @@ class LogTest { val buffer = ByteBuffer.allocate(256) val append = appendTransactionalToBuffer(buffer, pid, epoch, leaderEpoch = 1) append(0, 10) - appendEndTxnMarkerToBuffer(buffer, pid, epoch, 10L, ControlRecordType.COMMIT, - coordinatorEpoch = 0, leaderEpoch = 1) + appendEndTxnMarkerToBuffer(buffer, pid, epoch, 10L, ControlRecordType.COMMIT, leaderEpoch = 1) buffer.flip() log.appendAsFollower(MemoryRecords.readableRecords(buffer)) - appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 2, leaderEpoch = 1) - appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 2, leaderEpoch = 1) + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 2, leaderEpoch = 1) + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 2, leaderEpoch = 1) assertThrows(classOf[TransactionCoordinatorFencedException], - () => appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1, leaderEpoch = 1)) + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1, leaderEpoch = 1)) } @Test @@ -4544,10 +3232,10 @@ class LogTest { val epoch = 5.toShort val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) - appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1) assertThrows(classOf[InvalidProducerEpochException], - () => appendEndTxnMarkerAsLeader(log, producerId, (epoch - 1).toShort, ControlRecordType.ABORT, coordinatorEpoch = 1)) + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, (epoch - 1).toShort, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) } @Test @@ -4556,10 +3244,10 @@ class LogTest { val log = createLog(logDir, logConfig) val epoch = 0.toShort val pid = 1L - val appendPid = appendTransactionalAsLeader(log, pid, epoch) + val appendPid = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) appendPid(5) - appendNonTransactionalAsLeader(log, 3) + LogTestUtils.appendNonTransactionalAsLeader(log, 3) assertEquals(8L, log.logEndOffset) log.roll() @@ -4581,10 +3269,10 @@ class LogTest { val log = createLog(logDir, logConfig) val epoch = 0.toShort val pid = 1L - val appendPid = appendTransactionalAsLeader(log, pid, epoch) + val appendPid = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) appendPid(5) - appendNonTransactionalAsLeader(log, 3) + LogTestUtils.appendNonTransactionalAsLeader(log, 3) assertEquals(8L, log.logEndOffset) log.roll() @@ -4610,7 +3298,7 @@ class LogTest { val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) - val append = appendTransactionalAsLeader(log, pid, epoch) + val append = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) append(10) // Kind of a hack, but renaming the index to a directory ensures that the append @@ -4618,15 +3306,18 @@ class LogTest { log.activeSegment.txnIndex.renameTo(log.dir) // The append will be written to the log successfully, but the write to the index will fail - assertThrows(classOf[KafkaStorageException], - () => appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1)) + assertThrows( + classOf[KafkaStorageException], + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) assertEquals(11L, log.logEndOffset) assertEquals(0L, log.lastStableOffset) // Try the append a second time. The appended offset in the log should not increase // because the log dir is marked as failed. Nor will there be a write to the transaction // index. - assertThrows(classOf[KafkaStorageException], () => appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1)) + assertThrows( + classOf[KafkaStorageException], + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) assertEquals(11L, log.logEndOffset) assertEquals(0L, log.lastStableOffset) @@ -4700,14 +3391,14 @@ class LogTest { assertEquals(firstAppendInfo.firstOffset.map(_.messageOffset), log.firstUnstableOffset) // now first producer's transaction is aborted - val abortAppendInfo = appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) + val abortAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) log.updateHighWatermark(abortAppendInfo.lastOffset + 1) // LSO should now point to one less than the first offset of the second transaction assertEquals(secondAppendInfo.firstOffset.map(_.messageOffset), log.firstUnstableOffset) // commit the second transaction - val commitAppendInfo = appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.COMMIT) + val commitAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) log.updateHighWatermark(commitAppendInfo.lastOffset + 1) // now there should be no first unstable offset @@ -4741,7 +3432,7 @@ class LogTest { assertEquals(3L, log.logEndOffsetMetadata.segmentBaseOffset) // now abort the transaction - val abortAppendInfo = appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT) + val abortAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) log.updateHighWatermark(abortAppendInfo.lastOffset + 1) assertEquals(None, log.firstUnstableOffset) @@ -4767,61 +3458,6 @@ class LogTest { assertEquals(1, log.numberOfSegments) } - private def allAbortedTransactions(log: Log) = log.logSegments.flatMap(_.txnIndex.allAbortedTxns) - - private def appendTransactionalAsLeader( - log: Log, - producerId: Long, - producerEpoch: Short - ): Int => Unit = { - appendIdempotentAsLeader(log, producerId, producerEpoch, isTransactional = true) - } - - private def appendIdempotentAsLeader( - log: Log, - producerId: Long, - producerEpoch: Short, - isTransactional: Boolean = false - ): Int => Unit = { - var sequence = 0 - numRecords: Int => { - val simpleRecords = (sequence until sequence + numRecords).map { seq => - new SimpleRecord(mockTime.milliseconds(), s"$seq".getBytes) - } - - val records = if (isTransactional) { - MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, - producerEpoch, sequence, simpleRecords: _*) - } else { - MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, - producerEpoch, sequence, simpleRecords: _*) - } - - log.appendAsLeader(records, leaderEpoch = 0) - sequence += numRecords - } - } - - private def appendEndTxnMarkerAsLeader(log: Log, - producerId: Long, - producerEpoch: Short, - controlType: ControlRecordType, - coordinatorEpoch: Int = 0, - leaderEpoch: Int = 0, - timestamp: Long = mockTime.milliseconds()): LogAppendInfo = { - val records = endTxnRecords(controlType, producerId, producerEpoch, - coordinatorEpoch = coordinatorEpoch, timestamp = timestamp) - log.appendAsLeader(records, origin = AppendOrigin.Coordinator, leaderEpoch = leaderEpoch) - } - - private def appendNonTransactionalAsLeader(log: Log, numRecords: Int): Unit = { - val simpleRecords = (0 until numRecords).map { seq => - new SimpleRecord(s"$seq".getBytes) - } - val records = MemoryRecords.withRecords(CompressionType.NONE, simpleRecords: _*) - log.appendAsLeader(records, leaderEpoch = 0) - } - private def appendTransactionalToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, @@ -4864,14 +3500,6 @@ class LogTest { log.appendAsFollower(records) } - private def deleteProducerSnapshotFiles(): Unit = { - val files = logDir.listFiles.filter(f => f.isFile && f.getName.endsWith(Log.ProducerSnapshotFileSuffix)) - files.foreach(Utils.delete) - } - - private def listProducerSnapshotOffsets: Seq[Long] = - ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted - private def createLog(dir: File, config: LogConfig, brokerTopicStats: BrokerTopicStats = brokerTopicStats, @@ -4897,20 +3525,6 @@ class LogTest { (log, segmentWithOverflow) } - - private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]) = { - // method is called only in case of recovery from hard reset - LogTest.recoverAndCheck(logDir, config, expectedKeys, brokerTopicStats, mockTime, mockTime.scheduler) - } - - private def readLog(log: Log, - startOffset: Long, - maxLength: Int, - isolation: FetchIsolation = FetchLogEnd, - minOneMessage: Boolean = true): FetchDataInfo = { - log.read(startOffset, maxLength, isolation, minOneMessage) - } - } object LogTest { @@ -5055,19 +3669,4 @@ object LogTest { record <- batch.asScala if record.hasValue && record.hasKey) yield TestUtils.readString(record.key).toLong } - - def recoverAndCheck(logDir: File, config: LogConfig, expectedKeys: Iterable[Long], brokerTopicStats: BrokerTopicStats, time: Time, scheduler: Scheduler): Log = { - // Recover log file and check that after recovery, keys are as expected - // and all temporary files have been deleted - val recoveredLog = createLog(logDir, config, brokerTopicStats, scheduler, time, lastShutdownClean = false) - time.sleep(config.fileDeleteDelayMs + 1) - for (file <- logDir.listFiles) { - assertFalse(file.getName.endsWith(Log.DeletedFileSuffix), "Unexpected .deleted file after recovery") - assertFalse(file.getName.endsWith(Log.CleanedFileSuffix), "Unexpected .cleaned file after recovery") - assertFalse(file.getName.endsWith(Log.SwapFileSuffix), "Unexpected .swap file after recovery") - } - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertFalse(LogTest.hasOffsetOverflow(recoveredLog)) - recoveredLog - } } diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index b52d8d84c5b25..29777b2dec900 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -18,9 +18,18 @@ package kafka.log import java.io.File +import java.util.Properties -import org.apache.kafka.common.record.FileRecords -import org.apache.kafka.common.utils.Time +import kafka.server.checkpoints.LeaderEpochCheckpointFile +import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchIsolation, FetchLogEnd, LogDirFailureChannel} +import kafka.utils.{Scheduler, TestUtils} +import org.apache.kafka.common.Uuid +import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord} +import org.apache.kafka.common.utils.{Time, Utils} +import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} + +import scala.collection.Iterable +import scala.jdk.CollectionConverters._ object LogTestUtils { /** @@ -37,4 +46,233 @@ object LogTestUtils { new LogSegment(ms, idx, timeIdx, txnIndex, offset, indexIntervalBytes, 0, time) } + + def createLogConfig(segmentMs: Long = Defaults.SegmentMs, + segmentBytes: Int = Defaults.SegmentSize, + retentionMs: Long = Defaults.RetentionMs, + retentionBytes: Long = Defaults.RetentionSize, + segmentJitterMs: Long = Defaults.SegmentJitterMs, + cleanupPolicy: String = Defaults.CleanupPolicy, + maxMessageBytes: Int = Defaults.MaxMessageSize, + indexIntervalBytes: Int = Defaults.IndexInterval, + segmentIndexBytes: Int = Defaults.MaxIndexSize, + messageFormatVersion: String = Defaults.MessageFormatVersion, + fileDeleteDelayMs: Long = Defaults.FileDeleteDelayMs): LogConfig = { + val logProps = new Properties() + + logProps.put(LogConfig.SegmentMsProp, segmentMs: java.lang.Long) + logProps.put(LogConfig.SegmentBytesProp, segmentBytes: Integer) + logProps.put(LogConfig.RetentionMsProp, retentionMs: java.lang.Long) + logProps.put(LogConfig.RetentionBytesProp, retentionBytes: java.lang.Long) + logProps.put(LogConfig.SegmentJitterMsProp, segmentJitterMs: java.lang.Long) + logProps.put(LogConfig.CleanupPolicyProp, cleanupPolicy) + logProps.put(LogConfig.MaxMessageBytesProp, maxMessageBytes: Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, indexIntervalBytes: Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, segmentIndexBytes: Integer) + logProps.put(LogConfig.MessageFormatVersionProp, messageFormatVersion) + logProps.put(LogConfig.FileDeleteDelayMsProp, fileDeleteDelayMs: java.lang.Long) + LogConfig(logProps) + } + + def createLog(dir: File, + config: LogConfig, + brokerTopicStats: BrokerTopicStats, + scheduler: Scheduler, + time: Time, + logStartOffset: Long = 0L, + recoveryPoint: Long = 0L, + maxProducerIdExpirationMs: Int = 60 * 60 * 1000, + producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs, + lastShutdownClean: Boolean = true, + topicId: Option[Uuid] = None): Log = { + Log(dir = dir, + config = config, + logStartOffset = logStartOffset, + recoveryPoint = recoveryPoint, + scheduler = scheduler, + brokerTopicStats = brokerTopicStats, + time = time, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs, + logDirFailureChannel = new LogDirFailureChannel(10), + lastShutdownClean = lastShutdownClean, + topicId = topicId, + keepPartitionMetadataFile = true) + } + + /** + * Check if the given log contains any segment with records that cause offset overflow. + * @param log Log to check + * @return true if log contains at least one segment with offset overflow; false otherwise + */ + def hasOffsetOverflow(log: Log): Boolean = firstOverflowSegment(log).isDefined + + def firstOverflowSegment(log: Log): Option[LogSegment] = { + def hasOverflow(baseOffset: Long, batch: RecordBatch): Boolean = + batch.lastOffset > baseOffset + Int.MaxValue || batch.baseOffset < baseOffset + + for (segment <- log.logSegments) { + val overflowBatch = segment.log.batches.asScala.find(batch => hasOverflow(segment.baseOffset, batch)) + if (overflowBatch.isDefined) + return Some(segment) + } + None + } + + private def rawSegment(logDir: File, baseOffset: Long): FileRecords = + FileRecords.open(Log.logFile(logDir, baseOffset)) + + /** + * Initialize the given log directory with a set of segments, one of which will have an + * offset which overflows the segment + */ + def initializeLogDirWithOverflowedSegment(logDir: File): Unit = { + def writeSampleBatches(baseOffset: Long, segment: FileRecords): Long = { + def record(offset: Long) = { + val data = offset.toString.getBytes + new SimpleRecord(data, data) + } + + segment.append(MemoryRecords.withRecords(baseOffset, CompressionType.NONE, 0, + record(baseOffset))) + segment.append(MemoryRecords.withRecords(baseOffset + 1, CompressionType.NONE, 0, + record(baseOffset + 1), + record(baseOffset + 2))) + segment.append(MemoryRecords.withRecords(baseOffset + Int.MaxValue - 1, CompressionType.NONE, 0, + record(baseOffset + Int.MaxValue - 1))) + // Need to create the offset files explicitly to avoid triggering segment recovery to truncate segment. + Log.offsetIndexFile(logDir, baseOffset).createNewFile() + Log.timeIndexFile(logDir, baseOffset).createNewFile() + baseOffset + Int.MaxValue + } + + def writeNormalSegment(baseOffset: Long): Long = { + val segment = rawSegment(logDir, baseOffset) + try writeSampleBatches(baseOffset, segment) + finally segment.close() + } + + def writeOverflowSegment(baseOffset: Long): Long = { + val segment = rawSegment(logDir, baseOffset) + try { + val nextOffset = writeSampleBatches(baseOffset, segment) + writeSampleBatches(nextOffset, segment) + } finally segment.close() + } + + // We create three segments, the second of which contains offsets which overflow + var nextOffset = 0L + nextOffset = writeNormalSegment(nextOffset) + nextOffset = writeOverflowSegment(nextOffset) + writeNormalSegment(nextOffset) + } + + /* extract all the keys from a log */ + def keysInLog(log: Log): Iterable[Long] = { + for (logSegment <- log.logSegments; + batch <- logSegment.log.batches.asScala if !batch.isControlBatch; + record <- batch.asScala if record.hasValue && record.hasKey) + yield TestUtils.readString(record.key).toLong + } + + def recoverAndCheck(logDir: File, config: LogConfig, expectedKeys: Iterable[Long], brokerTopicStats: BrokerTopicStats, time: Time, scheduler: Scheduler): Log = { + // Recover log file and check that after recovery, keys are as expected + // and all temporary files have been deleted + val recoveredLog = createLog(logDir, config, brokerTopicStats, scheduler, time, lastShutdownClean = false) + time.sleep(config.fileDeleteDelayMs + 1) + for (file <- logDir.listFiles) { + assertFalse(file.getName.endsWith(Log.DeletedFileSuffix), "Unexpected .deleted file after recovery") + assertFalse(file.getName.endsWith(Log.CleanedFileSuffix), "Unexpected .cleaned file after recovery") + assertFalse(file.getName.endsWith(Log.SwapFileSuffix), "Unexpected .swap file after recovery") + } + assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) + assertFalse(LogTest.hasOffsetOverflow(recoveredLog)) + recoveredLog + } + + def appendEndTxnMarkerAsLeader(log: Log, + producerId: Long, + producerEpoch: Short, + controlType: ControlRecordType, + timestamp: Long, + coordinatorEpoch: Int = 0, + leaderEpoch: Int = 0): LogAppendInfo = { + val records = endTxnRecords(controlType, producerId, producerEpoch, + coordinatorEpoch = coordinatorEpoch, timestamp = timestamp) + log.appendAsLeader(records, origin = AppendOrigin.Coordinator, leaderEpoch = leaderEpoch) + } + + private def endTxnRecords(controlRecordType: ControlRecordType, + producerId: Long, + epoch: Short, + offset: Long = 0L, + coordinatorEpoch: Int, + partitionLeaderEpoch: Int = 0, + timestamp: Long): MemoryRecords = { + val marker = new EndTransactionMarker(controlRecordType, coordinatorEpoch) + MemoryRecords.withEndTransactionMarker(offset, timestamp, partitionLeaderEpoch, producerId, epoch, marker) + } + + def readLog(log: Log, + startOffset: Long, + maxLength: Int, + isolation: FetchIsolation = FetchLogEnd, + minOneMessage: Boolean = true): FetchDataInfo = { + log.read(startOffset, maxLength, isolation, minOneMessage) + } + + def allAbortedTransactions(log: Log): Iterable[AbortedTxn] = log.logSegments.flatMap(_.txnIndex.allAbortedTxns) + + def deleteProducerSnapshotFiles(logDir: File): Unit = { + val files = logDir.listFiles.filter(f => f.isFile && f.getName.endsWith(Log.ProducerSnapshotFileSuffix)) + files.foreach(Utils.delete) + } + + def listProducerSnapshotOffsets(logDir: File): Seq[Long] = + ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted + + def assertLeaderEpochCacheEmpty(log: Log): Unit = { + assertEquals(None, log.leaderEpochCache) + assertEquals(None, log.latestEpoch) + assertFalse(LeaderEpochCheckpointFile.newFile(log.dir).exists()) + } + + def appendNonTransactionalAsLeader(log: Log, numRecords: Int): Unit = { + val simpleRecords = (0 until numRecords).map { seq => + new SimpleRecord(s"$seq".getBytes) + } + val records = MemoryRecords.withRecords(CompressionType.NONE, simpleRecords: _*) + log.appendAsLeader(records, leaderEpoch = 0) + } + + def appendTransactionalAsLeader(log: Log, + producerId: Long, + producerEpoch: Short, + time: Time): Int => Unit = { + appendIdempotentAsLeader(log, producerId, producerEpoch, time, isTransactional = true) + } + + def appendIdempotentAsLeader(log: Log, + producerId: Long, + producerEpoch: Short, + time: Time, + isTransactional: Boolean = false): Int => Unit = { + var sequence = 0 + numRecords: Int => { + val simpleRecords = (sequence until sequence + numRecords).map { seq => + new SimpleRecord(time.milliseconds(), s"$seq".getBytes) + } + + val records = if (isTransactional) { + MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, + producerEpoch, sequence, simpleRecords: _*) + } else { + MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, + producerEpoch, sequence, simpleRecords: _*) + } + + log.appendAsLeader(records, leaderEpoch = 0) + sequence += numRecords + } + } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 4bbe7012c46cc..deb8827bc7dc8 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -47,12 +47,13 @@ import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.Mockito - import java.io.File import java.net.InetAddress +import java.nio.file.Files import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.{Collections, Optional, Properties} + import scala.collection.{Map, Seq, mutable} import scala.jdk.CollectionConverters._ @@ -1476,23 +1477,46 @@ class ReplicaManagerTest { props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) props.asScala ++= extraProps.asScala val config = KafkaConfig.fromProps(props) - + val logConfig = LogConfig() + val logDir = new File(new File(config.logDirs.head), s"$topic-$topicPartition") + Files.createDirectories(logDir.toPath) val mockScheduler = new MockScheduler(time) val mockBrokerTopicStats = new BrokerTopicStats val mockLogDirFailureChannel = new LogDirFailureChannel(config.logDirs.size) + val tp = new TopicPartition(topic, topicPartition) + val maxProducerIdExpirationMs = 30000 + val segments = new LogSegments(tp) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, tp, mockLogDirFailureChannel, logConfig.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(tp, logDir, maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + logDir, + tp, + logConfig, + mockScheduler, + time, + mockLogDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) val mockLog = new Log( - _dir = new File(new File(config.logDirs.head), s"$topic-0"), - config = LogConfig(), - logStartOffset = 0L, - recoveryPoint = 0L, + _dir = logDir, + config = logConfig, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, scheduler = mockScheduler, brokerTopicStats = mockBrokerTopicStats, time = time, - maxProducerIdExpirationMs = 30000, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = 30000, - topicPartition = new TopicPartition(topic, topicPartition), - producerStateManager = new ProducerStateManager(new TopicPartition(topic, topicPartition), - new File(new File(config.logDirs.head), s"$topic-$topicPartition"), 30000), + topicPartition = tp, + leaderEpochCache = leaderEpochCache, + producerStateManager = producerStateManager, logDirFailureChannel = mockLogDirFailureChannel, topicId = topicId, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index 913d3e2449a4b..159ace8103e5a 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -19,7 +19,8 @@ package kafka.utils import java.util.Properties import java.util.concurrent.atomic._ import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} -import kafka.log.{Log, LogConfig, LogManager, ProducerStateManager} + +import kafka.log.{LoadLogParams, Log, LogConfig, LogLoader, LogManager, LogSegments, ProducerStateManager} import kafka.server.{BrokerTopicStats, LogDirFailureChannel} import kafka.utils.TestUtils.retry import org.junit.jupiter.api.Assertions._ @@ -118,13 +119,31 @@ class SchedulerTest { val logDir = TestUtils.randomPartitionLogDir(tmpDir) val logConfig = LogConfig(new Properties()) val brokerTopicStats = new BrokerTopicStats - val recoveryPoint = 0L val maxProducerIdExpirationMs = 60 * 60 * 1000 val topicPartition = Log.parseTopicPartitionName(logDir) + val logDirFailureChannel = new LogDirFailureChannel(10) + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.messageFormatVersion.recordVersion) val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) - val log = new Log(logDir, logConfig, logStartOffset = 0, recoveryPoint = recoveryPoint, scheduler, + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + logConfig, + scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + val log = new Log(logDir, logConfig, segments = segments, logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, nextOffsetMetadata = offsets.nextOffsetMetadata, scheduler, brokerTopicStats, mockTime, maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs, - topicPartition, producerStateManager, new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true) + topicPartition, leaderEpochCache, producerStateManager, logDirFailureChannel, + topicId = None, keepPartitionMetadataFile = true) assertTrue(scheduler.taskRunning(log.producerExpireCheck)) log.close() assertFalse(scheduler.taskRunning(log.producerExpireCheck))