diff --git a/core/src/main/scala/kafka/log/LocalLog.scala b/core/src/main/scala/kafka/log/LocalLog.scala new file mode 100644 index 0000000000000..31d9006cac8b8 --- /dev/null +++ b/core/src/main/scala/kafka/log/LocalLog.scala @@ -0,0 +1,1010 @@ +/** + * 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 +import java.text.NumberFormat +import java.util.concurrent.atomic.AtomicLong +import java.util.regex.Pattern +import kafka.metrics.KafkaMetricsGroup +import kafka.server.{FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata} +import kafka.utils.{Logging, Scheduler} +import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.errors.{KafkaStorageException, OffsetOutOfRangeException} +import org.apache.kafka.common.message.FetchResponseData +import org.apache.kafka.common.record.MemoryRecords +import org.apache.kafka.common.utils.{Time, Utils} + +import scala.jdk.CollectionConverters._ +import scala.collection.{Seq, immutable} +import scala.collection.mutable.{ArrayBuffer, ListBuffer} + +/** + * Holds the result of splitting a segment into one or more segments, see LocalLog.splitOverflowedSegment(). + * + * @param deletedSegments segments deleted when splitting a segment + * @param newSegments new segments created when splitting a segment + */ +case class SplitSegmentResult(deletedSegments: Iterable[LogSegment], newSegments: Iterable[LogSegment]) + +/** + * An append-only log for storing messages locally. The log is a sequence of LogSegments, each with a base offset. + * New log segments are created according to a configurable policy that controls the size in bytes or time interval + * for a given segment. + * + * NOTE: this class is not thread-safe, and it relies on the thread safety provided by the Log class. + * + * @param _dir The directory in which log segments are created. + * @param config The log configuration settings + * @param segments The non-empty log segments recovered from disk + * @param 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 time The time instance used for checking the clock + * @param topicPartition The topic partition associated with this log + * @param logDirFailureChannel The LogDirFailureChannel instance to asynchronously handle Log dir failure + */ +class LocalLog(@volatile private var _dir: File, + @volatile private[log] var config: LogConfig, + private[log] val segments: LogSegments, + @volatile private[log] var recoveryPoint: Long, + @volatile private var nextOffsetMetadata: LogOffsetMetadata, + private[log] val scheduler: Scheduler, + private[log] val time: Time, + private[log] val topicPartition: TopicPartition, + private[log] val logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup { + + import kafka.log.LocalLog._ + + this.logIdent = s"[LocalLog partition=$topicPartition, dir=${dir.getParent}] " + + // The memory mapped buffer for index files of this log will be closed with either delete() or closeHandlers() + // After memory mapped buffer is closed, no disk IO operation should be performed for this log. + @volatile private[log] var isMemoryMappedBufferClosed = false + + // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks + @volatile private var _parentDir: String = dir.getParent + + // Last time the log was flushed + private val lastFlushedTime = new AtomicLong(time.milliseconds) + + private[log] def dir: File = _dir + + private[log] def name: String = dir.getName() + + private[log] def parentDir: String = _parentDir + + private[log] def parentDirFile: File = new File(_parentDir) + + private[log] def isFuture: Boolean = dir.getName.endsWith(LocalLog.FutureDirSuffix) + + private def maybeHandleIOException[T](msg: => String)(fun: => T): T = { + LocalLog.maybeHandleIOException(logDirFailureChannel, parentDir, msg) { + fun + } + } + + /** + * Rename the directory of the log + * @param name the new dir name + * @throws KafkaStorageException if rename fails + */ + private[log] def renameDir(name: String): Boolean = { + maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") { + val renamedDir = new File(dir.getParent, name) + Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath) + if (renamedDir != dir) { + _dir = renamedDir + _parentDir = renamedDir.getParent + segments.updateParentDir(renamedDir) + true + } else { + false + } + } + } + + /** + * Update the existing configuration to the new provided configuration. + * @param newConfig the new configuration to be updated to + */ + private[log] def updateConfig(newConfig: LogConfig): Unit = { + val oldConfig = config + config = newConfig + val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion + val newRecordVersion = newConfig.messageFormatVersion.recordVersion + if (newRecordVersion.precedes(oldRecordVersion)) + warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.") + } + + private[log] def checkIfMemoryMappedBufferClosed(): Unit = { + if (isMemoryMappedBufferClosed) + throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed") + } + + private[log] def updateRecoveryPoint(newRecoveryPoint: Long): Unit = { + recoveryPoint = newRecoveryPoint + } + + /** + * Update recoveryPoint to provided offset and mark the log as flushed, if the offset is greater + * than the existing recoveryPoint. + * + * @param offset the offset to be updated + */ + private[log] def markFlushed(offset: Long): Unit = { + checkIfMemoryMappedBufferClosed() + if (offset > recoveryPoint) { + updateRecoveryPoint(offset) + lastFlushedTime.set(time.milliseconds) + } + } + + /** + * The number of messages appended to the log since the last flush + */ + private[log] def unflushedMessages: Long = logEndOffset - recoveryPoint + + /** + * Flush local log segments for all offsets up to offset-1. + * Does not update the recovery point. + * + * @param offset The offset to flush up to (non-inclusive) + */ + private[log] def flush(offset: Long): Unit = { + val segmentsToFlush = segments.values(recoveryPoint, offset) + segmentsToFlush.foreach(_.flush()) + // If there are any new segments, we need to flush the parent directory for crash consistency. + segmentsToFlush.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath)) + } + + /** + * The time this log is last known to have been fully flushed to disk + */ + private[log] def lastFlushTime: Long = lastFlushedTime.get + + /** + * The offset metadata of the next message that will be appended to the log + */ + private[log] def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata + + /** + * The offset of the next message that will be appended to the log + */ + private[log] def logEndOffset: Long = nextOffsetMetadata.messageOffset + + /** + * Update end offset of the log, and update the recoveryPoint. + * + * @param endOffset the new end offset of the log + */ + private[log] def updateLogEndOffset(endOffset: Long): Unit = { + nextOffsetMetadata = LogOffsetMetadata(endOffset, segments.activeSegment.baseOffset, segments.activeSegment.size) + if (recoveryPoint > endOffset) { + updateRecoveryPoint(endOffset) + } + } + + /** + * Close file handlers used by log but don't write to disk. + * This is called if the log directory is offline. + */ + private[log] def closeHandlers(): Unit = { + segments.closeHandlers() + isMemoryMappedBufferClosed = true + } + + /** + * Closes the segments of the log. + */ + private[log] def close(): Unit = { + maybeHandleIOException(s"Error while renaming dir for $topicPartition in dir ${dir.getParent}") { + checkIfMemoryMappedBufferClosed() + segments.close() + } + } + + /** + * Completely delete this log directory with no delay. + */ + private[log] def deleteEmptyDir(): Unit = { + maybeHandleIOException(s"Error while deleting dir for $topicPartition in dir ${dir.getParent}") { + if (segments.nonEmpty) { + throw new IllegalStateException(s"Can not delete directory when ${segments.numberOfSegments} segments are still present") + } + if (!isMemoryMappedBufferClosed) { + throw new IllegalStateException(s"Can not delete directory when memory mapped buffer for log of $topicPartition is still open.") + } + Utils.delete(dir) + } + } + + /** + * Completely delete all segments with no delay. + * @return the deleted segments + */ + private[log] def deleteAllSegments(): Iterable[LogSegment] = { + maybeHandleIOException(s"Error while deleting all segments for $topicPartition in dir ${dir.getParent}") { + val deletableSegments = List[LogSegment]() ++ segments.values + removeAndDeleteSegments(segments.values, asyncDelete = false, LogDeletion(this)) + isMemoryMappedBufferClosed = true + deletableSegments + } + } + + /** + * Find segments starting from the oldest until the user-supplied predicate is false. + * A final segment that is empty will never be returned. + * + * @param predicate A function that takes in a candidate log segment, the next higher segment + * (if there is one). It returns true iff the segment is deletable. + * @return the segments ready to be deleted + */ + private[log] def deletableSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean): Iterable[LogSegment] = { + if (segments.isEmpty) { + Seq.empty + } else { + val deletable = ArrayBuffer.empty[LogSegment] + val segmentsIterator = segments.values.iterator + var segmentOpt = nextOption(segmentsIterator) + while (segmentOpt.isDefined) { + val segment = segmentOpt.get + val nextSegmentOpt = nextOption(segmentsIterator) + val isLastSegmentAndEmpty = nextSegmentOpt.isEmpty && segment.size == 0 + if (predicate(segment, nextSegmentOpt) && !isLastSegmentAndEmpty) { + deletable += segment + segmentOpt = nextSegmentOpt + } else { + segmentOpt = Option.empty + } + } + deletable + } + } + + /** + * This method deletes the given log segments 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 renames the index and log files by appending .deleted to the respective file name + * - 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 convert IOException to KafkaStorageException, the immediate caller + * is expected to catch and handle IOException. + * + * @param segmentsToDelete The log segments to schedule for deletion + * @param asyncDelete Whether the segment files should be deleted asynchronously + * @param reason The reason for the segment deletion + */ + private[log] def removeAndDeleteSegments(segmentsToDelete: Iterable[LogSegment], + asyncDelete: Boolean, + reason: SegmentDeletionReason): Unit = { + if (segmentsToDelete.nonEmpty) { + // Most callers hold an iterator into the `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. We should also pass only the materialized view of the + // iterator to the logic that actually deletes the segments. + val toDelete = segmentsToDelete.toList + reason.logReason(toDelete) + toDelete.foreach { segment => + segments.remove(segment.baseOffset) + } + LocalLog.deleteSegmentFiles(toDelete, asyncDelete, dir, topicPartition, config, scheduler, logDirFailureChannel, logIdent) + } + } + + /** + * Given a message offset, find its corresponding offset metadata in the log. + * If the message offset is out of range, throw an OffsetOutOfRangeException + */ + private[log] def convertToOffsetMetadataOrThrow(offset: Long): LogOffsetMetadata = { + val fetchDataInfo = read(offset, + maxLength = 1, + minOneMessage = false, + maxOffsetMetadata = nextOffsetMetadata, + includeAbortedTxns = false) + fetchDataInfo.fetchOffsetMetadata + } + + /** + * Read messages from the log. + * + * @param startOffset The offset to begin reading at + * @param maxLength The maximum number of bytes to read + * @param minOneMessage If this is true, the first message will be returned even if it exceeds `maxLength` (if one exists) + * @param maxOffsetMetadata The metadata of the maximum offset to be fetched + * @param includeAbortedTxns If true, aborted transactions are included + * @throws OffsetOutOfRangeException If startOffset is beyond the log end offset + * @return The fetch data information including fetch starting offset metadata and messages read. + */ + def read(startOffset: Long, + maxLength: Int, + minOneMessage: Boolean, + maxOffsetMetadata: LogOffsetMetadata, + includeAbortedTxns: Boolean): FetchDataInfo = { + maybeHandleIOException(s"Exception while reading from $topicPartition in dir ${dir.getParent}") { + trace(s"Reading maximum $maxLength bytes at offset $startOffset from log with " + + s"total length ${segments.sizeInBytes} bytes") + + val endOffsetMetadata = nextOffsetMetadata + val endOffset = endOffsetMetadata.messageOffset + var segmentOpt = segments.floorSegment(startOffset) + + // return error on attempt to read beyond the log end offset + if (startOffset > endOffset || segmentOpt.isEmpty) + throw new OffsetOutOfRangeException(s"Received request for offset $startOffset for partition $topicPartition, " + + s"but we only have log segments upto $endOffset.") + + if (startOffset == maxOffsetMetadata.messageOffset) + emptyFetchDataInfo(maxOffsetMetadata, includeAbortedTxns) + else if (startOffset > maxOffsetMetadata.messageOffset) + emptyFetchDataInfo(convertToOffsetMetadataOrThrow(startOffset), includeAbortedTxns) + else { + // Do the read on the segment with a base offset less than the target offset + // but if that segment doesn't contain any messages with an offset greater than that + // continue to read from successive segments until we get some messages or we reach the end of the log + var fetchDataInfo: FetchDataInfo = null + while (fetchDataInfo == null && segmentOpt.isDefined) { + val segment = segmentOpt.get + val baseOffset = segment.baseOffset + + val maxPosition = + // Use the max offset position if it is on this segment; otherwise, the segment size is the limit. + if (maxOffsetMetadata.segmentBaseOffset == segment.baseOffset) maxOffsetMetadata.relativePositionInSegment + else segment.size + + fetchDataInfo = segment.read(startOffset, maxLength, maxPosition, minOneMessage) + if (fetchDataInfo != null) { + if (includeAbortedTxns) + fetchDataInfo = addAbortedTransactions(startOffset, segment, fetchDataInfo) + } else segmentOpt = segments.higherSegment(baseOffset) + } + + if (fetchDataInfo != null) fetchDataInfo + else { + // okay we are beyond the end of the last segment with no data fetched although the start offset is in range, + // this can happen when all messages with offset larger than start offsets have been deleted. + // In this case, we will return the empty set with log end offset metadata + FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY) + } + } + } + } + + private[log] def append(lastOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, records: MemoryRecords): Unit = { + segments.activeSegment.append(largestOffset = lastOffset, largestTimestamp = largestTimestamp, + shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp, records = records) + updateLogEndOffset(lastOffset + 1) + } + + private def addAbortedTransactions(startOffset: Long, segment: LogSegment, + fetchInfo: FetchDataInfo): FetchDataInfo = { + val fetchSize = fetchInfo.records.sizeInBytes + val startOffsetPosition = OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset, + fetchInfo.fetchOffsetMetadata.relativePositionInSegment) + val upperBoundOffset = segment.fetchUpperBoundOffset(startOffsetPosition, fetchSize).getOrElse { + segments.higherSegment(segment.baseOffset).map(_.baseOffset).getOrElse(logEndOffset) + } + + val abortedTransactions = ListBuffer.empty[FetchResponseData.AbortedTransaction] + def accumulator(abortedTxns: List[AbortedTxn]): Unit = abortedTransactions ++= abortedTxns.map(_.asAbortedTransaction) + collectAbortedTransactions(startOffset, upperBoundOffset, segment, accumulator) + + FetchDataInfo(fetchOffsetMetadata = fetchInfo.fetchOffsetMetadata, + records = fetchInfo.records, + firstEntryIncomplete = fetchInfo.firstEntryIncomplete, + abortedTransactions = Some(abortedTransactions.toList)) + } + + private def collectAbortedTransactions(startOffset: Long, upperBoundOffset: Long, + startingSegment: LogSegment, + accumulator: List[AbortedTxn] => Unit): Unit = { + val higherSegments = segments.higherSegments(startingSegment.baseOffset).iterator + var segmentEntryOpt = Option(startingSegment) + while (segmentEntryOpt.isDefined) { + val segment = segmentEntryOpt.get + val searchResult = segment.collectAbortedTxns(startOffset, upperBoundOffset) + accumulator(searchResult.abortedTransactions) + if (searchResult.isComplete) + return + segmentEntryOpt = nextOption(higherSegments) + } + } + + private[log] def collectAbortedTransactions(logStartOffset: Long, baseOffset: Long, upperBoundOffset: Long): List[AbortedTxn] = { + val segmentEntry = segments.floorSegment(baseOffset) + val allAbortedTxns = ListBuffer.empty[AbortedTxn] + def accumulator(abortedTxns: List[AbortedTxn]): Unit = allAbortedTxns ++= abortedTxns + segmentEntry.foreach(segment => collectAbortedTransactions(logStartOffset, upperBoundOffset, segment, accumulator)) + allAbortedTxns.toList + } + + /** + * Roll the log over to a new active segment starting with the current logEndOffset. + * This will trim the index to the exact size of the number of entries it currently contains. + * + * @param expectedNextOffset The expected next offset after the segment is rolled + * + * @return The newly rolled segment + */ + private[log] def roll(expectedNextOffset: Option[Long] = None): LogSegment = { + maybeHandleIOException(s"Error while rolling log segment for $topicPartition in dir ${dir.getParent}") { + val start = time.hiResClockMs() + checkIfMemoryMappedBufferClosed() + val newOffset = math.max(expectedNextOffset.getOrElse(0L), logEndOffset) + val logFile = LocalLog.logFile(dir, newOffset) + val activeSegment = segments.activeSegment + if (segments.contains(newOffset)) { + // segment with the same base offset already exists and loaded + if (activeSegment.baseOffset == newOffset && activeSegment.size == 0) { + // We have seen this happen (see KAFKA-6388) after shouldRoll() returns true for an + // active segment of size zero because of one of the indexes is "full" (due to _maxEntries == 0). + warn(s"Trying to roll a new log segment with start offset $newOffset " + + s"=max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already " + + s"exists and is active with size 0. Size of time index: ${activeSegment.timeIndex.entries}," + + s" size of offset index: ${activeSegment.offsetIndex.entries}.") + removeAndDeleteSegments(Seq(activeSegment), asyncDelete = true, LogRoll(this)) + } else { + throw new KafkaException(s"Trying to roll a new log segment for topic partition $topicPartition with start offset $newOffset" + + s" =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already exists. Existing " + + s"segment is ${segments.get(newOffset)}.") + } + } else if (!segments.isEmpty && newOffset < activeSegment.baseOffset) { + throw new KafkaException( + s"Trying to roll a new log segment for topic partition $topicPartition with " + + s"start offset $newOffset =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) lower than start offset of the active segment $activeSegment") + } else { + val offsetIdxFile = offsetIndexFile(dir, newOffset) + val timeIdxFile = timeIndexFile(dir, newOffset) + val txnIdxFile = transactionIndexFile(dir, newOffset) + + for (file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) { + warn(s"Newly rolled segment file ${file.getAbsolutePath} already exists; deleting it first") + Files.delete(file.toPath) + } + + segments.lastSegment.foreach(_.onBecomeInactiveSegment()) + } + + val newSegment = LogSegment.open(dir, + baseOffset = newOffset, + config, + time = time, + initFileSize = config.initFileSize, + preallocate = config.preallocate) + segments.add(newSegment) + + // We need to update the segment base offset and append position data of the metadata when log rolls. + // The next offset should not change. + updateLogEndOffset(nextOffsetMetadata.messageOffset) + + info(s"Rolled new log segment at offset $newOffset in ${time.hiResClockMs() - start} ms.") + + newSegment + } + } + + /** + * Delete all data in the local log and start at the new offset. + * + * @param newOffset The new offset to start the log with + * @return the list of segments that were scheduled for deletion + */ + private[log] def truncateFullyAndStartAt(newOffset: Long): Iterable[LogSegment] = { + maybeHandleIOException(s"Error while truncating the entire log for $topicPartition in dir ${dir.getParent}") { + debug(s"Truncate and start at offset $newOffset") + checkIfMemoryMappedBufferClosed() + val segmentsToDelete = List[LogSegment]() ++ segments.values + removeAndDeleteSegments(segmentsToDelete, asyncDelete = true, LogTruncation(this)) + segments.add(LogSegment.open(dir, + baseOffset = newOffset, + config = config, + time = time, + initFileSize = config.initFileSize, + preallocate = config.preallocate)) + updateLogEndOffset(newOffset) + segmentsToDelete + } + } + + /** + * Truncate this log so that it ends with the greatest offset < targetOffset. + * + * @param targetOffset The offset to truncate to, an upper bound on all offsets in the log after truncation is complete. + * @return the list of segments that were scheduled for deletion + */ + private[log] def truncateTo(targetOffset: Long): Iterable[LogSegment] = { + val deletableSegments = List[LogSegment]() ++ segments.filter(segment => segment.baseOffset > targetOffset) + removeAndDeleteSegments(deletableSegments, asyncDelete = true, LogTruncation(this)) + segments.activeSegment.truncateTo(targetOffset) + updateLogEndOffset(targetOffset) + deletableSegments + } +} + +/** + * Helper functions for logs + */ +object LocalLog extends Logging { + + /** a log file */ + private[log] val LogFileSuffix = ".log" + + /** an index file */ + private[log] val IndexFileSuffix = ".index" + + /** a time index file */ + private[log] val TimeIndexFileSuffix = ".timeindex" + + /** an (aborted) txn index */ + private[log] val TxnIndexFileSuffix = ".txnindex" + + /** a file that is scheduled to be deleted */ + private[log] val DeletedFileSuffix = ".deleted" + + /** A temporary file that is being used for log cleaning */ + private[log] val CleanedFileSuffix = ".cleaned" + + /** A temporary file used when swapping files into the log */ + private[log] val SwapFileSuffix = ".swap" + + /** a directory that is scheduled to be deleted */ + private[log] val DeleteDirSuffix = "-delete" + + /** a directory that is used for future partition */ + private[log] val FutureDirSuffix = "-future" + + private[log] val DeleteDirPattern = Pattern.compile(s"^(\\S+)-(\\S+)\\.(\\S+)$DeleteDirSuffix") + private[log] val FutureDirPattern = Pattern.compile(s"^(\\S+)-(\\S+)\\.(\\S+)$FutureDirSuffix") + + private[log] val UnknownOffset = -1L + + /** + * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros + * so that ls sorts the files numerically. + * + * @param offset The offset to use in the file name + * @return The filename + */ + private[log] def filenamePrefixFromOffset(offset: Long): String = { + val nf = NumberFormat.getInstance() + nf.setMinimumIntegerDigits(20) + nf.setMaximumFractionDigits(0) + nf.setGroupingUsed(false) + nf.format(offset) + } + + /** + * Construct a log file name in the given dir with the given base offset and the given suffix + * + * @param dir The directory in which the log will reside + * @param offset The base offset of the log file + * @param suffix The suffix to be appended to the file name (e.g. "", ".deleted", ".cleaned", ".swap", etc.) + */ + private[log] def logFile(dir: File, offset: Long, suffix: String = ""): File = + new File(dir, filenamePrefixFromOffset(offset) + LogFileSuffix + suffix) + + /** + * Return a directory name to rename the log directory to for async deletion. + * The name will be in the following format: "topic-partitionId.uniqueId-delete". + * If the topic name is too long, it will be truncated to prevent the total name + * from exceeding 255 characters. + */ + private[log] def logDeleteDirName(topicPartition: TopicPartition): String = { + val uniqueId = java.util.UUID.randomUUID.toString.replaceAll("-", "") + val suffix = s"-${topicPartition.partition()}.${uniqueId}${DeleteDirSuffix}" + val prefixLength = Math.min(topicPartition.topic().size, 255 - suffix.size) + s"${topicPartition.topic().substring(0, prefixLength)}${suffix}" + } + + /** + * Return a future directory name for the given topic partition. The name will be in the following + * format: topic-partition.uniqueId-future where topic, partition and uniqueId are variables. + */ + private[log] def logFutureDirName(topicPartition: TopicPartition): String = { + logDirNameWithSuffix(topicPartition, FutureDirSuffix) + } + + private[log] def logDirNameWithSuffix(topicPartition: TopicPartition, suffix: String): String = { + val uniqueId = java.util.UUID.randomUUID.toString.replaceAll("-", "") + s"${logDirName(topicPartition)}.$uniqueId$suffix" + } + + /** + * Return a directory name for the given topic partition. The name will be in the following + * format: topic-partition where topic, partition are variables. + */ + private[log] def logDirName(topicPartition: TopicPartition): String = { + s"${topicPartition.topic}-${topicPartition.partition}" + } + + /** + * Construct an index file name in the given dir using the given base offset and the given suffix + * + * @param dir The directory in which the log will reside + * @param offset The base offset of the log file + * @param suffix The suffix to be appended to the file name ("", ".deleted", ".cleaned", ".swap", etc.) + */ + private[log] def offsetIndexFile(dir: File, offset: Long, suffix: String = ""): File = + new File(dir, filenamePrefixFromOffset(offset) + IndexFileSuffix + suffix) + + /** + * Construct a time index file name in the given dir using the given base offset and the given suffix + * + * @param dir The directory in which the log will reside + * @param offset The base offset of the log file + * @param suffix The suffix to be appended to the file name ("", ".deleted", ".cleaned", ".swap", etc.) + */ + private[log] def timeIndexFile(dir: File, offset: Long, suffix: String = ""): File = + new File(dir, filenamePrefixFromOffset(offset) + TimeIndexFileSuffix + suffix) + + /** + * Construct a transaction index file name in the given dir using the given base offset and the given suffix + * + * @param dir The directory in which the log will reside + * @param offset The base offset of the log file + * @param suffix The suffix to be appended to the file name ("", ".deleted", ".cleaned", ".swap", etc.) + */ + private[log] def transactionIndexFile(dir: File, offset: Long, suffix: String = ""): File = + new File(dir, filenamePrefixFromOffset(offset) + TxnIndexFileSuffix + suffix) + + private[log] def offsetFromFileName(filename: String): Long = { + filename.substring(0, filename.indexOf('.')).toLong + } + + private[log] def offsetFromFile(file: File): Long = { + offsetFromFileName(file.getName) + } + + /** + * Parse the topic and partition out of the directory name of a log + */ + private[log] def parseTopicPartitionName(dir: File): TopicPartition = { + if (dir == null) + throw new KafkaException("dir should not be null") + + def exception(dir: File): KafkaException = { + new KafkaException(s"Found directory ${dir.getCanonicalPath}, '${dir.getName}' is not in the form of " + + "topic-partition or topic-partition.uniqueId-delete (if marked for deletion).\n" + + "Kafka's log directories (and children) should only contain Kafka topic data.") + } + + val dirName = dir.getName + if (dirName == null || dirName.isEmpty || !dirName.contains('-')) + throw exception(dir) + if (dirName.endsWith(DeleteDirSuffix) && !DeleteDirPattern.matcher(dirName).matches || + dirName.endsWith(FutureDirSuffix) && !FutureDirPattern.matcher(dirName).matches) + throw exception(dir) + + val name: String = + if (dirName.endsWith(DeleteDirSuffix) || dirName.endsWith(FutureDirSuffix)) dirName.substring(0, dirName.lastIndexOf('.')) + else dirName + + val index = name.lastIndexOf('-') + val topic = name.substring(0, index) + val partitionString = name.substring(index + 1) + if (topic.isEmpty || partitionString.isEmpty) + throw exception(dir) + + val partition = + try partitionString.toInt + catch { case _: NumberFormatException => throw exception(dir) } + + new TopicPartition(topic, partition) + } + + private[log] def isIndexFile(file: File): Boolean = { + val filename = file.getName + filename.endsWith(IndexFileSuffix) || filename.endsWith(TimeIndexFileSuffix) || filename.endsWith(TxnIndexFileSuffix) + } + + private[log] def isLogFile(file: File): Boolean = + file.getPath.endsWith(LogFileSuffix) + + /** + * 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[log] 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) + } + } + + /** + * 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 logPrefix The logging prefix + * @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, + logPrefix: String): SplitSegmentResult = { + require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}") + require(segment.hasOverflow, s"Split operation is only permitted for segments with overflow, and the problem path is ${segment.log.file.getAbsoluteFile}") + + info(s"${logPrefix}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 = 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"${logPrefix}Replacing overflowed segment $segment with split segments $newSegments") + val newSegmentsToAdd = newSegments.toSeq + val deletedSegments = LocalLog.replaceSegments(existingSegments, newSegmentsToAdd, List(segment), + dir, topicPartition, config, scheduler, logDirFailureChannel, logPrefix) + SplitSegmentResult(deletedSegments.toSeq, newSegmentsToAdd) + } catch { + case e: Exception => + newSegments.foreach { splitSegment => + splitSegment.close() + splitSegment.deleteIfExists() + } + throw e + } + } + + /** + * 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 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 logPrefix The logging prefix + * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash + */ + private[log] def replaceSegments(existingSegments: LogSegments, + newSegments: Seq[LogSegment], + oldSegments: Seq[LogSegment], + dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + logDirFailureChannel: LogDirFailureChannel, + logPrefix: String, + isRecoveredSwapFile: Boolean = false): Iterable[LogSegment] = { + 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(CleanedFileSuffix, SwapFileSuffix)) + sortedNewSegments.reverse.foreach(existingSegments.add(_)) + val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet + + // delete the old files + val deletedNotReplaced = sortedOldSegments.map { seg => + // remove the index entry + if (seg.baseOffset != sortedNewSegments.head.baseOffset) + existingSegments.remove(seg.baseOffset) + deleteSegmentFiles( + List(seg), + asyncDelete = true, + dir, + topicPartition, + config, + scheduler, + logDirFailureChannel, + logPrefix) + if (newSegmentBaseOffsets.contains(seg.baseOffset)) Option.empty else Some(seg) + }.filter(item => item.isDefined).map(item => item.get) + + // okay we are safe now, remove the swap suffix + sortedNewSegments.foreach(_.changeFileSuffixes(SwapFileSuffix, "")) + Utils.flushDir(dir.toPath) + deletedNotReplaced + } + + /** + * Perform physical deletion of the index and log 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 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 logPrefix The logging prefix + * @throws IOException if the file can't be renamed and still exists + */ + private[log] def deleteSegmentFiles(segmentsToDelete: immutable.Iterable[LogSegment], + asyncDelete: Boolean, + dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + logDirFailureChannel: LogDirFailureChannel, + logPrefix: String): Unit = { + segmentsToDelete.foreach(_.changeFileSuffixes("", DeletedFileSuffix)) + + def deleteSegments(): Unit = { + info(s"${logPrefix}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 (asyncDelete) + scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs) + else + deleteSegments() + } + + private[log] def emptyFetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, + includeAbortedTxns: Boolean): FetchDataInfo = { + val abortedTransactions = + if (includeAbortedTxns) Some(List.empty[FetchResponseData.AbortedTransaction]) + else None + FetchDataInfo(fetchOffsetMetadata, + MemoryRecords.EMPTY, + abortedTransactions = abortedTransactions) + } + + private[log] def createNewCleanedSegment(dir: File, logConfig: LogConfig, baseOffset: Long): LogSegment = { + LogSegment.deleteIfExists(dir, baseOffset, fileSuffix = CleanedFileSuffix) + LogSegment.open(dir, baseOffset, logConfig, Time.SYSTEM, + fileSuffix = CleanedFileSuffix, initFileSize = logConfig.initFileSize, preallocate = logConfig.preallocate) + } + + /** + * Wraps the value of iterator.next() in an option. + * Note: this facility is a part of the Iterator class starting from scala v2.13. + * + * @param iterator + * @tparam T the type of object held within the iterator + * @return Some(iterator.next) if a next element exists, None otherwise. + */ + private def nextOption[T](iterator: Iterator[T]): Option[T] = { + if (iterator.hasNext) + Some(iterator.next()) + else + None + } +} + +trait SegmentDeletionReason { + def logReason(toDelete: List[LogSegment]): Unit +} + +case class LogTruncation(log: LocalLog) extends SegmentDeletionReason { + override def logReason(toDelete: List[LogSegment]): Unit = { + log.info(s"Deleting segments as part of log truncation: ${toDelete.mkString(",")}") + } +} + +case class LogRoll(log: LocalLog) extends SegmentDeletionReason { + override def logReason(toDelete: List[LogSegment]): Unit = { + log.info(s"Deleting segments as part of log roll: ${toDelete.mkString(",")}") + } +} + +case class LogDeletion(log: LocalLog) extends SegmentDeletionReason { + override def logReason(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/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 13f7598431e91..425a6e9548db6 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -19,11 +19,8 @@ package kafka.log import java.io.{File, IOException} import java.nio.file.Files -import java.text.NumberFormat import java.util.Optional -import java.util.concurrent.atomic._ import java.util.concurrent.TimeUnit -import java.util.regex.Pattern import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0} import kafka.common.{LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.AppendOrigin.RaftLeader @@ -44,7 +41,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} import scala.jdk.CollectionConverters._ -import scala.collection.mutable.{ArrayBuffer, ListBuffer} +import scala.collection.mutable.ListBuffer import scala.collection.{Seq, immutable, mutable} object LogAppendInfo { @@ -210,16 +207,16 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason { } /** - * An append-only log for storing messages. + * A log which presents a unified view of local and tiered log segments. * - * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment. + * The log consists of tiered and local segments with the tiered portion of the log being optional. There could be an + * overlap between the tiered and local segments. The active segment is always guaranteed to be local. If tiered segments + * are present, they always appear at the beginning of the log, followed by an optional region of overlap, followed by the local + * segments including the active segment. * - * New log segments are created according to a configurable policy that controls the size in bytes or time interval - * for a given segment. + * NOTE: this class handles state and behavior specific to tiered segments as well as any behavior combining both tiered + * and local segments. The state and behavior specific to local segments are handled by the encapsulated LocalLog instance. * - * @param _dir The directory in which log segments are created. - * @param config The log configuration settings - * @param segments The non-empty log segments recovered from disk * @param logStartOffset The earliest offset allowed to be exposed to kafka client. * The logStartOffset can be updated by : * - user's DeleteRecordsRequest @@ -232,13 +229,9 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason { * - 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 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 localLog The LocalLog instance containing non-empty log segments recovered from disk * @param brokerTopicStats Container for Broker Topic Yammer Metrics - * @param time The time instance used for checking the clock * @param producerIdExpirationCheckIntervalMs How often to check for producer ids which need to be expired - * @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 @@ -255,40 +248,22 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason { * will be deleted to avoid ID conflicts upon re-upgrade. */ @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, +class Log(@volatile var logStartOffset: Long, + private val localLog: LocalLog, brokerTopicStats: BrokerTopicStats, - val time: Time, val producerIdExpirationCheckIntervalMs: Int, - val topicPartition: TopicPartition, @volatile var leaderEpochCache: Option[LeaderEpochFileCache], val producerStateManager: ProducerStateManager, - logDirFailureChannel: LogDirFailureChannel, @volatile private var _topicId: Option[Uuid], val keepPartitionMetadataFile: Boolean) extends Logging with KafkaMetricsGroup { import kafka.log.Log._ - this.logIdent = s"[Log partition=$topicPartition, dir=${dir.getParent}] " + this.logIdent = s"[Log partition=$topicPartition, dir=$parentDir] " /* A lock that guards all modifications to the log */ private val lock = new Object - // The memory mapped buffer for index files of this log will be closed with either delete() or closeHandlers() - // After memory mapped buffer is closed, no disk IO operation should be performed for this log - @volatile private var isMemoryMappedBufferClosed = false - - // Cache value of parent directory to avoid allocations in hot paths like ReplicaManager.checkpointHighWatermarks - @volatile private var _parentDir: String = dir.getParent - - /* last time it was flushed */ - private val lastFlushedTime = new AtomicLong(time.milliseconds) - /* 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 @@ -355,28 +330,35 @@ class Log(@volatile private var _dir: File, def topicId: Option[Uuid] = _topicId - def dir: File = _dir + def dir: File = localLog.dir + + def parentDir: String = localLog.parentDir + + def parentDirFile: File = localLog.parentDirFile + + def name: String = localLog.name + + def recoveryPoint: Long = localLog.recoveryPoint - def parentDir: String = _parentDir + def topicPartition: TopicPartition = localLog.topicPartition - def parentDirFile: File = new File(_parentDir) + def time: Time = localLog.time + + def scheduler: Scheduler = localLog.scheduler + + def config: LogConfig = localLog.config + + def logDirFailureChannel: LogDirFailureChannel = localLog.logDirFailureChannel def updateConfig(newConfig: LogConfig): Unit = { - val oldConfig = this.config - this.config = newConfig + val oldConfig = localLog.config + localLog.updateConfig(newConfig) val oldRecordVersion = oldConfig.messageFormatVersion.recordVersion val newRecordVersion = newConfig.messageFormatVersion.recordVersion - if (newRecordVersion.precedes(oldRecordVersion)) - warn(s"Record format version has been downgraded from $oldRecordVersion to $newRecordVersion.") if (newRecordVersion.value != oldRecordVersion.value) initializeLeaderEpochCache() } - private def checkIfMemoryMappedBufferClosed(): Unit = { - if (isMemoryMappedBufferClosed) - throw new KafkaStorageException(s"The memory mapped buffer for log of $topicPartition is already closed") - } - def highWatermark: Long = highWatermarkMetadata.messageOffset /** @@ -401,7 +383,7 @@ class Log(@volatile private var _dir: File, * @return the updated high watermark offset */ def updateHighWatermark(highWatermarkMetadata: LogOffsetMetadata): Long = { - val endOffsetMetadata = logEndOffsetMetadata + val endOffsetMetadata = localLog.logEndOffsetMetadata val newHighWatermarkMetadata = if (highWatermarkMetadata.messageOffset < logStartOffset) { LogOffsetMetadata(logStartOffset) } else if (highWatermarkMetadata.messageOffset >= endOffsetMetadata.messageOffset) { @@ -426,7 +408,7 @@ class Log(@volatile private var _dir: File, def maybeIncrementHighWatermark(newHighWatermark: LogOffsetMetadata): Option[LogOffsetMetadata] = { if (newHighWatermark.messageOffset > logEndOffset) throw new IllegalArgumentException(s"High watermark $newHighWatermark update exceeds current " + - s"log end offset $logEndOffsetMetadata") + s"log end offset ${localLog.logEndOffsetMetadata}") lock.synchronized { val oldHighWatermark = fetchHighWatermarkMetadata @@ -448,7 +430,7 @@ class Log(@volatile private var _dir: File, * known, this will do a lookup in the index and cache the result. */ private def fetchHighWatermarkMetadata: LogOffsetMetadata = { - checkIfMemoryMappedBufferClosed() + localLog.checkIfMemoryMappedBufferClosed() val offsetMetadata = highWatermarkMetadata if (offsetMetadata.messageOffsetOnly) { @@ -487,7 +469,7 @@ class Log(@volatile private var _dir: File, private[log] def firstUnstableOffset: Option[Long] = firstUnstableOffsetMetadata.map(_.messageOffset) private def fetchLastStableOffsetMetadata: LogOffsetMetadata = { - checkIfMemoryMappedBufferClosed() + localLog.checkIfMemoryMappedBufferClosed() // cache the current high watermark to avoid a concurrent update invalidating the range check val highWatermarkMetadata = fetchHighWatermarkMetadata @@ -535,7 +517,7 @@ class Log(@volatile private var _dir: File, LogOffsetSnapshot( logStartOffset, - logEndOffsetMetadata, + localLog.logEndOffsetMetadata, highWatermark, lastStable ) @@ -557,9 +539,6 @@ class Log(@volatile private var _dir: File, } }, period = producerIdExpirationCheckIntervalMs, delay = producerIdExpirationCheckIntervalMs, unit = TimeUnit.MILLISECONDS) - /** The name of this log */ - def name = dir.getName() - private def recordVersion: RecordVersion = config.messageFormatVersion.recordVersion private def initializePartitionMetadata(): Unit = lock synchronized { @@ -579,17 +558,11 @@ class Log(@volatile private var _dir: File, leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion, logIdent) } - private def updateLogEndOffset(offset: Long): Unit = { - nextOffsetMetadata = LogOffsetMetadata(offset, activeSegment.baseOffset, activeSegment.size) - + private def updateHighWatermarkWithLogEndOffset(): Unit = { // Update the high watermark in case it has gotten ahead of the log end offset following a truncation // or if a new segment has been rolled and the offset metadata needs to be updated. - if (highWatermark >= offset) { - updateHighWatermarkMetadata(nextOffsetMetadata) - } - - if (this.recoveryPoint > offset) { - this.recoveryPoint = offset + if (highWatermark >= localLog.logEndOffset) { + updateHighWatermarkMetadata(localLog.logEndOffsetMetadata) } } @@ -600,8 +573,8 @@ class Log(@volatile private var _dir: File, updateHighWatermark(offset) } - if (this.recoveryPoint < offset) { - this.recoveryPoint = offset + if (localLog.recoveryPoint < offset) { + localLog.updateRecoveryPoint(offset) } } @@ -609,8 +582,8 @@ class Log(@volatile private var _dir: File, // free of all side-effects, i.e. it must not update any log-specific state. private def rebuildProducerState(lastOffset: Long, producerStateManager: ProducerStateManager): Unit = lock synchronized { - checkIfMemoryMappedBufferClosed() - Log.rebuildProducerState(producerStateManager, segments, logStartOffset, lastOffset, recordVersion, time, + localLog.checkIfMemoryMappedBufferClosed() + Log.rebuildProducerState(producerStateManager, localLog.segments, logStartOffset, lastOffset, recordVersion, time, reloadFromCleanShutdown = false, logIdent) } @@ -646,7 +619,7 @@ class Log(@volatile private var _dir: File, * The number of segments in the log. * Take care! this is an O(n) operation. */ - def numberOfSegments: Int = segments.numberOfSegments + def numberOfSegments: Int = localLog.segments.numberOfSegments /** * Close this log. @@ -655,32 +628,27 @@ class Log(@volatile private var _dir: File, def close(): Unit = { debug("Closing log") lock synchronized { - checkIfMemoryMappedBufferClosed() + localLog.checkIfMemoryMappedBufferClosed() producerExpireCheck.cancel(true) maybeHandleIOException(s"Error while renaming dir for $topicPartition in dir ${dir.getParent}") { // We take a snapshot at the last written offset to hopefully avoid the need to scan the log // after restarting and to ensure that we cannot inadvertently hit the upgrade optimization // (the clean shutdown file is written after the logs are all closed). producerStateManager.takeSnapshot() - segments.close() } + localLog.close() } } /** - * Rename the directory of the log + * Rename the directory of the local log * * @throws KafkaStorageException if rename fails */ def renameDir(name: String): Unit = { lock synchronized { maybeHandleIOException(s"Error while renaming dir for $topicPartition in log dir ${dir.getParent}") { - val renamedDir = new File(dir.getParent, name) - Utils.atomicMoveWithFallback(dir.toPath, renamedDir.toPath) - if (renamedDir != dir) { - _dir = renamedDir - _parentDir = renamedDir.getParent - segments.updateParentDir(renamedDir) + if (localLog.renameDir(name)) { producerStateManager.updateParentDir(dir) // re-initialize leader epoch cache so that LeaderEpochCheckpointFile.checkpoint can correctly reference // the checkpoint file in renamed log directory @@ -692,18 +660,17 @@ class Log(@volatile private var _dir: File, } /** - * Close file handlers used by log but don't write to disk. This is called if the log directory is offline + * Close file handlers used by this log but don't write to disk. This is called if the log directory is offline */ def closeHandlers(): Unit = { debug("Closing handlers") lock synchronized { - segments.closeHandlers() - isMemoryMappedBufferClosed = true + localLog.closeHandlers() } } /** - * Append this message set to the active segment of the log, assigning offsets and Partition Leader Epochs + * Append this message set to the active segment of the local log, assigning offsets and Partition Leader Epochs * * @param records The records to append * @param origin Declares the origin of the append which affects required validations @@ -722,7 +689,7 @@ class Log(@volatile private var _dir: File, } /** - * Append this message set to the active segment of the log without assigning offsets or Partition Leader Epochs + * Append this message set to the active segment of the local log without assigning offsets or Partition Leader Epochs * * @param records The records to append * @throws KafkaStorageException If the append fails due to an I/O error. @@ -740,7 +707,7 @@ class Log(@volatile private var _dir: File, } /** - * Append this message set to the active segment of the log, rolling over to a fresh segment if necessary. + * Append this message set to the active segment of the local log, rolling over to a fresh segment if necessary. * * This method will generally be responsible for assigning offsets to the messages, * however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid. @@ -777,10 +744,10 @@ class Log(@volatile private var _dir: File, // they are valid, insert them in the log lock synchronized { maybeHandleIOException(s"Error while appending records to $topicPartition in dir ${dir.getParent}") { - checkIfMemoryMappedBufferClosed() + localLog.checkIfMemoryMappedBufferClosed() if (validateAndAssignOffsets) { // assign offsets to the message set - val offset = new LongRef(nextOffsetMetadata.messageOffset) + val offset = new LongRef(localLog.logEndOffset) appendInfo.firstOffset = Some(LogOffsetMetadata(offset.value)) val now = time.milliseconds val validateAndOffsetAssignResult = try { @@ -833,7 +800,7 @@ class Log(@volatile private var _dir: File, throw new OffsetsOutOfOrderException(s"Out of order offsets found in append to $topicPartition: " + records.records.asScala.map(_.offset)) - if (appendInfo.firstOrLastOffsetOfFirstBatch < nextOffsetMetadata.messageOffset) { + if (appendInfo.firstOrLastOffsetOfFirstBatch < localLog.logEndOffset) { // we may still be able to recover if the log is empty // one example: fetching from log start offset on the leader which is not batch aligned, // which may happen as a result of AdminClient#deleteRecords() @@ -845,7 +812,7 @@ class Log(@volatile private var _dir: File, val firstOrLast = if (appendInfo.firstOffset.isDefined) "First offset" else "Last offset of the first batch" throw new UnexpectedAppendOffsetException( s"Unexpected offset in append to $topicPartition. $firstOrLast " + - s"${appendInfo.firstOrLastOffsetOfFirstBatch} is less than the next offset ${nextOffsetMetadata.messageOffset}. " + + s"${appendInfo.firstOrLastOffsetOfFirstBatch} is less than the next offset ${localLog.logEndOffset}. " + s"First 10 offsets in append: ${records.records.asScala.take(10).map(_.offset)}, last offset in" + s" append: ${appendInfo.lastOffset}. Log start offset = $logStartOffset", firstOffset, appendInfo.lastOffset) @@ -898,18 +865,14 @@ class Log(@volatile private var _dir: File, offsetMetadata.copy(segmentBaseOffset = segment.baseOffset, relativePositionInSegment = segment.size) } - segment.append(largestOffset = appendInfo.lastOffset, - largestTimestamp = appendInfo.maxTimestamp, - shallowOffsetOfMaxTimestamp = appendInfo.offsetOfMaxTimestamp, - records = validRecords) - - // Increment the log end offset. We do this immediately after the append because a + // Append the records, and increment the local log end offset immediately after the append because a // write to the transaction index below may fail and we want to ensure that the offsets // of future appends still grow monotonically. The resulting transaction index inconsistency // will be cleaned up after the log directory is recovered. Note that the end offset of the // ProducerStateManager will not be updated and the last stable offset will not advance // if the append to the transaction index fails. - updateLogEndOffset(appendInfo.lastOffset + 1) + localLog.append(appendInfo.lastOffset, appendInfo.maxTimestamp, appendInfo.offsetOfMaxTimestamp, validRecords) + updateHighWatermarkWithLogEndOffset() // update the producer state updatedProducers.values.foreach(producerAppendInfo => producerStateManager.update(producerAppendInfo)) @@ -931,10 +894,10 @@ class Log(@volatile private var _dir: File, trace(s"Appended message set with last offset: ${appendInfo.lastOffset}, " + s"first offset: ${appendInfo.firstOffset}, " + - s"next offset: ${nextOffsetMetadata.messageOffset}, " + + s"next offset: ${localLog.logEndOffset}, " + s"and messages: $validRecords") - if (unflushedMessages >= config.flushInterval) flush() + if (localLog.unflushedMessages >= config.flushInterval) flush() } appendInfo } @@ -961,7 +924,7 @@ class Log(@volatile private var _dir: File, } private def maybeIncrementFirstUnstableOffset(): Unit = lock synchronized { - checkIfMemoryMappedBufferClosed() + localLog.checkIfMemoryMappedBufferClosed() val updatedFirstStableOffset = producerStateManager.firstUnstableOffset match { case Some(logOffsetMetadata) if logOffsetMetadata.messageOffsetOnly || logOffsetMetadata.messageOffset < logStartOffset => @@ -997,7 +960,7 @@ class Log(@volatile private var _dir: File, throw new OffsetOutOfRangeException(s"Cannot increment the log start offset to $newLogStartOffset of partition $topicPartition " + s"since it is larger than the high watermark $highWatermark") - checkIfMemoryMappedBufferClosed() + localLog.checkIfMemoryMappedBufferClosed() if (newLogStartOffset > logStartOffset) { updatedLogStartOffset = true updateLogStartOffset(newLogStartOffset) @@ -1172,15 +1135,10 @@ class Log(@volatile private var _dir: File, } } - private def emptyFetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, - includeAbortedTxns: Boolean): FetchDataInfo = { - val abortedTransactions = - if (includeAbortedTxns) Some(List.empty[FetchResponseData.AbortedTransaction]) - else None - FetchDataInfo(fetchOffsetMetadata, - MemoryRecords.EMPTY, - firstEntryIncomplete = false, - abortedTransactions = abortedTransactions) + private def checkLogStartOffset(offset: Long): Unit = { + if (offset < logStartOffset) + throw new OffsetOutOfRangeException(s"Received request for offset $offset for partition $topicPartition, " + + s"but we only have log segments starting from offset: $logStartOffset.") } /** @@ -1197,105 +1155,17 @@ class Log(@volatile private var _dir: File, maxLength: Int, isolation: FetchIsolation, minOneMessage: Boolean): FetchDataInfo = { - maybeHandleIOException(s"Exception while reading from $topicPartition in dir ${dir.getParent}") { - trace(s"Reading maximum $maxLength bytes at offset $startOffset from log with " + - s"total length $size bytes") - - val includeAbortedTxns = isolation == FetchTxnCommitted - - // Because we don't use the lock for reading, the synchronization is a little bit tricky. - // We create the local variables to avoid race conditions with updates to the log. - val endOffsetMetadata = nextOffsetMetadata - val endOffset = endOffsetMetadata.messageOffset - var segmentOpt = segments.floorSegment(startOffset) - - // return error on attempt to read beyond the log end offset or read below log start offset - if (startOffset > endOffset || segmentOpt.isEmpty || startOffset < logStartOffset) - throw new OffsetOutOfRangeException(s"Received request for offset $startOffset for partition $topicPartition, " + - s"but we only have log segments in the range $logStartOffset to $endOffset.") - - val maxOffsetMetadata = isolation match { - case FetchLogEnd => endOffsetMetadata - case FetchHighWatermark => fetchHighWatermarkMetadata - case FetchTxnCommitted => fetchLastStableOffsetMetadata - } - - if (startOffset == maxOffsetMetadata.messageOffset) - emptyFetchDataInfo(maxOffsetMetadata, includeAbortedTxns) - else if (startOffset > maxOffsetMetadata.messageOffset) - emptyFetchDataInfo(convertToOffsetMetadataOrThrow(startOffset), includeAbortedTxns) - else { - // Do the read on the segment with a base offset less than the target offset - // but if that segment doesn't contain any messages with an offset greater than that - // continue to read from successive segments until we get some messages or we reach the end of the log - var fetchDataInfo: FetchDataInfo = null - while (fetchDataInfo == null && segmentOpt.isDefined) { - val segment = segmentOpt.get - val baseOffset = segment.baseOffset - - val maxPosition = - // Use the max offset position if it is on this segment; otherwise, the segment size is the limit. - if (maxOffsetMetadata.segmentBaseOffset == segment.baseOffset) maxOffsetMetadata.relativePositionInSegment - else segment.size - - fetchDataInfo = segment.read(startOffset, maxLength, maxPosition, minOneMessage) - if (fetchDataInfo != null) { - if (includeAbortedTxns) - fetchDataInfo = addAbortedTransactions(startOffset, segment, fetchDataInfo) - } else segmentOpt = segments.higherSegment(baseOffset) - } - - if (fetchDataInfo != null) fetchDataInfo - else { - // okay we are beyond the end of the last segment with no data fetched although the start offset is in range, - // this can happen when all messages with offset larger than start offsets have been deleted. - // In this case, we will return the empty set with log end offset metadata - FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY) - } - } + checkLogStartOffset(startOffset) + val maxOffsetMetadata = isolation match { + case FetchLogEnd => localLog.logEndOffsetMetadata + case FetchHighWatermark => fetchHighWatermarkMetadata + case FetchTxnCommitted => fetchLastStableOffsetMetadata } + localLog.read(startOffset, maxLength, minOneMessage, maxOffsetMetadata, isolation == FetchTxnCommitted) } private[log] def collectAbortedTransactions(startOffset: Long, upperBoundOffset: Long): List[AbortedTxn] = { - val segmentEntry = segments.floorSegment(startOffset) - val allAbortedTxns = ListBuffer.empty[AbortedTxn] - def accumulator(abortedTxns: List[AbortedTxn]): Unit = allAbortedTxns ++= abortedTxns - segmentEntry.foreach(segment => collectAbortedTransactions(logStartOffset, upperBoundOffset, segment, accumulator)) - allAbortedTxns.toList - } - - private def addAbortedTransactions(startOffset: Long, segment: LogSegment, - fetchInfo: FetchDataInfo): FetchDataInfo = { - val fetchSize = fetchInfo.records.sizeInBytes - val startOffsetPosition = OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset, - fetchInfo.fetchOffsetMetadata.relativePositionInSegment) - val upperBoundOffset = segment.fetchUpperBoundOffset(startOffsetPosition, fetchSize).getOrElse { - segments.higherSegment(segment.baseOffset).map(_.baseOffset).getOrElse(logEndOffset) - } - - val abortedTransactions = ListBuffer.empty[FetchResponseData.AbortedTransaction] - def accumulator(abortedTxns: List[AbortedTxn]): Unit = abortedTransactions ++= abortedTxns.map(_.asAbortedTransaction) - collectAbortedTransactions(startOffset, upperBoundOffset, segment, accumulator) - - FetchDataInfo(fetchOffsetMetadata = fetchInfo.fetchOffsetMetadata, - records = fetchInfo.records, - firstEntryIncomplete = fetchInfo.firstEntryIncomplete, - abortedTransactions = Some(abortedTransactions.toList)) - } - - private def collectAbortedTransactions(startOffset: Long, upperBoundOffset: Long, - startingSegment: LogSegment, - accumulator: List[AbortedTxn] => Unit): Unit = { - val higherSegments = segments.higherSegments(startingSegment.baseOffset).iterator - var segmentEntryOpt = Option(startingSegment) - while (segmentEntryOpt.isDefined) { - val segment = segmentEntryOpt.get - val searchResult = segment.collectAbortedTxns(startOffset, upperBoundOffset) - accumulator(searchResult.abortedTransactions) - if (searchResult.isComplete) - return - segmentEntryOpt = nextOption(higherSegments) - } + localLog.collectAbortedTransactions(logStartOffset, startOffset, upperBoundOffset) } /** @@ -1410,25 +1280,30 @@ class Log(@volatile private var _dir: File, * If the message offset is out of range, throw an OffsetOutOfRangeException */ private def convertToOffsetMetadataOrThrow(offset: Long): LogOffsetMetadata = { - val fetchDataInfo = read(offset, - maxLength = 1, - isolation = FetchLogEnd, - minOneMessage = false) - fetchDataInfo.fetchOffsetMetadata + checkLogStartOffset(offset) + localLog.convertToOffsetMetadataOrThrow(offset) } /** - * Delete any log segments matching the given predicate function, - * starting with the oldest segment and moving forward until a segment doesn't match. + * Delete any local log segments starting with the oldest segment and moving forward until until + * the user-supplied predicate is false or the segment containing the current high watermark is reached. + * We do not delete segments with offsets at or beyond the high watermark to ensure that the log start + * offset can never exceed it. If the high watermark has not yet been initialized, no segments are eligible + * for deletion. * * @param predicate A function that takes in a candidate log segment and the next higher segment * (if there is one) and returns true iff it is deletable + * @param reason The reason for the segment deletion * @return The number of segments deleted */ private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean, reason: SegmentDeletionReason): Int = { + def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = { + highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) && + predicate(segment, nextSegmentOpt) + } lock synchronized { - val deletable = deletableSegments(predicate) + val deletable = localLog.deletableSegments(shouldDelete) if (deletable.nonEmpty) deleteSegments(deletable, reason) else @@ -1441,13 +1316,14 @@ class Log(@volatile private var _dir: File, val numToDelete = deletable.size if (numToDelete > 0) { // we must always have at least one segment, so if we are going to delete all the segments, create a new one first - if (numberOfSegments == numToDelete) + if (localLog.segments.numberOfSegments == numToDelete) roll() lock synchronized { - checkIfMemoryMappedBufferClosed() + localLog.checkIfMemoryMappedBufferClosed() // remove the segments for lookups - removeAndDeleteSegments(deletable, asyncDelete = true, reason) - maybeIncrementLogStartOffset(segments.firstSegment.get.baseOffset, SegmentDeletion) + localLog.removeAndDeleteSegments(deletable, asyncDelete = true, reason) + deleteProducerSnapshots(deletable, asyncDelete = true) + maybeIncrementLogStartOffset(localLog.segments.firstSegmentBaseOffset.get, SegmentDeletion) } } numToDelete @@ -1455,50 +1331,10 @@ class Log(@volatile private var _dir: File, } /** - * Find segments starting from the oldest until the user-supplied predicate is false or the segment - * containing the current high watermark is reached. We do not delete segments with offsets at or beyond - * the high watermark to ensure that the log start offset can never exceed it. If the high watermark - * has not yet been initialized, no segments are eligible for deletion. - * - * A final segment that is empty will never be returned (since we would just end up re-creating it). - * - * @param predicate A function that takes in a candidate log segment and the next higher segment - * (if there is one) and returns true iff it is deletable - * @return the segments ready to be deleted - */ - private def deletableSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean): Iterable[LogSegment] = { - if (segments.isEmpty) { - Seq.empty - } else { - val deletable = ArrayBuffer.empty[LogSegment] - val segmentsIterator = segments.values.iterator - var segmentOpt = nextOption(segmentsIterator) - while (segmentOpt.isDefined) { - val segment = segmentOpt.get - val nextSegmentOpt = nextOption(segmentsIterator) - val (upperBoundOffset: Long, isLastSegmentAndEmpty: Boolean) = - nextSegmentOpt.map { - nextSegment => (nextSegment.baseOffset, false) - }.getOrElse { - (logEndOffset, segment.size == 0) - } - - if (highWatermark >= upperBoundOffset && predicate(segment, nextSegmentOpt) && !isLastSegmentAndEmpty) { - deletable += segment - segmentOpt = nextSegmentOpt - } else { - segmentOpt = Option.empty - } - } - deletable - } - } - - /** - * If topic deletion is enabled, delete any log segments that have either expired due to time based retention + * If topic deletion is enabled, delete any local log segments that have either expired due to time based retention * or because the log size is > retentionSize. * - * Whether or not deletion is enabled, delete any log segments that are before the log start offset + * Whether or not deletion is enabled, delete any local log segments that are before the log start offset */ def deleteOldSegments(): Int = { if (config.delete) { @@ -1518,7 +1354,7 @@ class Log(@volatile private var _dir: File, startMs - segment.largestTimestamp > config.retentionMs } - deleteOldSegments(shouldDelete, RetentionMsBreach) + deleteOldSegments(shouldDelete, RetentionMsBreach(this)) } private def deleteRetentionSizeBreachedSegments(): Int = { @@ -1533,7 +1369,7 @@ class Log(@volatile private var _dir: File, } } - deleteOldSegments(shouldDelete, RetentionSizeBreach) + deleteOldSegments(shouldDelete, RetentionSizeBreach(this)) } private def deleteLogStartOffsetBreachedSegments(): Int = { @@ -1541,42 +1377,41 @@ class Log(@volatile private var _dir: File, nextSegmentOpt.exists(_.baseOffset <= logStartOffset) } - deleteOldSegments(shouldDelete, StartOffsetBreach) + deleteOldSegments(shouldDelete, StartOffsetBreach(this)) } - def isFuture: Boolean = dir.getName.endsWith(Log.FutureDirSuffix) + def isFuture: Boolean = localLog.isFuture /** * The size of the log in bytes */ - def size: Long = Log.sizeInBytes(logSegments) + def size: Long = localLog.segments.sizeInBytes /** - * The offset metadata of the next message that will be appended to the log + * The offset of the next message that will be appended to the log */ - def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata + def logEndOffset: Long = localLog.logEndOffset /** - * The offset of the next message that will be appended to the log + * The offset metadata of the next message that will be appended to the log */ - def logEndOffset: Long = nextOffsetMetadata.messageOffset + def logEndOffsetMetadata: LogOffsetMetadata = localLog.logEndOffsetMetadata /** * Roll the log over to a new empty log segment if necessary. + * The segment will be rolled if one of the following conditions met: + * 1. The logSegment is full + * 2. The maxTime has elapsed since the timestamp of first message in the segment (or since the + * create time if the first message does not have a timestamp) + * 3. The index is full * * @param messagesSize The messages set size in bytes. * @param appendInfo log append information - * logSegment will be rolled if one of the following conditions met - *
    - *
  1. The logSegment is full - *
  2. The maxTime has elapsed since the timestamp of first message in the segment (or since the create time if - * the first message does not have a timestamp) - *
  3. The index is full - *
- * @return The currently active segment after (perhaps) rolling to a new segment + * + * @return The currently active segment after (perhaps) rolling to a new segment */ - private def maybeRoll(messagesSize: Int, appendInfo: LogAppendInfo): LogSegment = { - val segment = activeSegment + private def maybeRoll(messagesSize: Int, appendInfo: LogAppendInfo): LogSegment = lock synchronized { + val segment = localLog.segments.activeSegment val now = time.milliseconds val maxTimestampInMessages = appendInfo.maxTimestamp @@ -1612,137 +1447,69 @@ class Log(@volatile private var _dir: File, } /** - * Roll the log over to a new active segment starting with the current logEndOffset. - * This will trim the index to the exact size of the number of entries it currently contains. + * Roll the local log over to a new active segment starting with the expectedNextOffset (when provided), + * or localLog.logEndOffset otherwise. This will trim the index to the exact size of the number of entries + * it currently contains. * * @return The newly rolled segment */ - def roll(expectedNextOffset: Option[Long] = None): LogSegment = { - maybeHandleIOException(s"Error while rolling log segment for $topicPartition in dir ${dir.getParent}") { - val start = time.hiResClockMs() - lock synchronized { - checkIfMemoryMappedBufferClosed() - val newOffset = math.max(expectedNextOffset.getOrElse(0L), logEndOffset) - val logFile = Log.logFile(dir, newOffset) - - if (segments.contains(newOffset)) { - // segment with the same base offset already exists and loaded - if (activeSegment.baseOffset == newOffset && activeSegment.size == 0) { - // We have seen this happen (see KAFKA-6388) after shouldRoll() returns true for an - // active segment of size zero because of one of the indexes is "full" (due to _maxEntries == 0). - warn(s"Trying to roll a new log segment with start offset $newOffset " + - s"=max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already " + - s"exists and is active with size 0. Size of time index: ${activeSegment.timeIndex.entries}," + - s" size of offset index: ${activeSegment.offsetIndex.entries}.") - removeAndDeleteSegments(Seq(activeSegment), asyncDelete = true, LogRoll) - } else { - throw new KafkaException(s"Trying to roll a new log segment for topic partition $topicPartition with start offset $newOffset" + - s" =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) while it already exists. Existing " + - s"segment is ${segments.get(newOffset)}.") - } - } else if (!segments.isEmpty && newOffset < activeSegment.baseOffset) { - throw new KafkaException( - s"Trying to roll a new log segment for topic partition $topicPartition with " + - s"start offset $newOffset =max(provided offset = $expectedNextOffset, LEO = $logEndOffset) lower than start offset of the active segment $activeSegment") - } else { - val offsetIdxFile = offsetIndexFile(dir, newOffset) - val timeIdxFile = timeIndexFile(dir, newOffset) - val txnIdxFile = transactionIndexFile(dir, newOffset) - - for (file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) { - warn(s"Newly rolled segment file ${file.getAbsolutePath} already exists; deleting it first") - Files.delete(file.toPath) - } - - segments.lastSegment.foreach(_.onBecomeInactiveSegment()) - } - - // take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot - // offset align with the new segment offset since this ensures we can recover the segment by beginning - // with the corresponding snapshot file and scanning the segment data. Because the segment base offset - // may actually be ahead of the current producer state end offset (which corresponds to the log end offset), - // we manually override the state offset here prior to taking the snapshot. - producerStateManager.updateMapEndOffset(newOffset) - producerStateManager.takeSnapshot() - - val segment = LogSegment.open(dir, - baseOffset = newOffset, - config, - time = time, - initFileSize = config.initFileSize, - preallocate = config.preallocate) - addSegment(segment) - - // We need to update the segment base offset and append position data of the metadata when log rolls. - // The next offset should not change. - updateLogEndOffset(nextOffsetMetadata.messageOffset) - - // schedule an asynchronous flush of the old segment - scheduler.schedule("flush-log", () => flush(newOffset), delay = 0L) - - info(s"Rolled new log segment at offset $newOffset in ${time.hiResClockMs() - start} ms.") - - segment - } - } + def roll(expectedNextOffset: Option[Long] = None): LogSegment = lock synchronized { + val newSegment = localLog.roll(expectedNextOffset) + // Take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot + // offset align with the new segment offset since this ensures we can recover the segment by beginning + // with the corresponding snapshot file and scanning the segment data. Because the segment base offset + // may actually be ahead of the current producer state end offset (which corresponds to the log end offset), + // we manually override the state offset here prior to taking the snapshot. + producerStateManager.updateMapEndOffset(newSegment.baseOffset) + producerStateManager.takeSnapshot() + updateHighWatermarkWithLogEndOffset() + // Schedule an asynchronous flush of the old segment + scheduler.schedule("flush-log", () => flush(newSegment.baseOffset)) + newSegment } /** - * The number of messages appended to the log since the last flush - */ - private def unflushedMessages: Long = this.logEndOffset - this.recoveryPoint - - /** - * Flush all log segments + * Flush all local log segments */ - def flush(): Unit = flush(this.logEndOffset) + def flush(): Unit = flush(logEndOffset) /** - * Flush log segments for all offsets up to offset-1 + * Flush local log segments for all offsets up to offset-1 * * @param offset The offset to flush up to (non-inclusive); the new recovery point */ def flush(offset: Long): Unit = { maybeHandleIOException(s"Error while flushing log for $topicPartition in dir ${dir.getParent} with offset $offset") { - if (offset > this.recoveryPoint) { + if (offset > localLog.recoveryPoint) { debug(s"Flushing log up to offset $offset, last flushed: $lastFlushTime, current time: ${time.milliseconds()}, " + - s"unflushed: $unflushedMessages") - val segments = logSegments(this.recoveryPoint, offset) - segments.foreach(_.flush()) - // if there are any new segments, we need to flush the parent directory for crash consistency - segments.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath)) - + s"unflushed: ${localLog.unflushedMessages}") + localLog.flush(offset) lock synchronized { - checkIfMemoryMappedBufferClosed() - if (offset > this.recoveryPoint) { - this.recoveryPoint = offset - lastFlushedTime.set(time.milliseconds) - } + localLog.markFlushed(offset) } } } } /** - * Completely delete this log directory and all contents from the file system with no delay + * Completely delete the local log directory and all contents from the file system with no delay */ private[log] def delete(): Unit = { maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") { lock synchronized { - checkIfMemoryMappedBufferClosed() + localLog.checkIfMemoryMappedBufferClosed() producerExpireCheck.cancel(true) - removeAndDeleteSegments(logSegments, asyncDelete = false, LogDeletion) leaderEpochCache.foreach(_.clear()) - Utils.delete(dir) - // File handlers will be closed if this log is deleted - isMemoryMappedBufferClosed = true + val deletedSegments = localLog.deleteAllSegments() + deleteProducerSnapshots(deletedSegments, asyncDelete = false) + localLog.deleteEmptyDir() } } } // visible for testing private[log] def takeProducerSnapshot(): Unit = lock synchronized { - checkIfMemoryMappedBufferClosed() + localLog.checkIfMemoryMappedBufferClosed() producerStateManager.takeSnapshot() } @@ -1771,8 +1538,8 @@ class Log(@volatile private var _dir: File, maybeHandleIOException(s"Error while truncating log to offset $targetOffset for $topicPartition in dir ${dir.getParent}") { if (targetOffset < 0) throw new IllegalArgumentException(s"Cannot truncate partition $topicPartition to a negative offset (%d).".format(targetOffset)) - if (targetOffset >= logEndOffset) { - info(s"Truncating to $targetOffset has no effect as the largest offset in the log is ${logEndOffset - 1}") + if (targetOffset >= localLog.logEndOffset) { + info(s"Truncating to $targetOffset has no effect as the largest offset in the log is ${localLog.logEndOffset - 1}") // Always truncate epoch cache since we may have a conflicting epoch entry at the // end of the log from the leader. This could happen if this broker was a leader @@ -1786,19 +1553,17 @@ class Log(@volatile private var _dir: File, } else { info(s"Truncating to offset $targetOffset") lock synchronized { - checkIfMemoryMappedBufferClosed() - if (segments.firstSegment.get.baseOffset > targetOffset) { + localLog.checkIfMemoryMappedBufferClosed() + if (localLog.segments.firstSegmentBaseOffset.get > targetOffset) { truncateFullyAndStartAt(targetOffset) } else { - val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset) - removeAndDeleteSegments(deletable, asyncDelete = true, LogTruncation) - activeSegment.truncateTo(targetOffset) + val deletedSegments = localLog.truncateTo(targetOffset) + deleteProducerSnapshots(deletedSegments, asyncDelete = true) leaderEpochCache.foreach(_.truncateFromEnd(targetOffset)) - - completeTruncation( - startOffset = math.min(targetOffset, logStartOffset), - endOffset = targetOffset - ) + logStartOffset = math.min(targetOffset, logStartOffset) + rebuildProducerState(targetOffset, producerStateManager) + if (highWatermark >= localLog.logEndOffset) + updateHighWatermark(localLog.logEndOffsetMetadata) } true } @@ -1815,61 +1580,41 @@ class Log(@volatile private var _dir: File, maybeHandleIOException(s"Error while truncating the entire log for $topicPartition in dir ${dir.getParent}") { debug(s"Truncate and start at offset $newOffset") lock synchronized { - checkIfMemoryMappedBufferClosed() - removeAndDeleteSegments(logSegments, asyncDelete = true, LogTruncation) - addSegment(LogSegment.open(dir, - baseOffset = newOffset, - config = config, - time = time, - initFileSize = config.initFileSize, - preallocate = config.preallocate)) + localLog.truncateFullyAndStartAt(newOffset) leaderEpochCache.foreach(_.clearAndFlush()) producerStateManager.truncateFullyAndStartAt(newOffset) - - completeTruncation( - startOffset = newOffset, - endOffset = newOffset - ) + logStartOffset = newOffset + rebuildProducerState(newOffset, producerStateManager) + updateHighWatermark(localLog.logEndOffsetMetadata) } } } - private def completeTruncation( - startOffset: Long, - endOffset: Long - ): Unit = { - logStartOffset = startOffset - nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size) - recoveryPoint = math.min(recoveryPoint, endOffset) - rebuildProducerState(endOffset, producerStateManager) - updateHighWatermark(math.min(highWatermark, endOffset)) - } - /** * The time this log is last known to have been fully flushed to disk */ - def lastFlushTime: Long = lastFlushedTime.get + def lastFlushTime: Long = localLog.lastFlushTime /** * The active segment that is currently taking appends */ - def activeSegment = segments.lastSegment.get + def activeSegment: LogSegment = localLog.segments.activeSegment /** * All the log segments in this log ordered from oldest to newest */ - def logSegments: Iterable[LogSegment] = segments.values + def logSegments: Iterable[LogSegment] = localLog.segments.values /** * Get all segments beginning with the segment that includes "from" and ending with the segment * that includes up to "to-1" or the end of the log (if to > logEndOffset). */ def logSegments(from: Long, to: Long): Iterable[LogSegment] = lock synchronized { - segments.values(from, to) + localLog.segments.values(from, to) } def nonActiveLogSegmentsFrom(from: Long): Iterable[LogSegment] = lock synchronized { - segments.nonActiveLogSegmentsFrom(from) + localLog.segments.nonActiveLogSegmentsFrom(from) } override def toString: String = { @@ -1885,66 +1630,24 @@ class Log(@volatile private var _dir: File, logString.toString } - /** - * This method deletes the given log segments by doing the following for each of them: - *
    - *
  1. It removes the segment from the segment map so that it will no longer be used for reads. - *
  2. It renames the index and log files by appending .deleted to the respective file name - *
  3. 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 segments The log segments to schedule for deletion - * @param asyncDelete Whether the segment files should be deleted asynchronously - */ - private def removeAndDeleteSegments(segments: Iterable[LogSegment], - asyncDelete: Boolean, - reason: SegmentDeletionReason): Unit = { - if (segments.nonEmpty) { - lock synchronized { - // Most callers hold an iterator into the `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. We should also pass only the materialized view of the - // iterator to the logic that actually deletes the segments. - val toDelete = segments.toList - reason.logReason(this, toDelete) - toDelete.foreach { segment => - this.segments.remove(segment.baseOffset) - } - deleteSegmentFiles(toDelete, asyncDelete) - } - } - } - - private def deleteSegmentFiles(segments: immutable.Iterable[LogSegment], asyncDelete: Boolean, deleteProducerStateSnapshots: Boolean = true): Unit = { - Log.deleteSegmentFiles(segments, asyncDelete, deleteProducerStateSnapshots, dir, topicPartition, - config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent) - } - - private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = { + private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment]): Unit = { lock synchronized { - checkIfMemoryMappedBufferClosed() - Log.replaceSegments(segments, newSegments, oldSegments, isRecoveredSwapFile, dir, topicPartition, - config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent) + localLog.checkIfMemoryMappedBufferClosed() + val deletedSegments = Log.replaceSegments(localLog.segments, newSegments, oldSegments, dir, topicPartition, + config, scheduler, logDirFailureChannel, logIdent) + deleteProducerSnapshots(deletedSegments, asyncDelete = true) } } /** - * This function does not acquire Log.lock. The caller has to make sure log segments don't get deleted during - * this call, and also protects against calling this function on the same segment in parallel. - * - * Currently, it is used by LogCleaner threads on log compact non-active segments only with LogCleanerManager's lock - * to ensure no other logcleaner threads and retention thread can work on the same segment. - */ + * This function does not acquire Log.lock. The caller has to make sure log segments don't get deleted during + * this call, and also protects against calling this function on the same segment in parallel. + * + * Currently, it is used by LogCleaner threads on log compact non-active segments only with LogCleanerManager's lock + * to ensure no other logcleaner threads and retention thread can work on the same segment. + */ private[log] def getFirstBatchTimestampForSegments(segments: Iterable[LogSegment]): Iterable[Long] = { - segments.map { - segment => - segment.getFirstBatchTimestamp() - } + LogSegments.getFirstBatchTimestampForSegments(segments) } /** @@ -1962,65 +1665,50 @@ class Log(@volatile private var _dir: File, * @param segment The segment to add */ @threadsafe - private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.add(segment) + private[log] def addSegment(segment: LogSegment): LogSegment = localLog.segments.add(segment) private def maybeHandleIOException[T](msg: => String)(fun: => T): T = { - Log.maybeHandleIOException(logDirFailureChannel, parentDir, msg) { + LocalLog.maybeHandleIOException(logDirFailureChannel, parentDir, msg) { fun } } private[log] def splitOverflowedSegment(segment: LogSegment): List[LogSegment] = lock synchronized { - Log.splitOverflowedSegment(segment, segments, dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager, this.logIdent) + val result = Log.splitOverflowedSegment(segment, localLog.segments, dir, topicPartition, config, scheduler, logDirFailureChannel, logIdent) + deleteProducerSnapshots(result.deletedSegments, asyncDelete = true) + result.newSegments.toList } + private[log] def deleteProducerSnapshots(segments: Iterable[LogSegment], asyncDelete: Boolean): Unit = { + Log.deleteProducerSnapshots(segments, producerStateManager, asyncDelete, scheduler, config, logDirFailureChannel, parentDir, topicPartition) + } } -/** - * Helper functions for logs - */ object Log extends Logging { - /** a log file */ - val LogFileSuffix = ".log" + val LogFileSuffix = LocalLog.LogFileSuffix - /** an index file */ - val IndexFileSuffix = ".index" + val IndexFileSuffix = LocalLog.IndexFileSuffix - /** a time index file */ - val TimeIndexFileSuffix = ".timeindex" + val TimeIndexFileSuffix = LocalLog.TimeIndexFileSuffix val ProducerSnapshotFileSuffix = ".snapshot" - /** an (aborted) txn index */ - val TxnIndexFileSuffix = ".txnindex" - - /** a file that is scheduled to be deleted */ - val DeletedFileSuffix = ".deleted" + val TxnIndexFileSuffix = LocalLog.TxnIndexFileSuffix - /** A temporary file that is being used for log cleaning */ - val CleanedFileSuffix = ".cleaned" + val DeletedFileSuffix = LocalLog.DeletedFileSuffix - /** A temporary file used when swapping files into the log */ - val SwapFileSuffix = ".swap" + val CleanedFileSuffix = LocalLog.CleanedFileSuffix - /** Clean shutdown file that indicates the broker was cleanly shutdown in 0.8 and higher. - * This is used to avoid unnecessary recovery after a clean shutdown. In theory this could be - * avoided by passing in the recovery point, however finding the correct position to do this - * requires accessing the offset index which may not be safe in an unclean shutdown. - * For more information see the discussion in PR#2104 - */ - val CleanShutdownFile = ".kafka_cleanshutdown" + val SwapFileSuffix = LocalLog.SwapFileSuffix - /** a directory that is scheduled to be deleted */ - val DeleteDirSuffix = "-delete" + val DeleteDirSuffix = LocalLog.DeleteDirSuffix - /** a directory that is used for future partition */ - val FutureDirSuffix = "-future" + val FutureDirSuffix = LocalLog.FutureDirSuffix - private[log] val DeleteDirPattern = Pattern.compile(s"^(\\S+)-(\\S+)\\.(\\S+)$DeleteDirSuffix") - private[log] val FutureDirPattern = Pattern.compile(s"^(\\S+)-(\\S+)\\.(\\S+)$FutureDirSuffix") + private[log] val DeleteDirPattern = LocalLog.DeleteDirPattern + private[log] val FutureDirPattern = LocalLog.FutureDirPattern - val UnknownOffset = -1L + val UnknownOffset = LocalLog.UnknownOffset def apply(dir: File, config: LogConfig, @@ -2045,7 +1733,7 @@ object Log extends Logging { logDirFailureChannel, config.messageFormatVersion.recordVersion, s"[Log partition=$topicPartition, dir=${dir.getParent}] ") - val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs, time) + val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) val offsets = LogLoader.load(LoadLogParams( dir, topicPartition, @@ -2060,89 +1748,29 @@ object Log extends Logging { maxProducerIdExpirationMs, leaderEpochCache, producerStateManager)) - new Log(dir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, offsets.nextOffsetMetadata, scheduler, - brokerTopicStats, time, producerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, - producerStateManager, logDirFailureChannel, topicId, keepPartitionMetadataFile) - } - - /** - * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros - * so that ls sorts the files numerically. - * - * @param offset The offset to use in the file name - * @return The filename - */ - def filenamePrefixFromOffset(offset: Long): String = { - val nf = NumberFormat.getInstance() - nf.setMinimumIntegerDigits(20) - nf.setMaximumFractionDigits(0) - nf.setGroupingUsed(false) - nf.format(offset) + val localLog = new LocalLog(dir, config, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, scheduler, time, topicPartition, logDirFailureChannel) + new Log(offsets.logStartOffset, + localLog, + brokerTopicStats, + producerIdExpirationCheckIntervalMs, + leaderEpochCache, + producerStateManager, + topicId, + keepPartitionMetadataFile) } - /** - * Construct a log file name in the given dir with the given base offset and the given suffix - * - * @param dir The directory in which the log will reside - * @param offset The base offset of the log file - * @param suffix The suffix to be appended to the file name (e.g. "", ".deleted", ".cleaned", ".swap", etc.) - */ - def logFile(dir: File, offset: Long, suffix: String = ""): File = - new File(dir, filenamePrefixFromOffset(offset) + LogFileSuffix + suffix) - - /** - * Return a directory name to rename the log directory to for async deletion. - * The name will be in the following format: "topic-partitionId.uniqueId-delete". - * If the topic name is too long, it will be truncated to prevent the total name - * from exceeding 255 characters. - */ - def logDeleteDirName(topicPartition: TopicPartition): String = { - val uniqueId = java.util.UUID.randomUUID.toString.replaceAll("-", "") - val suffix = s"-${topicPartition.partition()}.${uniqueId}${DeleteDirSuffix}" - val prefixLength = Math.min(topicPartition.topic().size, 255 - suffix.size) - s"${topicPartition.topic().substring(0, prefixLength)}${suffix}" - } + def logFile(dir: File, offset: Long, suffix: String = ""): File = LocalLog.logFile(dir, offset, suffix) - /** - * Return a future directory name for the given topic partition. The name will be in the following - * format: topic-partition.uniqueId-future where topic, partition and uniqueId are variables. - */ - def logFutureDirName(topicPartition: TopicPartition): String = { - logDirNameWithSuffix(topicPartition, FutureDirSuffix) - } + def logDeleteDirName(topicPartition: TopicPartition): String = LocalLog.logDeleteDirName(topicPartition) - private def logDirNameWithSuffix(topicPartition: TopicPartition, suffix: String): String = { - val uniqueId = java.util.UUID.randomUUID.toString.replaceAll("-", "") - s"${logDirName(topicPartition)}.$uniqueId$suffix" - } + def logFutureDirName(topicPartition: TopicPartition): String = LocalLog.logFutureDirName(topicPartition) - /** - * Return a directory name for the given topic partition. The name will be in the following - * format: topic-partition where topic, partition are variables. - */ - def logDirName(topicPartition: TopicPartition): String = { - s"${topicPartition.topic}-${topicPartition.partition}" - } + def logDirName(topicPartition: TopicPartition): String = LocalLog.logDirName(topicPartition) - /** - * Construct an index file name in the given dir using the given base offset and the given suffix - * - * @param dir The directory in which the log will reside - * @param offset The base offset of the log file - * @param suffix The suffix to be appended to the file name ("", ".deleted", ".cleaned", ".swap", etc.) - */ - def offsetIndexFile(dir: File, offset: Long, suffix: String = ""): File = - new File(dir, filenamePrefixFromOffset(offset) + IndexFileSuffix + suffix) + def offsetIndexFile(dir: File, offset: Long, suffix: String = ""): File = LocalLog.offsetIndexFile(dir, offset, suffix) - /** - * Construct a time index file name in the given dir using the given base offset and the given suffix - * - * @param dir The directory in which the log will reside - * @param offset The base offset of the log file - * @param suffix The suffix to be appended to the file name ("", ".deleted", ".cleaned", ".swap", etc.) - */ - def timeIndexFile(dir: File, offset: Long, suffix: String = ""): File = - new File(dir, filenamePrefixFromOffset(offset) + TimeIndexFileSuffix + suffix) + def timeIndexFile(dir: File, offset: Long, suffix: String = ""): File = LocalLog.timeIndexFile(dir, offset, suffix) def deleteFileIfExists(file: File, suffix: String = ""): Unit = Files.deleteIfExists(new File(file.getPath + suffix).toPath) @@ -2150,83 +1778,25 @@ object Log extends Logging { /** * Construct a producer id snapshot file using the given offset. * - * @param dir The directory in which the log will reside + * @param dir The directory in which the log will reside * @param offset The last offset (exclusive) included in the snapshot */ def producerSnapshotFile(dir: File, offset: Long): File = - new File(dir, filenamePrefixFromOffset(offset) + ProducerSnapshotFileSuffix) - - /** - * Construct a transaction index file name in the given dir using the given base offset and the given suffix - * - * @param dir The directory in which the log will reside - * @param offset The base offset of the log file - * @param suffix The suffix to be appended to the file name ("", ".deleted", ".cleaned", ".swap", etc.) - */ - def transactionIndexFile(dir: File, offset: Long, suffix: String = ""): File = - new File(dir, filenamePrefixFromOffset(offset) + TxnIndexFileSuffix + suffix) - - def offsetFromFileName(filename: String): Long = { - filename.substring(0, filename.indexOf('.')).toLong - } - - def offsetFromFile(file: File): Long = { - offsetFromFileName(file.getName) - } - - /** - * Calculate a log's size (in bytes) based on its log segments - * - * @param segments The log segments to calculate the size of - * @return Sum of the log segments' sizes (in bytes) - */ - def sizeInBytes(segments: Iterable[LogSegment]): Long = - segments.map(_.size.toLong).sum + new File(dir, LocalLog.filenamePrefixFromOffset(offset) + ProducerSnapshotFileSuffix) - /** - * Parse the topic and partition out of the directory name of a log - */ - def parseTopicPartitionName(dir: File): TopicPartition = { - if (dir == null) - throw new KafkaException("dir should not be null") - - def exception(dir: File): KafkaException = { - new KafkaException(s"Found directory ${dir.getCanonicalPath}, '${dir.getName}' is not in the form of " + - "topic-partition or topic-partition.uniqueId-delete (if marked for deletion).\n" + - "Kafka's log directories (and children) should only contain Kafka topic data.") - } + def transactionIndexFile(dir: File, offset: Long, suffix: String = ""): File = LocalLog.transactionIndexFile(dir, offset, suffix) - val dirName = dir.getName - if (dirName == null || dirName.isEmpty || !dirName.contains('-')) - throw exception(dir) - if (dirName.endsWith(DeleteDirSuffix) && !DeleteDirPattern.matcher(dirName).matches || - dirName.endsWith(FutureDirSuffix) && !FutureDirPattern.matcher(dirName).matches) - throw exception(dir) + def offsetFromFileName(filename: String): Long = LocalLog.offsetFromFileName(filename) - val name: String = - if (dirName.endsWith(DeleteDirSuffix) || dirName.endsWith(FutureDirSuffix)) dirName.substring(0, dirName.lastIndexOf('.')) - else dirName + def offsetFromFile(file: File): Long = LocalLog.offsetFromFile(file) - val index = name.lastIndexOf('-') - val topic = name.substring(0, index) - val partitionString = name.substring(index + 1) - if (topic.isEmpty || partitionString.isEmpty) - throw exception(dir) + def sizeInBytes(segments: Iterable[LogSegment]): Long = LogSegments.sizeInBytes(segments) - val partition = - try partitionString.toInt - catch { case _: NumberFormatException => throw exception(dir) } + def parseTopicPartitionName(dir: File): TopicPartition = LocalLog.parseTopicPartitionName(dir) - new TopicPartition(topic, partition) - } + private[log] def isIndexFile(file: File): Boolean = LocalLog.isIndexFile(file) - private[log] def isIndexFile(file: File): Boolean = { - val filename = file.getName - filename.endsWith(IndexFileSuffix) || filename.endsWith(TimeIndexFileSuffix) || filename.endsWith(TxnIndexFileSuffix) - } - - private[log] def isLogFile(file: File): Boolean = - file.getPath.endsWith(LogFileSuffix) + private[log] def isLogFile(file: File): Boolean = LocalLog.isLogFile(file) private def loadProducersFromRecords(producerStateManager: ProducerStateManager, records: Records): Unit = { val loadedProducers = mutable.Map.empty[Long, ProducerAppendInfo] @@ -2261,8 +1831,8 @@ object Log extends Logging { * 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 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 * @param logPrefix The logging prefix @@ -2296,175 +1866,37 @@ object Log extends Logging { } } - /** - * 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 - * @param logPrefix The logging prefix - */ - 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, - logPrefix: String): 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)) - 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, - logPrefix) - } - // okay we are safe now, remove the swap suffix - sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, "")) - Utils.flushDir(dir.toPath) + private[log] def replaceSegments(existingSegments: LogSegments, + newSegments: Seq[LogSegment], + oldSegments: Seq[LogSegment], + dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + logDirFailureChannel: LogDirFailureChannel, + logPrefix: String, + isRecoveredSwapFile: Boolean = false): Iterable[LogSegment] = { + LocalLog.replaceSegments(existingSegments, + newSegments, + oldSegments, + dir, + topicPartition, + config, + scheduler, + logDirFailureChannel, + logPrefix, + isRecoveredSwapFile) } - /** - * 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 - * @param logPrefix The logging prefix - * @throws IOException if the file can't be renamed and still exists - */ private[log] def deleteSegmentFiles(segmentsToDelete: immutable.Iterable[LogSegment], asyncDelete: Boolean, - deleteProducerStateSnapshots: Boolean = true, dir: File, topicPartition: TopicPartition, config: LogConfig, scheduler: Scheduler, logDirFailureChannel: LogDirFailureChannel, - producerStateManager: ProducerStateManager, logPrefix: String): Unit = { - segmentsToDelete.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix)) - val snapshotsToDelete = if (deleteProducerStateSnapshots) - segmentsToDelete.flatMap { segment => - producerStateManager.removeAndMarkSnapshotForDeletion(segment.baseOffset)} - else - Seq() - - def deleteSegments(): Unit = { - info(s"${logPrefix}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() - } - snapshotsToDelete.foreach { snapshot => - snapshot.deleteIfExists() - } - } - } - - 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) - } + LocalLog.deleteSegmentFiles(segmentsToDelete, asyncDelete, dir, topicPartition, config, scheduler, logDirFailureChannel, logPrefix) } /** @@ -2472,12 +1904,12 @@ object Log extends Logging { * 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 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. * @param logPrefix The logging prefix @@ -2566,31 +1998,6 @@ object Log extends Logging { } } - /** - * 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 - * @param logPrefix The logging prefix - * @return List of new segments that replace the input segment - */ private[log] def splitOverflowedSegment(segment: LogSegment, existingSegments: LogSegments, dir: File, @@ -2598,71 +2005,38 @@ object Log extends Logging { config: LogConfig, scheduler: Scheduler, logDirFailureChannel: LogDirFailureChannel, - producerStateManager: ProducerStateManager, - logPrefix: String): List[LogSegment] = { - require(Log.isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}") - require(segment.hasOverflow, s"Split operation is only permitted for segments with overflow, and the problem path is ${segment.log.file.getAbsoluteFile}") - - info(s"${logPrefix}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"${logPrefix}Replacing overflowed segment $segment with split segments $newSegments") - replaceSegments(existingSegments, newSegments.toList, List(segment), isRecoveredSwapFile = false, - dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager, logPrefix) - newSegments.toList - } catch { - case e: Exception => - newSegments.foreach { splitSegment => - splitSegment.close() - splitSegment.deleteIfExists() + logPrefix: String): SplitSegmentResult = { + LocalLog.splitOverflowedSegment(segment, existingSegments, dir, topicPartition, config, scheduler, logDirFailureChannel, logPrefix) + } + + private[log] def deleteProducerSnapshots(segments: Iterable[LogSegment], + producerStateManager: ProducerStateManager, + asyncDelete: Boolean, + scheduler: Scheduler, + config: LogConfig, + logDirFailureChannel: LogDirFailureChannel, + parentDir: String, + topicPartition: TopicPartition): Unit = { + val snapshotsToDelete = segments.flatMap { segment => + producerStateManager.removeAndMarkSnapshotForDeletion(segment.baseOffset)} + def deleteProducerSnapshots(): Unit = { + LocalLog.maybeHandleIOException(logDirFailureChannel, + parentDir, + s"Error while deleting producer state snapshots for $topicPartition in dir $parentDir") { + snapshotsToDelete.foreach { snapshot => + snapshot.deleteIfExists() } - throw e + } } - } - /** - * Wraps the value of iterator.next() in an option. - * Note: this facility is a part of the Iterator class starting from scala v2.13. - * - * @param iterator - * @tparam T the type of object held within the iterator - * @return Some(iterator.next) if a next element exists, None otherwise. - */ - private def nextOption[T](iterator: Iterator[T]): Option[T] = { - if (iterator.hasNext) - Some(iterator.next()) + if (asyncDelete) + scheduler.schedule("delete-producer-snapshot", () => deleteProducerSnapshots(), delay = config.fileDeleteDelayMs) else - None + deleteProducerSnapshots() + } + + private[log] def createNewCleanedSegment(dir: File, logConfig: LogConfig, baseOffset: Long): LogSegment = { + LocalLog.createNewCleanedSegment(dir, logConfig, baseOffset) } } @@ -2677,12 +2051,8 @@ object LogMetricNames { } } -sealed trait SegmentDeletionReason { - def logReason(log: Log, toDelete: List[LogSegment]): Unit -} - -case object RetentionMsBreach extends SegmentDeletionReason { - override def logReason(log: Log, toDelete: List[LogSegment]): Unit = { +case class RetentionMsBreach(log: Log) extends SegmentDeletionReason { + override def logReason(toDelete: List[LogSegment]): Unit = { val retentionMs = log.config.retentionMs toDelete.foreach { segment => segment.largestRecordTimestamp match { @@ -2697,8 +2067,8 @@ case object RetentionMsBreach extends SegmentDeletionReason { } } -case object RetentionSizeBreach extends SegmentDeletionReason { - override def logReason(log: Log, toDelete: List[LogSegment]): Unit = { +case class RetentionSizeBreach(log: Log) extends SegmentDeletionReason { + override def logReason(toDelete: List[LogSegment]): Unit = { var size = log.size toDelete.foreach { segment => size -= segment.size @@ -2708,26 +2078,8 @@ case object RetentionSizeBreach extends SegmentDeletionReason { } } -case object StartOffsetBreach extends SegmentDeletionReason { - override def logReason(log: Log, toDelete: List[LogSegment]): Unit = { +case class StartOffsetBreach(log: Log) extends SegmentDeletionReason { + override def logReason(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(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(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(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/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 1f1d776f83b86..42eb2a69f552b 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -449,12 +449,6 @@ object LogCleaner { enableCleaner = config.logCleanerEnable) } - - 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) - } } /** @@ -562,7 +556,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.dir, log.config, segments.head.baseOffset) + val cleaned = Log.createNewCleanedSegment(log.dir, log.config, segments.head.baseOffset) transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) try { diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala index b4002f76aceef..1bc0f95baab37 100644 --- a/core/src/main/scala/kafka/log/LogLoader.scala +++ b/core/src/main/scala/kafka/log/LogLoader.scala @@ -74,8 +74,18 @@ case class LoadLogParams(dir: File, * This object is responsible for all activities related with recovery of log segments from disk. */ object LogLoader extends Logging { + + /** + * Clean shutdown file that indicates the broker was cleanly shutdown in 0.8 and higher. + * This is used to avoid unnecessary recovery after a clean shutdown. In theory this could be + * avoided by passing in the recovery point, however finding the correct position to do this + * requires accessing the offset index which may not be safe in an unclean shutdown. + * For more information see the discussion in PR#2104 + */ + val CleanShutdownFile = ".kafka_cleanshutdown" + /** - * Load the log segments from the log files on disk, and return the components of the loaded log. + * Load the log segments from the log files on disk, and returns the components of the loaded log. * Additionally, it also suitably updates the provided LeaderEpochFileCache and ProducerStateManager * to reflect the contents of the loaded log. * @@ -90,7 +100,6 @@ object LogLoader extends Logging { * overflow index offset */ def load(params: LoadLogParams): LoadedLogOffsets = { - // First pass: through the files in the log directory and remove any temporary files // and find any interrupted swap operations val swapFiles = removeTempFilesAndCollectSwapFiles(params) @@ -141,7 +150,6 @@ object LogLoader extends Logging { } } - // Fourth pass: 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. @@ -200,7 +208,6 @@ object LogLoader extends Logging { params.time, reloadFromCleanShutdown = params.hadCleanShutdown, params.logIdentifier) - val activeSegment = params.segments.lastSegment.get LoadedLogOffsets( newLogStartOffset, @@ -274,7 +281,7 @@ object LogLoader extends Logging { } catch { case e: LogSegmentOffsetOverflowException => info(s"${params.logIdentifier}Caught segment overflow error: ${e.getMessage}. Split segment and retry.") - Log.splitOverflowedSegment( + val result = Log.splitOverflowedSegment( e.segment, params.segments, params.dir, @@ -282,8 +289,8 @@ object LogLoader extends Logging { params.config, params.scheduler, params.logDirFailureChannel, - params.producerStateManager, params.logIdentifier) + deleteProducerSnapshotsAsync(result.deletedSegments, params) } } throw new IllegalStateException() @@ -493,14 +500,25 @@ object LogLoader extends Logging { Log.deleteSegmentFiles( toDelete, asyncDelete = true, - deleteProducerStateSnapshots = true, params.dir, params.topicPartition, params.config, params.scheduler, params.logDirFailureChannel, - params.producerStateManager, params.logIdentifier) + deleteProducerSnapshotsAsync(segmentsToDelete, params) } } + + private def deleteProducerSnapshotsAsync(segments: Iterable[LogSegment], + params: LoadLogParams): Unit = { + Log.deleteProducerSnapshots(segments, + params.producerStateManager, + asyncDelete = true, + params.scheduler, + params.config, + params.logDirFailureChannel, + params.dir.getParent, + params.topicPartition) + } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 3545ab5dfc563..c3b47718256ea 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -315,7 +315,7 @@ class LogManager(logDirs: Seq[File], val pool = Executors.newFixedThreadPool(numRecoveryThreadsPerDataDir) threadPools.append(pool) - val cleanShutdownFile = new File(dir, Log.CleanShutdownFile) + val cleanShutdownFile = new File(dir, LogLoader.CleanShutdownFile) if (cleanShutdownFile.exists) { info(s"Skipping recovery for all logs in $logDirAbsolutePath since clean shutdown file was found") // Cache the clean shutdown status and use that for rest of log loading workflow. Delete the CleanShutdownFile @@ -516,7 +516,7 @@ class LogManager(logDirs: Seq[File], // mark that the shutdown was clean by creating marker file debug(s"Writing clean shutdown marker at $dir") - CoreUtils.swallow(Files.createFile(new File(dir, Log.CleanShutdownFile).toPath), this) + CoreUtils.swallow(Files.createFile(new File(dir, LogLoader.CleanShutdownFile).toPath), this) } } } finally { diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 7ade6177642d7..8681562183a3c 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -494,6 +494,13 @@ class LogSegment private[log] (val log: FileRecords, txnIndex.renameTo(new File(CoreUtils.replaceSuffix(txnIndex.file.getPath, oldSuffix, newSuffix))) } + def hasSuffix(suffix: String): Boolean = { + log.file.getName.endsWith(suffix) && + lazyOffsetIndex.file.getName.endsWith(suffix) && + lazyTimeIndex.file.getName.endsWith(suffix) && + txnIndex.file.getName.endsWith(suffix) + } + /** * Append the largest time index entry to the time index and trim the log and indexes. * @@ -624,6 +631,10 @@ class LogSegment private[log] (val log: FileRecords, )) } + def deleted(): Boolean = { + !log.file.exists() && !lazyOffsetIndex.file.exists() && !lazyTimeIndex.file.exists() && !txnIndex.file.exists() + } + /** * The last modified time of this log segment as a unix time stamp */ diff --git a/core/src/main/scala/kafka/log/LogSegments.scala b/core/src/main/scala/kafka/log/LogSegments.scala index d6886d7fdef13..564586dc9b4c8 100644 --- a/core/src/main/scala/kafka/log/LogSegments.scala +++ b/core/src/main/scala/kafka/log/LogSegments.scala @@ -205,6 +205,11 @@ class LogSegments(topicPartition: TopicPartition) { @threadsafe def firstSegment: Option[LogSegment] = firstEntry.map(_.getValue) + /** + * @return the base offset of the log segment associated with the smallest offset, if it exists + */ + private[log] def firstSegmentBaseOffset: Option[Long] = firstSegment.map(_.baseOffset) + /** * @return the entry associated with the greatest offset, if it exists. */ @@ -228,4 +233,36 @@ class LogSegments(topicPartition: TopicPartition) { }.getOrElse(collection.immutable.Map[Long, LogSegment]().asJava) view.values.asScala } + + /** + * The active segment that is currently taking appends + */ + def activeSegment = lastSegment.get + + def sizeInBytes: Long = LogSegments.sizeInBytes(values) + + /** + * Returns an Iterable containing segments matching the provided predicate. + * + * @param predicate the predicate to be used for filtering segments. + */ + def filter(predicate: LogSegment => Boolean): Iterable[LogSegment] = values.filter(predicate) +} + +object LogSegments { + /** + * Calculate a log's size (in bytes) from the provided log segments. + * + * @param segments The log segments to calculate the size of + * @return Sum of the log segments' sizes (in bytes) + */ + def sizeInBytes(segments: Iterable[LogSegment]): Long = + segments.map(_.size.toLong).sum + + def getFirstBatchTimestampForSegments(segments: Iterable[LogSegment]): Iterable[Long] = { + segments.map { + segment => + segment.getFirstBatchTimestamp() + } + } } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 379a79af61377..c406136d66766 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -301,7 +301,10 @@ class PartitionLockTest extends Logging { maxProducerIdExpirationMs, leaderEpochCache, producerStateManager)) - new SlowLog(log, segments, offsets, leaderEpochCache, producerStateManager, mockTime, logDirFailureChannel, appendSemaphore) + val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition, + logDirFailureChannel) + new SlowLog(log, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore) } } when(offsetCheckpoints.fetch(ArgumentMatchers.anyString, ArgumentMatchers.eq(topicPartition))) @@ -365,28 +368,18 @@ class PartitionLockTest extends Logging { private class SlowLog( log: Log, - segments: LogSegments, - offsets: LoadedLogOffsets, + logStartOffset: Long, + localLog: LocalLog, leaderEpochCache: Option[LeaderEpochFileCache], producerStateManager: ProducerStateManager, - mockTime: MockTime, - logDirFailureChannel: LogDirFailureChannel, appendSemaphore: Semaphore ) extends Log( - log.dir, - log.config, - segments, - offsets.logStartOffset, - offsets.recoveryPoint, - offsets.nextOffsetMetadata, - mockTime.scheduler, + logStartOffset, + localLog, new BrokerTopicStats, - mockTime, log.producerIdExpirationCheckIntervalMs, - log.topicPartition, leaderEpochCache, producerStateManager, - logDirFailureChannel, _topicId = None, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 3efe27ba3aca2..cfcc50ab26568 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -257,7 +257,10 @@ class PartitionTest extends AbstractPartitionTest { maxProducerIdExpirationMs, leaderEpochCache, producerStateManager)) - new SlowLog(log, segments, offsets, leaderEpochCache, producerStateManager, mockTime, logDirFailureChannel, appendSemaphore) + val localLog = new LocalLog(log.dir, log.config, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, log.topicPartition, + logDirFailureChannel) + new SlowLog(log, offsets.logStartOffset, localLog, leaderEpochCache, producerStateManager, appendSemaphore) } } @@ -2030,28 +2033,18 @@ class PartitionTest extends AbstractPartitionTest { private class SlowLog( log: Log, - segments: LogSegments, - offsets: LoadedLogOffsets, + logStartOffset: Long, + localLog: LocalLog, leaderEpochCache: Option[LeaderEpochFileCache], producerStateManager: ProducerStateManager, - mockTime: MockTime, - logDirFailureChannel: LogDirFailureChannel, appendSemaphore: Semaphore ) extends Log( - log.dir, - log.config, - segments, - offsets.logStartOffset, - offsets.recoveryPoint, - offsets.nextOffsetMetadata, - mockTime.scheduler, + logStartOffset, + localLog, new BrokerTopicStats, - mockTime, log.producerIdExpirationCheckIntervalMs, - log.topicPartition, leaderEpochCache, producerStateManager, - logDirFailureChannel, _topicId = None, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala new file mode 100644 index 0000000000000..10884c437a338 --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala @@ -0,0 +1,729 @@ +/** + * 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 +import java.nio.channels.ClosedChannelException +import java.nio.charset.StandardCharsets +import java.util.regex.Pattern +import java.util.{Collections, Properties} + +import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata} +import kafka.utils.{MockTime, Scheduler, TestUtils} +import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.errors.KafkaStorageException +import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord} +import org.apache.kafka.common.utils.{Time, Utils} +import org.junit.jupiter.api.Assertions.{assertFalse, _} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} + +import scala.jdk.CollectionConverters._ + +class LocalLogTest { + + import kafka.log.LocalLogTest._ + + var config: KafkaConfig = null + val tmpDir: File = TestUtils.tempDir() + val logDir: File = TestUtils.randomPartitionLogDir(tmpDir) + val topicPartition = new TopicPartition("test_topic", 1) + val logDirFailureChannel = new LogDirFailureChannel(10) + val mockTime = new MockTime() + val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig()) + + @BeforeEach + def setUp(): Unit = { + val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1) + config = KafkaConfig.fromProps(props) + } + + @AfterEach + def tearDown(): Unit = { + try { + log.close() + } catch { + case _: KafkaStorageException => { + // ignore + } + } + Utils.delete(tmpDir) + } + + case class KeyValue(key: String, value: String) { + def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = { + new SimpleRecord(timestamp, key.getBytes, value.getBytes) + } + } + + object KeyValue { + def fromRecord(record: Record): KeyValue = { + val key = + if (record.hasKey) + StandardCharsets.UTF_8.decode(record.key()).toString + else + "" + val value = + if (record.hasValue) + StandardCharsets.UTF_8.decode(record.value()).toString + else + "" + KeyValue(key, value) + } + } + + private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = { + keyValues.map(kv => kv.toRecord()) + } + + private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = { + records.map(r => KeyValue.fromRecord(r)) + } + + private def appendRecords(records: Iterable[SimpleRecord], + log: LocalLog = log, + initialOffset: Long = 0L): Unit = { + log.append(lastOffset = initialOffset + records.size - 1, + largestTimestamp = records.head.timestamp, + shallowOffsetOfMaxTimestamp = initialOffset, + records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*)) + } + + private def readRecords(log: LocalLog = log, + startOffset: Long = 0L, + maxLength: => Int = log.segments.activeSegment.size, + minOneMessage: Boolean = false, + maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata, + includeAbortedTxns: Boolean = false): FetchDataInfo = { + log.read(startOffset, + maxLength, + minOneMessage = minOneMessage, + maxOffsetMetadata, + includeAbortedTxns = includeAbortedTxns) + } + + @Test + def testLogDeleteSegmentsSuccess(): Unit = { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record)) + log.roll() + assertEquals(2, log.segments.numberOfSegments) + assertFalse(logDir.listFiles.isEmpty) + val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values + val deletedSegments = log.deleteAllSegments() + assertTrue(log.segments.isEmpty) + assertEquals(segmentsBeforeDelete, deletedSegments) + assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed()) + assertTrue(logDir.exists) + } + + @Test + def testLogDeleteDirSuccessWhenEmptyAndFailureWhenNonEmpty(): Unit ={ + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record)) + log.roll() + assertEquals(2, log.segments.numberOfSegments) + assertFalse(logDir.listFiles.isEmpty) + + assertThrows(classOf[IllegalStateException], () => log.deleteEmptyDir()) + assertTrue(logDir.exists) + + log.deleteAllSegments() + log.deleteEmptyDir() + assertFalse(logDir.exists) + } + + @Test + def testUpdateConfig(): Unit = { + val oldConfig = log.config + assertEquals(oldConfig, log.config) + + val newConfig = createLogConfig(segmentBytes=oldConfig.segmentSize + 1) + log.updateConfig(newConfig) + assertEquals(newConfig, log.config) + } + + @Test + def testLogDirRenameToNewDir(): Unit = { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record)) + log.roll() + assertEquals(2, log.segments.numberOfSegments) + val newLogDir = TestUtils.randomPartitionLogDir(tmpDir) + assertTrue(log.renameDir(newLogDir.getName)) + assertFalse(logDir.exists()) + assertTrue(newLogDir.exists()) + assertEquals(newLogDir, log.dir) + assertEquals(newLogDir.getParent, log.parentDir) + assertEquals(newLogDir.getParent, log.dir.getParent) + log.segments.values.foreach(segment => assertEquals(newLogDir.getPath, segment.log.file().getParentFile.getPath)) + assertEquals(2, log.segments.numberOfSegments) + } + + @Test + def testLogDirRenameToExistingDir(): Unit = { + assertFalse(log.renameDir(log.dir.getName)) + } + + @Test + def testLogFlush(): Unit = { + assertEquals(0L, log.recoveryPoint) + assertEquals(mockTime.milliseconds, log.lastFlushTime) + + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record)) + mockTime.sleep(1) + val newSegment = log.roll() + log.flush(newSegment.baseOffset) + log.markFlushed(newSegment.baseOffset) + assertEquals(1L, log.recoveryPoint) + assertEquals(mockTime.milliseconds, log.lastFlushTime) + } + + @Test + def testLogAppend(): Unit = { + val fetchDataInfoBeforeAppend = readRecords(maxLength = 1) + assertTrue(fetchDataInfoBeforeAppend.records.records.asScala.isEmpty) + + mockTime.sleep(1) + val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) + appendRecords(kvsToRecords(keyValues)) + assertEquals(2L, log.logEndOffset) + assertEquals(0L, log.recoveryPoint) + val fetchDataInfo = readRecords() + assertEquals(2L, fetchDataInfo.records.records.asScala.size) + assertEquals(keyValues, recordsToKvs(fetchDataInfo.records.records.asScala)) + } + + @Test + def testLogCloseSuccess(): Unit = { + val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) + appendRecords(kvsToRecords(keyValues)) + log.close() + assertThrows(classOf[ClosedChannelException], () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L)) + } + + @Test + def testLogCloseIdempotent(): Unit = { + log.close() + // Check that LocalLog.close() is idempotent + log.close() + } + + @Test + def testLogCloseFailureWhenInMemoryBufferClosed(): Unit = { + val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) + appendRecords(kvsToRecords(keyValues)) + log.closeHandlers() + assertThrows(classOf[KafkaStorageException], () => log.close()) + } + + @Test + def testLogCloseHandlers(): Unit = { + val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) + appendRecords(kvsToRecords(keyValues)) + log.closeHandlers() + assertThrows(classOf[ClosedChannelException], + () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L)) + } + + @Test + def testLogCloseHandlersIdempotent(): Unit = { + log.closeHandlers() + // Check that LocalLog.closeHandlers() is idempotent + log.closeHandlers() + } + + private def testRemoveAndDeleteSegments(asyncDelete: Boolean): Unit = { + for (offset <- 0 to 8) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + log.roll() + } + + assertEquals(10L, log.segments.numberOfSegments) + + class TestDeletionReason extends SegmentDeletionReason { + private var _deletedSegments: Iterable[LogSegment] = List[LogSegment]() + + override def logReason(toDelete: List[LogSegment]): Unit = { + _deletedSegments = List[LogSegment]() ++ toDelete + } + + def deletedSegments: Iterable[LogSegment] = _deletedSegments + } + val reason = new TestDeletionReason() + val toDelete = List[LogSegment]() ++ log.segments.values + log.removeAndDeleteSegments(toDelete, asyncDelete = asyncDelete, reason) + if (asyncDelete) { + mockTime.sleep(log.config.fileDeleteDelayMs + 1) + } + assertTrue(log.segments.isEmpty) + assertEquals(toDelete, reason.deletedSegments) + toDelete.foreach(segment => assertTrue(segment.deleted())) + } + + @Test + def testRemoveAndDeleteSegmentsSync(): Unit = { + testRemoveAndDeleteSegments(asyncDelete = false) + } + + @Test + def testRemoveAndDeleteSegmentsAsync(): Unit = { + testRemoveAndDeleteSegments(asyncDelete = true) + } + + private def testDeleteSegmentFiles(asyncDelete: Boolean): Unit = { + for (offset <- 0 to 8) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + log.roll() + } + + assertEquals(10L, log.segments.numberOfSegments) + + val toDelete = List[LogSegment]() ++ log.segments.values + LocalLog.deleteSegmentFiles(toDelete, asyncDelete = asyncDelete, log.dir, log.topicPartition, log.config, log.scheduler, log.logDirFailureChannel, "") + if (asyncDelete) { + toDelete.foreach { + segment => + assertFalse(segment.deleted()) + assertTrue(segment.hasSuffix(LocalLog.DeletedFileSuffix)) + } + mockTime.sleep(log.config.fileDeleteDelayMs + 1) + } + toDelete.foreach(segment => assertTrue(segment.deleted())) + } + + @Test + def testDeleteSegmentFilesSync(): Unit = { + testDeleteSegmentFiles(asyncDelete = false) + } + + @Test + def testDeleteSegmentFilesAsync(): Unit = { + testDeleteSegmentFiles(asyncDelete = true) + } + + @Test + def testDeletableSegmentsFilter(): Unit = { + for (offset <- 0 to 8) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + log.roll() + } + + assertEquals(10, log.segments.numberOfSegments) + + { + val deletable = log.deletableSegments( + (segment: LogSegment, _: Option[LogSegment]) => segment.baseOffset <= 5) + val expected = log.segments.nonActiveLogSegmentsFrom(0L).filter(segment => segment.baseOffset <= 5).toList + assertEquals(6, expected.length) + assertEquals(expected, deletable.toList) + } + + { + val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment]) => true) + val expected = log.segments.nonActiveLogSegmentsFrom(0L).toList + assertEquals(9, expected.length) + assertEquals(expected, deletable.toList) + } + + { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = 9L) + val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment]) => true) + val expected = log.segments.values.toList + assertEquals(10, expected.length) + assertEquals(expected, deletable.toList) + } + } + + @Test + def testDeletableSegmentsIteration(): Unit = { + for (offset <- 0 to 8) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + log.roll() + } + + assertEquals(10L, log.segments.numberOfSegments) + + var offset = 0 + val deletableSegments = log.deletableSegments( + (segment: LogSegment, nextSegmentOpt: Option[LogSegment]) => { + assertEquals(offset, segment.baseOffset) + val floorSegmentOpt = log.segments.floorSegment(offset) + assertTrue(floorSegmentOpt.isDefined) + assertEquals(floorSegmentOpt.get, segment) + if (offset == log.logEndOffset) { + assertFalse(nextSegmentOpt.isDefined) + } else { + assertTrue(nextSegmentOpt.isDefined) + val higherSegmentOpt = log.segments.higherSegment(segment.baseOffset) + assertTrue(higherSegmentOpt.isDefined) + assertEquals(segment.baseOffset + 1, higherSegmentOpt.get.baseOffset) + assertEquals(higherSegmentOpt.get, nextSegmentOpt.get) + } + offset += 1 + true + }) + assertEquals(10L, log.segments.numberOfSegments) + assertEquals(log.segments.nonActiveLogSegmentsFrom(0L).toSeq, deletableSegments.toSeq) + } + + @Test + def testTruncateFullyAndStartAt(): Unit = { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + for (offset <- 0 to 7) { + appendRecords(List(record), initialOffset = offset) + if (offset % 2 != 0) + log.roll() + } + for (offset <- 8 to 12) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + } + assertEquals(5, log.segments.numberOfSegments) + val expected = List[LogSegment]() ++ log.segments.values + val deleted = log.truncateFullyAndStartAt(10L) + assertEquals(expected, deleted) + assertEquals(1, log.segments.numberOfSegments) + assertEquals(10L, log.segments.activeSegment.baseOffset) + assertEquals(0L, log.recoveryPoint) + assertEquals(10L, log.logEndOffset) + val fetchDataInfo = readRecords(startOffset = 10L) + assertTrue(fetchDataInfo.records.records.asScala.isEmpty) + } + + @Test + def testTruncateTo(): Unit = { + for (offset <- 0 to 11) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + if (offset % 3 == 2) + log.roll() + } + assertEquals(5, log.segments.numberOfSegments) + assertEquals(12L, log.logEndOffset) + + val expected = List[LogSegment]() ++ log.segments.values(9L, log.logEndOffset + 1) + // Truncate to an offset before the base offset of the active segment + val deleted = log.truncateTo(7L) + assertEquals(expected, deleted) + assertEquals(3, log.segments.numberOfSegments) + assertEquals(6L, log.segments.activeSegment.baseOffset) + assertEquals(0L, log.recoveryPoint) + assertEquals(7L, log.logEndOffset) + val fetchDataInfo = readRecords(startOffset = 6L) + assertEquals(1, fetchDataInfo.records.records.asScala.size) + assertEquals(Seq(KeyValue("", "a")), recordsToKvs(fetchDataInfo.records.records.asScala)) + + // Verify that we can still append to the active segment + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = 7L) + assertEquals(8L, log.logEndOffset) + } + + @Test + def testNonActiveSegmentsFrom(): Unit = { + for (i <- 0 until 5) { + val keyValues = Seq(KeyValue(i.toString, i.toString)) + appendRecords(kvsToRecords(keyValues), initialOffset = i) + log.roll() + } + + def nonActiveBaseOffsetsFrom(startOffset: Long): Seq[Long] = { + log.segments.nonActiveLogSegmentsFrom(startOffset).map(_.baseOffset).toSeq + } + + assertEquals(5L, log.segments.activeSegment.baseOffset) + assertEquals(0 until 5, nonActiveBaseOffsetsFrom(0L)) + assertEquals(Seq.empty, nonActiveBaseOffsetsFrom(5L)) + assertEquals(2 until 5, nonActiveBaseOffsetsFrom(2L)) + assertEquals(Seq.empty, nonActiveBaseOffsetsFrom(6L)) + } + + private def topicPartitionName(topic: String, partition: String): String = topic + "-" + partition + + @Test + def testParseTopicPartitionName(): Unit = { + val topic = "test_topic" + val partition = "143" + val dir = new File(logDir, topicPartitionName(topic, partition)) + val topicPartition = LocalLog.parseTopicPartitionName(dir) + assertEquals(topic, topicPartition.topic) + assertEquals(partition.toInt, topicPartition.partition) + } + + /** + * Tests that log directories with a period in their name that have been marked for deletion + * are parsed correctly by `Log.parseTopicPartitionName` (see KAFKA-5232 for details). + */ + @Test + def testParseTopicPartitionNameWithPeriodForDeletedTopic(): Unit = { + val topic = "foo.bar-testtopic" + val partition = "42" + val dir = new File(logDir, LocalLog.logDeleteDirName(new TopicPartition(topic, partition.toInt))) + val topicPartition = LocalLog.parseTopicPartitionName(dir) + assertEquals(topic, topicPartition.topic, "Unexpected topic name parsed") + assertEquals(partition.toInt, topicPartition.partition, "Unexpected partition number parsed") + } + + @Test + def testParseTopicPartitionNameForEmptyName(): Unit = { + val dir = new File("") + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(dir), + () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + } + + @Test + def testParseTopicPartitionNameForNull(): Unit = { + val dir: File = null + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(dir), + () => "KafkaException should have been thrown for dir: " + dir) + } + + @Test + def testParseTopicPartitionNameForMissingSeparator(): Unit = { + val topic = "test_topic" + val partition = "1999" + val dir = new File(logDir, topic + partition) + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(dir), + () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + // also test the "-delete" marker case + val deleteMarkerDir = new File(logDir, topic + partition + "." + LocalLog.DeleteDirSuffix) + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(deleteMarkerDir), + () => "KafkaException should have been thrown for dir: " + deleteMarkerDir.getCanonicalPath) + } + + @Test + def testParseTopicPartitionNameForMissingTopic(): Unit = { + val topic = "" + val partition = "1999" + val dir = new File(logDir, topicPartitionName(topic, partition)) + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(dir), + () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + + // also test the "-delete" marker case + val deleteMarkerDir = new File(logDir, LocalLog.logDeleteDirName(new TopicPartition(topic, partition.toInt))) + + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(deleteMarkerDir), + () => "KafkaException should have been thrown for dir: " + deleteMarkerDir.getCanonicalPath) + } + + @Test + def testParseTopicPartitionNameForMissingPartition(): Unit = { + val topic = "test_topic" + val partition = "" + val dir = new File(logDir.getPath + topicPartitionName(topic, partition)) + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(dir), + () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + + // also test the "-delete" marker case + val deleteMarkerDir = new File(logDir, topicPartitionName(topic, partition) + "." + LocalLog.DeleteDirSuffix) + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(deleteMarkerDir), + () => "KafkaException should have been thrown for dir: " + deleteMarkerDir.getCanonicalPath) + } + + @Test + def testParseTopicPartitionNameForInvalidPartition(): Unit = { + val topic = "test_topic" + val partition = "1999a" + val dir = new File(logDir, topicPartitionName(topic, partition)) + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(dir), + () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + + // also test the "-delete" marker case + val deleteMarkerDir = new File(logDir, topic + partition + "." + LocalLog.DeleteDirSuffix) + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(deleteMarkerDir), + () => "KafkaException should have been thrown for dir: " + deleteMarkerDir.getCanonicalPath) + } + + @Test + def testParseTopicPartitionNameForExistingInvalidDir(): Unit = { + val dir1 = new File(logDir.getPath + "/non_kafka_dir") + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(dir1), + () => "KafkaException should have been thrown for dir: " + dir1.getCanonicalPath) + val dir2 = new File(logDir.getPath + "/non_kafka_dir-delete") + assertThrows(classOf[KafkaException], () => LocalLog.parseTopicPartitionName(dir2), + () => "KafkaException should have been thrown for dir: " + dir2.getCanonicalPath) + } + + @Test + def testLogDeleteDirName(): Unit = { + val name1 = LocalLog.logDeleteDirName(new TopicPartition("foo", 3)) + assertTrue(name1.length <= 255) + assertTrue(Pattern.compile("foo-3\\.[0-9a-z]{32}-delete").matcher(name1).matches()) + assertTrue(LocalLog.DeleteDirPattern.matcher(name1).matches()) + assertFalse(LocalLog.FutureDirPattern.matcher(name1).matches()) + val name2 = LocalLog.logDeleteDirName( + new TopicPartition("n" + String.join("", Collections.nCopies(248, "o")), 5)) + assertEquals(255, name2.length) + assertTrue(Pattern.compile("n[o]{212}-5\\.[0-9a-z]{32}-delete").matcher(name2).matches()) + assertTrue(LocalLog.DeleteDirPattern.matcher(name2).matches()) + assertFalse(LocalLog.FutureDirPattern.matcher(name2).matches()) + } + + @Test + def testOffsetFromFile(): Unit = { + val offset = 23423423L + + val logFile = LocalLog.logFile(tmpDir, offset) + assertEquals(offset, LocalLog.offsetFromFile(logFile)) + + val offsetIndexFile = LocalLog.offsetIndexFile(tmpDir, offset) + assertEquals(offset, LocalLog.offsetFromFile(offsetIndexFile)) + + val timeIndexFile = LocalLog.timeIndexFile(tmpDir, offset) + assertEquals(offset, LocalLog.offsetFromFile(timeIndexFile)) + } + + @Test + def testRollSegmentThatAlreadyExists(): Unit = { + assertEquals(1, log.segments.numberOfSegments, "Log begins with a single empty segment.") + + // roll active segment with the same base offset of size zero should recreate the segment + log.roll(Some(0L)) + assertEquals(1, log.segments.numberOfSegments, "Expect 1 segment after roll() empty segment with base offset.") + + // should be able to append records to active segment + val keyValues1 = List(KeyValue("k1", "v1")) + appendRecords(kvsToRecords(keyValues1)) + assertEquals(0L, log.segments.activeSegment.baseOffset) + // make sure we can append more records + val keyValues2 = List(KeyValue("k2", "v2")) + appendRecords(keyValues2.map(_.toRecord(mockTime.milliseconds + 10)), initialOffset = 1L) + assertEquals(2, log.logEndOffset, "Expect two records in the log") + val readResult = readRecords() + assertEquals(2L, readResult.records.records.asScala.size) + assertEquals(keyValues1 ++ keyValues2, recordsToKvs(readResult.records.records.asScala)) + + // roll so that active segment is empty + log.roll() + assertEquals(2L, log.segments.activeSegment.baseOffset, "Expect base offset of active segment to be LEO") + assertEquals(2, log.segments.numberOfSegments, "Expect two segments.") + assertEquals(2L, log.logEndOffset) + } + + @Test + def testNewSegmentsAfterRoll(): Unit = { + assertEquals(1, log.segments.numberOfSegments, "Log begins with a single empty segment.") + + // roll active segment with the same base offset of size zero should recreate the segment + { + val newSegment = log.roll() + assertEquals(0L, newSegment.baseOffset) + assertEquals(1, log.segments.numberOfSegments) + assertEquals(0L, log.logEndOffset) + } + + appendRecords(List(KeyValue("k1", "v1").toRecord())) + + { + val newSegment = log.roll() + assertEquals(1L, newSegment.baseOffset) + assertEquals(2, log.segments.numberOfSegments) + assertEquals(1L, log.logEndOffset) + } + + appendRecords(List(KeyValue("k2", "v2").toRecord()), initialOffset = 1L) + + { + val newSegment = log.roll(Some(1L)) + assertEquals(2L, newSegment.baseOffset) + assertEquals(3, log.segments.numberOfSegments) + assertEquals(2L, log.logEndOffset) + } + } + + @Test + def testRollSegmentErrorWhenNextOffsetIsIllegal(): Unit = { + assertEquals(1, log.segments.numberOfSegments, "Log begins with a single empty segment.") + + val keyValues = List(KeyValue("k1", "v1"), KeyValue("k2", "v2"), KeyValue("k3", "v3")) + appendRecords(kvsToRecords(keyValues)) + assertEquals(0L, log.segments.activeSegment.baseOffset) + assertEquals(3, log.logEndOffset, "Expect two records in the log") + + // roll to create an empty active segment + log.roll() + assertEquals(3L, log.segments.activeSegment.baseOffset) + + // intentionally setup the logEndOffset to introduce an error later + log.updateLogEndOffset(1L) + + // expect an error because of attempt to roll to a new offset (1L) that's lower than the + // base offset (3L) of the active segment + assertThrows(classOf[KafkaException], () => log.roll()) + } + + private def createLocalLogWithActiveSegment(dir: File = logDir, + config: LogConfig, + segments: LogSegments = new LogSegments(topicPartition), + recoveryPoint: Long = 0L, + nextOffsetMetadata: LogOffsetMetadata = new LogOffsetMetadata(0L, 0L, 0), + scheduler: Scheduler = mockTime.scheduler, + time: Time = mockTime, + topicPartition: TopicPartition = topicPartition, + logDirFailureChannel: LogDirFailureChannel = logDirFailureChannel): LocalLog = { + segments.add(LogSegment.open(dir = dir, + baseOffset = 0L, + config, + time = time, + initFileSize = config.initFileSize, + preallocate = config.preallocate)) + new LocalLog(_dir = dir, + config = config, + segments = segments, + recoveryPoint = recoveryPoint, + nextOffsetMetadata = nextOffsetMetadata, + scheduler = scheduler, + time = time, + topicPartition = topicPartition, + logDirFailureChannel = logDirFailureChannel) + } +} + +object LocalLogTest { + 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) + } +} diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 19de8ea2703bb..4474b3f0c83e1 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -116,18 +116,18 @@ class LogCleanerManagerTest extends Logging { maxProducerIdExpirationMs, leaderEpochCache, producerStateManager)) + val localLog = new LocalLog(tpDir, config, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, time.scheduler, time, tp, logDirFailureChannel) // 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, - LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, - producerStateManager, logDirFailureChannel, _topicId = None, keepPartitionMetadataFile = true) { + class LogMock extends Log(offsets.logStartOffset, localLog, new BrokerTopicStats, + LogManager.ProducerIdExpirationCheckIntervalMs, leaderEpochCache, + producerStateManager, _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, offsets) + val log: Log = new LogMock() writeRecords(log = log, numBatches = logSegmentsCount * 2, recordsPerBatch = 10, @@ -361,7 +361,7 @@ class LogCleanerManagerTest extends Logging { logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) logProps.put(LogConfig.MinCleanableDirtyRatioProp, 0: Integer) val config = LogConfig(logProps) - log.config = config + log.updateConfig(config) // log cleanup inprogress, the log is not available for compaction val cleanable = cleanerManager.grabFilthiestCompactedLog(time) @@ -375,7 +375,7 @@ class LogCleanerManagerTest extends Logging { // update cleanup policy to delete logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Delete) val config2 = LogConfig(logProps) - log.config = config2 + log.updateConfig(config2) // compaction in progress, should have 0 log eligible for log cleanup val deletableLog2 = cleanerManager.pauseCleaningForNonCompactedPartitions() diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 816d6d4fdd0df..a7950f69596ad 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -148,7 +148,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati val log = cleaner.logs.get(topicPartitions(0)) val props = logConfigProperties(maxMessageSize = maxMessageSize) props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_9_0.version) - log.config = new LogConfig(props) + log.updateConfig(new LogConfig(props)) val appends = writeDups(numKeys = 100, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0) val startSize = log.size @@ -168,7 +168,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati // also add some messages with version 1 and version 2 to check that we handle mixed format versions correctly props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_11_0_IV0.version) - log.config = new LogConfig(props) + log.updateConfig(new LogConfig(props)) val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1) val dupsV2 = writeDups(startKey = 15, numKeys = 5, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V2) appends ++ dupsV0 ++ Seq((largeMessageKey, largeMessageValue, largeMessageOffset)) ++ dupsV1 ++ dupsV2 @@ -188,7 +188,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati val log = cleaner.logs.get(topicPartitions(0)) val props = logConfigProperties(maxMessageSize = maxMessageSize, segmentSize = 256) props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_9_0.version) - log.config = new LogConfig(props) + log.updateConfig(new LogConfig(props)) // with compression enabled, these messages will be written as a single message containing // all of the individual messages @@ -196,7 +196,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0) props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version) - log.config = new LogConfig(props) + log.updateConfig(new LogConfig(props)) var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1) appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index e5bdc87a3b1c7..67ac7434d6c6d 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -121,29 +121,22 @@ class LogCleanerTest { maxProducerIdExpirationMs, leaderEpochCache, producerStateManager)) - - val log = new Log(dir, - config = config, - segments = logSegments, - logStartOffset = offsets.logStartOffset, - recoveryPoint = offsets.recoveryPoint, - nextOffsetMetadata = offsets.nextOffsetMetadata, - scheduler = time.scheduler, + val localLog = new LocalLog(dir, config, logSegments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, time.scheduler, time, topicPartition, logDirFailureChannel) + val log = new Log(offsets.logStartOffset, + localLog, brokerTopicStats = new BrokerTopicStats, - time, producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs, - topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, producerStateManager = producerStateManager, - logDirFailureChannel = logDirFailureChannel, _topicId = None, keepPartitionMetadataFile = true) { - override def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = { + override def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment]): Unit = { deleteStartLatch.countDown() if (!deleteCompleteLatch.await(5000, TimeUnit.MILLISECONDS)) { throw new IllegalStateException("Log segment deletion timed out") } - super.replaceSegments(newSegments, oldSegments, isRecoveredSwapFile) + super.replaceSegments(newSegments, oldSegments) } } @@ -802,7 +795,7 @@ class LogCleanerTest { // Decrease the log's max message size logProps.put(LogConfig.MaxMessageBytesProp, largeMessageSize / 2: java.lang.Integer) - log.config = LogConfig.fromProps(logConfig.originals, logProps) + log.updateConfig(LogConfig.fromProps(logConfig.originals, logProps)) // pretend we have the following keys val keys = immutable.ListSet(1, 3, 5, 7, 9) diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 7fc7bcc827b16..c1a2164edc875 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -40,6 +40,7 @@ import scala.jdk.CollectionConverters._ class LogLoaderTest { var config: KafkaConfig = null val brokerTopicStats = new BrokerTopicStats + val maxProducerIdExpirationMs: Int = 60 * 60 * 1000 val tmpDir = TestUtils.tempDir() val logDir = TestUtils.randomPartitionLogDir(tmpDir) val mockTime = new MockTime() @@ -99,15 +100,17 @@ class LogLoaderTest { 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, - LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, - producerStateManager, logDirFailureChannel, None, true) + val localLog = new LocalLog(logDir, logConfig, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, + logDirFailureChannel) + new Log(offsets.logStartOffset, localLog, brokerTopicStats, + LogManager.ProducerIdExpirationCheckIntervalMs, leaderEpochCache, + producerStateManager, None, true) } } } - val cleanShutdownFile = new File(logDir, Log.CleanShutdownFile) + val cleanShutdownFile = new File(logDir, LogLoader.CleanShutdownFile) locally { val logManager: LogManager = interceptedLogManager(logConfig, logDirs, simulateError) log = logManager.getOrCreateLog(topicPartition, isNew = true, topicId = None) @@ -161,7 +164,7 @@ class LogLoaderTest { recoveryPoint: Long = 0L, scheduler: Scheduler = mockTime.scheduler, time: Time = mockTime, - maxProducerIdExpirationMs: Int = 60 * 60 * 1000, + maxProducerIdExpirationMs: Int = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs, lastShutdownClean: Boolean = true): Log = { LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint, @@ -280,10 +283,12 @@ class LogLoaderTest { leaderEpochCache, producerStateManager) val offsets = LogLoader.load(loadLogParams) - new Log(logDir, logConfig, interceptedLogSegments, offsets.logStartOffset, offsets.recoveryPoint, - offsets.nextOffsetMetadata, mockTime.scheduler, brokerTopicStats, mockTime, - LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, - producerStateManager, logDirFailureChannel, _topicId = None, keepPartitionMetadataFile = true) + val localLog = new LocalLog(logDir, logConfig, interceptedLogSegments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, + logDirFailureChannel) + new Log(offsets.logStartOffset, localLog, brokerTopicStats, + LogManager.ProducerIdExpirationCheckIntervalMs, leaderEpochCache, producerStateManager, + None, keepPartitionMetadataFile = true) } // Retain snapshots for the last 2 segments @@ -351,20 +356,15 @@ class LogLoaderTest { maxProducerIdExpirationMs, leaderEpochCache, stateManager)) - val log = new Log(logDir, - config, - segments = segments, - logStartOffset = offsets.logStartOffset, - recoveryPoint = offsets.recoveryPoint, - nextOffsetMetadata = offsets.nextOffsetMetadata, - scheduler = mockTime.scheduler, + val localLog = new LocalLog(logDir, config, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, + logDirFailureChannel) + val log = new Log(offsets.logStartOffset, + localLog, brokerTopicStats = brokerTopicStats, - time = mockTime, producerIdExpirationCheckIntervalMs = 30000, - topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, producerStateManager = stateManager, - logDirFailureChannel = logDirFailureChannel, _topicId = None, keepPartitionMetadataFile = true) @@ -398,7 +398,6 @@ class LogLoaderTest { 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() @@ -485,20 +484,15 @@ class LogLoaderTest { maxProducerIdExpirationMs, leaderEpochCache, stateManager)) - new Log(logDir, - config, - segments = segments, - logStartOffset = offsets.logStartOffset, - recoveryPoint = offsets.recoveryPoint, - nextOffsetMetadata = offsets.nextOffsetMetadata, - scheduler = mockTime.scheduler, + val localLog = new LocalLog(logDir, config, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, + logDirFailureChannel) + new Log(offsets.logStartOffset, + localLog, brokerTopicStats = brokerTopicStats, - time = mockTime, producerIdExpirationCheckIntervalMs = 30000, - topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, producerStateManager = stateManager, - logDirFailureChannel = logDirFailureChannel, _topicId = None, keepPartitionMetadataFile = true) @@ -546,20 +540,15 @@ class LogLoaderTest { maxProducerIdExpirationMs, leaderEpochCache, stateManager)) - new Log(logDir, - config, - segments = segments, - logStartOffset = offsets.logStartOffset, - recoveryPoint = offsets.recoveryPoint, - nextOffsetMetadata = offsets.nextOffsetMetadata, - scheduler = mockTime.scheduler, + val localLog = new LocalLog(logDir, config, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, + logDirFailureChannel) + new Log(offsets.logStartOffset, + localLog, brokerTopicStats = brokerTopicStats, - time = mockTime, producerIdExpirationCheckIntervalMs = 30000, - topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, producerStateManager = stateManager, - logDirFailureChannel = logDirFailureChannel, _topicId = None, keepPartitionMetadataFile = true) @@ -609,20 +598,15 @@ class LogLoaderTest { maxProducerIdExpirationMs, leaderEpochCache, stateManager)) - new Log(logDir, - config, - segments = segments, - logStartOffset = offsets.logStartOffset, - recoveryPoint = offsets.recoveryPoint, - nextOffsetMetadata = offsets.nextOffsetMetadata, - scheduler = mockTime.scheduler, + val localLog = new LocalLog(logDir, config, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, mockTime, topicPartition, + logDirFailureChannel) + new Log(offsets.logStartOffset, + localLog, brokerTopicStats = brokerTopicStats, - time = mockTime, producerIdExpirationCheckIntervalMs = 30000, - topicPartition = topicPartition, leaderEpochCache = leaderEpochCache, producerStateManager = stateManager, - logDirFailureChannel = logDirFailureChannel, _topicId = None, keepPartitionMetadataFile = true) @@ -1528,7 +1512,7 @@ class LogLoaderTest { assertEquals(startOffset, log.logStartOffset) assertEquals(startOffset, log.logEndOffset) // Validate that the remaining segment matches our expectations - val onlySegment = log.segments.firstSegment.get + val onlySegment = log.logSegments.head assertEquals(startOffset, onlySegment.baseOffset) assertTrue(onlySegment.log.file().exists()) assertTrue(onlySegment.lazyOffsetIndex.file.exists()) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index cd086fe047092..4e08b3427abd1 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -125,8 +125,8 @@ class LogManagerTest { logManagerForTest.get.shutdown() - assertFalse(Files.exists(new File(logDir1, Log.CleanShutdownFile).toPath)) - assertTrue(Files.exists(new File(logDir2, Log.CleanShutdownFile).toPath)) + assertFalse(Files.exists(new File(logDir1, LogLoader.CleanShutdownFile).toPath)) + assertTrue(Files.exists(new File(logDir2, LogLoader.CleanShutdownFile).toPath)) } finally { logManagerForTest.foreach(manager => manager.liveLogDirs.foreach(Utils.delete)) } diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala index 9d0765aed687e..0345acea8e6b8 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala @@ -21,6 +21,7 @@ import java.io.File import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.utils.{Time, Utils} +import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -223,4 +224,17 @@ class LogSegmentsTest { assertFalse(iterator.hasNext) } } + + @Test + def testSizeForLargeLogs(): Unit = { + val largeSize = Int.MaxValue.toLong * 2 + val logSegment: LogSegment = EasyMock.createMock(classOf[LogSegment]) + + EasyMock.expect(logSegment.size).andReturn(Int.MaxValue).anyTimes + EasyMock.replay(logSegment) + + assertEquals(Int.MaxValue, LogSegments.sizeInBytes(Seq(logSegment))) + assertEquals(largeSize, LogSegments.sizeInBytes(Seq(logSegment, logSegment))) + assertTrue(Log.sizeInBytes(Seq(logSegment, logSegment)) > Int.MaxValue) + } } diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index f515f131fc80f..d831cba03c247 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -21,16 +21,14 @@ import java.io._ import java.nio.ByteBuffer import java.nio.file.Files import java.util.concurrent.{Callable, Executors} -import java.util.regex.Pattern -import java.util.{Collections, Optional} +import java.util.Optional 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.{BrokerTopicStats, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, LogOffsetMetadata, PartitionMetadataFile} import kafka.utils._ -import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} +import org.apache.kafka.common.{InvalidRecordException, TopicPartition, Uuid} import org.apache.kafka.common.errors._ import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.record.FileRecords.TimestampAndOffset @@ -39,7 +37,6 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils} -import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -463,33 +460,8 @@ class LogTest { } @Test - def testLogDeleteDirName(): Unit = { - val name1 = Log.logDeleteDirName(new TopicPartition("foo", 3)) - assertTrue(name1.length <= 255) - assertTrue(Pattern.compile("foo-3\\.[0-9a-z]{32}-delete").matcher(name1).matches()) - assertTrue(Log.DeleteDirPattern.matcher(name1).matches()) - assertFalse(Log.FutureDirPattern.matcher(name1).matches()) - val name2 = Log.logDeleteDirName( - new TopicPartition("n" + String.join("", Collections.nCopies(248, "o")), 5)) - assertEquals(255, name2.length) - assertTrue(Pattern.compile("n[o]{212}-5\\.[0-9a-z]{32}-delete").matcher(name2).matches()) - assertTrue(Log.DeleteDirPattern.matcher(name2).matches()) - assertFalse(Log.FutureDirPattern.matcher(name2).matches()) - } - - @Test - def testOffsetFromFile(): Unit = { + def testOffsetFromProducerSnapshotFile(): Unit = { val offset = 23423423L - - val logFile = Log.logFile(tmpDir, offset) - assertEquals(offset, Log.offsetFromFile(logFile)) - - val offsetIndexFile = Log.offsetIndexFile(tmpDir, offset) - assertEquals(offset, Log.offsetFromFile(offsetIndexFile)) - - val timeIndexFile = Log.timeIndexFile(tmpDir, offset) - assertEquals(offset, Log.offsetFromFile(timeIndexFile)) - val snapshotFile = Log.producerSnapshotFile(tmpDir, offset) assertEquals(offset, Log.offsetFromFile(snapshotFile)) } @@ -499,7 +471,7 @@ class LogTest { * using the mock clock to force the log to roll and checks the number of segments. */ @Test - def testTimeBasedLogRoll(): Unit = { + def testTimeBasedLogRollDuringAppend(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) val logConfig = LogTestUtils.createLogConfig(segmentMs = 1 * 60 * 60L) @@ -606,28 +578,6 @@ class LogTest { assertThrows(classOf[OutOfOrderSequenceException], () => log.appendAsLeader(nextRecords, leaderEpoch = 0)) } - @Test - def testTruncateToEmptySegment(): Unit = { - val log = createLog(logDir, LogConfig()) - - // Force a segment roll by using a large offset. The first segment will be empty - val records = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds, "key".getBytes, "value".getBytes)), - baseOffset = Int.MaxValue.toLong + 200) - appendAsFollower(log, records) - assertEquals(0, log.logSegments.head.size) - assertEquals(2, log.logSegments.size) - - // Truncate to an offset before the base offset of the latest segment - log.truncateTo(0L) - assertEquals(1, log.logSegments.size) - - // Now verify that we can still append to the active segment - appendAsFollower(log, TestUtils.records(List(new SimpleRecord(mockTime.milliseconds, "key".getBytes, "value".getBytes)), - baseOffset = 100L)) - assertEquals(1, log.logSegments.size) - assertEquals(101L, log.logEndOffset) - } - @Test def testTruncateToEndOffsetClearsEpochCache(): Unit = { val log = createLog(logDir, LogConfig()) @@ -747,63 +697,6 @@ class LogTest { assertEquals(500, log.logEndOffset) } - @Test - def testNonActiveSegmentsFrom(): Unit = { - val logConfig = LogTestUtils.createLogConfig() - val log = createLog(logDir, logConfig) - - for (i <- 0 until 5) { - val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes) - log.appendAsLeader(TestUtils.records(List(record)), leaderEpoch = 0) - log.roll() - } - - def nonActiveBaseOffsetsFrom(startOffset: Long): Seq[Long] = { - log.nonActiveLogSegmentsFrom(startOffset).map(_.baseOffset).toSeq - } - - assertEquals(5L, log.activeSegment.baseOffset) - assertEquals(0 until 5, nonActiveBaseOffsetsFrom(0L)) - assertEquals(Seq.empty, nonActiveBaseOffsetsFrom(5L)) - assertEquals(2 until 5, nonActiveBaseOffsetsFrom(2L)) - assertEquals(Seq.empty, nonActiveBaseOffsetsFrom(6L)) - } - - @Test - def testInconsistentLogSegmentRange(): Unit = { - val logConfig = LogTestUtils.createLogConfig() - val log = createLog(logDir, logConfig) - - for (i <- 0 until 5) { - val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes) - log.appendAsLeader(TestUtils.records(List(record)), leaderEpoch = 0) - log.roll() - } - - assertThrows(classOf[IllegalArgumentException], () => log.logSegments(5, 1)) - } - - @Test - def testLogDelete(): Unit = { - val logConfig = LogTestUtils.createLogConfig() - val log = createLog(logDir, logConfig) - - for (i <- 0 to 100) { - val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes) - log.appendAsLeader(TestUtils.records(List(record)), leaderEpoch = 0) - log.roll() - } - - assertTrue(log.logSegments.nonEmpty) - assertFalse(logDir.listFiles.isEmpty) - - // delete the log - log.delete() - - assertEquals(0, log.logSegments.size) - assertFalse(logDir.exists) - } - /** * Test that "PeriodicProducerExpirationCheck" scheduled task gets canceled after log * is deleted. @@ -827,19 +720,6 @@ class LogTest { } } - @Test - def testSizeForLargeLogs(): Unit = { - val largeSize = Int.MaxValue.toLong * 2 - val logSegment: LogSegment = EasyMock.createMock(classOf[LogSegment]) - - EasyMock.expect(logSegment.size).andReturn(Int.MaxValue).anyTimes - EasyMock.replay(logSegment) - - assertEquals(Int.MaxValue, Log.sizeInBytes(Seq(logSegment))) - assertEquals(largeSize, Log.sizeInBytes(Seq(logSegment, logSegment))) - assertTrue(Log.sizeInBytes(Seq(logSegment, logSegment)) > Int.MaxValue) - } - @Test def testProducerIdMapOffsetUpdatedForNonIdempotentData(): Unit = { val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) @@ -2401,8 +2281,6 @@ class LogTest { assertEquals(Some(5), log.latestEpoch) } - - @Test def testSplitOnOffsetOverflow(): Unit = { // create a log such that one log segment has offsets that overflow, and call the split API on that segment @@ -2478,113 +2356,6 @@ class LogTest { assertFalse(LogTestUtils.hasOffsetOverflow(log)) } - @Test - def testParseTopicPartitionName(): Unit = { - val topic = "test_topic" - val partition = "143" - val dir = new File(logDir, topicPartitionName(topic, partition)) - val topicPartition = Log.parseTopicPartitionName(dir) - assertEquals(topic, topicPartition.topic) - assertEquals(partition.toInt, topicPartition.partition) - } - - /** - * Tests that log directories with a period in their name that have been marked for deletion - * are parsed correctly by `Log.parseTopicPartitionName` (see KAFKA-5232 for details). - */ - @Test - def testParseTopicPartitionNameWithPeriodForDeletedTopic(): Unit = { - val topic = "foo.bar-testtopic" - val partition = "42" - val dir = new File(logDir, Log.logDeleteDirName(new TopicPartition(topic, partition.toInt))) - val topicPartition = Log.parseTopicPartitionName(dir) - assertEquals(topic, topicPartition.topic, "Unexpected topic name parsed") - assertEquals(partition.toInt, topicPartition.partition, "Unexpected partition number parsed") - } - - @Test - def testParseTopicPartitionNameForEmptyName(): Unit = { - val dir = new File("") - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(dir), - () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) - } - - @Test - def testParseTopicPartitionNameForNull(): Unit = { - val dir: File = null - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(dir), - () => "KafkaException should have been thrown for dir: " + dir) - } - - @Test - def testParseTopicPartitionNameForMissingSeparator(): Unit = { - val topic = "test_topic" - val partition = "1999" - val dir = new File(logDir, topic + partition) - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(dir), - () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) - // also test the "-delete" marker case - val deleteMarkerDir = new File(logDir, topic + partition + "." + DeleteDirSuffix) - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(deleteMarkerDir), - () => "KafkaException should have been thrown for dir: " + deleteMarkerDir.getCanonicalPath) - } - - @Test - def testParseTopicPartitionNameForMissingTopic(): Unit = { - val topic = "" - val partition = "1999" - val dir = new File(logDir, topicPartitionName(topic, partition)) - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(dir), - () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) - - // also test the "-delete" marker case - val deleteMarkerDir = new File(logDir, Log.logDeleteDirName(new TopicPartition(topic, partition.toInt))) - - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(deleteMarkerDir), - () => "KafkaException should have been thrown for dir: " + deleteMarkerDir.getCanonicalPath) - } - - @Test - def testParseTopicPartitionNameForMissingPartition(): Unit = { - val topic = "test_topic" - val partition = "" - val dir = new File(logDir.getPath + topicPartitionName(topic, partition)) - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(dir), - () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) - - // also test the "-delete" marker case - val deleteMarkerDir = new File(logDir, topicPartitionName(topic, partition) + "." + DeleteDirSuffix) - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(deleteMarkerDir), - () => "KafkaException should have been thrown for dir: " + deleteMarkerDir.getCanonicalPath) - } - - @Test - def testParseTopicPartitionNameForInvalidPartition(): Unit = { - val topic = "test_topic" - val partition = "1999a" - val dir = new File(logDir, topicPartitionName(topic, partition)) - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(dir), - () => "KafkaException should have been thrown for dir: " + dir.getCanonicalPath) - - // also test the "-delete" marker case - val deleteMarkerDir = new File(logDir, topic + partition + "." + DeleteDirSuffix) - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(deleteMarkerDir), - () => "KafkaException should have been thrown for dir: " + deleteMarkerDir.getCanonicalPath) - } - - @Test - def testParseTopicPartitionNameForExistingInvalidDir(): Unit = { - val dir1 = new File(logDir.getPath + "/non_kafka_dir") - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(dir1), - () => "KafkaException should have been thrown for dir: " + dir1.getCanonicalPath) - val dir2 = new File(logDir.getPath + "/non_kafka_dir-delete") - assertThrows(classOf[KafkaException], () => Log.parseTopicPartitionName(dir2), - () => "KafkaException should have been thrown for dir: " + dir2.getCanonicalPath) - } - - def topicPartitionName(topic: String, partition: String): String = - topic + "-" + partition - @Test def testDeleteOldSegments(): Unit = { def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 30e8a6f20fff9..ad2178c05da6c 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -1663,21 +1663,15 @@ class ReplicaManagerTest { maxProducerIdExpirationMs, leaderEpochCache, producerStateManager)) + val localLog = new LocalLog(logDir, logConfig, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockScheduler, time, tp, mockLogDirFailureChannel) val mockLog = new Log( - _dir = logDir, - config = logConfig, - segments = segments, logStartOffset = offsets.logStartOffset, - recoveryPoint = offsets.recoveryPoint, - nextOffsetMetadata = offsets.nextOffsetMetadata, - scheduler = mockScheduler, + localLog = localLog, brokerTopicStats = mockBrokerTopicStats, - time = time, producerIdExpirationCheckIntervalMs = 30000, - topicPartition = tp, leaderEpochCache = leaderEpochCache, producerStateManager = producerStateManager, - logDirFailureChannel = mockLogDirFailureChannel, _topicId = topicId, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala index 6ff338fdc5912..5a32693c7c4b8 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala @@ -21,7 +21,7 @@ import java.io.{File, RandomAccessFile} import java.util.Properties import kafka.api.ApiVersion -import kafka.log.Log +import kafka.log.{Log, LogLoader} import kafka.server.KafkaConfig._ import kafka.server.{KafkaConfig, KafkaServer} import kafka.tools.DumpLogSegments @@ -153,7 +153,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness broker100.shutdown() //Delete the clean shutdown file to simulate crash - new File(broker100.config.logDirs.head, Log.CleanShutdownFile).delete() + new File(broker100.config.logDirs.head, LogLoader.CleanShutdownFile).delete() //Delete 5 messages from the leader's log on 100 deleteMessagesFromLogFile(5 * msg.length, broker100, 0) @@ -200,7 +200,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness brokers.foreach { b => b.shutdown() } //Delete the clean shutdown file to simulate crash - new File(brokers(0).config.logDirs(0), Log.CleanShutdownFile).delete() + new File(brokers(0).config.logDirs(0), LogLoader.CleanShutdownFile).delete() //Delete half the messages from the log file deleteMessagesFromLogFile(getLogFile(brokers(0), 0).length() / 2, brokers(0), 0) diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index b085ab0166513..2c61af772cc8d 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -19,8 +19,7 @@ package kafka.utils import java.util.Properties import java.util.concurrent.atomic._ import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} - -import kafka.log.{LoadLogParams, Log, LogConfig, LogLoader, LogManager, LogSegments, ProducerStateManager} +import kafka.log.{LoadLogParams, LocalLog, Log, LogConfig, LogLoader, LogManager, LogSegments, ProducerStateManager} import kafka.server.{BrokerTopicStats, LogDirFailureChannel} import kafka.utils.TestUtils.retry import org.junit.jupiter.api.Assertions._ @@ -139,10 +138,12 @@ class SchedulerTest { maxProducerIdExpirationMs, leaderEpochCache, producerStateManager)) - val log = new Log(logDir, logConfig, segments = segments, logStartOffset = offsets.logStartOffset, - recoveryPoint = offsets.recoveryPoint, nextOffsetMetadata = offsets.nextOffsetMetadata, scheduler, - brokerTopicStats, mockTime, LogManager.ProducerIdExpirationCheckIntervalMs, - topicPartition, leaderEpochCache, producerStateManager, logDirFailureChannel, + val localLog = new LocalLog(logDir, logConfig, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, scheduler, mockTime, topicPartition, logDirFailureChannel) + val log = new Log(logStartOffset = offsets.logStartOffset, + localLog = localLog, + brokerTopicStats, LogManager.ProducerIdExpirationCheckIntervalMs, + leaderEpochCache, producerStateManager, _topicId = None, keepPartitionMetadataFile = true) assertTrue(scheduler.taskRunning(log.producerExpireCheck)) log.close()