From 6d24769995ead499cb497431e47d8a3b39a28e2b Mon Sep 17 00:00:00 2001
From: Kowshik Prakasam
Date: Thu, 1 Apr 2021 23:49:12 -0700
Subject: [PATCH 1/7] KAFKA-12553: Refactor recovery logic
---
core/src/main/scala/kafka/log/Log.scala | 1097 +++++-------
.../src/main/scala/kafka/log/LogCleaner.scala | 11 +-
core/src/main/scala/kafka/log/LogConfig.scala | 7 +
core/src/main/scala/kafka/log/LogLoader.scala | 530 ++++++
.../kafka/cluster/PartitionLockTest.scala | 3 +
.../unit/kafka/cluster/PartitionTest.scala | 3 +
.../kafka/log/LogCleanerManagerTest.scala | 21 +-
.../scala/unit/kafka/log/LogCleanerTest.scala | 26 +-
.../scala/unit/kafka/log/LogLoaderTest.scala | 1444 +++++++++++++++
.../test/scala/unit/kafka/log/LogTest.scala | 1587 +----------------
.../scala/unit/kafka/log/LogTestUtils.scala | 256 ++-
.../kafka/server/ReplicaManagerTest.scala | 29 +-
.../unit/kafka/utils/SchedulerTest.scala | 14 +-
13 files changed, 2844 insertions(+), 2184 deletions(-)
create mode 100644 core/src/main/scala/kafka/log/LogLoader.scala
create mode 100644 core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index 1111d98bae1cf..871590be61982 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 log segments, these may be non-empty when 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,19 +261,21 @@ 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 {
+ val keepPartitionMetadataFile: Boolean = true) extends Logging with KafkaMetricsGroup {
import kafka.log.Log._
@@ -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))
}
@@ -2306,7 +1876,7 @@ class Log(@volatile private var _dir: File,
// removing the deleted segment, we should force materialization of the iterator here, so that results of the
// iteration remain valid and deterministic.
val toDelete = segments.toList
- reason.logReason(this, toDelete)
+ reason.logReason(this, toDelete, this)
toDelete.foreach { segment =>
this.segments.remove(segment.baseOffset)
}
@@ -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:
- *
- * - 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().
- *
- 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.
- *
- 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 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.
- *
- 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,105 +1930,26 @@ 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")
+/**
+ * Helper functions for logs
+ */
+object Log extends Logging {
+ /** a log file */
+ val LogFileSuffix = ".log"
- 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"
-
- /** an index file */
- val IndexFileSuffix = ".index"
+ /** an index file */
+ val IndexFileSuffix = ".index"
/** a time index file */
val TimeIndexFileSuffix = ".timeindex"
@@ -2586,11 +1999,15 @@ object Log {
logDirFailureChannel: LogDirFailureChannel,
lastShutdownClean: Boolean = true,
topicId: Option[Uuid],
- keepPartitionMetadataFile: Boolean): Log = {
+ keepPartitionMetadataFile: Boolean = true): Log = {
+ // create the log directory if it doesn't exist
+ Files.createDirectories(dir.toPath)
val topicPartition = Log.parseTopicPartitionName(dir)
- 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 logLoader = new LogLoader(dir, topicPartition, config, scheduler, time, logDirFailureChannel)
+ val logComponents = logLoader.load(logStartOffset, recoveryPoint, maxProducerIdExpirationMs, lastShutdownClean)
+ new Log(dir, config, logComponents.segments, logComponents.logStartOffset, logComponents.recoveryPoint, logComponents.nextOffsetMetadata, scheduler,
+ brokerTopicStats, time, maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs, topicPartition,
+ logComponents.leaderEpochCache, logComponents.producerStateManager, logDirFailureChannel, topicId, keepPartitionMetadataFile)
}
/**
@@ -2748,12 +2165,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 +2201,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 {
@@ -2797,20 +2590,22 @@ object LogMetricNames {
}
}
-sealed trait SegmentDeletionReason {
- def logReason(log: Log, toDelete: List[LogSegment]): Unit
+abstract class SegmentDeletionReason {
+ def logReason(logger: Logging, toDelete: List[LogSegment]): Unit = {}
+
+ def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {}
}
case object RetentionMsBreach extends SegmentDeletionReason {
- override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
+ override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
val retentionMs = log.config.retentionMs
toDelete.foreach { segment =>
segment.largestRecordTimestamp match {
case Some(_) =>
- log.info(s"Deleting segment $segment due to retention time ${retentionMs}ms breach based on the largest " +
+ logger.info(s"Deleting segment $segment due to retention time ${retentionMs}ms breach based on the largest " +
s"record timestamp in the segment")
case None =>
- log.info(s"Deleting segment $segment due to retention time ${retentionMs}ms breach based on the " +
+ logger.info(s"Deleting segment $segment due to retention time ${retentionMs}ms breach based on the " +
s"last modified time of the segment")
}
}
@@ -2818,42 +2613,42 @@ case object RetentionMsBreach extends SegmentDeletionReason {
}
case object RetentionSizeBreach extends SegmentDeletionReason {
- override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
+ override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
var size = log.size
toDelete.foreach { segment =>
size -= segment.size
- log.info(s"Deleting segment $segment due to retention size ${log.config.retentionSize} breach. Log size " +
+ logger.info(s"Deleting segment $segment due to retention size ${log.config.retentionSize} breach. Log size " +
s"after deletion will be $size.")
}
}
}
case object StartOffsetBreach extends SegmentDeletionReason {
- override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
- log.info(s"Deleting segments due to log start offset ${log.logStartOffset} breach: ${toDelete.mkString(",")}")
+ override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
+ logger.info(s"Deleting segments due to log start offset ${log.logStartOffset} breach: ${toDelete.mkString(",")}")
}
}
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(",")}")
+ override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
+ logger.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(",")}")
+ override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
+ logger.info(s"Deleting segments as part of log truncation: ${toDelete.mkString(",")}")
}
}
case object LogRoll extends SegmentDeletionReason {
- override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
- log.info(s"Deleting segments as part of log roll: ${toDelete.mkString(",")}")
+ override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
+ logger.info(s"Deleting segments as part of log roll: ${toDelete.mkString(",")}")
}
}
case object LogDeletion extends SegmentDeletionReason {
- override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
- log.info(s"Deleting segments as the log has been deleted: ${toDelete.mkString(",")}")
+ override def logReason(logger: Logging, toDelete: List[LogSegment]): Unit = {
+ logger.info(s"Deleting segments as the log has been deleted: ${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..df48e435f0108
--- /dev/null
+++ b/core/src/main/scala/kafka/log/LogLoader.scala
@@ -0,0 +1,530 @@
+/**
+ * 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.record.RecordVersion
+import org.apache.kafka.common.utils.Time
+
+import scala.collection.{Seq, Set, mutable}
+
+case class LogComponents(segments: LogSegments,
+ logStartOffset: Long,
+ recoveryPoint: Long,
+ nextOffsetMetadata: LogOffsetMetadata,
+ leaderEpochCache: Option[LeaderEpochFileCache],
+ producerStateManager: ProducerStateManager)
+
+/**
+ * This class is responsible for all activities related with recovery of log segments from disk.
+ * Additionally, it also suitably updates the LeaderEpochFileCache and ProducerStateManager to
+ * reflect the contents of the recovered segments.
+ *
+ * @param dir The directory in which log segments are created.
+ * @param topicPartition The topic partition associated with the Log
+ * @param config The log configuration settings
+ * @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
+ */
+class LogLoader(dir: File,
+ topicPartition: TopicPartition,
+ config: LogConfig,
+ scheduler: Scheduler,
+ time: Time,
+ logDirFailureChannel: LogDirFailureChannel) extends Logging {
+
+ this.logIdent = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}] "
+
+ protected def createLogSegments(topicPartition: TopicPartition): LogSegments = new LogSegments(topicPartition)
+
+ protected def createProducerStateManager(topicPartition: TopicPartition,
+ dir: File,
+ maxProducerIdExpirationMs: Int): ProducerStateManager =
+ new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+
+ protected def maybeCreateLeaderEpochCache(dir: File,
+ topicPartition: TopicPartition,
+ logDirFailureChannel: LogDirFailureChannel,
+ recordVersion: RecordVersion): Option[LeaderEpochFileCache] =
+ Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion)
+
+ /**
+ * Load the log segments from the log files on disk, and return the components of the loaded log.
+ *
+ * This method does not need to convert IOException to KafkaStorageException because it is only
+ * called before all logs are loaded.
+ *
+ * @param 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 hadCleanShutdown Boolean flag to indicate whether the associated Log previously had a
+ * clean shutdown
+ *
+ * @return the components of the Log successfully loaded from disk
+ *
+ * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that
+ * overflow index offset; or when we find an unexpected
+ * number of .log files with overflow
+ */
+ def load(logStartOffsetCheckpoint: Long,
+ recoveryPointCheckpoint: Long,
+ maxProducerIdExpirationMs: Int,
+ hadCleanShutdown: Boolean = true): LogComponents = {
+ val leaderEpochFileCache = maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion)
+ val producerStateManager = createProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+ val components = LogComponents(
+ createLogSegments(topicPartition),
+ logStartOffsetCheckpoint,
+ recoveryPointCheckpoint,
+ LogOffsetMetadata(0, 0, 0),
+ leaderEpochFileCache,
+ producerStateManager)
+ // 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(components, {
+ // 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().
+ components.segments.close()
+ components.segments.clear()
+ loadSegmentFiles(components, maxProducerIdExpirationMs)
+ })
+
+ completeSwapOperations(swapFiles, maxProducerIdExpirationMs, components)
+
+ val (newRecoveryPoint: Long, nextOffset: Long) = {
+ if (!dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) {
+ val (newRecoveryPoint, nextOffset) = retryOnOffsetOverflow(components, {
+ recoverLog(components, maxProducerIdExpirationMs, hadCleanShutdown)
+ })
+
+ // reset the index size of the currently active log segment to allow more entries
+ components.segments.lastSegment.get.resizeIndexes(config.maxIndexSize)
+ (newRecoveryPoint, nextOffset)
+ } else {
+ if (components.segments.isEmpty) {
+ components.segments.add(LogSegment.open(dir = dir,
+ baseOffset = 0,
+ config,
+ time = time,
+ initFileSize = config.initFileSize))
+ }
+ (components.recoveryPoint, 0L)
+ }
+ }
+
+ components.leaderEpochCache.foreach(_.truncateFromEnd(nextOffset))
+ val newLogStartOffset = math.max(components.logStartOffset, components.segments.firstSegment.get.baseOffset)
+ // The earliest leader epoch may not be flushed during a hard failure. Recover it here.
+ components.leaderEpochCache.foreach(_.truncateFromStart(components.logStartOffset))
+
+ // Any segment loading or recovery code must not use producerStateManager, so that we can build the full state here
+ // from scratch.
+ if (!components.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.
+ components.producerStateManager.removeStraySnapshots(components.segments.baseOffsets.toSeq)
+ Log.rebuildProducerState(
+ components.producerStateManager,
+ components.segments,
+ newLogStartOffset,
+ nextOffset,
+ config.messageFormatVersion.recordVersion,
+ time,
+ reloadFromCleanShutdown = hadCleanShutdown)
+
+ val activeSegment = components.segments.lastSegment.get
+ LogComponents(
+ components.segments,
+ newLogStartOffset,
+ newRecoveryPoint,
+ LogOffsetMetadata(nextOffset, activeSegment.baseOffset, activeSegment.size),
+ components.leaderEpochCache,
+ components.producerStateManager)
+ }
+
+ /**
+ * 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 (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"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
+ }
+
+ /**
+ * 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 components The components of the log being loaded
+ * @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](components: LogComponents, fn: => T): T = {
+ while (true) {
+ try {
+ return fn
+ } catch {
+ case e: LogSegmentOffsetOverflowException =>
+ info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
+ Log.splitOverflowedSegment(e.segment, components.segments, dir, topicPartition, config, scheduler, logDirFailureChannel, components.producerStateManager)
+ }
+ }
+ throw new IllegalStateException()
+ }
+
+ /**
+ * Loads segments from disk into the provided LogComponents.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 components The components of the log being loaded
+ * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is
+ * considered expired
+ * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset
+ */
+ private def loadSegmentFiles(components: LogComponents, maxProducerIdExpirationMs: Int): Unit = {
+ // load segments in ascending order because transactional data from one segment may depend on the
+ // segments that come before it
+ for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+ if (isIndexFile(file)) {
+ // if it is an index file, make sure it has a corresponding .log file
+ val offset = offsetFromFile(file)
+ val logFile = 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)
+
+ 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, maxProducerIdExpirationMs, components)
+ 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, maxProducerIdExpirationMs, components)
+ }
+ components.segments.add(segment)
+ }
+ }
+ }
+
+ /**
+ * Just recovers the given segment, without adding it to the provided LogComponents.segments.
+ *
+ * @param segment Segment to recover
+ * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is
+ * considered expired
+ * @param components The components of the log being loaded
+ *
+ * @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,
+ maxProducerIdExpirationMs: Int,
+ components: LogComponents): Int = {
+ val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+ Log.rebuildProducerState(
+ producerStateManager,
+ components.segments,
+ components.logStartOffset,
+ segment.baseOffset,
+ config.messageFormatVersion.recordVersion,
+ time,
+ reloadFromCleanShutdown = false)
+ val bytesTruncated = segment.recover(producerStateManager, components.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 files
+ *
+ * @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],
+ maxProducerIdExpirationMs: Int,
+ components: LogComponents): 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,
+ config,
+ time = time,
+ fileSuffix = Log.SwapFileSuffix)
+ info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
+ recoverSegment(swapSegment, maxProducerIdExpirationMs, components)
+
+ // 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 = components.segments.values(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+ segment.readNextOffset > swapSegment.baseOffset
+ }
+ Log.replaceSegments(components.segments, Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true,
+ dir, topicPartition, config, scheduler, logDirFailureChannel, components.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 components The components of the log being loaded
+ * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is
+ * considered expired
+ * @param hadCleanShutdown Boolean flag to indicate whether the associated Log previously had a
+ * clean shutdown
+ *
+ * @return a tuple containing (newRecoveryPoint, nextOffset).
+ *
+ * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow
+ */
+ private[log] def recoverLog(components: LogComponents, maxProducerIdExpirationMs: Int, hadCleanShutdown: Boolean): (Long, Long) = {
+ /** return the log end offset if valid */
+ def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
+ if (components.segments.nonEmpty) {
+ val logEndOffset = components.segments.lastSegment.get.readNextOffset
+ if (logEndOffset >= components.logStartOffset)
+ Some(logEndOffset)
+ else {
+ warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($components.logStartOffset). " +
+ "This could happen if segment files were deleted from the file system.")
+ removeAndDeleteSegmentsAsync(components, components.segments.values)
+ components.leaderEpochCache.foreach(_.clearAndFlush())
+ components.producerStateManager.truncateFullyAndStartAt(components.logStartOffset)
+ None
+ }
+ } else None
+ }
+
+ // If we have the clean shutdown marker, skip recovery.
+ if (!hadCleanShutdown) {
+ val unflushed = components.segments.values(components.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, maxProducerIdExpirationMs, components)
+ } 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}")
+ removeAndDeleteSegmentsAsync(components, unflushed.toList)
+ truncated = true
+ }
+ }
+ }
+
+ val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
+
+ if (components.segments.isEmpty) {
+ // no existing segments, create a new mutable segment beginning at logStartOffset
+ components.segments.add(LogSegment.open(dir = dir,
+ baseOffset = components.logStartOffset,
+ config,
+ time = time,
+ initFileSize = config.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 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.
+ (hadCleanShutdown, logEndOffsetOption) match {
+ case (true, Some(logEndOffset)) =>
+ (logEndOffset, logEndOffset)
+ case _ =>
+ val logEndOffset = logEndOffsetOption.getOrElse(components.segments.lastSegment.get.readNextOffset)
+ (Math.min(components.recoveryPoint, 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 can either schedule an asynchronous delete operation to occur in the future or perform
+ * the deletion synchronously. Asynchronous deletion allows reads to happen concurrently without
+ * synchronization and without the possibility of physically deleting a file while it is being
+ * read.
+ *
+ * This method does not need to convert IOException to KafkaStorageException because it is either
+ * called before all logs are loaded or the immediate caller will catch and handle IOException
+ *
+ * @param components The components of the log being loaded
+ * @param segmentsToDelete The log segments to schedule for deletion
+ */
+ private def removeAndDeleteSegmentsAsync(components: LogComponents,
+ segmentsToDelete: Iterable[LogSegment]): Unit = {
+ if (segmentsToDelete.nonEmpty) {
+ // As most callers hold an iterator into the `components.segments` collection and
+ // `removeAndDeleteSegment` mutates it by removing the deleted segment, we should force
+ // materialization of the iterator here, so that results of the iteration remain valid and
+ // deterministic.
+ val toDelete = segmentsToDelete.toList
+ LogRecovery.logReason(this, toDelete)
+ toDelete.foreach { segment =>
+ components.segments.remove(segment.baseOffset)
+ }
+ Log.deleteSegmentFiles(
+ segmentsToDelete,
+ asyncDelete = true,
+ deleteProducerStateSnapshots = true,
+ dir,
+ topicPartition,
+ config,
+ scheduler,
+ logDirFailureChannel,
+ components.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..293dee3bfef4e 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
@@ -344,14 +344,17 @@ class PartitionLockTest extends Logging {
private class SlowLog(log: Log, mockTime: MockTime, appendSemaphore: Semaphore) extends Log(
log.dir,
log.config,
+ log.segments,
log.logStartOffset,
log.recoveryPoint,
+ log.nextOffsetMetadata,
mockTime.scheduler,
new BrokerTopicStats,
log.time,
log.maxProducerIdExpirationMs,
log.producerIdExpirationCheckIntervalMs,
log.topicPartition,
+ log.leaderEpochCache,
log.producerStateManager,
new LogDirFailureChannel(1),
topicId = None,
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index dd671e543e29c..de96e180df695 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -1937,14 +1937,17 @@ class PartitionTest extends AbstractPartitionTest {
private class SlowLog(log: Log, mockTime: MockTime, appendSemaphore: Semaphore) extends Log(
log.dir,
log.config,
+ log.segments,
log.logStartOffset,
log.recoveryPoint,
+ log.nextOffsetMetadata,
mockTime.scheduler,
new BrokerTopicStats,
log.time,
log.maxProducerIdExpirationMs,
log.producerIdExpirationCheckIntervalMs,
log.topicPartition,
+ log.leaderEpochCache,
log.producerStateManager,
new LogDirFailureChannel(1),
topicId = None,
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
index 16aac94dc5df6..085e5279f2061 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,25 @@ class LogCleanerManagerTest extends Logging {
val logSegmentSize = TestUtils.singletonRecords("test".getBytes).sizeInBytes * 10
val logSegmentsCount = 2
val tpDir = new File(logDir, "A-1")
-
+ Files.createDirectories(tpDir.toPath)
+ val logDirFailureChannel = new LogDirFailureChannel(10)
+ val config = createLowRetentionLogConfig(logSegmentSize, LogConfig.Compact)
+ val maxProducerIdExpirationMs = 60 * 60 * 1000
+ val logLoader = new LogLoader(tpDir, tp, config, time.scheduler, time, logDirFailureChannel)
+ val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
// 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) {
-
+ class LogMock(dir: File, config: LogConfig, logComponents: LogComponents)
+ extends Log(dir, config, logComponents.segments, logComponents.logStartOffset,
+ logComponents.recoveryPoint, logComponents.nextOffsetMetadata, time.scheduler,
+ new BrokerTopicStats, time, maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs,
+ topicPartition, logComponents.leaderEpochCache, logComponents.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, logComponents)
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..b10db26f3ded3 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -100,19 +100,27 @@ 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 logLoader = new LogLoader(dir, topicPartition, config, time.scheduler, time, logDirFailureChannel)
+ val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
val log = new Log(dir,
- config = LogConfig.fromProps(logConfig.originals, logProps),
- logStartOffset = 0L,
- recoveryPoint = 0L,
+ config = config,
+ segments = logComponents.segments,
+ logStartOffset = logComponents.logStartOffset,
+ recoveryPoint = logComponents.recoveryPoint,
+ nextOffsetMetadata = logComponents.nextOffsetMetadata,
scheduler = time.scheduler,
- brokerTopicStats = new BrokerTopicStats, time,
- maxProducerIdExpirationMs = 60 * 60 * 1000,
+ brokerTopicStats = new BrokerTopicStats,
+ time,
+ maxProducerIdExpirationMs = maxProducerIdExpirationMs,
producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs,
topicPartition = topicPartition,
- producerStateManager = producerStateManager,
- logDirFailureChannel = new LogDirFailureChannel(10),
+ leaderEpochCache = logComponents.leaderEpochCache,
+ producerStateManager = logComponents.producerStateManager,
+ logDirFailureChannel = logDirFailureChannel,
topicId = None,
keepPartitionMetadataFile = true) {
override def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = {
@@ -1755,7 +1763,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..cf9c7e4203057
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
@@ -0,0 +1,1444 @@
+/**
+ * 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
+ 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 maxProducerIdExpirationMs = 60 * 60 * 1000
+ val logLoader = new LogLoader(logDir, topicPartition, config, time.scheduler,
+ time, logDirFailureChannel) {
+ override def recoverLog(components: LogComponents, maxProducerIdExpirationMs: Int, hadCleanShutdown: Boolean): (Long, Long) = {
+ if (simulateError)
+ throw new RuntimeException
+ cleanShutdownInterceptedValue = hadCleanShutdown
+ super.recoverLog(components, maxProducerIdExpirationMs, hadCleanShutdown)
+ }
+ }
+ val logComponents = logLoader.load(logStartOffset, logRecoveryPoint, maxProducerIdExpirationMs, hadCleanShutdown)
+ new Log(logDir, config, logComponents.segments, logComponents.logStartOffset, logComponents.recoveryPoint,
+ logComponents.nextOffsetMetadata, time.scheduler, brokerTopicStats, time, maxPidExpirationMs,
+ LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, logComponents.leaderEpochCache,
+ logComponents.producerStateManager, logDirFailureChannel, None, true)
+ }
+ }
+ }
+
+ 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 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 interceptedLogLoader = new LogLoader(
+ logDir,
+ topicPartition,
+ logConfig,
+ mockTime.scheduler,
+ mockTime,
+ logDirFailureChannel) {
+ override def createLogSegments(topicPartition: TopicPartition): LogSegments = {
+ interceptedLogSegments
+ }
+ }
+ val logComponents = interceptedLogLoader.load(0L, recoveryPoint, maxProducerIdExpirationMs, hadCleanShutdown = false)
+ new Log(logDir, logConfig, logComponents.segments, logComponents.logStartOffset, logComponents.recoveryPoint,
+ logComponents.nextOffsetMetadata, mockTime.scheduler, brokerTopicStats, mockTime,
+ maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition,
+ logComponents.leaderEpochCache, logComponents.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 logLoader = new LogLoader(logDir, topicPartition, config, mockTime.scheduler, mockTime, logDirFailureChannel) {
+ override def createProducerStateManager(topicPartition: TopicPartition,
+ dir: File,
+ maxProducerIdExpirationMs: Int): ProducerStateManager = stateManager
+ }
+ val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs, hadCleanShutdown = false)
+ val log = new Log(logDir,
+ config,
+ segments = logComponents.segments,
+ logStartOffset = logComponents.logStartOffset,
+ recoveryPoint = logComponents.recoveryPoint,
+ nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ scheduler = mockTime.scheduler,
+ brokerTopicStats = brokerTopicStats,
+ time = mockTime,
+ maxProducerIdExpirationMs = maxProducerIdExpirationMs,
+ producerIdExpirationCheckIntervalMs = 30000,
+ topicPartition = topicPartition,
+ leaderEpochCache = logComponents.leaderEpochCache,
+ producerStateManager = logComponents.producerStateManager,
+ 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 logLoader = new LogLoader(logDir, topicPartition, config, mockTime.scheduler, mockTime, logDirFailureChannel) {
+ override def createProducerStateManager(topicPartition: TopicPartition,
+ dir: File,
+ maxProducerIdExpirationMs: Int): ProducerStateManager = stateManager
+ }
+ val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs, hadCleanShutdown = false)
+ new Log(logDir,
+ config,
+ segments = logComponents.segments,
+ logStartOffset = logComponents.logStartOffset,
+ recoveryPoint = logComponents.recoveryPoint,
+ nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ scheduler = mockTime.scheduler,
+ brokerTopicStats = brokerTopicStats,
+ time = mockTime,
+ maxProducerIdExpirationMs = maxProducerIdExpirationMs,
+ producerIdExpirationCheckIntervalMs = 30000,
+ topicPartition = Log.parseTopicPartitionName(logDir),
+ leaderEpochCache = logComponents.leaderEpochCache,
+ producerStateManager = logComponents.producerStateManager,
+ 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 logLoader = new LogLoader(logDir, topicPartition, config, mockTime.scheduler, mockTime, logDirFailureChannel) {
+ override def createProducerStateManager(topicPartition: TopicPartition,
+ dir: File,
+ maxProducerIdExpirationMs: Int): ProducerStateManager = stateManager
+ }
+ val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
+ new Log(logDir,
+ config,
+ segments = logComponents.segments,
+ logStartOffset = logComponents.logStartOffset,
+ recoveryPoint = logComponents.recoveryPoint,
+ nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ scheduler = mockTime.scheduler,
+ brokerTopicStats = brokerTopicStats,
+ time = mockTime,
+ maxProducerIdExpirationMs = maxProducerIdExpirationMs,
+ producerIdExpirationCheckIntervalMs = 30000,
+ topicPartition = Log.parseTopicPartitionName(logDir),
+ leaderEpochCache = logComponents.leaderEpochCache,
+ producerStateManager = logComponents.producerStateManager,
+ 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 logLoader = new LogLoader(logDir, topicPartition, config, mockTime.scheduler, mockTime, logDirFailureChannel) {
+ override def createProducerStateManager(topicPartition: TopicPartition,
+ dir: File,
+ maxProducerIdExpirationMs: Int): ProducerStateManager = stateManager
+ }
+ val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
+ new Log(logDir,
+ config,
+ segments = logComponents.segments,
+ logStartOffset = logComponents.logStartOffset,
+ recoveryPoint = logComponents.recoveryPoint,
+ nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ scheduler = mockTime.scheduler,
+ brokerTopicStats = brokerTopicStats,
+ time = mockTime,
+ maxProducerIdExpirationMs = maxProducerIdExpirationMs,
+ producerIdExpirationCheckIntervalMs = 30000,
+ topicPartition = Log.parseTopicPartitionName(logDir),
+ leaderEpochCache = logComponents.leaderEpochCache,
+ producerStateManager = logComponents.producerStateManager,
+ 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..410d1fbf807fc 100644
--- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
@@ -18,9 +18,19 @@
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, Record, RecordBatch, SimpleRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse}
+
+import scala.collection.Iterable
+import scala.collection.mutable.ListBuffer
+import scala.jdk.CollectionConverters._
object LogTestUtils {
/**
@@ -37,4 +47,246 @@ 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)
+ }
+
+ /**
+ * 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)
+ }
+
+ def allRecords(log: Log): List[Record] = {
+ val recordsFound = ListBuffer[Record]()
+ for (logSegment <- log.logSegments) {
+ for (batch <- logSegment.log.batches.asScala) {
+ recordsFound ++= batch.iterator().asScala
+ }
+ }
+ recordsFound.toList
+ }
+
+ def verifyRecordsInLog(log: Log, expectedRecords: List[Record]): Unit = {
+ assertEquals(expectedRecords, allRecords(log))
+ }
+
+ /* 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..732926c6e88e8 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,31 @@ 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 logLoader = new LogLoader(logDir, tp, logConfig, mockScheduler, time, mockLogDirFailureChannel)
+ val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
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 = logComponents.segments,
+ logStartOffset = logComponents.logStartOffset,
+ recoveryPoint = logComponents.recoveryPoint,
+ nextOffsetMetadata = logComponents.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 = logComponents.leaderEpochCache,
+ producerStateManager = logComponents.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..12ee6951c075c 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.{Log, LogConfig, LogLoader, LogManager}
import kafka.server.{BrokerTopicStats, LogDirFailureChannel}
import kafka.utils.TestUtils.retry
import org.junit.jupiter.api.Assertions._
@@ -118,13 +119,16 @@ 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 producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs)
- val log = new Log(logDir, logConfig, logStartOffset = 0, recoveryPoint = recoveryPoint, scheduler,
+ val logDirFailureChannel = new LogDirFailureChannel(10)
+ val logLoader = new LogLoader(logDir, topicPartition, logConfig, scheduler, mockTime, logDirFailureChannel)
+ val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
+ val log = new Log(logDir, logConfig, segments = logComponents.segments, logStartOffset = logComponents.logStartOffset,
+ recoveryPoint = logComponents.recoveryPoint, nextOffsetMetadata = logComponents.nextOffsetMetadata, scheduler,
brokerTopicStats, mockTime, maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs,
- topicPartition, producerStateManager, new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true)
+ topicPartition, logComponents.leaderEpochCache, logComponents.producerStateManager, logDirFailureChannel,
+ topicId = None, keepPartitionMetadataFile = true)
assertTrue(scheduler.taskRunning(log.producerExpireCheck))
log.close()
assertFalse(scheduler.taskRunning(log.producerExpireCheck))
From 500365cb757f50c3c03ae6bca479af61790aee8d Mon Sep 17 00:00:00 2001
From: Kowshik Prakasam
Date: Thu, 8 Apr 2021 10:23:54 -0700
Subject: [PATCH 2/7] Fix doc
---
core/src/main/scala/kafka/log/LogLoader.scala | 3 +--
1 file changed, 1 insertion(+), 2 deletions(-)
diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala
index df48e435f0108..e51a53e0082ed 100644
--- a/core/src/main/scala/kafka/log/LogLoader.scala
+++ b/core/src/main/scala/kafka/log/LogLoader.scala
@@ -492,8 +492,7 @@ class LogLoader(dir: File,
* 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 can either schedule an asynchronous delete operation to occur in the future or perform
- * the deletion synchronously. Asynchronous deletion allows reads to happen concurrently without
+ * - 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.
*
From 14b979994a862767559c9be238182ec6892c58e4 Mon Sep 17 00:00:00 2001
From: Kowshik Prakasam
Date: Mon, 12 Apr 2021 02:01:00 -0700
Subject: [PATCH 3/7] Address comments
---
core/src/main/scala/kafka/log/Log.scala | 44 ++++++++-----------
core/src/main/scala/kafka/log/LogLoader.scala | 2 +-
.../scala/unit/kafka/log/LogTestUtils.scala | 20 ++-------
3 files changed, 22 insertions(+), 44 deletions(-)
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index 871590be61982..ae5f3c39312ee 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -275,7 +275,7 @@ class Log(@volatile private var _dir: File,
val producerStateManager: ProducerStateManager,
logDirFailureChannel: LogDirFailureChannel,
@volatile var topicId: Option[Uuid],
- val keepPartitionMetadataFile: Boolean = true) extends Logging with KafkaMetricsGroup {
+ val keepPartitionMetadataFile: Boolean) extends Logging with KafkaMetricsGroup {
import kafka.log.Log._
@@ -1876,7 +1876,7 @@ class Log(@volatile private var _dir: File,
// removing the deleted segment, we should force materialization of the iterator here, so that results of the
// iteration remain valid and deterministic.
val toDelete = segments.toList
- reason.logReason(this, toDelete, this)
+ reason.logReason(this, toDelete)
toDelete.foreach { segment =>
this.segments.remove(segment.baseOffset)
}
@@ -1999,7 +1999,7 @@ object Log extends Logging {
logDirFailureChannel: LogDirFailureChannel,
lastShutdownClean: Boolean = true,
topicId: Option[Uuid],
- keepPartitionMetadataFile: Boolean = true): Log = {
+ keepPartitionMetadataFile: Boolean): Log = {
// create the log directory if it doesn't exist
Files.createDirectories(dir.toPath)
val topicPartition = Log.parseTopicPartitionName(dir)
@@ -2590,22 +2590,20 @@ object LogMetricNames {
}
}
-abstract class SegmentDeletionReason {
- def logReason(logger: Logging, toDelete: List[LogSegment]): Unit = {}
-
- def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {}
+sealed trait SegmentDeletionReason {
+ def logReason(log: Log, toDelete: List[LogSegment]): Unit
}
case object RetentionMsBreach extends SegmentDeletionReason {
- override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
+ override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
val retentionMs = log.config.retentionMs
toDelete.foreach { segment =>
segment.largestRecordTimestamp match {
case Some(_) =>
- logger.info(s"Deleting segment $segment due to retention time ${retentionMs}ms breach based on the largest " +
+ log.info(s"Deleting segment $segment due to retention time ${retentionMs}ms breach based on the largest " +
s"record timestamp in the segment")
case None =>
- logger.info(s"Deleting segment $segment due to retention time ${retentionMs}ms breach based on the " +
+ log.info(s"Deleting segment $segment due to retention time ${retentionMs}ms breach based on the " +
s"last modified time of the segment")
}
}
@@ -2613,42 +2611,36 @@ case object RetentionMsBreach extends SegmentDeletionReason {
}
case object RetentionSizeBreach extends SegmentDeletionReason {
- override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
+ override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
var size = log.size
toDelete.foreach { segment =>
size -= segment.size
- logger.info(s"Deleting segment $segment due to retention size ${log.config.retentionSize} breach. Log size " +
+ log.info(s"Deleting segment $segment due to retention size ${log.config.retentionSize} breach. Log size " +
s"after deletion will be $size.")
}
}
}
case object StartOffsetBreach extends SegmentDeletionReason {
- override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
- logger.info(s"Deleting segments due to log start offset ${log.logStartOffset} breach: ${toDelete.mkString(",")}")
- }
-}
-
-case object LogRecovery extends SegmentDeletionReason {
- override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
- logger.info(s"Deleting segments as part of log recovery: ${toDelete.mkString(",")}")
+ override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
+ log.info(s"Deleting segments due to log start offset ${log.logStartOffset} breach: ${toDelete.mkString(",")}")
}
}
case object LogTruncation extends SegmentDeletionReason {
- override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
- logger.info(s"Deleting segments as part of log truncation: ${toDelete.mkString(",")}")
+ override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
+ log.info(s"Deleting segments as part of log truncation: ${toDelete.mkString(",")}")
}
}
case object LogRoll extends SegmentDeletionReason {
- override def logReason(logger: Logging, toDelete: List[LogSegment], log: Log): Unit = {
- logger.info(s"Deleting segments as part of log roll: ${toDelete.mkString(",")}")
+ override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
+ log.info(s"Deleting segments as part of log roll: ${toDelete.mkString(",")}")
}
}
case object LogDeletion extends SegmentDeletionReason {
- override def logReason(logger: Logging, toDelete: List[LogSegment]): Unit = {
- logger.info(s"Deleting segments as the log has been deleted: ${toDelete.mkString(",")}")
+ override def logReason(log: Log, toDelete: List[LogSegment]): Unit = {
+ log.info(s"Deleting segments as the log has been deleted: ${toDelete.mkString(",")}")
}
}
diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala
index e51a53e0082ed..d04f9d0524706 100644
--- a/core/src/main/scala/kafka/log/LogLoader.scala
+++ b/core/src/main/scala/kafka/log/LogLoader.scala
@@ -510,7 +510,7 @@ class LogLoader(dir: File,
// materialization of the iterator here, so that results of the iteration remain valid and
// deterministic.
val toDelete = segmentsToDelete.toList
- LogRecovery.logReason(this, toDelete)
+ info(s"Deleting segments as part of log recovery: ${toDelete.mkString(",")}")
toDelete.foreach { segment =>
components.segments.remove(segment.baseOffset)
}
diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
index 410d1fbf807fc..29777b2dec900 100644
--- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
@@ -24,12 +24,11 @@ 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, Record, RecordBatch, SimpleRecord}
+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.collection.mutable.ListBuffer
import scala.jdk.CollectionConverters._
object LogTestUtils {
@@ -97,7 +96,8 @@ object LogTestUtils {
producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs,
logDirFailureChannel = new LogDirFailureChannel(10),
lastShutdownClean = lastShutdownClean,
- topicId = topicId)
+ topicId = topicId,
+ keepPartitionMetadataFile = true)
}
/**
@@ -167,20 +167,6 @@ object LogTestUtils {
writeNormalSegment(nextOffset)
}
- def allRecords(log: Log): List[Record] = {
- val recordsFound = ListBuffer[Record]()
- for (logSegment <- log.logSegments) {
- for (batch <- logSegment.log.batches.asScala) {
- recordsFound ++= batch.iterator().asScala
- }
- }
- recordsFound.toList
- }
-
- def verifyRecordsInLog(log: Log, expectedRecords: List[Record]): Unit = {
- assertEquals(expectedRecords, allRecords(log))
- }
-
/* extract all the keys from a log */
def keysInLog(log: Log): Iterable[Long] = {
for (logSegment <- log.logSegments;
From 41d6eda2d89d81975e9a260075974a0683862443 Mon Sep 17 00:00:00 2001
From: Kowshik Prakasam
Date: Tue, 13 Apr 2021 02:13:36 -0700
Subject: [PATCH 4/7] Address comments
---
core/src/main/scala/kafka/log/Log.scala | 23 +-
core/src/main/scala/kafka/log/LogLoader.scala | 359 +++++++++---------
.../kafka/log/LogCleanerManagerTest.scala | 34 +-
.../scala/unit/kafka/log/LogCleanerTest.scala | 32 +-
.../scala/unit/kafka/log/LogLoaderTest.scala | 267 ++++++++-----
.../kafka/server/ReplicaManagerTest.scala | 31 +-
.../unit/kafka/utils/SchedulerTest.scala | 27 +-
7 files changed, 458 insertions(+), 315 deletions(-)
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index ae5f3c39312ee..ca508920079b6 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -2003,11 +2003,26 @@ object Log extends Logging {
// create the log directory if it doesn't exist
Files.createDirectories(dir.toPath)
val topicPartition = Log.parseTopicPartitionName(dir)
- val logLoader = new LogLoader(dir, topicPartition, config, scheduler, time, logDirFailureChannel)
- val logComponents = logLoader.load(logStartOffset, recoveryPoint, maxProducerIdExpirationMs, lastShutdownClean)
- new Log(dir, config, logComponents.segments, logComponents.logStartOffset, logComponents.recoveryPoint, logComponents.nextOffsetMetadata, scheduler,
+ val segments = 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,
+ 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,
- logComponents.leaderEpochCache, logComponents.producerStateManager, logDirFailureChannel, topicId, keepPartitionMetadataFile)
+ leaderEpochCache, producerStateManager, logDirFailureChannel, topicId, keepPartitionMetadataFile)
}
/**
diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala
index d04f9d0524706..0ae8bbdf2d665 100644
--- a/core/src/main/scala/kafka/log/LogLoader.scala
+++ b/core/src/main/scala/kafka/log/LogLoader.scala
@@ -27,41 +27,55 @@ 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.record.RecordVersion
import org.apache.kafka.common.utils.Time
import scala.collection.{Seq, Set, mutable}
-case class LogComponents(segments: LogSegments,
- logStartOffset: Long,
- recoveryPoint: Long,
- nextOffsetMetadata: LogOffsetMetadata,
- leaderEpochCache: Option[LeaderEpochFileCache],
- producerStateManager: ProducerStateManager)
+case class LoadedLogOffsets(logStartOffset: Long,
+ recoveryPoint: Long,
+ nextOffsetMetadata: LogOffsetMetadata)
/**
- * This class is responsible for all activities related with recovery of log segments from disk.
- * Additionally, it also suitably updates the LeaderEpochFileCache and ProducerStateManager to
- * reflect the contents of the recovered segments.
- *
- * @param dir The directory in which log segments are created.
- * @param topicPartition The topic partition associated with the Log
- * @param config The log configuration settings
+ * @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
*/
-class LogLoader(dir: File,
- topicPartition: TopicPartition,
- config: LogConfig,
- scheduler: Scheduler,
- time: Time,
- logDirFailureChannel: LogDirFailureChannel) extends Logging {
+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.logIdent = 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 {
- protected def createLogSegments(topicPartition: TopicPartition): LogSegments = new LogSegments(topicPartition)
+ /*protected def createLogSegments(topicPartition: TopicPartition): LogSegments = new LogSegments(topicPartition)
protected def createProducerStateManager(topicPartition: TopicPartition,
dir: File,
@@ -72,110 +86,94 @@ class LogLoader(dir: File,
topicPartition: TopicPartition,
logDirFailureChannel: LogDirFailureChannel,
recordVersion: RecordVersion): Option[LeaderEpochFileCache] =
- Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion)
+ Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion)*/
/**
* Load the log segments from the log files on disk, and return the components of the loaded log.
+ * Additionally, it also suitably updates the provided LeaderEpochFileCache and ProducerStateManager
+ * to reflect the contents of the loaded log.
*
* This method does not need to convert IOException to KafkaStorageException because it is only
* called before all logs are loaded.
*
- * @param 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 hadCleanShutdown Boolean flag to indicate whether the associated Log previously had a
- * clean shutdown
+ * @param params The parameters for the log being loaded from disk
*
- * @return the components of the Log successfully loaded from disk
+ * @return the offsets of the Log successfully loaded from disk
*
* @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that
* overflow index offset; or when we find an unexpected
* number of .log files with overflow
*/
- def load(logStartOffsetCheckpoint: Long,
- recoveryPointCheckpoint: Long,
- maxProducerIdExpirationMs: Int,
- hadCleanShutdown: Boolean = true): LogComponents = {
- val leaderEpochFileCache = maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion)
- val producerStateManager = createProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
- val components = LogComponents(
- createLogSegments(topicPartition),
- logStartOffsetCheckpoint,
- recoveryPointCheckpoint,
- LogOffsetMetadata(0, 0, 0),
- leaderEpochFileCache,
- producerStateManager)
+ 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()
+ 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(components, {
+ 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().
- components.segments.close()
- components.segments.clear()
- loadSegmentFiles(components, maxProducerIdExpirationMs)
+ params.segments.close()
+ params.segments.clear()
+ loadSegmentFiles(params)
})
- completeSwapOperations(swapFiles, maxProducerIdExpirationMs, components)
+ completeSwapOperations(swapFiles, params)
val (newRecoveryPoint: Long, nextOffset: Long) = {
- if (!dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) {
- val (newRecoveryPoint, nextOffset) = retryOnOffsetOverflow(components, {
- recoverLog(components, maxProducerIdExpirationMs, hadCleanShutdown)
+ 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
- components.segments.lastSegment.get.resizeIndexes(config.maxIndexSize)
+ params.segments.lastSegment.get.resizeIndexes(params.config.maxIndexSize)
(newRecoveryPoint, nextOffset)
} else {
- if (components.segments.isEmpty) {
- components.segments.add(LogSegment.open(dir = dir,
- baseOffset = 0,
- config,
- time = time,
- initFileSize = config.initFileSize))
+ if (params.segments.isEmpty) {
+ params.segments.add(
+ LogSegment.open(
+ dir = params.dir,
+ baseOffset = 0,
+ params.config,
+ time = params.time,
+ initFileSize = params.config.initFileSize))
}
- (components.recoveryPoint, 0L)
+ (params.recoveryPointCheckpoint, 0L)
}
}
- components.leaderEpochCache.foreach(_.truncateFromEnd(nextOffset))
- val newLogStartOffset = math.max(components.logStartOffset, components.segments.firstSegment.get.baseOffset)
+ 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.
- components.leaderEpochCache.foreach(_.truncateFromStart(components.logStartOffset))
+ 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 (!components.producerStateManager.isEmpty)
+ 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.
- components.producerStateManager.removeStraySnapshots(components.segments.baseOffsets.toSeq)
+ params.producerStateManager.removeStraySnapshots(params.segments.baseOffsets.toSeq)
Log.rebuildProducerState(
- components.producerStateManager,
- components.segments,
+ params.producerStateManager,
+ params.segments,
newLogStartOffset,
nextOffset,
- config.messageFormatVersion.recordVersion,
- time,
- reloadFromCleanShutdown = hadCleanShutdown)
+ params.config.messageFormatVersion.recordVersion,
+ params.time,
+ reloadFromCleanShutdown = params.hadCleanShutdown)
- val activeSegment = components.segments.lastSegment.get
- LogComponents(
- components.segments,
+ val activeSegment = params.segments.lastSegment.get
+ LoadedLogOffsets(
newLogStartOffset,
newRecoveryPoint,
- LogOffsetMetadata(nextOffset, activeSegment.baseOffset, activeSegment.size),
- components.leaderEpochCache,
- components.producerStateManager)
+ LogOffsetMetadata(nextOffset, activeSegment.baseOffset, activeSegment.size))
}
/**
@@ -183,28 +181,29 @@ class LogLoader(dir: File,
* 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(): Set[File] = {
+ private def removeTempFilesAndCollectSwapFiles(params: LoadLogParams): Set[File] = {
def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
- info(s"Deleting index files with suffix $suffix for baseFile $baseFile")
+ info(s"${params.logIdentifier} 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)
+ 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 <- dir.listFiles if file.isFile) {
+ 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"Deleting stray temporary file ${file.getAbsolutePath}")
+ 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)
@@ -214,7 +213,7 @@ class LogLoader(dir: File,
// 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.")
+ info(s"${params.logIdentifier} Found file ${file.getAbsolutePath} from interrupted swap operation.")
if (Log.isIndexFile(baseFile)) {
deleteIndicesIfExist(baseFile)
} else if (Log.isLogFile(baseFile)) {
@@ -229,7 +228,7 @@ class LogLoader(dir: File,
// 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")
+ 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)
@@ -237,7 +236,7 @@ class LogLoader(dir: File,
// 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}")
+ debug(s"${params.logIdentifier} Deleting stray .clean file ${file.getAbsolutePath}")
Files.deleteIfExists(file.toPath)
}
@@ -249,101 +248,105 @@ class LogLoader(dir: File,
* 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 components The components of the log being loaded
+ * @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](components: LogComponents, fn: => T): T = {
+ private def retryOnOffsetOverflow[T](params: LoadLogParams, fn: => T): T = {
while (true) {
try {
return fn
} catch {
case e: LogSegmentOffsetOverflowException =>
- info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.")
- Log.splitOverflowedSegment(e.segment, components.segments, dir, topicPartition, config, scheduler, logDirFailureChannel, components.producerStateManager)
+ 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 LogComponents.segments.
+ * 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 components The components of the log being loaded
- * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is
- * considered expired
+ * @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(components: LogComponents, maxProducerIdExpirationMs: Int): Unit = {
+ 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 <- dir.listFiles.sortBy(_.getName) if file.isFile) {
+ 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(dir, offset)
+ val logFile = Log.logFile(params.dir, offset)
if (!logFile.exists) {
- warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.")
+ 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(dir, baseOffset).exists()
- val segment = LogSegment.open(dir = dir,
+ val timeIndexFileNewlyCreated = !Log.timeIndexFile(params.dir, baseOffset).exists()
+ val segment = LogSegment.open(
+ dir = params.dir,
baseOffset = baseOffset,
- config,
- time = time,
+ params.config,
+ time = params.time,
fileAlreadyExists = true)
try segment.sanityCheck(timeIndexFileNewlyCreated)
catch {
case _: NoSuchFileException =>
- error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " +
- "recovering segment and rebuilding index files...")
- recoverSegment(segment, maxProducerIdExpirationMs, components)
+ 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"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, maxProducerIdExpirationMs, components)
+ 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)
}
- components.segments.add(segment)
+ params.segments.add(segment)
}
}
}
/**
- * Just recovers the given segment, without adding it to the provided LogComponents.segments.
+ * Just recovers the given segment, without adding it to the provided params.segments.
*
* @param segment Segment to recover
- * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is
- * considered expired
- * @param components The components of the log being loaded
+ * @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,
- maxProducerIdExpirationMs: Int,
- components: LogComponents): Int = {
- val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
+ private def recoverSegment(segment: LogSegment, params: LoadLogParams): Int = {
+ val producerStateManager = new ProducerStateManager(params.topicPartition, params.dir, params.maxProducerIdExpirationMs)
Log.rebuildProducerState(
producerStateManager,
- components.segments,
- components.logStartOffset,
+ params.segments,
+ params.logStartOffsetCheckpoint,
segment.baseOffset,
- config.messageFormatVersion.recordVersion,
- time,
+ params.config.messageFormatVersion.recordVersion,
+ params.time,
reloadFromCleanShutdown = false)
- val bytesTruncated = segment.recover(producerStateManager, components.leaderEpochCache)
+ 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()
@@ -358,7 +361,8 @@ class LogLoader(dir: 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 files
+ * @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
@@ -368,18 +372,17 @@ class LogLoader(dir: File,
* and manual intervention might be required to get out of it.
*/
private def completeSwapOperations(swapFiles: Set[File],
- maxProducerIdExpirationMs: Int,
- components: LogComponents): Unit = {
+ 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,
- config,
- time = time,
+ params.config,
+ time = params.time,
fileSuffix = Log.SwapFileSuffix)
info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.")
- recoverSegment(swapSegment, maxProducerIdExpirationMs, components)
+ recoverSegment(swapSegment, params)
// We create swap files for two cases:
// (1) Log cleaning where multiple segments are merged into one, and
@@ -389,11 +392,20 @@ class LogLoader(dir: File,
// 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 = components.segments.values(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
+ val oldSegments = params.segments.values(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment =>
segment.readNextOffset > swapSegment.baseOffset
}
- Log.replaceSegments(components.segments, Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true,
- dir, topicPartition, config, scheduler, logDirFailureChannel, components.producerStateManager)
+ Log.replaceSegments(
+ params.segments,
+ Seq(swapSegment),
+ oldSegments.toSeq,
+ isRecoveredSwapFile = true,
+ params.dir,
+ params.topicPartition,
+ params.config,
+ params.scheduler,
+ params.logDirFailureChannel,
+ params.producerStateManager)
}
}
@@ -406,56 +418,53 @@ class LogLoader(dir: File,
* This method does not need to convert IOException to KafkaStorageException because it is only
* called before all logs are loaded.
*
- * @param components The components of the log being loaded
- * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is
- * considered expired
- * @param hadCleanShutdown Boolean flag to indicate whether the associated Log previously had a
- * clean shutdown
+ * @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(components: LogComponents, maxProducerIdExpirationMs: Int, hadCleanShutdown: Boolean): (Long, Long) = {
+ private[log] def recoverLog(params: LoadLogParams): (Long, Long) = {
/** return the log end offset if valid */
def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = {
- if (components.segments.nonEmpty) {
- val logEndOffset = components.segments.lastSegment.get.readNextOffset
- if (logEndOffset >= components.logStartOffset)
+ 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 ($components.logStartOffset). " +
+ 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(components, components.segments.values)
- components.leaderEpochCache.foreach(_.clearAndFlush())
- components.producerStateManager.truncateFullyAndStartAt(components.logStartOffset)
+ 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 (!hadCleanShutdown) {
- val unflushed = components.segments.values(components.recoveryPoint, Long.MaxValue).iterator
+ 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"Recovering unflushed segment ${segment.baseOffset}")
+ info(s"${params.logIdentifier} Recovering unflushed segment ${segment.baseOffset}")
val truncatedBytes =
try {
- recoverSegment(segment, maxProducerIdExpirationMs, components)
+ recoverSegment(segment, params)
} 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")
+ 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"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}")
- removeAndDeleteSegmentsAsync(components, unflushed.toList)
+ warn(s"${params.logIdentifier} Corruption found in segment ${segment.baseOffset}," +
+ s" truncating to offset ${segment.readNextOffset}")
+ removeAndDeleteSegmentsAsync(unflushed.toList, params)
truncated = true
}
}
@@ -463,14 +472,16 @@ class LogLoader(dir: File,
val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd()
- if (components.segments.isEmpty) {
+ if (params.segments.isEmpty) {
// no existing segments, create a new mutable segment beginning at logStartOffset
- components.segments.add(LogSegment.open(dir = dir,
- baseOffset = components.logStartOffset,
- config,
- time = time,
- initFileSize = config.initFileSize,
- preallocate = config.preallocate))
+ 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
@@ -479,12 +490,12 @@ class LogLoader(dir: File,
// 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.
- (hadCleanShutdown, logEndOffsetOption) match {
+ (params.hadCleanShutdown, logEndOffsetOption) match {
case (true, Some(logEndOffset)) =>
(logEndOffset, logEndOffset)
case _ =>
- val logEndOffset = logEndOffsetOption.getOrElse(components.segments.lastSegment.get.readNextOffset)
- (Math.min(components.recoveryPoint, logEndOffset), logEndOffset)
+ val logEndOffset = logEndOffsetOption.getOrElse(params.segments.lastSegment.get.readNextOffset)
+ (Math.min(params.recoveryPointCheckpoint, logEndOffset), logEndOffset)
}
}
@@ -499,31 +510,31 @@ class LogLoader(dir: File,
* 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 components The components of the log being loaded
* @param segmentsToDelete The log segments to schedule for deletion
+ * @param params The parameters for the log being loaded from disk
*/
- private def removeAndDeleteSegmentsAsync(components: LogComponents,
- segmentsToDelete: Iterable[LogSegment]): Unit = {
+ private def removeAndDeleteSegmentsAsync(segmentsToDelete: Iterable[LogSegment],
+ params: LoadLogParams): Unit = {
if (segmentsToDelete.nonEmpty) {
- // As most callers hold an iterator into the `components.segments` collection and
- // `removeAndDeleteSegment` mutates it by removing the deleted segment, we should force
+ // 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 =>
- components.segments.remove(segment.baseOffset)
+ params.segments.remove(segment.baseOffset)
}
Log.deleteSegmentFiles(
segmentsToDelete,
asyncDelete = true,
deleteProducerStateSnapshots = true,
- dir,
- topicPartition,
- config,
- scheduler,
- logDirFailureChannel,
- components.producerStateManager)
+ params.dir,
+ params.topicPartition,
+ params.config,
+ params.scheduler,
+ params.logDirFailureChannel,
+ params.producerStateManager)
}
}
}
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
index 085e5279f2061..cdaa89de9e34e 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
@@ -99,21 +99,35 @@ class LogCleanerManagerTest extends Logging {
val logDirFailureChannel = new LogDirFailureChannel(10)
val config = createLowRetentionLogConfig(logSegmentSize, LogConfig.Compact)
val maxProducerIdExpirationMs = 60 * 60 * 1000
- val logLoader = new LogLoader(tpDir, tp, config, time.scheduler, time, logDirFailureChannel)
- val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
- // the exception should be catched and the partition that caused it marked as uncleanable
- class LogMock(dir: File, config: LogConfig, logComponents: LogComponents)
- extends Log(dir, config, logComponents.segments, logComponents.logStartOffset,
- logComponents.recoveryPoint, logComponents.nextOffsetMetadata, time.scheduler,
- new BrokerTopicStats, time, maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs,
- topicPartition, logComponents.leaderEpochCache, logComponents.producerStateManager,
- logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) {
+ 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, config, logComponents)
+ 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 b10db26f3ded3..de55724d6ba71 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -104,22 +104,38 @@ class LogCleanerTest {
val topicPartition = Log.parseTopicPartitionName(dir)
val logDirFailureChannel = new LogDirFailureChannel(10)
val maxProducerIdExpirationMs = 60 * 60 * 1000
- val logLoader = new LogLoader(dir, topicPartition, config, time.scheduler, time, logDirFailureChannel)
- val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
+ 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 = config,
- segments = logComponents.segments,
- logStartOffset = logComponents.logStartOffset,
- recoveryPoint = logComponents.recoveryPoint,
- nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ segments = logSegments,
+ logStartOffset = offsets.logStartOffset,
+ recoveryPoint = offsets.recoveryPoint,
+ nextOffsetMetadata = offsets.nextOffsetMetadata,
scheduler = time.scheduler,
brokerTopicStats = new BrokerTopicStats,
time,
maxProducerIdExpirationMs = maxProducerIdExpirationMs,
producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs,
topicPartition = topicPartition,
- leaderEpochCache = logComponents.leaderEpochCache,
- producerStateManager = logComponents.producerStateManager,
+ leaderEpochCache = leaderEpochCache,
+ producerStateManager = producerStateManager,
logDirFailureChannel = logDirFailureChannel,
topicId = None,
keepPartitionMetadataFile = true) {
diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
index cf9c7e4203057..0d6e7e3dfc9d4 100644
--- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
@@ -69,11 +69,12 @@ class LogLoaderTest {
var log: Log = null
val time = new MockTime()
var cleanShutdownInterceptedValue = false
- var simulateError = 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]): LogManager = {
+ 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,
@@ -82,55 +83,66 @@ class LogLoaderTest {
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 logLoader = new LogLoader(logDir, topicPartition, config, time.scheduler,
- time, logDirFailureChannel) {
- override def recoverLog(components: LogComponents, maxProducerIdExpirationMs: Int, hadCleanShutdown: Boolean): (Long, Long) = {
- if (simulateError)
- throw new RuntimeException
- cleanShutdownInterceptedValue = hadCleanShutdown
- super.recoverLog(components, maxProducerIdExpirationMs, hadCleanShutdown)
- }
- }
- val logComponents = logLoader.load(logStartOffset, logRecoveryPoint, maxProducerIdExpirationMs, hadCleanShutdown)
- new Log(logDir, config, logComponents.segments, logComponents.logStartOffset, logComponents.recoveryPoint,
- logComponents.nextOffsetMetadata, time.scheduler, brokerTopicStats, time, maxPidExpirationMs,
- LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, logComponents.leaderEpochCache,
- logComponents.producerStateManager, logDirFailureChannel, None, true)
+ 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)
- 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")
+ 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
@@ -252,23 +264,28 @@ class LogLoaderTest {
super.add(wrapper)
}
}
- val interceptedLogLoader = new LogLoader(
+ 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) {
- override def createLogSegments(topicPartition: TopicPartition): LogSegments = {
- interceptedLogSegments
- }
- }
- val logComponents = interceptedLogLoader.load(0L, recoveryPoint, maxProducerIdExpirationMs, hadCleanShutdown = false)
- new Log(logDir, logConfig, logComponents.segments, logComponents.logStartOffset, logComponents.recoveryPoint,
- logComponents.nextOffsetMetadata, mockTime.scheduler, brokerTopicStats, 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,
- logComponents.leaderEpochCache, logComponents.producerStateManager, logDirFailureChannel,
- topicId = None, keepPartitionMetadataFile = true)
+ leaderEpochCache, producerStateManager, logDirFailureChannel, topicId = None,
+ keepPartitionMetadataFile = true)
}
// Retain snapshots for the last 2 segments
@@ -320,26 +337,36 @@ class LogLoaderTest {
val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1)
val config = LogConfig(new Properties())
val maxProducerIdExpirationMs = 300000
- val logLoader = new LogLoader(logDir, topicPartition, config, mockTime.scheduler, mockTime, logDirFailureChannel) {
- override def createProducerStateManager(topicPartition: TopicPartition,
- dir: File,
- maxProducerIdExpirationMs: Int): ProducerStateManager = stateManager
- }
- val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs, hadCleanShutdown = false)
+ 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 = logComponents.segments,
- logStartOffset = logComponents.logStartOffset,
- recoveryPoint = logComponents.recoveryPoint,
- nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ 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 = logComponents.leaderEpochCache,
- producerStateManager = logComponents.producerStateManager,
+ leaderEpochCache = leaderEpochCache,
+ producerStateManager = stateManager,
logDirFailureChannel = logDirFailureChannel,
topicId = None,
keepPartitionMetadataFile = true)
@@ -444,27 +471,37 @@ class LogLoaderTest {
val config = LogConfig(logProps)
val maxProducerIdExpirationMs = 300000
val logDirFailureChannel = null
- val logLoader = new LogLoader(logDir, topicPartition, config, mockTime.scheduler, mockTime, logDirFailureChannel) {
- override def createProducerStateManager(topicPartition: TopicPartition,
- dir: File,
- maxProducerIdExpirationMs: Int): ProducerStateManager = stateManager
- }
- val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs, hadCleanShutdown = false)
+ 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 = logComponents.segments,
- logStartOffset = logComponents.logStartOffset,
- recoveryPoint = logComponents.recoveryPoint,
- nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ segments = segments,
+ logStartOffset = offsets.logStartOffset,
+ recoveryPoint = offsets.recoveryPoint,
+ nextOffsetMetadata = offsets.nextOffsetMetadata,
scheduler = mockTime.scheduler,
brokerTopicStats = brokerTopicStats,
time = mockTime,
maxProducerIdExpirationMs = maxProducerIdExpirationMs,
producerIdExpirationCheckIntervalMs = 30000,
- topicPartition = Log.parseTopicPartitionName(logDir),
- leaderEpochCache = logComponents.leaderEpochCache,
- producerStateManager = logComponents.producerStateManager,
- logDirFailureChannel,
+ topicPartition = topicPartition,
+ leaderEpochCache = leaderEpochCache,
+ producerStateManager = stateManager,
+ logDirFailureChannel = logDirFailureChannel,
topicId = None,
keepPartitionMetadataFile = true)
@@ -496,27 +533,37 @@ class LogLoaderTest {
val config = LogConfig(logProps)
val maxProducerIdExpirationMs = 300000
val logDirFailureChannel = null
- val logLoader = new LogLoader(logDir, topicPartition, config, mockTime.scheduler, mockTime, logDirFailureChannel) {
- override def createProducerStateManager(topicPartition: TopicPartition,
- dir: File,
- maxProducerIdExpirationMs: Int): ProducerStateManager = stateManager
- }
- val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
+ 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 = logComponents.segments,
- logStartOffset = logComponents.logStartOffset,
- recoveryPoint = logComponents.recoveryPoint,
- nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ segments = segments,
+ logStartOffset = offsets.logStartOffset,
+ recoveryPoint = offsets.recoveryPoint,
+ nextOffsetMetadata = offsets.nextOffsetMetadata,
scheduler = mockTime.scheduler,
brokerTopicStats = brokerTopicStats,
time = mockTime,
maxProducerIdExpirationMs = maxProducerIdExpirationMs,
producerIdExpirationCheckIntervalMs = 30000,
- topicPartition = Log.parseTopicPartitionName(logDir),
- leaderEpochCache = logComponents.leaderEpochCache,
- producerStateManager = logComponents.producerStateManager,
- logDirFailureChannel,
+ topicPartition = topicPartition,
+ leaderEpochCache = leaderEpochCache,
+ producerStateManager = stateManager,
+ logDirFailureChannel = logDirFailureChannel,
topicId = None,
keepPartitionMetadataFile = true)
@@ -550,27 +597,37 @@ class LogLoaderTest {
val config = LogConfig(logProps)
val maxProducerIdExpirationMs = 300000
val logDirFailureChannel = null
- val logLoader = new LogLoader(logDir, topicPartition, config, mockTime.scheduler, mockTime, logDirFailureChannel) {
- override def createProducerStateManager(topicPartition: TopicPartition,
- dir: File,
- maxProducerIdExpirationMs: Int): ProducerStateManager = stateManager
- }
- val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
+ 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 = logComponents.segments,
- logStartOffset = logComponents.logStartOffset,
- recoveryPoint = logComponents.recoveryPoint,
- nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ segments = segments,
+ logStartOffset = offsets.logStartOffset,
+ recoveryPoint = offsets.recoveryPoint,
+ nextOffsetMetadata = offsets.nextOffsetMetadata,
scheduler = mockTime.scheduler,
brokerTopicStats = brokerTopicStats,
time = mockTime,
maxProducerIdExpirationMs = maxProducerIdExpirationMs,
producerIdExpirationCheckIntervalMs = 30000,
- topicPartition = Log.parseTopicPartitionName(logDir),
- leaderEpochCache = logComponents.leaderEpochCache,
- producerStateManager = logComponents.producerStateManager,
- logDirFailureChannel,
+ topicPartition = topicPartition,
+ leaderEpochCache = leaderEpochCache,
+ producerStateManager = stateManager,
+ logDirFailureChannel = logDirFailureChannel,
topicId = None,
keepPartitionMetadataFile = true)
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 732926c6e88e8..deb8827bc7dc8 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -1485,23 +1485,38 @@ class ReplicaManagerTest {
val mockLogDirFailureChannel = new LogDirFailureChannel(config.logDirs.size)
val tp = new TopicPartition(topic, topicPartition)
val maxProducerIdExpirationMs = 30000
- val logLoader = new LogLoader(logDir, tp, logConfig, mockScheduler, time, mockLogDirFailureChannel)
- val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
+ 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 = logDir,
config = logConfig,
- segments = logComponents.segments,
- logStartOffset = logComponents.logStartOffset,
- recoveryPoint = logComponents.recoveryPoint,
- nextOffsetMetadata = logComponents.nextOffsetMetadata,
+ segments = segments,
+ logStartOffset = offsets.logStartOffset,
+ recoveryPoint = offsets.recoveryPoint,
+ nextOffsetMetadata = offsets.nextOffsetMetadata,
scheduler = mockScheduler,
brokerTopicStats = mockBrokerTopicStats,
time = time,
maxProducerIdExpirationMs = maxProducerIdExpirationMs,
producerIdExpirationCheckIntervalMs = 30000,
topicPartition = tp,
- leaderEpochCache = logComponents.leaderEpochCache,
- producerStateManager = logComponents.producerStateManager,
+ 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 12ee6951c075c..159ace8103e5a 100644
--- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
@@ -20,7 +20,7 @@ import java.util.Properties
import java.util.concurrent.atomic._
import java.util.concurrent.{CountDownLatch, Executors, TimeUnit}
-import kafka.log.{Log, LogConfig, LogLoader, LogManager}
+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._
@@ -122,12 +122,27 @@ class SchedulerTest {
val maxProducerIdExpirationMs = 60 * 60 * 1000
val topicPartition = Log.parseTopicPartitionName(logDir)
val logDirFailureChannel = new LogDirFailureChannel(10)
- val logLoader = new LogLoader(logDir, topicPartition, logConfig, scheduler, mockTime, logDirFailureChannel)
- val logComponents = logLoader.load(0L, 0L, maxProducerIdExpirationMs)
- val log = new Log(logDir, logConfig, segments = logComponents.segments, logStartOffset = logComponents.logStartOffset,
- recoveryPoint = logComponents.recoveryPoint, nextOffsetMetadata = logComponents.nextOffsetMetadata, scheduler,
+ val segments = new LogSegments(topicPartition)
+ val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.messageFormatVersion.recordVersion)
+ val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs)
+ 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, logComponents.leaderEpochCache, logComponents.producerStateManager, logDirFailureChannel,
+ topicPartition, leaderEpochCache, producerStateManager, logDirFailureChannel,
topicId = None, keepPartitionMetadataFile = true)
assertTrue(scheduler.taskRunning(log.producerExpireCheck))
log.close()
From e7a2c749b4e13e66bb896de6aa1428ff58d34a8b Mon Sep 17 00:00:00 2001
From: Kowshik Prakasam
Date: Wed, 14 Apr 2021 02:43:15 -0700
Subject: [PATCH 5/7] Minor improvements
---
core/src/main/scala/kafka/log/Log.scala | 2 +-
core/src/main/scala/kafka/log/LogLoader.scala | 14 --------------
2 files changed, 1 insertion(+), 15 deletions(-)
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index ca508920079b6..c7b50baf98281 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -246,7 +246,7 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason {
* @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 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.
diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala
index 0ae8bbdf2d665..79e0cd086da92 100644
--- a/core/src/main/scala/kafka/log/LogLoader.scala
+++ b/core/src/main/scala/kafka/log/LogLoader.scala
@@ -74,20 +74,6 @@ case class LoadLogParams(dir: File,
* This object is responsible for all activities related with recovery of log segments from disk.
*/
object LogLoader extends Logging {
-
- /*protected def createLogSegments(topicPartition: TopicPartition): LogSegments = new LogSegments(topicPartition)
-
- protected def createProducerStateManager(topicPartition: TopicPartition,
- dir: File,
- maxProducerIdExpirationMs: Int): ProducerStateManager =
- new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
-
- protected def maybeCreateLeaderEpochCache(dir: File,
- topicPartition: TopicPartition,
- logDirFailureChannel: LogDirFailureChannel,
- recordVersion: RecordVersion): Option[LeaderEpochFileCache] =
- Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion)*/
-
/**
* 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
From 9e40d0fac31a0135641a6a3f5d2e824bc0a38c8d Mon Sep 17 00:00:00 2001
From: Kowshik Prakasam
Date: Thu, 15 Apr 2021 00:12:00 -0700
Subject: [PATCH 6/7] Address comments from Jun
---
core/src/main/scala/kafka/log/Log.scala | 4 ++--
core/src/main/scala/kafka/log/LogLoader.scala | 9 ++++-----
2 files changed, 6 insertions(+), 7 deletions(-)
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index c7b50baf98281..edbabb88feef6 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -222,7 +222,7 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason {
*
* @param _dir The directory in which log segments are created.
* @param config The log configuration settings
- * @param segments The log segments, these may be non-empty when recovered from disk
+ * @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
@@ -2344,7 +2344,7 @@ object Log extends Logging {
*
* 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
+ * 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
diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala
index 79e0cd086da92..6b28ec51bb91d 100644
--- a/core/src/main/scala/kafka/log/LogLoader.scala
+++ b/core/src/main/scala/kafka/log/LogLoader.scala
@@ -79,16 +79,15 @@ object LogLoader extends Logging {
* Additionally, it also suitably updates the provided LeaderEpochFileCache and ProducerStateManager
* to reflect the contents of the loaded log.
*
- * This method does not need to convert IOException to KafkaStorageException because it is only
- * called before all logs are loaded.
+ * 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; or when we find an unexpected
- * number of .log files with overflow
+ * overflow index offset
*/
def load(params: LoadLogParams): LoadedLogOffsets = {
// first do a pass through the files in the log directory and remove any temporary files
@@ -128,7 +127,7 @@ object LogLoader extends Logging {
time = params.time,
initFileSize = params.config.initFileSize))
}
- (params.recoveryPointCheckpoint, 0L)
+ (0L, 0L)
}
}
From eaf8519fa26ecfe6278edadee83e3604ae496be8 Mon Sep 17 00:00:00 2001
From: Kowshik Prakasam
Date: Fri, 16 Apr 2021 10:23:41 -0700
Subject: [PATCH 7/7] Fix broken PartitionLockTest
---
.../kafka/cluster/PartitionLockTest.scala | 50 ++++++++++++++----
.../unit/kafka/cluster/PartitionTest.scala | 51 +++++++++++++++----
2 files changed, 79 insertions(+), 22 deletions(-)
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
index 293dee3bfef4e..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,22 +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.segments,
- log.logStartOffset,
- log.recoveryPoint,
- log.nextOffsetMetadata,
+ segments,
+ offsets.logStartOffset,
+ offsets.recoveryPoint,
+ offsets.nextOffsetMetadata,
mockTime.scheduler,
new BrokerTopicStats,
- log.time,
+ mockTime,
log.maxProducerIdExpirationMs,
log.producerIdExpirationCheckIntervalMs,
log.topicPartition,
- log.leaderEpochCache,
- log.producerStateManager,
- new LogDirFailureChannel(1),
+ leaderEpochCache,
+ producerStateManager,
+ logDirFailureChannel,
topicId = None,
keepPartitionMetadataFile = true) {
@@ -365,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 de96e180df695..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,22 +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.segments,
- log.logStartOffset,
- log.recoveryPoint,
- log.nextOffsetMetadata,
+ segments,
+ offsets.logStartOffset,
+ offsets.recoveryPoint,
+ offsets.nextOffsetMetadata,
mockTime.scheduler,
new BrokerTopicStats,
- log.time,
+ mockTime,
log.maxProducerIdExpirationMs,
log.producerIdExpirationCheckIntervalMs,
log.topicPartition,
- log.leaderEpochCache,
- log.producerStateManager,
- new LogDirFailureChannel(1),
+ leaderEpochCache,
+ producerStateManager,
+ logDirFailureChannel,
topicId = None,
keepPartitionMetadataFile = true) {