diff --git a/build.gradle b/build.gradle
index 4753be6ccc6af..2320291e32a24 100644
--- a/build.gradle
+++ b/build.gradle
@@ -3737,7 +3737,7 @@ project(':connect:mirror') {
testImplementation project(':core')
testImplementation project(':test-common:test-common-runtime')
testImplementation project(':server')
- testImplementation project(':server-common').sourceSets.test.output
+ testImplementation project(':server-common')
testRuntimeOnly project(':connect:runtime')
diff --git a/checkstyle/import-control-server-common.xml b/checkstyle/import-control-server-common.xml
index 2b5c9002c9b9e..8d85dffa341f0 100644
--- a/checkstyle/import-control-server-common.xml
+++ b/checkstyle/import-control-server-common.xml
@@ -130,6 +130,7 @@
+
diff --git a/checkstyle/import-control-storage.xml b/checkstyle/import-control-storage.xml
index d96311d2897c0..e3bee65b6eb1d 100644
--- a/checkstyle/import-control-storage.xml
+++ b/checkstyle/import-control-storage.xml
@@ -88,6 +88,7 @@
+
diff --git a/config/log4j2.yaml b/config/log4j2.yaml
index 7ee6f001e18ea..49bcf78d136c6 100644
--- a/config/log4j2.yaml
+++ b/config/log4j2.yaml
@@ -133,7 +133,17 @@ Configuration:
AppenderRef:
ref: ControllerAppender
# LogCleaner logger
- - name: kafka.log.LogCleaner
+ - name: org.apache.kafka.storage.internals.log.LogCleaner
+ level: INFO
+ additivity: false
+ AppenderRef:
+ ref: CleanerAppender
+ - name: org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread
+ level: INFO
+ additivity: false
+ AppenderRef:
+ ref: CleanerAppender
+ - name: org.apache.kafka.storage.internals.log.Cleaner
level: INFO
additivity: false
AppenderRef:
diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala
deleted file mode 100644
index 50b160056b5fd..0000000000000
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ /dev/null
@@ -1,1307 +0,0 @@
-/**
- * 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.lang.{Long => JLong}
-import java.nio._
-import java.util
-import java.util.{Date, Optional}
-import java.util.concurrent.TimeUnit
-import kafka.log.LogCleaner.{CleanerRecopyPercentMetricName, DeadThreadCountMetricName, MaxBufferUtilizationPercentMetricName, MaxCleanTimeMetricName, MaxCompactionDelayMetricsName}
-import kafka.server.{BrokerReconfigurable, KafkaConfig}
-import kafka.utils.Logging
-import org.apache.kafka.common.{KafkaException, TopicPartition}
-import org.apache.kafka.common.config.ConfigException
-import org.apache.kafka.common.errors.{CorruptRecordException, KafkaStorageException}
-import org.apache.kafka.common.record.MemoryRecords.RecordFilter
-import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention
-import org.apache.kafka.common.record._
-import org.apache.kafka.common.utils.{BufferSupplier, Time}
-import org.apache.kafka.server.config.ServerConfigs
-import org.apache.kafka.server.metrics.KafkaMetricsGroup
-import org.apache.kafka.server.util.ShutdownableThread
-import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, LastRecord, LogCleanerManager, LogCleaningAbortedException, LogCleaningException, LogDirFailureChannel, LogSegment, LogSegmentOffsetOverflowException, LogToClean, OffsetMap, PreCleanStats, SkimpyOffsetMap, ThreadShutdownException, TransactionIndex, UnifiedLog}
-import org.apache.kafka.storage.internals.utils.Throttler
-
-import scala.jdk.CollectionConverters._
-import scala.collection.mutable.ListBuffer
-import scala.collection.{Iterable, Seq, Set, mutable}
-import scala.jdk.OptionConverters.{RichOption, RichOptional}
-import scala.util.control.ControlThrowable
-
-/**
- * The cleaner is responsible for removing obsolete records from logs which have the "compact" retention strategy.
- * A message with key K and offset O is obsolete if there exists a message with key K and offset O' such that O < O'.
- *
- * Each log can be thought of being split into two sections of segments: a "clean" section which has previously been cleaned followed by a
- * "dirty" section that has not yet been cleaned. The dirty section is further divided into the "cleanable" section followed by an "uncleanable" section.
- * The uncleanable section is excluded from cleaning. The active log segment is always uncleanable. If there is a
- * compaction lag time set, segments whose largest message timestamp is within the compaction lag time of the cleaning operation are also uncleanable.
- *
- * The cleaning is carried out by a pool of background threads. Each thread chooses the dirtiest log that has the "compact" retention policy
- * and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log.
- *
- * To clean a log the cleaner first builds a mapping of key=>last_offset for the dirty section of the log. See {@link OffsetMap} for details of
- * the implementation of the mapping.
- *
- * Once the key=>last_offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a
- * higher offset than what is found in the segment (i.e. messages with a key that appears in the dirty section of the log).
- *
- * To avoid segments shrinking to very small sizes with repeated cleanings we implement a rule by which if we will merge successive segments when
- * doing a cleaning if their log and index size are less than the maximum log and index size prior to the clean beginning.
- *
- * Cleaned segments are swapped into the log as they become available.
- *
- * One nuance that the cleaner must handle is log truncation. If a log is truncated while it is being cleaned the cleaning of that log is aborted.
- *
- * Messages with null payload are treated as deletes for the purpose of log compaction. This means that they receive special treatment by the cleaner.
- * The cleaner will only retain delete records for a period of time to avoid accumulating space indefinitely. This period of time is configurable on a per-topic
- * basis and is measured from the time the segment enters the clean portion of the log (at which point any prior message with that key has been removed).
- * Delete markers in the clean section of the log that are older than this time will not be retained when log segments are being recopied as part of cleaning.
- * This time is tracked by setting the base timestamp of a record batch with delete markers when the batch is recopied in the first cleaning that encounters
- * it. The relative timestamps of the records in the batch are also modified when recopied in this cleaning according to the new base timestamp of the batch.
- *
- * Note that cleaning is more complicated with the idempotent/transactional producer capabilities. The following
- * are the key points:
- *
- * 1. In order to maintain sequence number continuity for active producers, we always retain the last batch
- * from each producerId, even if all the records from the batch have been removed. The batch will be removed
- * once the producer either writes a new batch or is expired due to inactivity.
- * 2. We do not clean beyond the last stable offset. This ensures that all records observed by the cleaner have
- * been decided (i.e. committed or aborted). In particular, this allows us to use the transaction index to
- * collect the aborted transactions ahead of time.
- * 3. Records from aborted transactions are removed by the cleaner immediately without regard to record keys.
- * 4. Transaction markers are retained until all record batches from the same transaction have been removed and
- * a sufficient amount of time has passed to reasonably ensure that an active consumer wouldn't consume any
- * data from the transaction prior to reaching the offset of the marker. This follows the same logic used for
- * tombstone deletion.
- *
- * @param initialConfig Initial configuration parameters for the cleaner. Actual config may be dynamically updated.
- * @param logDirs The directories where offset checkpoints reside
- * @param logs The map of logs
- * @param logDirFailureChannel The channel used to add offline log dirs that may be encountered when cleaning the log
- * @param time A way to control the passage of time
- */
-class LogCleaner(initialConfig: CleanerConfig,
- val logDirs: Seq[File],
- val logs: util.concurrent.ConcurrentMap[TopicPartition, UnifiedLog],
- val logDirFailureChannel: LogDirFailureChannel,
- time: Time = Time.SYSTEM) extends Logging with BrokerReconfigurable {
- // Visible for test.
- private[log] val metricsGroup = new KafkaMetricsGroup(this.getClass)
-
- /* Log cleaner configuration which may be dynamically updated */
- @volatile private var config = initialConfig
-
- /* for managing the state of partitions being cleaned. package-private to allow access in tests */
- private[log] val cleanerManager = new LogCleanerManager(logDirs.asJava, logs, logDirFailureChannel)
-
- /* a throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate */
- private[log] val throttler = new Throttler(config.maxIoBytesPerSecond, 300, "cleaner-io", "bytes", time)
-
- private[log] val cleaners = mutable.ArrayBuffer[CleanerThread]()
-
- /**
- * @param f to compute the result
- * @return the max value or 0 if there is no cleaner
- */
- private[log] def maxOverCleanerThreads(f: CleanerThread => Double): Double =
- cleaners.map(f).maxOption.getOrElse(0.0d)
-
- /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */
- metricsGroup.newGauge(MaxBufferUtilizationPercentMetricName,
- () => (maxOverCleanerThreads(_.lastStats.bufferUtilization) * 100).toInt)
-
- /* a metric to track the recopy rate of each thread's last cleaning */
- metricsGroup.newGauge(CleanerRecopyPercentMetricName, () => {
- val stats = cleaners.map(_.lastStats)
- val recopyRate = stats.iterator.map(_.bytesWritten).sum.toDouble / math.max(stats.iterator.map(_.bytesRead).sum, 1)
- (100 * recopyRate).toInt
- })
-
- /* a metric to track the maximum cleaning time for the last cleaning from each thread */
- metricsGroup.newGauge(MaxCleanTimeMetricName, () => maxOverCleanerThreads(_.lastStats.elapsedSecs).toInt)
-
- // a metric to track delay between the time when a log is required to be compacted
- // as determined by max compaction lag and the time of last cleaner run.
- metricsGroup.newGauge(MaxCompactionDelayMetricsName,
- () => (maxOverCleanerThreads(_.lastPreCleanStats.maxCompactionDelayMs.toDouble) / 1000).toInt)
-
- metricsGroup.newGauge(DeadThreadCountMetricName, () => deadThreadCount)
-
- private[log] def deadThreadCount: Int = cleaners.count(_.isThreadFailed)
-
- /**
- * Start the background cleaner threads
- */
- def startup(): Unit = {
- info("Starting the log cleaner")
- (0 until config.numThreads).foreach { i =>
- val cleaner = new CleanerThread(i)
- cleaners += cleaner
- cleaner.start()
- }
- }
-
- /**
- * Stop the background cleaner threads
- */
- private[this] def shutdownCleaners(): Unit = {
- info("Shutting down the log cleaner.")
- cleaners.foreach(_.shutdown())
- cleaners.clear()
- }
-
- /**
- * Stop the background cleaner threads
- */
- def shutdown(): Unit = {
- try {
- shutdownCleaners()
- } finally {
- removeMetrics()
- }
- }
-
- /**
- * Remove metrics
- */
- def removeMetrics(): Unit = {
- LogCleaner.MetricNames.foreach(metricsGroup.removeMetric)
- cleanerManager.removeMetrics()
- }
-
- /**
- * @return A set of configs that is reconfigurable in LogCleaner
- */
- override def reconfigurableConfigs: Set[String] = {
- LogCleaner.ReconfigurableConfigs
- }
-
- /**
- * Validate the new cleaner threads num is reasonable
- *
- * @param newConfig A submitted new KafkaConfig instance that contains new cleaner config
- */
- override def validateReconfiguration(newConfig: KafkaConfig): Unit = {
- val numThreads = LogCleaner.cleanerConfig(newConfig).numThreads
- val currentThreads = config.numThreads
- if (numThreads < 1)
- throw new ConfigException(s"Log cleaner threads should be at least 1")
- if (numThreads < currentThreads / 2)
- throw new ConfigException(s"Log cleaner threads cannot be reduced to less than half the current value $currentThreads")
- if (numThreads > currentThreads * 2)
- throw new ConfigException(s"Log cleaner threads cannot be increased to more than double the current value $currentThreads")
-
- }
-
- /**
- * Reconfigure log clean config. The will:
- * 1. update desiredRatePerSec in Throttler with logCleanerIoMaxBytesPerSecond, if necessary
- * 2. stop current log cleaners and create new ones.
- * That ensures that if any of the cleaners had failed, new cleaners are created to match the new config.
- *
- * @param oldConfig the old log cleaner config
- * @param newConfig the new log cleaner config reconfigured
- */
- override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = {
- config = LogCleaner.cleanerConfig(newConfig)
-
- val maxIoBytesPerSecond = config.maxIoBytesPerSecond
- if (maxIoBytesPerSecond != oldConfig.logCleanerIoMaxBytesPerSecond) {
- info(s"Updating logCleanerIoMaxBytesPerSecond: $maxIoBytesPerSecond")
- throttler.updateDesiredRatePerSec(maxIoBytesPerSecond)
- }
- // call shutdownCleaners() instead of shutdown to avoid unnecessary deletion of metrics
- shutdownCleaners()
- startup()
- }
-
- /**
- * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of
- * the partition is aborted.
- *
- * @param topicPartition The topic and partition to abort cleaning
- */
- def abortCleaning(topicPartition: TopicPartition): Unit = {
- cleanerManager.abortCleaning(topicPartition)
- }
-
- /**
- * Update checkpoint file to remove partitions if necessary.
- *
- * @param dataDir The data dir to be updated if necessary
- * @param partitionToRemove The topicPartition to be removed, default none
- */
- def updateCheckpoints(dataDir: File, partitionToRemove: Option[TopicPartition] = None): Unit = {
- cleanerManager.updateCheckpoints(dataDir, Optional.empty(), partitionToRemove.toJava)
- }
-
- /**
- * Alter the checkpoint directory for the `topicPartition`, to remove the data in `sourceLogDir`, and add the data in `destLogDir`
- * Generally occurs when the disk balance ends and replaces the previous file with the future file
- *
- * @param topicPartition The topic and partition to alter checkpoint
- * @param sourceLogDir The source log dir to remove checkpoint
- * @param destLogDir The dest log dir to remove checkpoint
- */
- def alterCheckpointDir(topicPartition: TopicPartition, sourceLogDir: File, destLogDir: File): Unit = {
- cleanerManager.alterCheckpointDir(topicPartition, sourceLogDir, destLogDir)
- }
-
- /**
- * Stop cleaning logs in the provided directory when handling log dir failure
- *
- * @param dir the absolute path of the log dir
- */
- def handleLogDirFailure(dir: String): Unit = {
- cleanerManager.handleLogDirFailure(dir)
- }
-
- /**
- * Truncate cleaner offset checkpoint for the given partition if its checkpoint offset is larger than the given offset
- *
- * @param dataDir The data dir to be truncated if necessary
- * @param topicPartition The topic and partition to truncate checkpoint offset
- * @param offset The given offset to be compared
- */
- def maybeTruncateCheckpoint(dataDir: File, topicPartition: TopicPartition, offset: Long): Unit = {
- cleanerManager.maybeTruncateCheckpoint(dataDir, topicPartition, offset)
- }
-
- /**
- * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition.
- * This call blocks until the cleaning of the partition is aborted and paused.
- *
- * @param topicPartition The topic and partition to abort and pause cleaning
- */
- def abortAndPauseCleaning(topicPartition: TopicPartition): Unit = {
- cleanerManager.abortAndPauseCleaning(topicPartition)
- }
-
- /**
- * Resume the cleaning of paused partitions.
- *
- * @param topicPartitions The collection of topicPartitions to be resumed cleaning
- */
- def resumeCleaning(topicPartitions: Iterable[TopicPartition]): Unit = {
- cleanerManager.resumeCleaning(topicPartitions.toList.asJava)
- }
-
- /**
- * For testing, a way to know when work has completed. This method waits until the
- * cleaner has processed up to the given offset on the specified topic/partition
- *
- * @param topicPartition The topic and partition to be cleaned
- * @param offset The first dirty offset that the cleaner doesn't have to clean
- * @param maxWaitMs The maximum time in ms to wait for cleaner
- *
- * @return A boolean indicating whether the work has completed before timeout
- */
- def awaitCleaned(topicPartition: TopicPartition, offset: Long, maxWaitMs: Long = 60000L): Boolean = {
- def isCleaned = Option(cleanerManager.allCleanerCheckpoints.get(topicPartition)).fold(false)(_ >= offset)
- var remainingWaitMs = maxWaitMs
- while (!isCleaned && remainingWaitMs > 0) {
- val sleepTime = math.min(100, remainingWaitMs)
- Thread.sleep(sleepTime)
- remainingWaitMs -= sleepTime
- }
- isCleaned
- }
-
- /**
- * To prevent race between retention and compaction,
- * retention threads need to make this call to obtain:
- *
- * @return A list of log partitions that retention threads can safely work on
- */
- def pauseCleaningForNonCompactedPartitions(): Iterable[(TopicPartition, UnifiedLog)] = {
- cleanerManager.pauseCleaningForNonCompactedPartitions().asScala.map(entry => (entry.getKey, entry.getValue))
- }
-
- // Only for testing
- private[kafka] def currentConfig: CleanerConfig = config
-
- // Only for testing
- private[log] def cleanerCount: Int = cleaners.size
-
- /**
- * The cleaner threads do the actual log cleaning. Each thread processes does its cleaning repeatedly by
- * choosing the dirtiest log, cleaning it, and then swapping in the cleaned segments.
- */
- private[log] class CleanerThread(threadId: Int)
- extends ShutdownableThread(s"kafka-log-cleaner-thread-$threadId", false) with Logging {
- protected override def loggerName: String = classOf[LogCleaner].getName
-
- this.logIdent = logPrefix
-
- if (config.dedupeBufferSize / config.numThreads > Int.MaxValue)
- warn("Cannot use more than 2G of cleaner buffer space per cleaner thread, ignoring excess buffer space...")
-
- val cleaner = new Cleaner(id = threadId,
- offsetMap = new SkimpyOffsetMap(math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt,
- config.hashAlgorithm),
- ioBufferSize = config.ioBufferSize / config.numThreads / 2,
- maxIoBufferSize = config.maxMessageSize,
- dupBufferLoadFactor = config.dedupeBufferLoadFactor,
- throttler = throttler,
- time = time,
- checkDone = checkDone)
-
- @volatile var lastStats: CleanerStats = new CleanerStats()
- @volatile var lastPreCleanStats: PreCleanStats = new PreCleanStats()
-
- /**
- * Check if the cleaning for a partition is aborted. If so, throw an exception.
- *
- * @param topicPartition The topic and partition to check
- */
- private def checkDone(topicPartition: TopicPartition): Unit = {
- if (!isRunning)
- throw new ThreadShutdownException
- cleanerManager.checkCleaningAborted(topicPartition)
- }
-
- /**
- * The main loop for the cleaner thread
- * Clean a log if there is a dirty log available, otherwise sleep for a bit
- */
- override def doWork(): Unit = {
- val cleaned = tryCleanFilthiestLog()
- if (!cleaned)
- pause(config.backoffMs, TimeUnit.MILLISECONDS)
-
- cleanerManager.maintainUncleanablePartitions()
- }
-
- /**
- * Cleans a log if there is a dirty log available
- *
- * @return whether a log was cleaned
- */
- private def tryCleanFilthiestLog(): Boolean = {
- try {
- cleanFilthiestLog()
- } catch {
- case e: LogCleaningException =>
- warn(s"Unexpected exception thrown when cleaning log ${e.log}. Marking its partition (${e.log.topicPartition}) as uncleanable", e)
- cleanerManager.markPartitionUncleanable(e.log.parentDir, e.log.topicPartition)
-
- false
- }
- }
-
- @throws(classOf[LogCleaningException])
- private def cleanFilthiestLog(): Boolean = {
- val preCleanStats = new PreCleanStats()
- val ltc = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats).toScala
- val cleaned = ltc match {
- case None =>
- false
- case Some(cleanable) =>
- // there's a log, clean it
- this.lastPreCleanStats = preCleanStats
- try {
- cleanLog(cleanable)
- true
- } catch {
- case e @ (_: ThreadShutdownException | _: ControlThrowable) => throw e
- case e: Exception => throw new LogCleaningException(cleanable.log, e.getMessage, e)
- }
- }
- val deletable = cleanerManager.deletableLogs().asScala
- try {
- deletable.foreach { case (_, log) =>
- try {
- log.deleteOldSegments()
- } catch {
- case e @ (_: ThreadShutdownException | _: ControlThrowable) => throw e
- case e: Exception => throw new LogCleaningException(log, e.getMessage, e)
- }
- }
- } finally {
- cleanerManager.doneDeleting(deletable.keys.toList.asJava)
- }
-
- cleaned
- }
-
- private def cleanLog(cleanable: LogToClean): Unit = {
- val startOffset = cleanable.firstDirtyOffset
- var endOffset = startOffset
- try {
- val (nextDirtyOffset, cleanerStats) = cleaner.clean(cleanable)
- endOffset = nextDirtyOffset
- recordStats(cleaner.id, cleanable.log.name, startOffset, endOffset, cleanerStats)
- } catch {
- case _: LogCleaningAbortedException => // task can be aborted, let it go.
- case _: KafkaStorageException => // partition is already offline. let it go.
- case e: IOException =>
- val logDirectory = cleanable.log.parentDir
- val msg = s"Failed to clean up log for ${cleanable.topicPartition} in dir $logDirectory due to IOException"
- logDirFailureChannel.maybeAddOfflineLogDir(logDirectory, msg, e)
- } finally {
- cleanerManager.doneCleaning(cleanable.topicPartition, cleanable.log.parentDirFile, endOffset)
- }
- }
-
- /**
- * Log out statistics on a single run of the cleaner.
- *
- * @param id The cleaner thread id
- * @param name The cleaned log name
- * @param from The cleaned offset that is the first dirty offset to begin
- * @param to The cleaned offset that is the first not cleaned offset to end
- * @param stats The statistics for this round of cleaning
- */
- private def recordStats(id: Int, name: String, from: Long, to: Long, stats: CleanerStats): Unit = {
- this.lastStats = stats
- def mb(bytes: Double) = bytes / (1024*1024)
- val message =
- "%n\tLog cleaner thread %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) +
- "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead.toDouble),
- stats.elapsedSecs,
- mb(stats.bytesRead.toDouble / stats.elapsedSecs)) +
- "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead.toDouble),
- stats.elapsedIndexSecs,
- mb(stats.mapBytesRead.toDouble) / stats.elapsedIndexSecs,
- 100 * stats.elapsedIndexSecs / stats.elapsedSecs) +
- "\tBuffer utilization: %.1f%%%n".format(100 * stats.bufferUtilization) +
- "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead.toDouble),
- stats.elapsedSecs - stats.elapsedIndexSecs,
- mb(stats.bytesRead.toDouble) / (stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs) / stats.elapsedSecs) +
- "\tStart size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesRead.toDouble), stats.messagesRead) +
- "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten.toDouble), stats.messagesWritten) +
- "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead),
- 100.0 * (1.0 - stats.messagesWritten.toDouble/stats.messagesRead))
- info(message)
- if (lastPreCleanStats.delayedPartitions > 0) {
- info("\tCleanable partitions: %d, Delayed partitions: %d, max delay: %d".format(lastPreCleanStats.cleanablePartitions, lastPreCleanStats.delayedPartitions, lastPreCleanStats.maxCompactionDelayMs))
- }
- if (stats.invalidMessagesRead > 0) {
- warn("\tFound %d invalid messages during compaction.".format(stats.invalidMessagesRead))
- }
- }
-
- }
-}
-
-object LogCleaner {
- val ReconfigurableConfigs: Set[String] = Set(
- CleanerConfig.LOG_CLEANER_THREADS_PROP,
- CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP,
- CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP,
- CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP,
- ServerConfigs.MESSAGE_MAX_BYTES_CONFIG,
- CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP,
- CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP
- )
-
- def cleanerConfig(config: KafkaConfig): CleanerConfig = {
- new CleanerConfig(config.logCleanerThreads,
- config.logCleanerDedupeBufferSize,
- config.logCleanerDedupeBufferLoadFactor,
- config.logCleanerIoBufferSize,
- config.messageMaxBytes,
- config.logCleanerIoMaxBytesPerSecond,
- config.logCleanerBackoffMs,
- config.logCleanerEnable)
-
- }
-
- // Visible for test.
- private[log] val MaxBufferUtilizationPercentMetricName = "max-buffer-utilization-percent"
- private val CleanerRecopyPercentMetricName = "cleaner-recopy-percent"
- private[log] val MaxCleanTimeMetricName = "max-clean-time-secs"
- private[log] val MaxCompactionDelayMetricsName = "max-compaction-delay-secs"
- private val DeadThreadCountMetricName = "DeadThreadCount"
- // package private for testing
- private[log] val MetricNames = Set(
- MaxBufferUtilizationPercentMetricName,
- CleanerRecopyPercentMetricName,
- MaxCleanTimeMetricName,
- MaxCompactionDelayMetricsName,
- DeadThreadCountMetricName)
-}
-
-/**
- * This class holds the actual logic for cleaning a log
- * @param id An identifier used for logging
- * @param offsetMap The map used for deduplication
- * @param ioBufferSize The size of the buffers to use. Memory usage will be 2x this number as there is a read and write buffer.
- * @param maxIoBufferSize The maximum size of a message that can appear in the log
- * @param dupBufferLoadFactor The maximum percent full for the deduplication buffer
- * @param throttler The throttler instance to use for limiting I/O rate.
- * @param time The time instance
- * @param checkDone Check if the cleaning for a partition is finished or aborted.
- */
-private[log] class Cleaner(val id: Int,
- val offsetMap: OffsetMap,
- ioBufferSize: Int,
- maxIoBufferSize: Int,
- dupBufferLoadFactor: Double,
- throttler: Throttler,
- time: Time,
- checkDone: TopicPartition => Unit) extends Logging {
-
- protected override def loggerName: String = classOf[LogCleaner].getName
-
- this.logIdent = s"Cleaner $id: "
-
- /* buffer used for read i/o */
- private var readBuffer = ByteBuffer.allocate(ioBufferSize)
-
- /* buffer used for write i/o */
- private var writeBuffer = ByteBuffer.allocate(ioBufferSize)
-
- private val decompressionBufferSupplier = BufferSupplier.create()
-
- require(offsetMap.slots * dupBufferLoadFactor > 1, "offset map is too small to fit in even a single message, so log cleaning will never make progress. You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads")
-
- /**
- * Clean the given log
- *
- * @param cleanable The log to be cleaned
- *
- * @return The first offset not cleaned and the statistics for this round of cleaning
- */
- private[log] def clean(cleanable: LogToClean): (Long, CleanerStats) = {
- doClean(cleanable, time.milliseconds())
- }
-
- /**
- * Clean the given log
- *
- * @param cleanable The log to be cleaned
- * @param currentTime The current timestamp for doing cleaning
- *
- * @return The first offset not cleaned and the statistics for this round of cleaning
- * */
- private[log] def doClean(cleanable: LogToClean, currentTime: Long): (Long, CleanerStats) = {
- info("Beginning cleaning of log %s".format(cleanable.log.name))
-
- // figure out the timestamp below which it is safe to remove delete tombstones
- // this position is defined to be a configurable time beneath the last modified time of the last clean segment
- // this timestamp is only used on the older message formats older than MAGIC_VALUE_V2
- val legacyDeleteHorizonMs =
- cleanable.log.logSegments(0, cleanable.firstDirtyOffset).asScala.lastOption match {
- case None => 0L
- case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs
- }
-
- val log = cleanable.log
- val stats = new CleanerStats()
-
- // build the offset map
- info("Building offset map for %s...".format(cleanable.log.name))
- val upperBoundOffset = cleanable.firstUncleanableOffset
- buildOffsetMap(log, cleanable.firstDirtyOffset, upperBoundOffset, offsetMap, stats)
- val endOffset = offsetMap.latestOffset + 1
- stats.indexDone()
-
- // determine the timestamp up to which the log will be cleaned
- // this is the lower of the last active segment and the compaction lag
- val cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset).asScala.lastOption.map(_.lastModified).getOrElse(0L)
-
- // group the segments and clean the groups
- info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to upper bound deletion horizon %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(legacyDeleteHorizonMs)))
- val transactionMetadata = new CleanedTransactionMetadata
-
- val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize,
- log.config.maxIndexSize, cleanable.firstUncleanableOffset)
- for (group <- groupedSegments)
- cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, legacyDeleteHorizonMs, upperBoundOffset)
-
- // record buffer utilization
- stats.bufferUtilization = offsetMap.utilization
-
- stats.allDone()
-
- (endOffset, stats)
- }
-
- /**
- * Clean a group of segments into a single replacement segment
- *
- * @param log The log being cleaned
- * @param segments The group of segments being cleaned
- * @param map The offset map to use for cleaning segments
- * @param currentTime The current time in milliseconds
- * @param stats Collector for cleaning statistics
- * @param transactionMetadata State of ongoing transactions which is carried between the cleaning
- * of the grouped segments
- * @param legacyDeleteHorizonMs The delete horizon used for tombstones whose version is less than 2
- * @param upperBoundOffsetOfCleaningRound The upper bound offset of this round of cleaning
- */
- private[log] def cleanSegments(log: UnifiedLog,
- segments: Seq[LogSegment],
- map: OffsetMap,
- currentTime: Long,
- stats: CleanerStats,
- transactionMetadata: CleanedTransactionMetadata,
- legacyDeleteHorizonMs: Long,
- upperBoundOffsetOfCleaningRound: Long): Unit = {
- // create a new segment with a suffix appended to the name of the log and indexes
- val cleaned = UnifiedLog.createNewCleanedSegment(log.dir, log.config, segments.head.baseOffset)
- transactionMetadata.cleanedIndex = Some(cleaned.txnIndex)
-
- try {
- // clean segments into the new destination segment
- val iter = segments.iterator
- var currentSegmentOpt: Option[LogSegment] = Some(iter.next())
- val lastOffsetOfActiveProducers = log.lastRecordsOfActiveProducers
-
- while (currentSegmentOpt.isDefined) {
- val currentSegment = currentSegmentOpt.get
- val nextSegmentOpt = if (iter.hasNext) Some(iter.next()) else None
-
- // Note that it is important to collect aborted transactions from the full log segment
- // range since we need to rebuild the full transaction index for the new segment.
- val startOffset = currentSegment.baseOffset
- val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(currentSegment.readNextOffset)
- val abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset)
- transactionMetadata.addAbortedTransactions(abortedTransactions.asScala.toList)
-
- val retainLegacyDeletesAndTxnMarkers = currentSegment.lastModified > legacyDeleteHorizonMs
- info(s"Cleaning $currentSegment in log ${log.name} into ${cleaned.baseOffset} " +
- s"with an upper bound deletion horizon $legacyDeleteHorizonMs computed from " +
- s"the segment last modified time of ${currentSegment.lastModified}," +
- s"${if(retainLegacyDeletesAndTxnMarkers) "retaining" else "discarding"} deletes.")
-
- try {
- cleanInto(log.topicPartition, currentSegment.log, cleaned, map, retainLegacyDeletesAndTxnMarkers, log.config.deleteRetentionMs,
- log.config.maxMessageSize, transactionMetadata, lastOffsetOfActiveProducers,
- upperBoundOffsetOfCleaningRound, stats, currentTime = currentTime)
- } catch {
- case e: LogSegmentOffsetOverflowException =>
- // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from
- // scratch once the split is complete.
- info(s"Caught segment overflow error during cleaning: ${e.getMessage}")
- log.splitOverflowedSegment(currentSegment)
- throw new LogCleaningAbortedException()
- }
- currentSegmentOpt = nextSegmentOpt
- }
-
- cleaned.onBecomeInactiveSegment()
- // flush new segment to disk before swap
- cleaned.flush()
-
- // update the modification date to retain the last modified date of the original files
- val modified = segments.last.lastModified
- cleaned.setLastModified(modified)
-
- // swap in new segment
- info(s"Swapping in cleaned segment $cleaned for segment(s) $segments in log $log")
- log.replaceSegments(util.List.of(cleaned), segments.asJava)
- } catch {
- case e: LogCleaningAbortedException =>
- try cleaned.deleteIfExists()
- catch {
- case deleteException: Exception =>
- e.addSuppressed(deleteException)
- } finally throw e
- }
- }
-
- /**
- * Clean the given source log segment into the destination segment using the key=>offset mapping
- * provided
- *
- * @param topicPartition The topic and partition of the log segment to clean
- * @param sourceRecords The dirty log segment
- * @param dest The cleaned log segment
- * @param map The key=>offset mapping
- * @param retainLegacyDeletesAndTxnMarkers Should tombstones (lower than version 2) and markers be retained while cleaning this segment
- * @param deleteRetentionMs Defines how long a tombstone should be kept as defined by log configuration
- * @param maxLogMessageSize The maximum message size of the corresponding topic
- * @param transactionMetadata The state of ongoing transactions which is carried between the cleaning of the grouped segments
- * @param lastRecordsOfActiveProducers The active producers and its last data offset
- * @param upperBoundOffsetOfCleaningRound Next offset of the last batch in the source segment
- * @param stats Collector for cleaning statistics
- * @param currentTime The time at which the clean was initiated
- */
- private[log] def cleanInto(topicPartition: TopicPartition,
- sourceRecords: FileRecords,
- dest: LogSegment,
- map: OffsetMap,
- retainLegacyDeletesAndTxnMarkers: Boolean,
- deleteRetentionMs: Long,
- maxLogMessageSize: Int,
- transactionMetadata: CleanedTransactionMetadata,
- lastRecordsOfActiveProducers: util.Map[JLong, LastRecord],
- upperBoundOffsetOfCleaningRound: Long,
- stats: CleanerStats,
- currentTime: Long): Unit = {
- val logCleanerFilter: RecordFilter = new RecordFilter(currentTime, deleteRetentionMs) {
- var discardBatchRecords: Boolean = _
-
- override def checkBatchRetention(batch: RecordBatch): RecordFilter.BatchRetentionResult = {
- // we piggy-back on the tombstone retention logic to delay deletion of transaction markers.
- // note that we will never delete a marker until all the records from that transaction are removed.
- val canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata)
-
- if (batch.isControlBatch)
- discardBatchRecords = canDiscardBatch && batch.deleteHorizonMs().isPresent && batch.deleteHorizonMs().getAsLong <= this.currentTime
- else
- discardBatchRecords = canDiscardBatch
-
- def isBatchLastRecordOfProducer: Boolean = {
- // We retain the batch in order to preserve the state of active producers. There are three cases:
- // 1) The producer is no longer active, which means we can delete all records for that producer.
- // 2) The producer is still active and has a last data offset. We retain the batch that contains
- // this offset since it also contains the last sequence number for this producer.
- // 3) The last entry in the log is a transaction marker. We retain this marker since it has the
- // last producer epoch, which is needed to ensure fencing.
- lastRecordsOfActiveProducers.asScala.get(batch.producerId).exists { lastRecord =>
- if (lastRecord.lastDataOffset.isPresent) {
- batch.lastOffset == lastRecord.lastDataOffset.getAsLong
- } else {
- batch.isControlBatch && batch.producerEpoch == lastRecord.producerEpoch
- }
- }
- }
-
- val batchRetention: BatchRetention =
- if (batch.hasProducerId && isBatchLastRecordOfProducer)
- BatchRetention.RETAIN_EMPTY
- else if (batch.nextOffset == upperBoundOffsetOfCleaningRound) {
- // retain the last batch of the cleaning round, even if it's empty, so that last offset information
- // is not lost after cleaning.
- BatchRetention.RETAIN_EMPTY
- } else if (discardBatchRecords)
- BatchRetention.DELETE
- else
- BatchRetention.DELETE_EMPTY
- new RecordFilter.BatchRetentionResult(batchRetention, canDiscardBatch && batch.isControlBatch)
- }
-
- override def shouldRetainRecord(batch: RecordBatch, record: Record): Boolean = {
- if (discardBatchRecords)
- // The batch is only retained to preserve producer sequence information; the records can be removed
- false
- else if (batch.isControlBatch)
- true
- else
- Cleaner.this.shouldRetainRecord(map, retainLegacyDeletesAndTxnMarkers, batch, record, stats, currentTime = this.currentTime)
- }
- }
-
- var position = 0
- while (position < sourceRecords.sizeInBytes) {
- checkDone(topicPartition)
- // read a chunk of messages and copy any that are to be retained to the write buffer to be written out
- readBuffer.clear()
- writeBuffer.clear()
-
- sourceRecords.readInto(readBuffer, position)
- val records = MemoryRecords.readableRecords(readBuffer)
- throttler.maybeThrottle(records.sizeInBytes)
- val result = records.filterTo(logCleanerFilter, writeBuffer, decompressionBufferSupplier)
-
- stats.readMessages(result.messagesRead, result.bytesRead)
- stats.recopyMessages(result.messagesRetained, result.bytesRetained)
-
- position += result.bytesRead
-
- // if any messages are to be retained, write them out
- val outputBuffer = result.outputBuffer
- if (outputBuffer.position() > 0) {
- outputBuffer.flip()
- val retained = MemoryRecords.readableRecords(outputBuffer)
- // it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads
- // after `Log.replaceSegments` (which acquires the lock) is called
- dest.append(result.maxOffset, retained)
- throttler.maybeThrottle(outputBuffer.limit())
- }
-
- // if we read bytes but didn't get even one complete batch, our I/O buffer is too small, grow it and try again
- // `result.bytesRead` contains bytes from `messagesRead` and any discarded batches.
- if (readBuffer.limit() > 0 && result.bytesRead == 0)
- growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records)
- }
- restoreBuffers()
- }
-
-
- /**
- * Grow buffers to process next batch of records from `sourceRecords.` Buffers are doubled in size
- * up to a maximum of `maxLogMessageSize`. In some scenarios, a record could be bigger than the
- * current maximum size configured for the log. For example:
- * 1. A compacted topic using compression may contain a message set slightly larger than max.message.bytes
- * 2. max.message.bytes of a topic could have been reduced after writing larger messages
- * In these cases, grow the buffer to hold the next batch.
- *
- * @param sourceRecords The dirty log segment records to process
- * @param position The current position in the read buffer to read from
- * @param maxLogMessageSize The maximum record size in bytes for the topic
- * @param memoryRecords The memory records in read buffer
- */
- private def growBuffersOrFail(sourceRecords: FileRecords,
- position: Int,
- maxLogMessageSize: Int,
- memoryRecords: MemoryRecords): Unit = {
-
- val maxSize = if (readBuffer.capacity >= maxLogMessageSize) {
- val nextBatchSize = memoryRecords.firstBatchSize
- val logDesc = s"log segment ${sourceRecords.file} at position $position"
- if (nextBatchSize == null)
- throw new IllegalStateException(s"Could not determine next batch size for $logDesc")
- if (nextBatchSize <= 0)
- throw new IllegalStateException(s"Invalid batch size $nextBatchSize for $logDesc")
- if (nextBatchSize <= readBuffer.capacity)
- throw new IllegalStateException(s"Batch size $nextBatchSize < buffer size ${readBuffer.capacity}, but not processed for $logDesc")
- val bytesLeft = sourceRecords.channel.size - position
- if (nextBatchSize > bytesLeft)
- throw new CorruptRecordException(s"Log segment may be corrupt, batch size $nextBatchSize > $bytesLeft bytes left in segment for $logDesc")
- nextBatchSize.intValue
- } else
- maxLogMessageSize
-
- growBuffers(maxSize)
- }
-
- /**
- * Check if a batch should be discard by cleaned transaction state
- *
- * @param batch The batch of records to check
- * @param transactionMetadata The maintained transaction state about cleaning
- *
- * @return if the batch can be discarded
- */
- private def shouldDiscardBatch(batch: RecordBatch,
- transactionMetadata: CleanedTransactionMetadata): Boolean = {
- if (batch.isControlBatch)
- transactionMetadata.onControlBatchRead(batch)
- else
- transactionMetadata.onBatchRead(batch)
- }
-
- /**
- * Check if a record should be retained
- *
- * @param map The offset map(key=>offset) to use for cleaning segments
- * @param retainDeletesForLegacyRecords Should tombstones (lower than version 2) and markers be retained while cleaning this segment
- * @param batch The batch of records that the record belongs to
- * @param record The record to check
- * @param stats The collector for cleaning statistics
- * @param currentTime The current time that used to compare with the delete horizon time of the batch when judging a non-legacy record
- *
- * @return if the record can be retained
- */
- private def shouldRetainRecord(map: OffsetMap,
- retainDeletesForLegacyRecords: Boolean,
- batch: RecordBatch,
- record: Record,
- stats: CleanerStats,
- currentTime: Long): Boolean = {
- val pastLatestOffset = record.offset > map.latestOffset
- if (pastLatestOffset)
- return true
-
- if (record.hasKey) {
- val key = record.key
- val foundOffset = map.get(key)
- /* First,the message must have the latest offset for the key
- * then there are two cases in which we can retain a message:
- * 1) The message has value
- * 2) The message doesn't has value but it can't be deleted now.
- */
- val latestOffsetForKey = record.offset() >= foundOffset
- val legacyRecord = batch.magic() < RecordBatch.MAGIC_VALUE_V2
- def shouldRetainDeletes = {
- if (!legacyRecord)
- !batch.deleteHorizonMs().isPresent || currentTime < batch.deleteHorizonMs().getAsLong
- else
- retainDeletesForLegacyRecords
- }
- val isRetainedValue = record.hasValue || shouldRetainDeletes
- latestOffsetForKey && isRetainedValue
- } else {
- stats.invalidMessage()
- false
- }
- }
-
- /**
- * Double the I/O buffer capacity
- *
- * @param maxLogMessageSize The maximum record size in bytes allowed
- */
- private def growBuffers(maxLogMessageSize: Int): Unit = {
- val maxBufferSize = math.max(maxLogMessageSize, maxIoBufferSize)
- if (readBuffer.capacity >= maxBufferSize || writeBuffer.capacity >= maxBufferSize)
- throw new IllegalStateException("This log contains a message larger than maximum allowable size of %s.".format(maxBufferSize))
- val newSize = math.min(this.readBuffer.capacity * 2, maxBufferSize)
- info(s"Growing cleaner I/O buffers from ${readBuffer.capacity} bytes to $newSize bytes.")
- this.readBuffer = ByteBuffer.allocate(newSize)
- this.writeBuffer = ByteBuffer.allocate(newSize)
- }
-
- /**
- * Restore the I/O buffer capacity to its original size
- */
- private def restoreBuffers(): Unit = {
- if (this.readBuffer.capacity > this.ioBufferSize)
- this.readBuffer = ByteBuffer.allocate(this.ioBufferSize)
- if (this.writeBuffer.capacity > this.ioBufferSize)
- this.writeBuffer = ByteBuffer.allocate(this.ioBufferSize)
- }
-
- /**
- * Group the segments in a log into groups totaling less than a given size. the size is enforced separately for the log data and the index data.
- * We collect a group of such segments together into a single
- * destination segment. This prevents segment sizes from shrinking too much.
- *
- * @param segments The log segments to group
- * @param maxSize the maximum size in bytes for the total of all log data in a group
- * @param maxIndexSize the maximum size in bytes for the total of all index data in a group
- * @param firstUncleanableOffset The upper(exclusive) offset to clean to
- *
- * @return A list of grouped segments
- */
- private[log] def groupSegmentsBySize(segments: util.Collection[LogSegment], maxSize: Int, maxIndexSize: Int, firstUncleanableOffset: Long): List[Seq[LogSegment]] = {
- var grouped = List[List[LogSegment]]()
- var segs = segments.asScala.toList
- while (segs.nonEmpty) {
- var group = List(segs.head)
- var logSize = segs.head.size.toLong
- var indexSize = segs.head.offsetIndex.sizeInBytes.toLong
- var timeIndexSize = segs.head.timeIndex.sizeInBytes.toLong
- segs = segs.tail
- while (segs.nonEmpty &&
- logSize + segs.head.size <= maxSize &&
- indexSize + segs.head.offsetIndex.sizeInBytes <= maxIndexSize &&
- timeIndexSize + segs.head.timeIndex.sizeInBytes <= maxIndexSize &&
- //if first segment size is 0, we don't need to do the index offset range check.
- //this will avoid empty log left every 2^31 message.
- (segs.head.size == 0 ||
- lastOffsetForFirstSegment(segs, firstUncleanableOffset) - group.last.baseOffset <= Int.MaxValue)) {
- group = segs.head :: group
- logSize += segs.head.size
- indexSize += segs.head.offsetIndex.sizeInBytes
- timeIndexSize += segs.head.timeIndex.sizeInBytes
- segs = segs.tail
- }
- grouped ::= group.reverse
- }
- grouped.reverse
- }
-
- /**
- * We want to get the last offset in the first log segment in segs.
- * LogSegment.nextOffset() gives the exact last offset in a segment, but can be expensive since it requires
- * scanning the segment from the last index entry.
- * Therefore, we estimate the last offset of the first log segment by using
- * the base offset of the next segment in the list.
- * If the next segment doesn't exist, first Uncleanable Offset will be used.
- *
- * @param segs Remaining segments to group.
- * @param firstUncleanableOffset The upper(exclusive) offset to clean to
- * @return The estimated last offset for the first segment in segs
- */
- private def lastOffsetForFirstSegment(segs: List[LogSegment], firstUncleanableOffset: Long): Long = {
- if (segs.size > 1) {
- /* if there is a next segment, use its base offset as the bounding offset to guarantee we know
- * the worst case offset */
- segs(1).baseOffset - 1
- } else {
- //for the last segment in the list, use the first uncleanable offset.
- firstUncleanableOffset - 1
- }
- }
-
- /**
- * Build a map of key_hash => offset for the keys in the cleanable dirty portion of the log to use in cleaning.
- * @param log The log to use
- * @param start The offset at which dirty messages begin
- * @param end The ending offset for the map that is being built
- * @param map The map in which to store the mappings
- * @param stats Collector for cleaning statistics
- */
- private[log] def buildOffsetMap(log: UnifiedLog,
- start: Long,
- end: Long,
- map: OffsetMap,
- stats: CleanerStats): Unit = {
- map.clear()
- val dirty = log.logSegments(start, end).asScala
- val nextSegmentStartOffsets = new ListBuffer[Long]
- if (dirty.nonEmpty) {
- for (nextSegment <- dirty.tail) nextSegmentStartOffsets.append(nextSegment.baseOffset)
- nextSegmentStartOffsets.append(end)
- }
- info("Building offset map for log %s for %d segments in offset range [%d, %d).".format(log.name, dirty.size, start, end))
-
- val transactionMetadata = new CleanedTransactionMetadata
- val abortedTransactions = log.collectAbortedTransactions(start, end)
- transactionMetadata.addAbortedTransactions(abortedTransactions.asScala.toList)
-
- // Add all the cleanable dirty segments. We must take at least map.slots * load_factor,
- // but we may be able to fit more (if there is lots of duplication in the dirty section of the log)
- var full = false
- for ((segment, nextSegmentStartOffset) <- dirty.zip(nextSegmentStartOffsets) if !full) {
- checkDone(log.topicPartition)
-
- full = buildOffsetMapForSegment(log.topicPartition, segment, map, start, nextSegmentStartOffset, log.config.maxMessageSize,
- transactionMetadata, stats)
- if (full)
- debug("Offset map is full, %d segments fully mapped, segment with base offset %d is partially mapped".format(dirty.indexOf(segment), segment.baseOffset))
- }
- info("Offset map for log %s complete.".format(log.name))
- }
-
- /**
- * Add the messages in the given segment to the offset map
- *
- * @param topicPartition The topic and partition of the log segment to build offset
- * @param segment The segment to index
- * @param map The map in which to store the key=>offset mapping
- * @param startOffset The offset at which dirty messages begin
- * @param nextSegmentStartOffset The base offset for next segment when building current segment
- * @param maxLogMessageSize The maximum size in bytes for record allowed
- * @param transactionMetadata The state of ongoing transactions for the log between offset range to build
- * @param stats Collector for cleaning statistics
- *
- * @return If the map was filled whilst loading from this segment
- */
- private def buildOffsetMapForSegment(topicPartition: TopicPartition,
- segment: LogSegment,
- map: OffsetMap,
- startOffset: Long,
- nextSegmentStartOffset: Long,
- maxLogMessageSize: Int,
- transactionMetadata: CleanedTransactionMetadata,
- stats: CleanerStats): Boolean = {
- var position = segment.offsetIndex.lookup(startOffset).position
- val maxDesiredMapSize = (map.slots * this.dupBufferLoadFactor).toInt
- while (position < segment.log.sizeInBytes) {
- checkDone(topicPartition)
- readBuffer.clear()
- try {
- segment.log.readInto(readBuffer, position)
- } catch {
- case e: Exception =>
- throw new KafkaException(s"Failed to read from segment $segment of partition $topicPartition " +
- "while loading offset map", e)
- }
- val records = MemoryRecords.readableRecords(readBuffer)
- throttler.maybeThrottle(records.sizeInBytes)
-
- val startPosition = position
- for (batch <- records.batches.asScala) {
- if (batch.isControlBatch) {
- transactionMetadata.onControlBatchRead(batch)
- stats.indexMessagesRead(1)
- } else {
- val isAborted = transactionMetadata.onBatchRead(batch)
- if (isAborted) {
- // If the batch is aborted, do not bother populating the offset map.
- // Note that abort markers are supported in v2 and above, which means count is defined.
- stats.indexMessagesRead(batch.countOrNull)
- } else {
- val recordsIterator = batch.streamingIterator(decompressionBufferSupplier)
- try {
- for (record <- recordsIterator.asScala) {
- if (record.hasKey && record.offset >= startOffset) {
- if (map.size < maxDesiredMapSize)
- map.put(record.key, record.offset)
- else
- return true
- }
- stats.indexMessagesRead(1)
- }
- } finally recordsIterator.close()
- }
- }
-
- if (batch.lastOffset >= startOffset)
- map.updateLatestOffset(batch.lastOffset)
- }
- val bytesRead = records.validBytes
- position += bytesRead
- stats.indexBytesRead(bytesRead)
-
- // if we didn't read even one complete message, our read buffer may be too small
- if (position == startPosition)
- growBuffersOrFail(segment.log, position, maxLogMessageSize, records)
- }
-
- // In the case of offsets gap, fast forward to latest expected offset in this segment.
- map.updateLatestOffset(nextSegmentStartOffset - 1L)
-
- restoreBuffers()
- false
- }
-}
-
-/**
- * A simple struct for collecting stats about log cleaning
- */
-private class CleanerStats(time: Time = Time.SYSTEM) {
- val startTime = time.milliseconds
- var mapCompleteTime: Long = -1L
- var endTime: Long = -1L
- var bytesRead = 0L
- var bytesWritten = 0L
- var mapBytesRead = 0L
- var mapMessagesRead = 0L
- var messagesRead = 0L
- var invalidMessagesRead = 0L
- var messagesWritten = 0L
- var bufferUtilization = 0.0d
-
- def readMessages(messagesRead: Int, bytesRead: Int): Unit = {
- this.messagesRead += messagesRead
- this.bytesRead += bytesRead
- }
-
- def invalidMessage(): Unit = {
- invalidMessagesRead += 1
- }
-
- def recopyMessages(messagesWritten: Int, bytesWritten: Int): Unit = {
- this.messagesWritten += messagesWritten
- this.bytesWritten += bytesWritten
- }
-
- def indexMessagesRead(size: Int): Unit = {
- mapMessagesRead += size
- }
-
- def indexBytesRead(size: Int): Unit = {
- mapBytesRead += size
- }
-
- def indexDone(): Unit = {
- mapCompleteTime = time.milliseconds
- }
-
- def allDone(): Unit = {
- endTime = time.milliseconds
- }
-
- def elapsedSecs: Double = (endTime - startTime) / 1000.0
-
- def elapsedIndexSecs: Double = (mapCompleteTime - startTime) / 1000.0
-
-}
-
-/**
- * This is a helper class to facilitate tracking transaction state while cleaning the log. It maintains a set
- * of the ongoing aborted and committed transactions as the cleaner is working its way through the log. This
- * class is responsible for deciding when transaction markers can be removed and is therefore also responsible
- * for updating the cleaned transaction index accordingly.
- */
-private[log] class CleanedTransactionMetadata {
- private val ongoingCommittedTxns = mutable.Set.empty[Long]
- private val ongoingAbortedTxns = mutable.Map.empty[Long, AbortedTransactionMetadata]
- // Minheap of aborted transactions sorted by the transaction first offset
- private val abortedTransactions = mutable.PriorityQueue.empty[AbortedTxn](new Ordering[AbortedTxn] {
- override def compare(x: AbortedTxn, y: AbortedTxn): Int = java.lang.Long.compare(x.firstOffset, y.firstOffset)
- }.reverse)
-
- // Output cleaned index to write retained aborted transactions
- var cleanedIndex: Option[TransactionIndex] = None
-
- /**
- * Update the cleaned transaction state with the new found aborted transactions that has just been traversed.
- *
- * @param abortedTransactions The new found aborted transactions to add
- */
- def addAbortedTransactions(abortedTransactions: List[AbortedTxn]): Unit = {
- this.abortedTransactions ++= abortedTransactions
- }
-
- /**
- * Update the cleaned transaction state with a control batch that has just been traversed by the cleaner.
- * Return true if the control batch can be discarded.
- *
- * @param controlBatch The control batch that been traversed
- *
- * @return True if the control batch can be discarded
- */
- def onControlBatchRead(controlBatch: RecordBatch): Boolean = {
- consumeAbortedTxnsUpTo(controlBatch.lastOffset)
-
- val controlRecordIterator = controlBatch.iterator
- if (controlRecordIterator.hasNext) {
- val controlRecord = controlRecordIterator.next()
- val controlType = ControlRecordType.parse(controlRecord.key)
- val producerId = controlBatch.producerId
- controlType match {
- case ControlRecordType.ABORT =>
- ongoingAbortedTxns.remove(producerId) match {
- // Retain the marker until all batches from the transaction have been removed.
- case Some(abortedTxnMetadata) if abortedTxnMetadata.lastObservedBatchOffset.isDefined =>
- cleanedIndex.foreach(_.append(abortedTxnMetadata.abortedTxn))
- false
- case _ => true
- }
-
- case ControlRecordType.COMMIT =>
- // This marker is eligible for deletion if we didn't traverse any batches from the transaction
- !ongoingCommittedTxns.remove(producerId)
-
- case _ => false
- }
- } else {
- // An empty control batch was already cleaned, so it's safe to discard
- true
- }
- }
-
- private def consumeAbortedTxnsUpTo(offset: Long): Unit = {
- while (abortedTransactions.headOption.exists(_.firstOffset <= offset)) {
- val abortedTxn = abortedTransactions.dequeue()
- ongoingAbortedTxns.getOrElseUpdate(abortedTxn.producerId, new AbortedTransactionMetadata(abortedTxn))
- }
- }
-
- /**
- * Update the transactional state for the incoming non-control batch. If the batch is part of
- * an aborted transaction, return true to indicate that it is safe to discard.
- *
- * @param batch The batch to read when updating the transactional state
- *
- * @return Whether the batch is part of an aborted transaction or not
- */
- def onBatchRead(batch: RecordBatch): Boolean = {
- consumeAbortedTxnsUpTo(batch.lastOffset)
- if (batch.isTransactional) {
- ongoingAbortedTxns.get(batch.producerId) match {
- case Some(abortedTransactionMetadata) =>
- abortedTransactionMetadata.lastObservedBatchOffset = Some(batch.lastOffset)
- true
- case None =>
- ongoingCommittedTxns += batch.producerId
- false
- }
- } else {
- false
- }
- }
-
-}
-
-private class AbortedTransactionMetadata(val abortedTxn: AbortedTxn) {
- var lastObservedBatchOffset: Option[Long] = None
-
- override def toString: String = s"(txn: $abortedTxn, lastOffset: $lastObservedBatchOffset)"
-}
diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala
index 4b255e9a66e32..fc5b15e8fc6d0 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -42,11 +42,12 @@ import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsem
import java.util.{Collections, Optional, OptionalLong, Properties}
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.util.{FileLock, Scheduler}
-import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, LogOffsetsListener, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog}
+import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, LogOffsetsListener, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog}
import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, OffsetCheckpointFile}
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import java.util
+import java.util.stream.Collectors
/**
* The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning.
@@ -629,7 +630,7 @@ class LogManager(logDirs: Seq[File],
initialTaskDelayMs)
}
if (cleanerConfig.enableCleaner) {
- _cleaner = new LogCleaner(cleanerConfig, liveLogDirs, currentLogs, logDirFailureChannel, time = time)
+ _cleaner = new LogCleaner(cleanerConfig, liveLogDirs.asJava, currentLogs, logDirFailureChannel, time)
_cleaner.startup()
}
}
@@ -894,7 +895,7 @@ class LogManager(logDirs: Seq[File],
*/
private def resumeCleaning(topicPartition: TopicPartition): Unit = {
if (cleaner != null) {
- cleaner.resumeCleaning(Seq(topicPartition))
+ cleaner.resumeCleaning(util.Set.of(topicPartition))
info(s"Cleaning for partition $topicPartition is resumed")
}
}
@@ -1286,7 +1287,7 @@ class LogManager(logDirs: Seq[File],
if (cleaner != null && !isFuture) {
cleaner.abortCleaning(topicPartition)
if (checkpoint) {
- cleaner.updateCheckpoints(removedLog.parentDirFile, partitionToRemove = Option(topicPartition))
+ cleaner.updateCheckpoints(removedLog.parentDirFile, Optional.of(topicPartition))
}
}
if (isStray) {
@@ -1344,7 +1345,7 @@ class LogManager(logDirs: Seq[File],
val logsByDirCached = logsByDir
logDirs.foreach { logDir =>
- if (cleaner != null) cleaner.updateCheckpoints(logDir)
+ if (cleaner != null) cleaner.updateCheckpoints(logDir, Optional.empty())
val logsToCheckpoint = logsInDir(logsByDirCached, logDir)
checkpointRecoveryOffsetsInDir(logDir, logsToCheckpoint)
checkpointLogStartOffsetsInDir(logDir, logsToCheckpoint)
@@ -1382,19 +1383,22 @@ class LogManager(logDirs: Seq[File],
val startMs = time.milliseconds
// clean current logs.
- val deletableLogs = {
+ val deletableLogs: util.Map[TopicPartition, UnifiedLog] = {
if (cleaner != null) {
// prevent cleaner from working on same partitions when changing cleanup policy
cleaner.pauseCleaningForNonCompactedPartitions()
} else {
- currentLogs.asScala.filter {
- case (_, log) => !log.config.compact
- }
+ currentLogs.entrySet().stream()
+ .filter(e => !e.getValue.config.compact)
+ .collect(Collectors.toMap(
+ (e: util.Map.Entry[TopicPartition, UnifiedLog]) => e.getKey,
+ (e: util.Map.Entry[TopicPartition, UnifiedLog]) => e.getValue
+ ))
}
}
try {
- deletableLogs.foreach {
+ deletableLogs.forEach {
case (topicPartition, log) =>
debug(s"Garbage collecting '${log.name}'")
total += log.deleteOldSegments()
@@ -1408,7 +1412,7 @@ class LogManager(logDirs: Seq[File],
}
} finally {
if (cleaner != null) {
- cleaner.resumeCleaning(deletableLogs.map(_._1))
+ cleaner.resumeCleaning(deletableLogs.keySet())
}
}
@@ -1548,7 +1552,7 @@ object LogManager {
LogConfig.validateBrokerLogConfigValues(defaultProps, config.remoteLogManagerConfig.isRemoteStorageSystemEnabled)
val defaultLogConfig = new LogConfig(defaultProps)
- val cleanerConfig = LogCleaner.cleanerConfig(config)
+ val cleanerConfig = new CleanerConfig(config)
val transactionLogConfig = new TransactionLogConfig(config)
new LogManager(logDirs = config.logDirs.map(new File(_).getAbsoluteFile),
diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
index 6db9e0a6d0a93..82c1ceb53c3a1 100755
--- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
+++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
@@ -21,7 +21,7 @@ import java.util
import java.util.{Collections, Properties}
import java.util.concurrent.CopyOnWriteArrayList
import java.util.concurrent.locks.ReentrantReadWriteLock
-import kafka.log.{LogCleaner, LogManager}
+import kafka.log.LogManager
import kafka.network.{DataPlaneAcceptor, SocketServer}
import kafka.raft.KafkaRaftManager
import kafka.server.DynamicBrokerConfig._
@@ -46,7 +46,7 @@ import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, MetricConfigs}
import org.apache.kafka.server.telemetry.ClientTelemetry
import org.apache.kafka.snapshot.RecordsSnapshotReader
-import org.apache.kafka.storage.internals.log.LogConfig
+import org.apache.kafka.storage.internals.log.{LogCleaner, LogConfig}
import scala.collection._
import scala.jdk.CollectionConverters._
@@ -89,7 +89,7 @@ object DynamicBrokerConfig {
private[server] val DynamicProducerStateManagerConfig = Set(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_CONFIG, TransactionLogConfig.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG)
val AllDynamicConfigs = DynamicSecurityConfigs ++
- LogCleaner.ReconfigurableConfigs ++
+ LogCleaner.RECONFIGURABLE_CONFIGS.asScala ++
DynamicLogConfig.ReconfigurableConfigs ++
DynamicThreadPool.RECONFIGURABLE_CONFIGS.asScala ++
Set(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG) ++
diff --git a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
index b2209f2973277..3e761620efd9c 100644
--- a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
@@ -283,7 +283,7 @@ class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) {
val broker = cluster.brokers.asScala.head._2
val log = broker.logManager.getLog(tp).get
log.roll()
- assertTrue(broker.logManager.cleaner.awaitCleaned(tp, 0))
+ assertTrue(broker.logManager.cleaner.awaitCleaned(tp, 0, 60000L))
}
private def withAdmin(f: Admin => Unit): Unit = {
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
index d1e04f8197a5f..0ab2328c53f79 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
@@ -1041,7 +1041,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
configs.get(topicResource1).get(TopicConfig.RETENTION_MS_CONFIG).value)
val maxMessageBytes2 = configs.get(topicResource2).get(TopicConfig.MAX_MESSAGE_BYTES_CONFIG)
- assertEquals(LogConfig.DEFAULT_MAX_MESSAGE_BYTES.toString, maxMessageBytes2.value)
+ assertEquals(ServerLogConfigs.MAX_MESSAGE_BYTES_DEFAULT.toString, maxMessageBytes2.value)
assertEquals(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, maxMessageBytes2.name)
assertTrue(maxMessageBytes2.isDefault)
assertFalse(maxMessageBytes2.isSensitive)
@@ -3467,7 +3467,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
assertEquals(2, configs.size)
assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString, configs.get(topic1Resource).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
- assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(topic1Resource).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value)
+ assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, configs.get(topic1Resource).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value)
assertEquals("0.9", configs.get(topic2Resource).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
// Check invalid use of append/subtract operation types
@@ -4120,12 +4120,12 @@ object PlaintextAdminIntegrationTest {
assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString,
configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
- assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE,
+ assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT,
configs.get(topicResource1).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value)
assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value)
- assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value)
+ assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value)
// Alter configs with validateOnly = true: first and third are invalid, second is valid
alterConfigs.put(topicResource1, util.Arrays.asList(
@@ -4149,11 +4149,11 @@ object PlaintextAdminIntegrationTest {
assertEquals(LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO.toString,
configs.get(topicResource1).get(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG).value)
- assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE,
+ assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT,
configs.get(topicResource1).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value)
assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value)
- assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value)
+ assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value)
}
}
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala
index 6487197df37cc..6e3bbf4aed701 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala
@@ -26,7 +26,7 @@ import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.{InvalidTimestampException, RecordTooLargeException, SerializationException, TimeoutException}
import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch, Records, TimestampType}
import org.apache.kafka.common.serialization.ByteArraySerializer
-import org.apache.kafka.storage.internals.log.LogConfig
+import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Timeout
import org.junit.jupiter.params.ParameterizedTest
@@ -262,7 +262,7 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
val valueLengthSize = 3
val overhead = Records.LOG_OVERHEAD + DefaultRecordBatch.RECORD_BATCH_OVERHEAD + DefaultRecord.MAX_RECORD_OVERHEAD +
keyLengthSize + headerLengthSize + valueLengthSize
- val valueSize = LogConfig.DEFAULT_MAX_MESSAGE_BYTES - overhead
+ val valueSize = ServerLogConfigs.MAX_MESSAGE_BYTES_DEFAULT - overhead
val record0 = new ProducerRecord(topic, new Array[Byte](0), new Array[Byte](valueSize))
assertEquals(record0.value.length, producer.send(record0).get.serializedValueSize)
diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala
index 70b65368111ba..64c40c230a5e4 100644
--- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala
@@ -30,10 +30,9 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.utils.SecurityUtils
import org.apache.kafka.common.security.token.delegation.DelegationToken
import org.apache.kafka.security.authorizer.AclEntry.{WILDCARD_HOST, WILDCARD_PRINCIPAL_STRING}
-import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, ServerConfigs}
+import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, ServerConfigs, ServerLogConfigs}
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
import org.apache.kafka.server.authorizer.{Authorizer => JAuthorizer}
-import org.apache.kafka.storage.internals.log.LogConfig
import org.apache.kafka.test.TestUtils.assertFutureThrows
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout}
@@ -584,7 +583,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
assertEquals(100000, segmentBytesConfig.value.toLong)
assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source)
val compressionConfig = topicConfigs.find(_.name == TopicConfig.COMPRESSION_TYPE_CONFIG).get
- assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, compressionConfig.value)
+ assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, compressionConfig.value)
assertEquals(ConfigEntry.ConfigSource.DEFAULT_CONFIG, compressionConfig.source)
assertFutureThrows(classOf[TopicAuthorizationException], result.numPartitions(topic2))
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
index 717a1c2f81864..e7ad10140f62a 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
@@ -36,6 +36,7 @@ import org.apache.kafka.metadata.MetadataCache
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion, RequestLocal, TransactionVersion}
import org.apache.kafka.server.common.TransactionVersion.{TV_0, TV_2}
import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey
+import org.apache.kafka.server.config.ServerLogConfigs
import org.apache.kafka.server.storage.log.FetchIsolation
import org.apache.kafka.server.util.MockScheduler
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, LogConfig, LogOffsetMetadata, UnifiedLog}
@@ -1134,7 +1135,7 @@ class TransactionStateManagerTest {
val partitionIds = 0 until numPartitions
loadTransactionsForPartitions(partitionIds)
- expectLogConfig(partitionIds, LogConfig.DEFAULT_MAX_MESSAGE_BYTES)
+ expectLogConfig(partitionIds, ServerLogConfigs.MAX_MESSAGE_BYTES_DEFAULT)
txnMetadata1.txnLastUpdateTimestamp = time.milliseconds() - txnConfig.transactionalIdExpirationMs
txnMetadata1.state = txnState
diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
index 98b8dafbc5b91..acf21e69ec377 100644
--- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
@@ -16,9 +16,6 @@
*/
package kafka.log
-import java.io.File
-import java.nio.file.Files
-import java.util.{Optional, Properties}
import kafka.utils.TestUtils
import kafka.utils.Implicits._
import org.apache.kafka.common.TopicPartition
@@ -28,10 +25,14 @@ import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.server.util.MockTime
-import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog}
+import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog}
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import org.junit.jupiter.api.{AfterEach, Tag}
+import java.io.File
+import java.nio.file.Files
+import java.util
+import java.util.{Optional, Properties}
import scala.collection.Seq
import scala.collection.mutable.ListBuffer
import scala.util.Random
@@ -93,7 +94,7 @@ abstract class AbstractLogCleanerIntegrationTest {
cleanerIoBufferSize: Option[Int] = None,
propertyOverrides: Properties = new Properties()): LogCleaner = {
- val logMap = new java.util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog]()
+ val logMap = new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog]()
for (partition <- partitions) {
val dir = new File(logDir, s"${partition.topic}-${partition.partition}")
Files.createDirectories(dir.toPath)
@@ -133,10 +134,10 @@ abstract class AbstractLogCleanerIntegrationTest {
backoffMs,
true)
new LogCleaner(cleanerConfig,
- logDirs = Array(logDir),
- logs = logMap,
- logDirFailureChannel = new LogDirFailureChannel(1),
- time = time)
+ util.List.of(logDir),
+ logMap,
+ new LogDirFailureChannel(1),
+ time)
}
private var ctr = 0
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
index 96f9ac6e7a134..38aa789729a9a 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
@@ -169,7 +169,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
val firstBlockCleanableSegmentOffset = activeSegAtT0.baseOffset
// the first block should get cleaned
- cleaner.awaitCleaned(new TopicPartition("log", 0), firstBlockCleanableSegmentOffset)
+ cleaner.awaitCleaned(new TopicPartition("log", 0), firstBlockCleanableSegmentOffset, 60000L)
val read1 = readFromLog(log)
val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints.get(new TopicPartition("log", 0))
@@ -181,7 +181,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
time.sleep(maxCompactionLagMs + 1)
// the second block should get cleaned. only zero keys left
- cleaner.awaitCleaned(new TopicPartition("log", 0), activeSegAtT1.baseOffset)
+ cleaner.awaitCleaned(new TopicPartition("log", 0), activeSegAtT1.baseOffset, 60000L)
val read2 = readFromLog(log)
@@ -222,10 +222,10 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
cleaner.startup()
assertEquals(0, cleaner.deadThreadCount)
// we simulate the unexpected error with an interrupt
- cleaner.cleaners.foreach(_.interrupt())
+ cleaner.cleaners.forEach(_.interrupt())
// wait until interruption is propagated to all the threads
TestUtils.waitUntilTrue(
- () => cleaner.cleaners.foldLeft(true)((result, thread) => {
+ () => cleaner.cleaners.asScala.foldLeft(true)((result, thread) => {
thread.isThreadFailed && result
}), "Threads didn't terminate unexpectedly"
)
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
index c47395711bb6e..c632f2c0bf1dd 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
@@ -84,7 +84,7 @@ class LogCleanerLagIntegrationTest extends AbstractLogCleanerIntegrationTest wit
val firstBlock1SegmentBaseOffset = activeSegAtT0.baseOffset
// the first block should get cleaned
- cleaner.awaitCleaned(new TopicPartition("log", 0), activeSegAtT0.baseOffset)
+ cleaner.awaitCleaned(new TopicPartition("log", 0), activeSegAtT0.baseOffset, 60000L)
// check the data is the same
val read1 = readFromLog(log)
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
index 1befc2caa8767..f93d703f07777 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
@@ -382,7 +382,7 @@ class LogCleanerManagerTest extends Logging {
assertEquals(0, cleanable.size, "should have 0 logs ready to be compacted")
// log cleanup finished, and log can be picked up for compaction
- cleanerManager.resumeCleaning(deletableLog.asScala.map(_.getKey).toList.asJava)
+ cleanerManager.resumeCleaning(deletableLog.asScala.map(_.getKey).toSet.asJava)
val cleanable2 = cleanerManager.grabFilthiestCompactedLog(time, new PreCleanStats()).toScala
assertEquals(1, cleanable2.size, "should have 1 logs ready to be compacted")
@@ -501,9 +501,9 @@ class LogCleanerManagerTest extends Logging {
val pausedPartitions = cleanerManager.pauseCleaningForNonCompactedPartitions()
// Log truncation happens due to unclean leader election
cleanerManager.abortAndPauseCleaning(log.topicPartition)
- cleanerManager.resumeCleaning(Seq(log.topicPartition).asJava)
+ cleanerManager.resumeCleaning(Set(log.topicPartition).asJava)
// log cleanup finishes and pausedPartitions are resumed
- cleanerManager.resumeCleaning(pausedPartitions.asScala.map(_.getKey).toList.asJava)
+ cleanerManager.resumeCleaning(pausedPartitions.asScala.map(_.getKey).toSet.asJava)
assertEquals(Optional.empty(), cleanerManager.cleaningState(log.topicPartition))
}
@@ -522,7 +522,7 @@ class LogCleanerManagerTest extends Logging {
// Broker processes StopReplicaRequest with delete=true
cleanerManager.abortCleaning(log.topicPartition)
// log cleanup finishes and pausedPartitions are resumed
- cleanerManager.resumeCleaning(pausedPartitions.asScala.map(_.getKey).toList.asJava)
+ cleanerManager.resumeCleaning(pausedPartitions.asScala.map(_.getKey).toSet.asJava)
assertEquals(Optional.empty(), cleanerManager.cleaningState(log.topicPartition))
}
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
index 0923bafbaa1a0..e0d3ac5601d23 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
@@ -18,7 +18,7 @@
package kafka.log
import java.io.File
-import java.util.Properties
+import java.util.{Optional, Properties}
import kafka.server.KafkaConfig
import kafka.utils._
import org.apache.kafka.common.TopicPartition
@@ -85,7 +85,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
// force a checkpoint
// and make sure its gone from checkpoint file
cleaner.logs.remove(topicPartitions(0))
- cleaner.updateCheckpoints(logDir, partitionToRemove = Option(topicPartitions(0)))
+ cleaner.updateCheckpoints(logDir, Optional.of(topicPartitions(0)))
val checkpoints = new OffsetCheckpointFile(new File(logDir, LogCleanerManager.OFFSET_CHECKPOINT_FILE), null).read()
// we expect partition 0 to be gone
assertFalse(checkpoints.containsKey(topicPartitions(0)))
@@ -280,7 +280,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
// Verify no cleaning with LogCleanerIoBufferSizeProp=1
val firstDirty = log.activeSegment.baseOffset
val topicPartition = new TopicPartition("log", 0)
- cleaner.awaitCleaned(topicPartition, firstDirty, maxWaitMs = 10)
+ cleaner.awaitCleaned(topicPartition, firstDirty, 10)
assertTrue(cleaner.cleanerManager.allCleanerCheckpoints.isEmpty, "Should not have cleaned")
def kafkaConfigWithCleanerConfig(cleanerConfig: CleanerConfig): KafkaConfig = {
@@ -317,7 +317,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
// wait until cleaning up to base_offset, note that cleaning happens only when "log dirty ratio" is higher than
// TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG
val topicPartition = new TopicPartition(topic, partitionId)
- cleaner.awaitCleaned(topicPartition, firstDirty)
+ cleaner.awaitCleaned(topicPartition, firstDirty, 60000L)
val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints.get(topicPartition)
assertTrue(lastCleaned >= firstDirty, s"log cleaner should have processed up to offset $firstDirty, but lastCleaned=$lastCleaned")
}
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
index 049e1793a139e..09a9d1c40f140 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -17,7 +17,6 @@
package kafka.log
-import kafka.log.LogCleaner.{MaxBufferUtilizationPercentMetricName, MaxCleanTimeMetricName, MaxCompactionDelayMetricsName}
import kafka.server.KafkaConfig
import kafka.utils.{CoreUtils, Logging, TestUtils}
import org.apache.kafka.common.TopicPartition
@@ -25,11 +24,11 @@ import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.CorruptRecordException
import org.apache.kafka.common.record._
-import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics}
import org.apache.kafka.server.util.MockTime
-import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanerConfig, LocalLog, LogAppendInfo, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog}
+import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanedTransactionMetadata, Cleaner, CleanerConfig, CleanerStats, LocalLog, LogAppendInfo, LogCleaner, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog}
import org.apache.kafka.storage.internals.utils.Throttler
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import org.junit.jupiter.api.Assertions._
@@ -45,6 +44,7 @@ import java.nio.file.Paths
import java.util
import java.util.{Optional, Properties}
import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, TimeUnit}
+import java.util.function.Consumer
import scala.collection._
import scala.jdk.CollectionConverters._
@@ -77,10 +77,10 @@ class LogCleanerTest extends Logging {
val mockMetricsGroupCtor = mockConstruction(classOf[KafkaMetricsGroup])
try {
val logCleaner = new LogCleaner(new CleanerConfig(true),
- logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()),
- logs = new ConcurrentHashMap[TopicPartition, UnifiedLog](),
- logDirFailureChannel = new LogDirFailureChannel(1),
- time = time)
+ util.List.of(TestUtils.tempDir(), TestUtils.tempDir()),
+ new ConcurrentHashMap[TopicPartition, UnifiedLog](),
+ new LogDirFailureChannel(1),
+ time)
val metricsToVerify = new java.util.HashMap[String, java.util.List[java.util.Map[String, String]]]()
logCleaner.cleanerManager.gaugeMetricNameWithTag.asScala.foreach { metricNameAndTags =>
val tags = new java.util.ArrayList[java.util.Map[String, String]]()
@@ -91,11 +91,11 @@ class LogCleanerTest extends Logging {
logCleaner.shutdown()
val mockMetricsGroup = mockMetricsGroupCtor.constructed.get(0)
- val numMetricsRegistered = LogCleaner.MetricNames.size
+ val numMetricsRegistered = LogCleaner.METRIC_NAMES.size
verify(mockMetricsGroup, times(numMetricsRegistered)).newGauge(anyString(), any())
// verify that each metric in `LogCleaner` is removed
- LogCleaner.MetricNames.foreach(verify(mockMetricsGroup).removeMetric(_))
+ LogCleaner.METRIC_NAMES.forEach(verify(mockMetricsGroup).removeMetric(_))
// verify that each metric in `LogCleanerManager` is removed
val mockLogCleanerManagerMetricsGroup = mockMetricsGroupCtor.constructed.get(1)
@@ -123,20 +123,20 @@ class LogCleanerTest extends Logging {
@Test
def testMetricsActiveAfterReconfiguration(): Unit = {
val logCleaner = new LogCleaner(new CleanerConfig(true),
- logDirs = Array(TestUtils.tempDir()),
- logs = new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
- logDirFailureChannel = new LogDirFailureChannel(1),
- time = time)
+ util.List.of(TestUtils.tempDir()),
+ new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
+ new LogDirFailureChannel(1),
+ time)
try {
logCleaner.startup()
- var nonexistent = LogCleaner.MetricNames.diff(KafkaYammerMetrics.defaultRegistry.allMetrics().keySet().asScala.map(_.getName))
+ var nonexistent = LogCleaner.METRIC_NAMES.asScala.diff(KafkaYammerMetrics.defaultRegistry.allMetrics().keySet().asScala.map(_.getName))
assertEquals(0, nonexistent.size, s"$nonexistent should be existent")
logCleaner.reconfigure(new KafkaConfig(TestUtils.createBrokerConfig(1)),
new KafkaConfig(TestUtils.createBrokerConfig(1)))
- nonexistent = LogCleaner.MetricNames.diff(KafkaYammerMetrics.defaultRegistry.allMetrics().keySet().asScala.map(_.getName))
+ nonexistent = LogCleaner.METRIC_NAMES.asScala.diff(KafkaYammerMetrics.defaultRegistry.allMetrics().keySet().asScala.map(_.getName))
assertEquals(0, nonexistent.size, s"$nonexistent should be existent")
} finally logCleaner.shutdown()
}
@@ -165,10 +165,10 @@ class LogCleanerTest extends Logging {
// clean the log
val segments = log.logSegments.asScala.take(3).toSeq
- val stats = new CleanerStats()
+ val stats = new CleanerStats(Time.SYSTEM)
val expectedBytesRead = segments.map(_.size).sum
val shouldRemain = LogTestUtils.keysInLog(log).filterNot(keys.contains)
- cleaner.cleanSegments(log, segments, map, 0L, stats, new CleanedTransactionMetadata, -1, segments.last.readNextOffset)
+ cleaner.cleanSegments(log, segments.asJava, map, 0L, stats, new CleanedTransactionMetadata, -1, segments.last.readNextOffset)
assertEquals(shouldRemain, LogTestUtils.keysInLog(log))
assertEquals(expectedBytesRead, stats.bytesRead)
}
@@ -257,9 +257,9 @@ class LogCleanerTest extends Logging {
val offsetMap = new FakeOffsetMap(Int.MaxValue)
val cleaner = makeCleaner(Int.MaxValue)
val segments = log.logSegments(0, log.activeSegment.baseOffset).asScala.toSeq
- val stats = new CleanerStats()
+ val stats = new CleanerStats(Time.SYSTEM)
cleaner.buildOffsetMap(log, 0, log.activeSegment.baseOffset, offsetMap, stats)
- cleaner.cleanSegments(log, segments, offsetMap, 0L, stats, new CleanedTransactionMetadata, -1, segments.last.readNextOffset)
+ cleaner.cleanSegments(log, segments.asJava, offsetMap, 0L, stats, new CleanedTransactionMetadata, -1, segments.last.readNextOffset)
// Validate based on the file name that log segment file is renamed exactly once for async deletion
assertEquals(expectedFileName, firstLogFile.file().getPath)
@@ -426,7 +426,7 @@ class LogCleanerTest extends Logging {
val segments = log.logSegments(0, log.activeSegment.baseOffset).asScala.toSeq
val stats = new CleanerStats(time)
cleaner.buildOffsetMap(log, dirtyOffset, log.activeSegment.baseOffset, offsetMap, stats)
- cleaner.cleanSegments(log, segments, offsetMap, time.milliseconds(), stats, new CleanedTransactionMetadata, Long.MaxValue, segments.last.readNextOffset)
+ cleaner.cleanSegments(log, segments.asJava, offsetMap, time.milliseconds(), stats, new CleanedTransactionMetadata, Long.MaxValue, segments.last.readNextOffset)
dirtyOffset = offsetMap.latestOffset + 1
}
@@ -525,7 +525,7 @@ class LogCleanerTest extends Logging {
log.appendAsLeader(abortMarker(pid1, producerEpoch), 0, AppendOrigin.COORDINATOR)
// we have only cleaned the records in the first segment
- val dirtyOffset = cleaner.clean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false))._1
+ val dirtyOffset = cleaner.clean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false)).getKey
assertEquals(List(2, 3, 5, 6, 6, 7, 7, 8, 8, 9, 9, 10), LogTestUtils.keysInLog(log))
log.roll()
@@ -558,7 +558,7 @@ class LogCleanerTest extends Logging {
log.roll()
// cannot remove the marker in this pass because there are still valid records
- var dirtyOffset = cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)._1
+ var dirtyOffset = cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), largeTimestamp).getKey
assertEquals(List(1, 3, 2), LogTestUtils.keysInLog(log))
assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log))
@@ -567,17 +567,17 @@ class LogCleanerTest extends Logging {
log.roll()
// the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction
- dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)._1
+ dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), largeTimestamp).getKey
assertEquals(List(2, 1, 3), LogTestUtils.keysInLog(log))
assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log))
// clean again with same timestamp to verify marker is not removed early
- dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)._1
+ dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), largeTimestamp).getKey
assertEquals(List(2, 1, 3), LogTestUtils.keysInLog(log))
assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log))
// clean again with max timestamp to verify the marker is removed
- dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), currentTime = Long.MaxValue)._1
+ dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), Long.MaxValue).getKey
assertEquals(List(2, 1, 3), LogTestUtils.keysInLog(log))
assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log))
}
@@ -605,7 +605,7 @@ class LogCleanerTest extends Logging {
log.appendAsLeader(commitMarker(producerId, producerEpoch), 0, AppendOrigin.COORDINATOR)
log.roll()
- cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)
+ cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), largeTimestamp)
assertEquals(List(2), LogTestUtils.keysInLog(log))
assertEquals(List(1, 3, 4), offsetsInLog(log))
@@ -644,14 +644,14 @@ class LogCleanerTest extends Logging {
// first time through the records are removed
// Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}]
- var dirtyOffset = cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)._1
+ var dirtyOffset = cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), largeTimestamp).getKey
assertEquals(List(2, 3), LogTestUtils.keysInLog(log))
assertEquals(List(4, 5, 6, 7), offsetsInLog(log))
assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log))
// the empty batch remains if cleaned again because it still holds the last sequence
// Expected State: [{Producer1: EmptyBatch}, {Producer2: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}]
- dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)._1
+ dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), largeTimestamp).getKey
assertEquals(List(2, 3), LogTestUtils.keysInLog(log))
assertEquals(List(4, 5, 6, 7), offsetsInLog(log))
assertEquals(List(1, 3, 4, 5, 6, 7), lastOffsetsPerBatchInLog(log))
@@ -665,7 +665,7 @@ class LogCleanerTest extends Logging {
// Expected State: [{Producer1: EmptyBatch}, {Producer2: Commit}, {2}, {3}, {Producer1: Commit}, {Producer2: 1}, {Producer2: Commit}]
// The deleteHorizon for {Producer2: Commit} is still not set yet.
- dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)._1
+ dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), largeTimestamp).getKey
assertEquals(List(2, 3, 1), LogTestUtils.keysInLog(log))
assertEquals(List(4, 5, 6, 7, 8, 9), offsetsInLog(log))
assertEquals(List(1, 4, 5, 6, 7, 8, 9), lastOffsetsPerBatchInLog(log))
@@ -704,7 +704,7 @@ class LogCleanerTest extends Logging {
// the empty control batch does not cause an exception when cleaned
// Expected State: [{Producer1: EmptyBatch}], [{2}, {3}]
- dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), currentTime = Long.MaxValue)._1
+ dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), Long.MaxValue).getKey
assertEquals(List(2, 3), LogTestUtils.keysInLog(log))
assertEquals(List(1, 2), offsetsInLog(log))
assertEquals(List(0, 1, 2), lastOffsetsPerBatchInLog(log))
@@ -751,12 +751,12 @@ class LogCleanerTest extends Logging {
// Both the batch and the marker should remain after cleaning. The batch is retained
// because it is the last entry for this producerId. The marker is retained because
// there are still batches remaining from this transaction.
- cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)
+ cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), largeTimestamp)
assertEquals(List(1), offsetsInLog(log))
assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log))
// The empty batch and the marker is still retained after a second cleaning.
- cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), currentTime = Long.MaxValue)
+ cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), Long.MaxValue)
assertEquals(List(1), offsetsInLog(log))
assertEquals(List(0, 1), lastOffsetsPerBatchInLog(log))
}
@@ -780,7 +780,7 @@ class LogCleanerTest extends Logging {
log.roll()
// Aborted records are removed, but the abort marker is still preserved.
- val dirtyOffset = cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)._1
+ val dirtyOffset = cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), largeTimestamp).getKey
assertEquals(List(3), LogTestUtils.keysInLog(log))
assertEquals(List(3, 4, 5), offsetsInLog(log))
@@ -818,7 +818,7 @@ class LogCleanerTest extends Logging {
// Both transactional batches will be cleaned. The last one will remain in the log
// as an empty batch in order to preserve the producer sequence number and epoch
- cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)
+ cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), largeTimestamp)
assertEquals(List(1, 3, 4, 5), offsetsInLog(log))
assertEquals(List(1, 2, 3, 4, 5), lastOffsetsPerBatchInLog(log))
@@ -854,7 +854,7 @@ class LogCleanerTest extends Logging {
assertAbortedTransactionIndexed()
// first time through the records are removed
- var dirtyOffset = cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)._1
+ var dirtyOffset = cleaner.doClean(new LogToClean(log, 0L, log.activeSegment.baseOffset, false), largeTimestamp).getKey
assertAbortedTransactionIndexed()
assertEquals(List(), LogTestUtils.keysInLog(log))
assertEquals(List(2), offsetsInLog(log)) // abort marker is retained
@@ -871,7 +871,7 @@ class LogCleanerTest extends Logging {
appendProducer(Seq(1))
log.roll()
- dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), currentTime = largeTimestamp)._1
+ dirtyOffset = cleaner.doClean(new LogToClean(log, dirtyOffset, log.activeSegment.baseOffset, false), largeTimestamp).getKey
assertAbortedTransactionIndexed()
assertEquals(List(1), LogTestUtils.keysInLog(log))
assertEquals(List(2, 3), offsetsInLog(log)) // abort marker is not yet gone because we read the empty batch
@@ -912,8 +912,8 @@ class LogCleanerTest extends Logging {
keys.foreach(k => map.put(key(k), Long.MaxValue))
// clean the log
- val stats = new CleanerStats()
- cleaner.cleanSegments(log, Seq(log.logSegments.asScala.head), map, 0L, stats, new CleanedTransactionMetadata, -1, log.logSegments.asScala.head.readNextOffset)
+ val stats = new CleanerStats(Time.SYSTEM)
+ cleaner.cleanSegments(log, Seq(log.logSegments.asScala.head).asJava, map, 0L, stats, new CleanedTransactionMetadata, -1, log.logSegments.asScala.head.readNextOffset)
val shouldRemain = LogTestUtils.keysInLog(log).filterNot(keys.contains)
assertEquals(shouldRemain, LogTestUtils.keysInLog(log))
}
@@ -926,7 +926,7 @@ class LogCleanerTest extends Logging {
val (log, offsetMap) = createLogWithMessagesLargerThanMaxSize(largeMessageSize = 1024 * 1024)
val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024)
- cleaner.cleanSegments(log, Seq(log.logSegments.asScala.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata, -1, log.logSegments.asScala.head.readNextOffset)
+ cleaner.cleanSegments(log, Seq(log.logSegments.asScala.head).asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM), new CleanedTransactionMetadata, -1, log.logSegments.asScala.head.readNextOffset)
val shouldRemain = LogTestUtils.keysInLog(log).filter(k => !offsetMap.map.containsKey(k.toString))
assertEquals(shouldRemain, LogTestUtils.keysInLog(log))
}
@@ -945,7 +945,7 @@ class LogCleanerTest extends Logging {
val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024)
assertThrows(classOf[CorruptRecordException], () =>
- cleaner.cleanSegments(log, Seq(log.logSegments.asScala.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata, -1, log.logSegments.asScala.head.readNextOffset)
+ cleaner.cleanSegments(log, Seq(log.logSegments.asScala.head).asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM), new CleanedTransactionMetadata, -1, log.logSegments.asScala.head.readNextOffset)
)
}
@@ -962,7 +962,7 @@ class LogCleanerTest extends Logging {
val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024)
assertThrows(classOf[CorruptRecordException], () =>
- cleaner.cleanSegments(log, Seq(log.logSegments.asScala.head), offsetMap, 0L, new CleanerStats, new CleanedTransactionMetadata, -1, log.logSegments.asScala.head.readNextOffset)
+ cleaner.cleanSegments(log, Seq(log.logSegments.asScala.head).asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM), new CleanedTransactionMetadata, -1, log.logSegments.asScala.head.readNextOffset)
)
}
@@ -1035,8 +1035,9 @@ class LogCleanerTest extends Logging {
val initialLogSize = log.size
- val (endOffset, stats) = cleaner.clean(new LogToClean(log, 2, log.activeSegment.baseOffset, false))
- assertEquals(5, endOffset)
+ val endOffsetAndStats = cleaner.clean(new LogToClean(log, 2, log.activeSegment.baseOffset, false))
+ val stats = endOffsetAndStats.getValue
+ assertEquals(5, endOffsetAndStats.getKey)
assertEquals(5, stats.messagesRead)
assertEquals(initialLogSize, stats.bytesRead)
assertEquals(2, stats.messagesWritten)
@@ -1292,7 +1293,7 @@ class LogCleanerTest extends Logging {
log.appendAsLeader(record(log.logEndOffset.toInt, log.logEndOffset.toInt), 0)
val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages
- val (_, stats) = cleaner.clean(new LogToClean(log, 0, log.activeSegment.baseOffset, false))
+ val stats = cleaner.clean(new LogToClean(log, 0, log.activeSegment.baseOffset, false)).getValue
assertEquals(0, unkeyedMessageCountInLog(log), "Log should only contain keyed messages after cleaning.")
assertEquals(expectedSizeAfterCleaning, log.size, "Log should only contain keyed messages after cleaning.")
@@ -1346,7 +1347,7 @@ class LogCleanerTest extends Logging {
keys.foreach(k => map.put(key(k), Long.MaxValue))
val segments = log.logSegments.asScala.take(3).toSeq
assertThrows(classOf[LogCleaningAbortedException], () =>
- cleaner.cleanSegments(log, segments, map, 0L, new CleanerStats(),
+ cleaner.cleanSegments(log, segments.asJava, map, 0L, new CleanerStats(Time.SYSTEM),
new CleanedTransactionMetadata, -1, segments.last.readNextOffset)
)
}
@@ -1371,8 +1372,8 @@ class LogCleanerTest extends Logging {
// clean the log
val segments = log.logSegments.asScala.take(3).toSeq
- val stats = new CleanerStats()
- cleaner.cleanSegments(log, segments, map, 0L, stats, new CleanedTransactionMetadata, -1, segments.last.readNextOffset)
+ val stats = new CleanerStats(Time.SYSTEM)
+ cleaner.cleanSegments(log, segments.asJava, map, 0L, stats, new CleanedTransactionMetadata, -1, segments.last.readNextOffset)
assertEquals(2, log.logSegments.size)
assertEquals(1, log.logSegments.asScala.head.log.batches.asScala.size, "one batch should be retained in the cleaned segment")
val retainedBatch = log.logSegments.asScala.head.log.batches.asScala.head
@@ -1400,34 +1401,34 @@ class LogCleanerTest extends Logging {
}
// grouping by very large values should result in a single group with all the segments in it
- var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue, log.logEndOffset)
+ var groups = cleaner.groupSegmentsBySize(log.logSegments, Int.MaxValue, Int.MaxValue, log.logEndOffset)
assertEquals(1, groups.size)
- assertEquals(log.numberOfSegments, groups.head.size)
+ assertEquals(log.numberOfSegments, groups.get(0).size)
checkSegmentOrder(groups)
// grouping by very small values should result in all groups having one entry
- groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue, log.logEndOffset)
+ groups = cleaner.groupSegmentsBySize(log.logSegments, 1, Int.MaxValue, log.logEndOffset)
assertEquals(log.numberOfSegments, groups.size)
- assertTrue(groups.forall(_.size == 1), "All groups should be singletons.")
+ assertTrue(groups.asScala.forall(_.size == 1), "All groups should be singletons.")
checkSegmentOrder(groups)
- groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = 1, log.logEndOffset)
+ groups = cleaner.groupSegmentsBySize(log.logSegments, Int.MaxValue, 1, log.logEndOffset)
assertEquals(log.numberOfSegments, groups.size)
- assertTrue(groups.forall(_.size == 1), "All groups should be singletons.")
+ assertTrue(groups.asScala.forall(_.size == 1), "All groups should be singletons.")
checkSegmentOrder(groups)
val groupSize = 3
// check grouping by log size
val logSize = log.logSegments.asScala.take(groupSize).map(_.size).sum + 1
- groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue, log.logEndOffset)
+ groups = cleaner.groupSegmentsBySize(log.logSegments, logSize, Int.MaxValue, log.logEndOffset)
checkSegmentOrder(groups)
- assertTrue(groups.dropRight(1).forall(_.size == groupSize), "All but the last group should be the target size.")
+ assertTrue(groups.asScala.dropRight(1).forall(_.size == groupSize), "All but the last group should be the target size.")
// check grouping by index size
val indexSize = log.logSegments.asScala.take(groupSize).map(_.offsetIndex.sizeInBytes).sum + 1
- groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize, log.logEndOffset)
+ groups = cleaner.groupSegmentsBySize(log.logSegments, Int.MaxValue, indexSize, log.logEndOffset)
checkSegmentOrder(groups)
- assertTrue(groups.dropRight(1).forall(_.size == groupSize),
+ assertTrue(groups.asScala.dropRight(1).forall(_.size == groupSize),
"All but the last group should be the target size.")
}
@@ -1458,7 +1459,7 @@ class LogCleanerTest extends Logging {
val notCleanableSegments = 1
assertEquals(totalSegments, log.numberOfSegments)
- var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue, firstUncleanableOffset)
+ var groups = cleaner.groupSegmentsBySize(log.logSegments, Int.MaxValue, Int.MaxValue, firstUncleanableOffset)
//because index file uses 4 byte relative index offset and current segments all none empty,
//segments will not group even their size is very small.
assertEquals(totalSegments - notCleanableSegments, groups.size)
@@ -1468,7 +1469,7 @@ class LogCleanerTest extends Logging {
assertEquals(0, log.logSegments.asScala.head.size)
//after clean we got 2 empty segment, they will group together this time
- groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue, firstUncleanableOffset)
+ groups = cleaner.groupSegmentsBySize(log.logSegments, Int.MaxValue, Int.MaxValue, firstUncleanableOffset)
val noneEmptySegment = 1
assertEquals(noneEmptySegment + 1, groups.size)
@@ -1504,14 +1505,14 @@ class LogCleanerTest extends Logging {
assertEquals(Int.MaxValue, log.activeSegment.offsetIndex.lastOffset)
// grouping should result in a single group with maximum relative offset of Int.MaxValue
- var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue, log.logEndOffset)
+ var groups = cleaner.groupSegmentsBySize(log.logSegments, Int.MaxValue, Int.MaxValue, log.logEndOffset)
assertEquals(1, groups.size)
// append another message, making last offset of second segment > Int.MaxValue
log.appendAsLeader(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes), 0)
// grouping should not group the two segments to ensure that maximum relative offset in each group <= Int.MaxValue
- groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue, log.logEndOffset)
+ groups = cleaner.groupSegmentsBySize(log.logSegments, Int.MaxValue, Int.MaxValue, log.logEndOffset)
assertEquals(2, groups.size)
checkSegmentOrder(groups)
@@ -1519,10 +1520,10 @@ class LogCleanerTest extends Logging {
while (log.numberOfSegments < 4)
log.appendAsLeader(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes), 0)
- groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue, log.logEndOffset)
+ groups = cleaner.groupSegmentsBySize(log.logSegments, Int.MaxValue, Int.MaxValue, log.logEndOffset)
assertEquals(log.numberOfSegments - 1, groups.size)
- for (group <- groups)
- assertTrue(group.last.offsetIndex.lastOffset - group.head.offsetIndex.baseOffset <= Int.MaxValue,
+ for (group <- groups.asScala)
+ assertTrue(group.asScala.last.offsetIndex.lastOffset - group.asScala.head.offsetIndex.baseOffset <= Int.MaxValue,
"Relative offset greater than Int.MaxValue")
checkSegmentOrder(groups)
}
@@ -1562,17 +1563,17 @@ class LogCleanerTest extends Logging {
"index.lastOffset is reporting the wrong last offset")
// grouping should result in two groups because the second segment takes the offset range > MaxInt
- val groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue, log.logEndOffset)
+ val groups = cleaner.groupSegmentsBySize(log.logSegments, Int.MaxValue, Int.MaxValue, log.logEndOffset)
assertEquals(2, groups.size)
- for (group <- groups)
- assertTrue(group.last.readNextOffset - 1 - group.head.baseOffset <= Int.MaxValue,
+ for (group <- groups.asScala)
+ assertTrue(group.asScala.last.readNextOffset - 1 - group.asScala.head.baseOffset <= Int.MaxValue,
"Relative offset greater than Int.MaxValue")
checkSegmentOrder(groups)
}
- private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]): Unit = {
- val offsets = groups.flatMap(_.map(_.baseOffset))
+ private def checkSegmentOrder(groups: util.List[util.List[LogSegment]]): Unit = {
+ val offsets = groups.asScala.flatMap(_.asScala.map(_.baseOffset))
assertEquals(offsets.sorted, offsets, "Offsets should be in increasing order.")
}
@@ -1589,7 +1590,7 @@ class LogCleanerTest extends Logging {
writeToLog(log, (start until end) zip (start until end))
def checkRange(map: FakeOffsetMap, start: Int, end: Int): Unit = {
- val stats = new CleanerStats()
+ val stats = new CleanerStats(Time.SYSTEM)
cleaner.buildOffsetMap(log, start, end, map, stats)
val endOffset = map.latestOffset + 1
assertEquals(end, endOffset, "Last offset should be the end offset.")
@@ -1635,7 +1636,7 @@ class LogCleanerTest extends Logging {
// Try to clean segment with offset overflow. This will trigger log split and the cleaning itself must abort.
assertThrows(classOf[LogCleaningAbortedException], () =>
- cleaner.cleanSegments(log, Seq(segmentWithOverflow), offsetMap, 0L, new CleanerStats(),
+ cleaner.cleanSegments(log, Seq(segmentWithOverflow).asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM),
new CleanedTransactionMetadata, -1, segmentWithOverflow.readNextOffset)
)
assertEquals(numSegmentsInitial + 1, log.logSegments.size)
@@ -1645,7 +1646,7 @@ class LogCleanerTest extends Logging {
// Clean each segment now that split is complete.
val upperBoundOffset = log.logSegments.asScala.last.readNextOffset
for (segmentToClean <- log.logSegments.asScala)
- cleaner.cleanSegments(log, List(segmentToClean), offsetMap, 0L, new CleanerStats(),
+ cleaner.cleanSegments(log, List(segmentToClean).asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM),
new CleanedTransactionMetadata, -1, upperBoundOffset)
assertEquals(expectedKeysAfterCleaning, LogTestUtils.keysInLog(log))
assertFalse(LogTestUtils.hasOffsetOverflow(log))
@@ -1688,7 +1689,7 @@ class LogCleanerTest extends Logging {
val upperBoundOffset = log.activeSegment.baseOffset
// clean the log
- cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq, offsetMap, 0L, new CleanerStats(),
+ cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq.asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM),
new CleanedTransactionMetadata, -1, upperBoundOffset)
// clear scheduler so that async deletes don't run
time.scheduler.clear()
@@ -1704,7 +1705,7 @@ class LogCleanerTest extends Logging {
log = recoverAndCheck(config, allKeys)
// clean again
- cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq, offsetMap, 0L, new CleanerStats(),
+ cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq.asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM),
new CleanedTransactionMetadata, -1, upperBoundOffset)
// clear scheduler so that async deletes don't run
time.scheduler.clear()
@@ -1721,7 +1722,7 @@ class LogCleanerTest extends Logging {
log = recoverAndCheck(config, allKeys)
// clean again
- cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq, offsetMap, 0L, new CleanerStats(),
+ cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq.asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM),
new CleanedTransactionMetadata, -1, upperBoundOffset)
// clear scheduler so that async deletes don't run
time.scheduler.clear()
@@ -1743,7 +1744,7 @@ class LogCleanerTest extends Logging {
}
for (k <- 1 until messageCount by 2)
offsetMap.put(key(k), Long.MaxValue)
- cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq, offsetMap, 0L, new CleanerStats(),
+ cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq.asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM),
new CleanedTransactionMetadata, -1, upperBoundOffset)
// clear scheduler so that async deletes don't run
time.scheduler.clear()
@@ -1761,7 +1762,7 @@ class LogCleanerTest extends Logging {
}
for (k <- 1 until messageCount by 2)
offsetMap.put(key(k), Long.MaxValue)
- cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq, offsetMap, 0L, new CleanerStats(),
+ cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq.asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM),
new CleanedTransactionMetadata, -1, upperBoundOffset)
// clear scheduler so that async deletes don't run
time.scheduler.clear()
@@ -1779,7 +1780,7 @@ class LogCleanerTest extends Logging {
}
for (k <- 1 until messageCount by 2)
offsetMap.put(key(k), Long.MaxValue)
- cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq, offsetMap, 0L, new CleanerStats(),
+ cleaner.cleanSegments(log, log.logSegments.asScala.take(9).toSeq.asJava, offsetMap, 0L, new CleanerStats(Time.SYSTEM),
new CleanedTransactionMetadata, -1, upperBoundOffset)
// clear scheduler so that async deletes don't run
time.scheduler.clear()
@@ -1808,7 +1809,7 @@ class LogCleanerTest extends Logging {
val offsetEnd = 7206178L
val offsetSeq = Seq(offsetStart, offsetEnd)
writeToLog(log, (keyStart until keyEnd) zip (keyStart until keyEnd), offsetSeq)
- cleaner.buildOffsetMap(log, keyStart, offsetEnd + 1L, map, new CleanerStats())
+ cleaner.buildOffsetMap(log, keyStart, offsetEnd + 1L, map, new CleanerStats(Time.SYSTEM))
assertEquals(offsetEnd, map.latestOffset, "Last offset should be the end offset.")
assertEquals(keyEnd - keyStart, map.size, "Should have the expected number of messages in the map.")
assertEquals(0L, map.get(key(0)), "Map should contain first value")
@@ -1832,7 +1833,7 @@ class LogCleanerTest extends Logging {
log.appendAsLeader(record(4,4), 0)
log.roll()
- val stats = new CleanerStats()
+ val stats = new CleanerStats(Time.SYSTEM)
cleaner.buildOffsetMap(log, 2, Int.MaxValue, map, stats)
assertEquals(2, map.size)
assertEquals(-1, map.get(key(0)))
@@ -1958,7 +1959,7 @@ class LogCleanerTest extends Logging {
// active segment record
log.appendAsFollower(messageWithOffset(1015, 1015, 11L), Int.MaxValue)
- val (nextDirtyOffset, _) = cleaner.clean(new LogToClean(log, 0L, log.activeSegment.baseOffset, true))
+ val nextDirtyOffset = cleaner.clean(new LogToClean(log, 0L, log.activeSegment.baseOffset, true)).getKey
assertEquals(log.activeSegment.baseOffset, nextDirtyOffset,
"Cleaning point should pass offset gap")
}
@@ -1977,7 +1978,7 @@ class LogCleanerTest extends Logging {
// active segment record
log.appendAsFollower(messageWithOffset(1015, 1015, 30L), Int.MaxValue)
- val (nextDirtyOffset, _) = cleaner.clean(new LogToClean(log, 0L, log.activeSegment.baseOffset, true))
+ val nextDirtyOffset = cleaner.clean(new LogToClean(log, 0L, log.activeSegment.baseOffset, true)).getKey
assertEquals(log.activeSegment.baseOffset, nextDirtyOffset,
"Cleaning point should pass offset gap in multiple segments")
}
@@ -1986,10 +1987,10 @@ class LogCleanerTest extends Logging {
@Test
def testMaxCleanTimeSecs(): Unit = {
val logCleaner = new LogCleaner(new CleanerConfig(true),
- logDirs = Array(TestUtils.tempDir()),
- logs = new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
- logDirFailureChannel = new LogDirFailureChannel(1),
- time = time)
+ util.List.of(TestUtils.tempDir()),
+ new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
+ new LogDirFailureChannel(1),
+ time)
def checkGauge(name: String): Unit = {
val gauge = logCleaner.metricsGroup.newGauge(name, () => 999)
@@ -2009,11 +2010,11 @@ class LogCleanerTest extends Logging {
val oldKafkaProps = TestUtils.createBrokerConfig(1)
oldKafkaProps.setProperty(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "10000000")
- val logCleaner = new LogCleaner(LogCleaner.cleanerConfig(new KafkaConfig(oldKafkaProps)),
- logDirs = Array(TestUtils.tempDir()),
- logs = new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
- logDirFailureChannel = new LogDirFailureChannel(1),
- time = time) {
+ val logCleaner = new LogCleaner(new CleanerConfig(new KafkaConfig(oldKafkaProps)),
+ util.List.of(TestUtils.tempDir()),
+ new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
+ new LogDirFailureChannel(1),
+ time) {
// shutdown() and startup() are called in LogCleaner.reconfigure().
// Empty startup() and shutdown() to ensure that no unnecessary log cleaner threads remain after this test.
override def startup(): Unit = {}
@@ -2038,14 +2039,14 @@ class LogCleanerTest extends Logging {
def testMaxBufferUtilizationPercentMetric(): Unit = {
val logCleaner = new LogCleaner(
new CleanerConfig(true),
- logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()),
- logs = new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
- logDirFailureChannel = new LogDirFailureChannel(1),
- time = time
+ util.List.of(TestUtils.tempDir(), TestUtils.tempDir()),
+ new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
+ new LogDirFailureChannel(1),
+ time
)
def assertMaxBufferUtilizationPercent(expected: Int): Unit = {
- val gauge = logCleaner.metricsGroup.newGauge(MaxBufferUtilizationPercentMetricName,
+ val gauge = logCleaner.metricsGroup.newGauge(LogCleaner.MAX_BUFFER_UTILIZATION_PERCENT_METRIC_NAME,
() => (logCleaner.maxOverCleanerThreads(_.lastStats.bufferUtilization) * 100).toInt)
assertEquals(expected, gauge.value())
}
@@ -2057,29 +2058,29 @@ class LogCleanerTest extends Logging {
val cleaners = logCleaner.cleaners
val cleaner1 = new logCleaner.CleanerThread(1)
- cleaner1.lastStats = new CleanerStats(time)
- cleaner1.lastStats.bufferUtilization = 0.75
- cleaners += cleaner1
+ cleaner1.setLastStats(new CleanerStats(time))
+ cleaner1.lastStats.setBufferUtilization(0.75)
+ cleaners.add(cleaner1)
val cleaner2 = new logCleaner.CleanerThread(2)
- cleaner2.lastStats = new CleanerStats(time)
- cleaner2.lastStats.bufferUtilization = 0.85
- cleaners += cleaner2
+ cleaner2.setLastStats(new CleanerStats(time))
+ cleaner2.lastStats.setBufferUtilization(0.85)
+ cleaners.add(cleaner2)
val cleaner3 = new logCleaner.CleanerThread(3)
- cleaner3.lastStats = new CleanerStats(time)
- cleaner3.lastStats.bufferUtilization = 0.65
- cleaners += cleaner3
+ cleaner3.setLastStats(new CleanerStats(time))
+ cleaner3.lastStats.setBufferUtilization(0.65)
+ cleaners.add(cleaner3)
// expect the gauge value to reflect the maximum bufferUtilization
assertMaxBufferUtilizationPercent(85)
// Update bufferUtilization and verify the gauge value updates
- cleaner1.lastStats.bufferUtilization = 0.9
+ cleaner1.lastStats.setBufferUtilization(0.9)
assertMaxBufferUtilizationPercent(90)
// All CleanerThreads have the same bufferUtilization
- cleaners.foreach(_.lastStats.bufferUtilization = 0.5)
+ cleaners.forEach(_.lastStats.setBufferUtilization(0.5))
assertMaxBufferUtilizationPercent(50)
} finally {
logCleaner.shutdown()
@@ -2090,14 +2091,14 @@ class LogCleanerTest extends Logging {
def testMaxCleanTimeMetric(): Unit = {
val logCleaner = new LogCleaner(
new CleanerConfig(true),
- logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()),
- logs = new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
- logDirFailureChannel = new LogDirFailureChannel(1),
- time = time
+ util.List.of(TestUtils.tempDir(), TestUtils.tempDir()),
+ new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
+ new LogDirFailureChannel(1),
+ time
)
def assertMaxCleanTime(expected: Int): Unit = {
- val gauge = logCleaner.metricsGroup.newGauge(MaxCleanTimeMetricName,
+ val gauge = logCleaner.metricsGroup.newGauge(LogCleaner.MAX_CLEAN_TIME_METRIC_NAME,
() => logCleaner.maxOverCleanerThreads(_.lastStats.elapsedSecs).toInt)
assertEquals(expected, gauge.value())
}
@@ -2109,29 +2110,29 @@ class LogCleanerTest extends Logging {
val cleaners = logCleaner.cleaners
val cleaner1 = new logCleaner.CleanerThread(1)
- cleaner1.lastStats = new CleanerStats(time)
- cleaner1.lastStats.endTime = cleaner1.lastStats.startTime + 1_000L
- cleaners += cleaner1
+ cleaner1.setLastStats(new CleanerStats(time))
+ cleaner1.lastStats.setEndTime(cleaner1.lastStats.startTime + 1_000L)
+ cleaners.add(cleaner1)
val cleaner2 = new logCleaner.CleanerThread(2)
- cleaner2.lastStats = new CleanerStats(time)
- cleaner2.lastStats.endTime = cleaner2.lastStats.startTime + 2_000L
- cleaners += cleaner2
+ cleaner2.setLastStats(new CleanerStats(time))
+ cleaner2.lastStats.setEndTime(cleaner2.lastStats.startTime + 2_000L)
+ cleaners.add(cleaner2)
val cleaner3 = new logCleaner.CleanerThread(3)
- cleaner3.lastStats = new CleanerStats(time)
- cleaner3.lastStats.endTime = cleaner3.lastStats.startTime + 3_000L
- cleaners += cleaner3
+ cleaner3.setLastStats(new CleanerStats(time))
+ cleaner3.lastStats.setEndTime(cleaner3.lastStats.startTime + 3_000L)
+ cleaners.add(cleaner3)
// expect the gauge value to reflect the maximum cleanTime
assertMaxCleanTime(3)
// Update cleanTime and verify the gauge value updates
- cleaner1.lastStats.endTime = cleaner1.lastStats.startTime + 4_000L
+ cleaner1.lastStats.setEndTime(cleaner1.lastStats.startTime + 4_000L)
assertMaxCleanTime(4)
// All CleanerThreads have the same cleanTime
- cleaners.foreach(cleaner => cleaner.lastStats.endTime = cleaner.lastStats.startTime + 1_500L)
+ cleaners.forEach(cleaner => cleaner.lastStats.setEndTime(cleaner.lastStats.startTime + 1_500L))
assertMaxCleanTime(1)
} finally {
logCleaner.shutdown()
@@ -2142,14 +2143,14 @@ class LogCleanerTest extends Logging {
def testMaxCompactionDelayMetrics(): Unit = {
val logCleaner = new LogCleaner(
new CleanerConfig(true),
- logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()),
- logs = new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
- logDirFailureChannel = new LogDirFailureChannel(1),
- time = time
+ util.List.of(TestUtils.tempDir(), TestUtils.tempDir()),
+ new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](),
+ new LogDirFailureChannel(1),
+ time
)
def assertMaxCompactionDelay(expected: Int): Unit = {
- val gauge = logCleaner.metricsGroup.newGauge(MaxCompactionDelayMetricsName,
+ val gauge = logCleaner.metricsGroup.newGauge(LogCleaner.MAX_COMPACTION_DELAY_METRICS_NAME,
() => (logCleaner.maxOverCleanerThreads(_.lastPreCleanStats.maxCompactionDelayMs.toDouble) / 1000).toInt)
assertEquals(expected, gauge.value())
}
@@ -2161,19 +2162,19 @@ class LogCleanerTest extends Logging {
val cleaners = logCleaner.cleaners
val cleaner1 = new logCleaner.CleanerThread(1)
- cleaner1.lastStats = new CleanerStats(time)
+ cleaner1.setLastStats(new CleanerStats(time))
cleaner1.lastPreCleanStats.maxCompactionDelayMs(1_000L)
- cleaners += cleaner1
+ cleaners.add(cleaner1)
val cleaner2 = new logCleaner.CleanerThread(2)
- cleaner2.lastStats = new CleanerStats(time)
+ cleaner2.setLastStats(new CleanerStats(time))
cleaner2.lastPreCleanStats.maxCompactionDelayMs(2_000L)
- cleaners += cleaner2
+ cleaners.add(cleaner2)
val cleaner3 = new logCleaner.CleanerThread(3)
- cleaner3.lastStats = new CleanerStats(time)
+ cleaner3.setLastStats(new CleanerStats(time))
cleaner3.lastPreCleanStats.maxCompactionDelayMs(3_000L)
- cleaners += cleaner3
+ cleaners.add(cleaner3)
// expect the gauge value to reflect the maximum CompactionDelay
assertMaxCompactionDelay(3)
@@ -2183,7 +2184,7 @@ class LogCleanerTest extends Logging {
assertMaxCompactionDelay(4)
// All CleanerThreads have the same CompactionDelay
- cleaners.foreach(_.lastPreCleanStats.maxCompactionDelayMs(1_500L))
+ cleaners.forEach(_.lastPreCleanStats.maxCompactionDelayMs(1_500L))
assertMaxCompactionDelay(1)
} finally {
logCleaner.shutdown()
@@ -2244,15 +2245,15 @@ class LogCleanerTest extends Logging {
)
}
- private def makeCleaner(capacity: Int, checkDone: TopicPartition => Unit = _ => (), maxMessageSize: Int = 64*1024) =
- new Cleaner(id = 0,
- offsetMap = new FakeOffsetMap(capacity),
- ioBufferSize = maxMessageSize,
- maxIoBufferSize = maxMessageSize,
- dupBufferLoadFactor = 0.75,
- throttler = throttler,
- time = time,
- checkDone = checkDone)
+ private def makeCleaner(capacity: Int, checkDone: Consumer[TopicPartition] = _ => (), maxMessageSize: Int = 64*1024) =
+ new Cleaner(0,
+ new FakeOffsetMap(capacity),
+ maxMessageSize,
+ maxMessageSize,
+ 0.75,
+ throttler,
+ time,
+ checkDone)
private def writeToLog(log: UnifiedLog, seq: Iterable[(Int, Int)]): Iterable[Long] = {
for ((key, value) <- seq)
@@ -2345,7 +2346,7 @@ class LogCleanerTest extends Logging {
private def runTwoPassClean(cleaner: Cleaner, logToClean: LogToClean, currentTime: Long,
tombstoneRetentionMs: Long = 86400000) : Long = {
cleaner.doClean(logToClean, currentTime)
- cleaner.doClean(logToClean, currentTime + tombstoneRetentionMs + 1)._1
+ cleaner.doClean(logToClean, currentTime + tombstoneRetentionMs + 1).getKey
}
}
diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
index b6909848253d8..65fa7a29f5c02 100644
--- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
@@ -67,7 +67,7 @@ object LogTestUtils {
localRetentionBytes: Long = LogConfig.DEFAULT_LOCAL_RETENTION_BYTES,
segmentJitterMs: Long = LogConfig.DEFAULT_SEGMENT_JITTER_MS,
cleanupPolicy: String = ServerLogConfigs.LOG_CLEANUP_POLICY_DEFAULT,
- maxMessageBytes: Int = LogConfig.DEFAULT_MAX_MESSAGE_BYTES,
+ maxMessageBytes: Int = ServerLogConfigs.MAX_MESSAGE_BYTES_DEFAULT,
indexIntervalBytes: Int = ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DEFAULT,
segmentIndexBytes: Int = ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_DEFAULT,
fileDeleteDelayMs: Long = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
index 257f777885b20..6628fa600b849 100755
--- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
@@ -43,7 +43,7 @@ import org.apache.kafka.server.storage.log.{FetchIsolation, UnexpectedAppendOffs
import org.apache.kafka.server.util.{KafkaScheduler, MockTime, Scheduler}
import org.apache.kafka.storage.internals.checkpoint.{LeaderEpochCheckpointFile, PartitionMetadataFile}
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
-import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, EpochEntry, LogConfig, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetResultHolder, OffsetsOutOfOrderException, ProducerStateManager, ProducerStateManagerConfig, RecordValidationException, UnifiedLog, VerificationGuard}
+import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, Cleaner, EpochEntry, LogConfig, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetResultHolder, OffsetsOutOfOrderException, ProducerStateManager, ProducerStateManagerConfig, RecordValidationException, UnifiedLog, VerificationGuard}
import org.apache.kafka.storage.internals.utils.Throttler
import org.apache.kafka.storage.log.metrics.{BrokerTopicMetrics, BrokerTopicStats}
import org.junit.jupiter.api.Assertions._
@@ -1076,14 +1076,14 @@ class UnifiedLogTest {
val log = createLog(logDir, logConfig)
val pid1 = 1L
val epoch = 0.toShort
- val cleaner = new Cleaner(id = 0,
- offsetMap = new FakeOffsetMap(Int.MaxValue),
- ioBufferSize = 64 * 1024,
- maxIoBufferSize = 64 * 1024,
- dupBufferLoadFactor = 0.75,
- throttler = new Throttler(Double.MaxValue, Long.MaxValue, "throttler", "entries", mockTime),
- time = mockTime,
- checkDone = _ => {})
+ val cleaner = new Cleaner(0,
+ new FakeOffsetMap(Int.MaxValue),
+ 64 * 1024,
+ 64 * 1024,
+ 0.75,
+ new Throttler(Double.MaxValue, Long.MaxValue, "throttler", "entries", mockTime),
+ mockTime,
+ tp => {})
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes, "a".getBytes())), producerId = pid1,
producerEpoch = epoch, sequence = 0), 0)
diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
index 8030c5060dd60..1459cd64f7dd0 100755
--- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala
@@ -594,7 +594,7 @@ class DynamicBrokerConfigTest {
config.dynamicConfig.initialize(None)
assertEquals(SocketServerConfigs.MAX_CONNECTIONS_DEFAULT, config.maxConnections)
- assertEquals(LogConfig.DEFAULT_MAX_MESSAGE_BYTES, config.messageMaxBytes)
+ assertEquals(ServerLogConfigs.MAX_MESSAGE_BYTES_DEFAULT, config.messageMaxBytes)
var newProps = new Properties()
newProps.put(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, "9999")
diff --git a/docs/upgrade.html b/docs/upgrade.html
index 3ffa47f0b0242..5ad7e22fc3cbd 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -24,6 +24,14 @@
+ - Common
+
+ -
+ The logger class name for LogCleaner has been updated from
kafka.log.LogCleaner
to org.apache.kafka.storage.internals.log.LogCleaner
in the log4j2.yaml configuration file.
+ Added loggers for org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread
and org.apache.kafka.storage.internals.log.Cleaner
classes to CleanerAppender.
+
+
+
- Producer
- The
flush
method now detects potential deadlocks and prohibits its use inside a callback. This change prevents unintended blocking behavior, which was a known risk in earlier versions.
diff --git a/server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerConfigs.java
similarity index 97%
rename from server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java
rename to server-common/src/main/java/org/apache/kafka/server/config/ServerConfigs.java
index 82cf9fd34bf21..9906320fe9d07 100644
--- a/server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java
+++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerConfigs.java
@@ -22,7 +22,6 @@
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.record.BrokerCompressionType;
-import org.apache.kafka.storage.internals.log.LogConfig;
import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH;
import static org.apache.kafka.common.config.ConfigDef.Importance.LOW;
@@ -122,7 +121,7 @@ public class ServerConfigs {
"will also be early start listeners. A listener should not appear in this list if it accepts external traffic.";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(BROKER_ID_CONFIG, INT, BROKER_ID_DEFAULT, HIGH, BROKER_ID_DOC)
- .define(MESSAGE_MAX_BYTES_CONFIG, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, MESSAGE_MAX_BYTES_DOC)
+ .define(MESSAGE_MAX_BYTES_CONFIG, INT, ServerLogConfigs.MAX_MESSAGE_BYTES_DEFAULT, atLeast(0), HIGH, MESSAGE_MAX_BYTES_DOC)
.define(NUM_IO_THREADS_CONFIG, INT, NUM_IO_THREADS_DEFAULT, atLeast(1), HIGH, NUM_IO_THREADS_DOC)
.define(NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG, INT, null, HIGH, NUM_REPLICA_ALTER_LOG_DIRS_THREADS_DOC)
.define(BACKGROUND_THREADS_CONFIG, INT, BACKGROUND_THREADS_DEFAULT, atLeast(1), HIGH, BACKGROUND_THREADS_DOC)
@@ -137,7 +136,7 @@ public class ServerConfigs {
/** ********* Controlled shutdown configuration ***********/
.define(CONTROLLED_SHUTDOWN_ENABLE_CONFIG, BOOLEAN, CONTROLLED_SHUTDOWN_ENABLE_DEFAULT, MEDIUM, CONTROLLED_SHUTDOWN_ENABLE_DOC)
.define(DELETE_TOPIC_ENABLE_CONFIG, BOOLEAN, DELETE_TOPIC_ENABLE_DEFAULT, HIGH, DELETE_TOPIC_ENABLE_DOC)
- .define(COMPRESSION_TYPE_CONFIG, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, ConfigDef.ValidString.in(BrokerCompressionType.names().toArray(new String[0])), HIGH, COMPRESSION_TYPE_DOC)
+ .define(COMPRESSION_TYPE_CONFIG, STRING, ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, ConfigDef.ValidString.in(BrokerCompressionType.names().toArray(new String[0])), HIGH, COMPRESSION_TYPE_DOC)
.define(COMPRESSION_GZIP_LEVEL_CONFIG, INT, CompressionType.GZIP.defaultLevel(), CompressionType.GZIP.levelValidator(), MEDIUM, COMPRESSION_GZIP_LEVEL_DOC)
.define(COMPRESSION_LZ4_LEVEL_CONFIG, INT, CompressionType.LZ4.defaultLevel(), CompressionType.LZ4.levelValidator(), MEDIUM, COMPRESSION_LZ4_LEVEL_DOC)
.define(COMPRESSION_ZSTD_LEVEL_CONFIG, INT, CompressionType.ZSTD.defaultLevel(), CompressionType.ZSTD.levelValidator(), MEDIUM, COMPRESSION_ZSTD_LEVEL_DOC)
diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java
index aeb4de47cf386..2c2d44ef012e5 100644
--- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java
+++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java
@@ -18,6 +18,8 @@
package org.apache.kafka.server.config;
import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.server.record.BrokerCompressionType;
import static org.apache.kafka.server.config.ServerTopicConfigSynonyms.LOG_PREFIX;
@@ -155,4 +157,7 @@ public class ServerLogConfigs {
public static final Long LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT = 30000L;
public static final String LOG_DIR_FAILURE_TIMEOUT_MS_DOC = "If the broker is unable to successfully communicate to the controller that some log " +
"directory has failed for longer than this time, the broker will fail and shut down.";
+
+ public static final int MAX_MESSAGE_BYTES_DEFAULT = 1024 * 1024 + Records.LOG_OVERHEAD;
+ public static final String COMPRESSION_TYPE_DEFAULT = BrokerCompressionType.PRODUCER.name;
}
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanedTransactionMetadata.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanedTransactionMetadata.java
new file mode 100644
index 0000000000000..d6e9642e59950
--- /dev/null
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanedTransactionMetadata.java
@@ -0,0 +1,164 @@
+/*
+ * 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 org.apache.kafka.storage.internals.log;
+
+import org.apache.kafka.common.record.ControlRecordType;
+import org.apache.kafka.common.record.Record;
+import org.apache.kafka.common.record.RecordBatch;
+
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+/**
+ * This is a helper class to facilitate tracking transaction state while cleaning the log. It maintains a set
+ * of the ongoing aborted and committed transactions as the cleaner is working its way through the log. This
+ * class is responsible for deciding when transaction markers can be removed and is therefore also responsible
+ * for updating the cleaned transaction index accordingly.
+ */
+public class CleanedTransactionMetadata {
+ private final Set ongoingCommittedTxns = new HashSet<>();
+ private final Map ongoingAbortedTxns = new HashMap<>();
+
+ /**
+ * Minheap of aborted transactions sorted by the transaction first offset
+ */
+ private final PriorityQueue abortedTransactions = new PriorityQueue<>(
+ Comparator.comparingLong(AbortedTxn::firstOffset)
+ );
+
+ /**
+ * Output cleaned index to write retained aborted transactions
+ */
+ private Optional cleanedIndex = Optional.empty();
+
+ /**
+ * Update the cleaned index.
+ *
+ * @param cleanedIndex The new cleaned index
+ */
+ public void setCleanedIndex(Optional cleanedIndex) {
+ this.cleanedIndex = cleanedIndex;
+ }
+
+ /**
+ * Update the cleaned transaction state with the new found aborted transactions that has just been traversed.
+ *
+ * @param abortedTransactions The new found aborted transactions to add
+ */
+ public void addAbortedTransactions(List abortedTransactions) {
+ this.abortedTransactions.addAll(abortedTransactions);
+ }
+
+ /**
+ * Update the cleaned transaction state with a control batch that has just been traversed by the cleaner.
+ * Return true if the control batch can be discarded.
+ *
+ * @param controlBatch The control batch that been traversed
+ *
+ * @return True if the control batch can be discarded
+ */
+ public boolean onControlBatchRead(RecordBatch controlBatch) {
+ consumeAbortedTxnsUpTo(controlBatch.lastOffset());
+
+ Iterator controlRecordIterator = controlBatch.iterator();
+ if (controlRecordIterator.hasNext()) {
+ Record controlRecord = controlRecordIterator.next();
+ ControlRecordType controlType = ControlRecordType.parse(controlRecord.key());
+ long producerId = controlBatch.producerId();
+
+ switch (controlType) {
+ case ABORT:
+ AbortedTransactionMetadata abortedTxnMetadata = ongoingAbortedTxns.remove(producerId);
+
+ // Retain the marker until all batches from the transaction have been removed.
+ if (abortedTxnMetadata != null && abortedTxnMetadata.lastObservedBatchOffset.isPresent()) {
+ cleanedIndex.ifPresent(index -> {
+ try {
+ index.append(abortedTxnMetadata.abortedTxn);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ return false;
+ }
+
+ return true;
+
+ case COMMIT:
+ // This marker is eligible for deletion if we didn't traverse any batches from the transaction
+ return !ongoingCommittedTxns.remove(producerId);
+
+ default:
+ return false;
+ }
+ } else {
+ // An empty control batch was already cleaned, so it's safe to discard
+ return true;
+ }
+ }
+
+ private void consumeAbortedTxnsUpTo(long offset) {
+ while (!abortedTransactions.isEmpty() && abortedTransactions.peek().firstOffset() <= offset) {
+ AbortedTxn abortedTxn = abortedTransactions.poll();
+ if (abortedTxn != null) {
+ ongoingAbortedTxns.computeIfAbsent(abortedTxn.producerId(), id -> new AbortedTransactionMetadata(abortedTxn));
+ }
+ }
+ }
+
+ /**
+ * Update the transactional state for the incoming non-control batch. If the batch is part of
+ * an aborted transaction, return true to indicate that it is safe to discard.
+ *
+ * @param batch The batch to read when updating the transactional state
+ *
+ * @return Whether the batch is part of an aborted transaction or not
+ */
+ public boolean onBatchRead(RecordBatch batch) {
+ consumeAbortedTxnsUpTo(batch.lastOffset());
+ if (batch.isTransactional()) {
+ Optional metadata = Optional.ofNullable(ongoingAbortedTxns.get(batch.producerId()));
+
+ if (metadata.isPresent()) {
+ metadata.get().lastObservedBatchOffset = Optional.of(batch.lastOffset());
+ return true;
+ } else {
+ ongoingCommittedTxns.add(batch.producerId());
+ return false;
+ }
+ } else {
+ return false;
+ }
+ }
+
+ private static class AbortedTransactionMetadata {
+ Optional lastObservedBatchOffset = Optional.empty();
+ final AbortedTxn abortedTxn;
+
+ public AbortedTransactionMetadata(AbortedTxn abortedTxn) {
+ this.abortedTxn = abortedTxn;
+ }
+ }
+}
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java
new file mode 100644
index 0000000000000..b7d4ccedb2ec1
--- /dev/null
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java
@@ -0,0 +1,766 @@
+/*
+ * 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 org.apache.kafka.storage.internals.log;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.record.FileRecords;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MutableRecordBatch;
+import org.apache.kafka.common.record.Record;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.utils.BufferSupplier;
+import org.apache.kafka.common.utils.CloseableIterator;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.storage.internals.utils.Throttler;
+
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.DigestException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Consumer;
+
+/**
+ * This class holds the actual logic for cleaning a log.
+ */
+public class Cleaner {
+ private final BufferSupplier decompressionBufferSupplier = BufferSupplier.create();
+
+ private final Logger logger;
+ private final int id;
+ private final OffsetMap offsetMap;
+ private final int ioBufferSize;
+ private final int maxIoBufferSize;
+ private final double dupBufferLoadFactor;
+ private final Throttler throttler;
+ private final Time time;
+ private final Consumer checkDone;
+
+ /**
+ * Buffer used for read i/o
+ */
+ private ByteBuffer readBuffer;
+
+ /**
+ * Buffer used for write i/o
+ */
+ private ByteBuffer writeBuffer;
+
+ /**
+ *
+ * @param id An identifier used for logging
+ * @param offsetMap The map used for deduplication
+ * @param ioBufferSize The size of the buffers to use. Memory usage will be 2x this number as there is a read and write buffer.
+ * @param maxIoBufferSize The maximum size of a message that can appear in the log
+ * @param dupBufferLoadFactor The maximum percent full for the deduplication buffer
+ * @param throttler The throttler instance to use for limiting I/O rate
+ * @param time The time instance
+ * @param checkDone Check if the cleaning for a partition is finished or aborted
+ */
+ public Cleaner(int id,
+ OffsetMap offsetMap,
+ int ioBufferSize,
+ int maxIoBufferSize,
+ double dupBufferLoadFactor,
+ Throttler throttler,
+ Time time,
+ Consumer checkDone) {
+ this.id = id;
+ this.offsetMap = offsetMap;
+ this.ioBufferSize = ioBufferSize;
+ this.maxIoBufferSize = maxIoBufferSize;
+ this.dupBufferLoadFactor = dupBufferLoadFactor;
+ this.throttler = throttler;
+ this.time = time;
+ this.checkDone = checkDone;
+ logger = new LogContext("Cleaner " + id + ": ").logger(Cleaner.class);
+
+ readBuffer = ByteBuffer.allocate(ioBufferSize);
+ writeBuffer = ByteBuffer.allocate(ioBufferSize);
+
+ assert offsetMap.slots() * dupBufferLoadFactor > 1 :
+ "offset map is too small to fit in even a single message, so log cleaning will never make progress. " +
+ "You can increase log.cleaner.dedupe.buffer.size or decrease log.cleaner.threads";
+ }
+
+ public int id() {
+ return id;
+ }
+
+ // Only for testing
+ public OffsetMap offsetMap() {
+ return offsetMap;
+ }
+
+ /**
+ * Clean the given log.
+ *
+ * @param cleanable The log to be cleaned
+ *
+ * @return The first offset not cleaned and the statistics for this round of cleaning
+ */
+ public Map.Entry clean(LogToClean cleanable) throws IOException, DigestException {
+ return doClean(cleanable, time.milliseconds());
+ }
+
+ /**
+ * Clean the given log.
+ *
+ * @param cleanable The log to be cleaned
+ * @param currentTime The current timestamp for doing cleaning
+ *
+ * @return The first offset not cleaned and the statistics for this round of cleaning
+ * */
+ public Map.Entry doClean(LogToClean cleanable, long currentTime) throws IOException, DigestException {
+ UnifiedLog log = cleanable.log();
+
+ logger.info("Beginning cleaning of log {}", log.name());
+
+ // figure out the timestamp below which it is safe to remove delete tombstones
+ // this position is defined to be a configurable time beneath the last modified time of the last clean segment
+ // this timestamp is only used on the older message formats older than MAGIC_VALUE_V2
+ List segments = log.logSegments(0, cleanable.firstDirtyOffset());
+ long legacyDeleteHorizonMs = segments.isEmpty()
+ ? 0L
+ : segments.get(segments.size() - 1).lastModified() - log.config().deleteRetentionMs;
+
+ CleanerStats stats = new CleanerStats(Time.SYSTEM);
+
+ // build the offset map
+ logger.info("Building offset map for {}...", log.name());
+ long upperBoundOffset = cleanable.firstUncleanableOffset();
+ buildOffsetMap(log, cleanable.firstDirtyOffset(), upperBoundOffset, offsetMap, stats);
+ long endOffset = offsetMap.latestOffset() + 1;
+ stats.indexDone();
+
+ // determine the timestamp up to which the log will be cleaned
+ // this is the lower of the last active segment and the compaction lag
+ segments = log.logSegments(0, cleanable.firstUncleanableOffset());
+ long cleanableHorizonMs = segments.isEmpty()
+ ? 0L
+ : segments.get(segments.size() - 1).lastModified();
+
+ // group the segments and clean the groups
+ logger.info("Cleaning log {} (cleaning prior to {}, discarding tombstones prior to upper bound deletion horizon {})...",
+ log.name(), new Date(cleanableHorizonMs), new Date(legacyDeleteHorizonMs));
+ CleanedTransactionMetadata transactionMetadata = new CleanedTransactionMetadata();
+
+ List> groupedSegments = groupSegmentsBySize(
+ log.logSegments(0, endOffset),
+ log.config().segmentSize,
+ log.config().maxIndexSize,
+ cleanable.firstUncleanableOffset()
+ );
+
+ for (List group : groupedSegments) {
+ cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, legacyDeleteHorizonMs, upperBoundOffset);
+ }
+
+ // record buffer utilization
+ stats.bufferUtilization = offsetMap.utilization();
+
+ stats.allDone();
+
+ return Map.entry(endOffset, stats);
+ }
+
+ /**
+ * Clean a group of segments into a single replacement segment.
+ *
+ * @param log The log being cleaned
+ * @param segments The group of segments being cleaned
+ * @param map The offset map to use for cleaning segments
+ * @param currentTime The current time in milliseconds
+ * @param stats Collector for cleaning statistics
+ * @param transactionMetadata State of ongoing transactions which is carried between the cleaning
+ * of the grouped segments
+ * @param legacyDeleteHorizonMs The delete horizon used for tombstones whose version is less than 2
+ * @param upperBoundOffsetOfCleaningRound The upper bound offset of this round of cleaning
+ */
+ @SuppressWarnings("finally")
+ public void cleanSegments(UnifiedLog log,
+ List segments,
+ OffsetMap map,
+ long currentTime,
+ CleanerStats stats,
+ CleanedTransactionMetadata transactionMetadata,
+ long legacyDeleteHorizonMs,
+ long upperBoundOffsetOfCleaningRound) throws IOException {
+ // create a new segment with a suffix appended to the name of the log and indexes
+ LogSegment cleaned = UnifiedLog.createNewCleanedSegment(log.dir(), log.config(), segments.get(0).baseOffset());
+ transactionMetadata.setCleanedIndex(Optional.of(cleaned.txnIndex()));
+
+ try {
+ // clean segments into the new destination segment
+ Iterator iter = segments.iterator();
+ Optional currentSegmentOpt = Optional.of(iter.next());
+ Map lastOffsetOfActiveProducers = log.lastRecordsOfActiveProducers();
+
+ while (currentSegmentOpt.isPresent()) {
+ LogSegment currentSegment = currentSegmentOpt.get();
+ Optional nextSegmentOpt = iter.hasNext() ? Optional.of(iter.next()) : Optional.empty();
+
+ // Note that it is important to collect aborted transactions from the full log segment
+ // range since we need to rebuild the full transaction index for the new segment.
+ long startOffset = currentSegment.baseOffset();
+ long upperBoundOffset = nextSegmentOpt.map(LogSegment::baseOffset).orElse(currentSegment.readNextOffset());
+ List abortedTransactions = log.collectAbortedTransactions(startOffset, upperBoundOffset);
+ transactionMetadata.addAbortedTransactions(abortedTransactions);
+
+ boolean retainLegacyDeletesAndTxnMarkers = currentSegment.lastModified() > legacyDeleteHorizonMs;
+ logger.info(
+ "Cleaning {} in log {} into {} with an upper bound deletion horizon {} computed from " +
+ "the segment last modified time of {},{} deletes.",
+ currentSegment, log.name(), cleaned.baseOffset(), legacyDeleteHorizonMs, currentSegment.lastModified(),
+ retainLegacyDeletesAndTxnMarkers ? "retaining" : "discarding"
+ );
+
+ try {
+ cleanInto(
+ log.topicPartition(),
+ currentSegment.log(),
+ cleaned,
+ map,
+ retainLegacyDeletesAndTxnMarkers,
+ log.config().deleteRetentionMs,
+ log.config().maxMessageSize(),
+ transactionMetadata,
+ lastOffsetOfActiveProducers,
+ upperBoundOffsetOfCleaningRound,
+ stats,
+ currentTime
+ );
+ } catch (LogSegmentOffsetOverflowException e) {
+ // Split the current segment. It's also safest to abort the current cleaning process, so that we retry from
+ // scratch once the split is complete.
+ logger.info("Caught segment overflow error during cleaning: {}", e.getMessage());
+ log.splitOverflowedSegment(currentSegment);
+ throw new LogCleaningAbortedException();
+ }
+ currentSegmentOpt = nextSegmentOpt;
+ }
+
+ cleaned.onBecomeInactiveSegment();
+ // flush new segment to disk before swap
+ cleaned.flush();
+
+ // update the modification date to retain the last modified date of the original files
+ long modified = segments.get(segments.size() - 1).lastModified();
+ cleaned.setLastModified(modified);
+
+ // swap in new segment
+ logger.info("Swapping in cleaned segment {} for segment(s) {} in log {}", cleaned, segments, log);
+ log.replaceSegments(List.of(cleaned), segments);
+ } catch (LogCleaningAbortedException e) {
+ try {
+ cleaned.deleteIfExists();
+ } catch (Exception deleteException) {
+ e.addSuppressed(deleteException);
+ } finally {
+ throw e;
+ }
+ }
+ }
+
+ /**
+ * Clean the given source log segment into the destination segment using the key=>offset mapping
+ * provided.
+ *
+ * @param topicPartition The topic and partition of the log segment to clean
+ * @param sourceRecords The dirty log segment
+ * @param dest The cleaned log segment
+ * @param map The key=>offset mapping
+ * @param retainLegacyDeletesAndTxnMarkers Should tombstones (lower than version 2) and markers be retained while cleaning this segment
+ * @param deleteRetentionMs Defines how long a tombstone should be kept as defined by log configuration
+ * @param maxLogMessageSize The maximum message size of the corresponding topic
+ * @param transactionMetadata The state of ongoing transactions which is carried between the cleaning of the grouped segments
+ * @param lastRecordsOfActiveProducers The active producers and its last data offset
+ * @param upperBoundOffsetOfCleaningRound Next offset of the last batch in the source segment
+ * @param stats Collector for cleaning statistics
+ * @param currentTime The time at which the clean was initiated
+ */
+ private void cleanInto(TopicPartition topicPartition,
+ FileRecords sourceRecords,
+ LogSegment dest,
+ OffsetMap map,
+ boolean retainLegacyDeletesAndTxnMarkers,
+ long deleteRetentionMs,
+ int maxLogMessageSize,
+ CleanedTransactionMetadata transactionMetadata,
+ Map lastRecordsOfActiveProducers,
+ long upperBoundOffsetOfCleaningRound,
+ CleanerStats stats,
+ long currentTime) throws IOException {
+ MemoryRecords.RecordFilter logCleanerFilter = new MemoryRecords.RecordFilter(currentTime, deleteRetentionMs) {
+ private boolean discardBatchRecords;
+
+ @Override
+ public BatchRetentionResult checkBatchRetention(RecordBatch batch) {
+ // we piggy-back on the tombstone retention logic to delay deletion of transaction markers.
+ // note that we will never delete a marker until all the records from that transaction are removed.
+ boolean canDiscardBatch = shouldDiscardBatch(batch, transactionMetadata);
+
+ if (batch.isControlBatch()) {
+ discardBatchRecords = canDiscardBatch && batch.deleteHorizonMs().isPresent() && batch.deleteHorizonMs().getAsLong() <= this.currentTime;
+ } else {
+ discardBatchRecords = canDiscardBatch;
+ }
+
+ // We retain the batch in order to preserve the state of active producers. There are three cases:
+ // 1) The producer is no longer active, which means we can delete all records for that producer.
+ // 2) The producer is still active and has a last data offset. We retain the batch that contains
+ // this offset since it also contains the last sequence number for this producer.
+ // 3) The last entry in the log is a transaction marker. We retain this marker since it has the
+ // last producer epoch, which is needed to ensure fencing.
+ boolean isBatchLastRecordOfProducer = Optional.ofNullable(lastRecordsOfActiveProducers.get(batch.producerId()))
+ .map(lastRecord -> {
+ if (lastRecord.lastDataOffset.isPresent()) {
+ return batch.lastOffset() == lastRecord.lastDataOffset.getAsLong();
+ } else {
+ return batch.isControlBatch() && batch.producerEpoch() == lastRecord.producerEpoch;
+ }
+ })
+ .orElse(false);
+
+ BatchRetention batchRetention;
+ if (batch.hasProducerId() && isBatchLastRecordOfProducer)
+ batchRetention = BatchRetention.RETAIN_EMPTY;
+ else if (batch.nextOffset() == upperBoundOffsetOfCleaningRound) {
+ // retain the last batch of the cleaning round, even if it's empty, so that last offset information
+ // is not lost after cleaning.
+ batchRetention = BatchRetention.RETAIN_EMPTY;
+ } else if (discardBatchRecords)
+ batchRetention = BatchRetention.DELETE;
+ else
+ batchRetention = BatchRetention.DELETE_EMPTY;
+
+ return new BatchRetentionResult(batchRetention, canDiscardBatch && batch.isControlBatch());
+ }
+
+ @Override
+ public boolean shouldRetainRecord(RecordBatch batch, Record record) {
+ if (discardBatchRecords) {
+ // The batch is only retained to preserve producer sequence information; the records can be removed
+ return false;
+ } else if (batch.isControlBatch()) {
+ return true;
+ } else {
+ try {
+ return Cleaner.this.shouldRetainRecord(map, retainLegacyDeletesAndTxnMarkers, batch, record, stats, currentTime);
+ } catch (DigestException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+ };
+
+ int position = 0;
+ while (position < sourceRecords.sizeInBytes()) {
+ checkDone.accept(topicPartition);
+ // read a chunk of messages and copy any that are to be retained to the write buffer to be written out
+ readBuffer.clear();
+ writeBuffer.clear();
+
+ sourceRecords.readInto(readBuffer, position);
+ MemoryRecords records = MemoryRecords.readableRecords(readBuffer);
+ throttler.maybeThrottle(records.sizeInBytes());
+ MemoryRecords.FilterResult result = records.filterTo(logCleanerFilter, writeBuffer, decompressionBufferSupplier);
+
+ stats.readMessages(result.messagesRead(), result.bytesRead());
+ stats.recopyMessages(result.messagesRetained(), result.bytesRetained());
+
+ position += result.bytesRead();
+
+ // if any messages are to be retained, write them out
+ ByteBuffer outputBuffer = result.outputBuffer();
+ if (outputBuffer.position() > 0) {
+ outputBuffer.flip();
+ MemoryRecords retained = MemoryRecords.readableRecords(outputBuffer);
+ // it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads
+ // after `Log.replaceSegments` (which acquires the lock) is called
+ dest.append(result.maxOffset(), retained);
+ throttler.maybeThrottle(outputBuffer.limit());
+ }
+
+ // if we read bytes but didn't get even one complete batch, our I/O buffer is too small, grow it and try again
+ // `result.bytesRead` contains bytes from `messagesRead` and any discarded batches.
+ if (readBuffer.limit() > 0 && result.bytesRead() == 0)
+ growBuffersOrFail(sourceRecords, position, maxLogMessageSize, records);
+ }
+ restoreBuffers();
+ }
+
+
+ /**
+ * Grow buffers to process next batch of records from `sourceRecords.` Buffers are doubled in size
+ * up to a maximum of `maxLogMessageSize`. In some scenarios, a record could be bigger than the
+ * current maximum size configured for the log. For example:
+ *
+ * - A compacted topic using compression may contain a message set slightly larger than max.message.bytes
+ * - max.message.bytes of a topic could have been reduced after writing larger messages
+ *
+ * In these cases, grow the buffer to hold the next batch.
+ *
+ * @param sourceRecords The dirty log segment records to process
+ * @param position The current position in the read buffer to read from
+ * @param maxLogMessageSize The maximum record size in bytes for the topic
+ * @param memoryRecords The memory records in read buffer
+ */
+ private void growBuffersOrFail(FileRecords sourceRecords,
+ int position,
+ int maxLogMessageSize,
+ MemoryRecords memoryRecords) throws IOException {
+ int maxSize;
+
+ if (readBuffer.capacity() >= maxLogMessageSize) {
+ Integer nextBatchSize = memoryRecords.firstBatchSize();
+ String logDesc = String.format("log segment %s at position %d", sourceRecords.file(), position);
+ if (nextBatchSize == null) {
+ throw new IllegalStateException("Could not determine next batch size for " + logDesc);
+ }
+ if (nextBatchSize <= 0) {
+ throw new IllegalStateException("Invalid batch size " + nextBatchSize + " for " + logDesc);
+ }
+ if (nextBatchSize <= readBuffer.capacity()) {
+ throw new IllegalStateException("Batch size " + nextBatchSize + " < buffer size " + readBuffer.capacity() + ", but not processed for " + logDesc);
+ }
+ long bytesLeft = sourceRecords.channel().size() - position;
+ if (nextBatchSize > bytesLeft) {
+ throw new CorruptRecordException("Log segment may be corrupt, batch size " + nextBatchSize + " > " + bytesLeft + " bytes left in segment for " + logDesc);
+ }
+
+ maxSize = nextBatchSize;
+ } else {
+ maxSize = maxLogMessageSize;
+ }
+
+ growBuffers(maxSize);
+ }
+
+ /**
+ * Check if a batch should be discarded by cleaned transaction state.
+ *
+ * @param batch The batch of records to check
+ * @param transactionMetadata The maintained transaction state about cleaning
+ *
+ * @return if the batch can be discarded
+ */
+ private boolean shouldDiscardBatch(RecordBatch batch,
+ CleanedTransactionMetadata transactionMetadata) {
+ if (batch.isControlBatch())
+ return transactionMetadata.onControlBatchRead(batch);
+ else
+ return transactionMetadata.onBatchRead(batch);
+ }
+
+ /**
+ * Check if a record should be retained.
+ *
+ * @param map The offset map(key=>offset) to use for cleaning segments
+ * @param retainDeletesForLegacyRecords Should tombstones (lower than version 2) and markers be retained while cleaning this segment
+ * @param batch The batch of records that the record belongs to
+ * @param record The record to check
+ * @param stats The collector for cleaning statistics
+ * @param currentTime The current time that used to compare with the delete horizon time of the batch when judging a non-legacy record
+ *
+ * @return if the record can be retained
+ */
+ private boolean shouldRetainRecord(OffsetMap map,
+ boolean retainDeletesForLegacyRecords,
+ RecordBatch batch,
+ Record record,
+ CleanerStats stats,
+ long currentTime) throws DigestException {
+ boolean pastLatestOffset = record.offset() > map.latestOffset();
+ if (pastLatestOffset) {
+ return true;
+ }
+
+ if (record.hasKey()) {
+ ByteBuffer key = record.key();
+ long foundOffset = map.get(key);
+ /* First,the message must have the latest offset for the key
+ * then there are two cases in which we can retain a message:
+ * 1) The message has value
+ * 2) The message doesn't have value but it can't be deleted now.
+ */
+ boolean latestOffsetForKey = record.offset() >= foundOffset;
+ boolean legacyRecord = batch.magic() < RecordBatch.MAGIC_VALUE_V2;
+
+ boolean shouldRetainDeletes;
+ if (!legacyRecord) {
+ shouldRetainDeletes = batch.deleteHorizonMs().isEmpty() || currentTime < batch.deleteHorizonMs().getAsLong();
+ } else {
+ shouldRetainDeletes = retainDeletesForLegacyRecords;
+ }
+
+ boolean isRetainedValue = record.hasValue() || shouldRetainDeletes;
+ return latestOffsetForKey && isRetainedValue;
+ } else {
+ stats.invalidMessage();
+ return false;
+ }
+ }
+
+ /**
+ * Double the I/O buffer capacity.
+ *
+ * @param maxLogMessageSize The maximum record size in bytes allowed
+ */
+ private void growBuffers(int maxLogMessageSize) {
+ int maxBufferSize = Math.max(maxLogMessageSize, maxIoBufferSize);
+ if (readBuffer.capacity() >= maxBufferSize || writeBuffer.capacity() >= maxBufferSize)
+ throw new IllegalStateException("This log contains a message larger than maximum allowable size of " + maxBufferSize + ".");
+ int newSize = Math.min(readBuffer.capacity() * 2, maxBufferSize);
+ logger.info("Growing cleaner I/O buffers from {} bytes to {} bytes.", readBuffer.capacity(), newSize);
+ readBuffer = ByteBuffer.allocate(newSize);
+ writeBuffer = ByteBuffer.allocate(newSize);
+ }
+
+ /**
+ * Restore the I/O buffer capacity to its original size.
+ */
+ private void restoreBuffers() {
+ if (readBuffer.capacity() > ioBufferSize)
+ readBuffer = ByteBuffer.allocate(ioBufferSize);
+ if (writeBuffer.capacity() > ioBufferSize)
+ writeBuffer = ByteBuffer.allocate(ioBufferSize);
+ }
+
+ /**
+ * Group the segments in a log into groups totaling less than a given size. the size is enforced separately for the log data and the index data.
+ * We collect a group of such segments together into a single destination segment.
+ * This prevents segment sizes from shrinking too much.
+ *
+ * @param segments The log segments to group
+ * @param maxSize the maximum size in bytes for the total of all log data in a group
+ * @param maxIndexSize the maximum size in bytes for the total of all index data in a group
+ * @param firstUncleanableOffset The upper(exclusive) offset to clean to
+ *
+ * @return A list of grouped segments
+ */
+ public List> groupSegmentsBySize(List segments, int maxSize, int maxIndexSize, long firstUncleanableOffset) throws IOException {
+ List> grouped = new ArrayList<>();
+
+ while (!segments.isEmpty()) {
+ List group = new ArrayList<>();
+ group.add(segments.get(0));
+
+ long logSize = segments.get(0).size();
+ long indexSize = segments.get(0).offsetIndex().sizeInBytes();
+ long timeIndexSize = segments.get(0).timeIndex().sizeInBytes();
+
+ segments = segments.subList(1, segments.size());
+
+ while (!segments.isEmpty() &&
+ logSize + segments.get(0).size() <= maxSize &&
+ indexSize + segments.get(0).offsetIndex().sizeInBytes() <= maxIndexSize &&
+ timeIndexSize + segments.get(0).timeIndex().sizeInBytes() <= maxIndexSize &&
+ //if first segment size is 0, we don't need to do the index offset range check.
+ //this will avoid empty log left every 2^31 message.
+ (segments.get(0).size() == 0 ||
+ lastOffsetForFirstSegment(segments, firstUncleanableOffset) - group.get(group.size() - 1).baseOffset() <= Integer.MAX_VALUE)) {
+ group.add(0, segments.get(0));
+ logSize += segments.get(0).size();
+ indexSize += segments.get(0).offsetIndex().sizeInBytes();
+ timeIndexSize += segments.get(0).timeIndex().sizeInBytes();
+ segments = segments.subList(1, segments.size());
+ }
+
+ Collections.reverse(group);
+ grouped.add(0, group);
+ }
+
+ Collections.reverse(grouped);
+
+ return grouped;
+ }
+
+ /**
+ * We want to get the last offset in the first log segment in segs.
+ * LogSegment.nextOffset() gives the exact last offset in a segment, but can be expensive since it requires
+ * scanning the segment from the last index entry.
+ * Therefore, we estimate the last offset of the first log segment by using
+ * the base offset of the next segment in the list.
+ * If the next segment doesn't exist, first Uncleanable Offset will be used.
+ *
+ * @param segs Remaining segments to group
+ * @param firstUncleanableOffset The upper(exclusive) offset to clean to
+ * @return The estimated last offset for the first segment in segs
+ */
+ private long lastOffsetForFirstSegment(List segs, long firstUncleanableOffset) {
+ if (segs.size() > 1) {
+ /* if there is a next segment, use its base offset as the bounding offset to guarantee we know
+ * the worst case offset */
+ return segs.get(1).baseOffset() - 1;
+ } else {
+ //for the last segment in the list, use the first uncleanable offset.
+ return firstUncleanableOffset - 1;
+ }
+ }
+
+ /**
+ * Build a map of key_hash => offset for the keys in the cleanable dirty portion of the log to use in cleaning.
+ *
+ * @param log The log to use
+ * @param start The offset at which dirty messages begin
+ * @param end The ending offset for the map that is being built
+ * @param map The map in which to store the mappings
+ * @param stats Collector for cleaning statistics
+ */
+ public void buildOffsetMap(UnifiedLog log,
+ long start,
+ long end,
+ OffsetMap map,
+ CleanerStats stats) throws IOException, DigestException {
+ map.clear();
+ List dirty = log.logSegments(start, end);
+ List nextSegmentStartOffsets = new ArrayList<>();
+ if (!dirty.isEmpty()) {
+ for (int i = 1; i < dirty.size(); i++) {
+ nextSegmentStartOffsets.add(dirty.get(i).baseOffset());
+ }
+ nextSegmentStartOffsets.add(end);
+ }
+ logger.info("Building offset map for log {} for {} segments in offset range [{}, {}).", log.name(), dirty.size(), start, end);
+
+ CleanedTransactionMetadata transactionMetadata = new CleanedTransactionMetadata();
+ List abortedTransactions = log.collectAbortedTransactions(start, end);
+ transactionMetadata.addAbortedTransactions(abortedTransactions);
+
+ // Add all the cleanable dirty segments. We must take at least map.slots * load_factor,
+ // but we may be able to fit more (if there is lots of duplication in the dirty section of the log)
+ boolean full = false;
+ for (int i = 0; i < dirty.size() && !full; i++) {
+ LogSegment segment = dirty.get(i);
+ long nextSegmentStartOffset = nextSegmentStartOffsets.get(i);
+
+ checkDone.accept(log.topicPartition());
+
+ full = buildOffsetMapForSegment(
+ log.topicPartition(),
+ segment,
+ map,
+ start,
+ nextSegmentStartOffset,
+ log.config().maxMessageSize(),
+ transactionMetadata,
+ stats
+ );
+ if (full) {
+ logger.debug("Offset map is full, {} segments fully mapped, segment with base offset {} is partially mapped",
+ dirty.indexOf(segment), segment.baseOffset());
+ }
+ }
+
+ logger.info("Offset map for log {} complete.", log.name());
+ }
+
+ /**
+ * Add the messages in the given segment to the offset map.
+ *
+ * @param topicPartition The topic and partition of the log segment to build offset
+ * @param segment The segment to index
+ * @param map The map in which to store the key=>offset mapping
+ * @param startOffset The offset at which dirty messages begin
+ * @param nextSegmentStartOffset The base offset for next segment when building current segment
+ * @param maxLogMessageSize The maximum size in bytes for record allowed
+ * @param transactionMetadata The state of ongoing transactions for the log between offset range to build
+ * @param stats Collector for cleaning statistics
+ *
+ * @return If the map was filled whilst loading from this segment
+ */
+ private boolean buildOffsetMapForSegment(TopicPartition topicPartition,
+ LogSegment segment,
+ OffsetMap map,
+ long startOffset,
+ long nextSegmentStartOffset,
+ int maxLogMessageSize,
+ CleanedTransactionMetadata transactionMetadata,
+ CleanerStats stats) throws IOException, DigestException {
+ int position = segment.offsetIndex().lookup(startOffset).position;
+ int maxDesiredMapSize = (int) (map.slots() * dupBufferLoadFactor);
+
+ while (position < segment.log().sizeInBytes()) {
+ checkDone.accept(topicPartition);
+ readBuffer.clear();
+ try {
+ segment.log().readInto(readBuffer, position);
+ } catch (Exception e) {
+ throw new KafkaException("Failed to read from segment " + segment + " of partition " + topicPartition +
+ " while loading offset map", e);
+ }
+ MemoryRecords records = MemoryRecords.readableRecords(readBuffer);
+ throttler.maybeThrottle(records.sizeInBytes());
+
+ int startPosition = position;
+ for (MutableRecordBatch batch : records.batches()) {
+ if (batch.isControlBatch()) {
+ transactionMetadata.onControlBatchRead(batch);
+ stats.indexMessagesRead(1);
+ } else {
+ boolean isAborted = transactionMetadata.onBatchRead(batch);
+ if (isAborted) {
+ // If the batch is aborted, do not bother populating the offset map.
+ // Note that abort markers are supported in v2 and above, which means count is defined.
+ stats.indexMessagesRead(batch.countOrNull());
+ } else {
+ try (CloseableIterator recordsIterator = batch.streamingIterator(decompressionBufferSupplier)) {
+ for (Record record : (Iterable) () -> recordsIterator) {
+ if (record.hasKey() && record.offset() >= startOffset) {
+ if (map.size() < maxDesiredMapSize) {
+ map.put(record.key(), record.offset());
+ } else {
+ return true;
+ }
+ }
+ stats.indexMessagesRead(1);
+ }
+ }
+ }
+ }
+
+ if (batch.lastOffset() >= startOffset)
+ map.updateLatestOffset(batch.lastOffset());
+ }
+ int bytesRead = records.validBytes();
+ position += bytesRead;
+ stats.indexBytesRead(bytesRead);
+
+ // if we didn't read even one complete message, our read buffer may be too small
+ if (position == startPosition)
+ growBuffersOrFail(segment.log(), position, maxLogMessageSize, records);
+ }
+
+ // In the case of offsets gap, fast forward to latest expected offset in this segment.
+ map.updateLatestOffset(nextSegmentStartOffset - 1L);
+
+ restoreBuffers();
+
+ return false;
+ }
+}
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java
index 4ae7779f5e84a..b3f814628640a 100644
--- a/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerConfig.java
@@ -16,8 +16,10 @@
*/
package org.apache.kafka.storage.internals.log;
+import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.server.config.ServerConfigs;
import org.apache.kafka.server.config.ServerTopicConfigSynonyms;
import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM;
@@ -129,6 +131,22 @@ public CleanerConfig(int numThreads,
this.enableCleaner = enableCleaner;
}
+ /**
+ * Create an instance of this class.
+ *
+ * @param config the {@link AbstractConfig} instance for initialize all variables
+ */
+ public CleanerConfig(AbstractConfig config) {
+ this.numThreads = config.getInt(LOG_CLEANER_THREADS_PROP);
+ this.dedupeBufferSize = config.getLong(LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP);
+ this.dedupeBufferLoadFactor = config.getDouble(LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP);
+ this.ioBufferSize = config.getInt(LOG_CLEANER_IO_BUFFER_SIZE_PROP);
+ this.maxMessageSize = config.getInt(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG);
+ this.maxIoBytesPerSecond = config.getDouble(LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP);
+ this.backoffMs = config.getLong(LOG_CLEANER_BACKOFF_MS_PROP);
+ this.enableCleaner = config.getBoolean(LOG_CLEANER_ENABLE_PROP);
+ }
+
public String hashAlgorithm() {
return HASH_ALGORITHM;
}
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerStats.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerStats.java
new file mode 100644
index 0000000000000..3dff18b930b76
--- /dev/null
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerStats.java
@@ -0,0 +1,137 @@
+/*
+ * 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 org.apache.kafka.storage.internals.log;
+
+import org.apache.kafka.common.utils.Time;
+
+/**
+ * A simple struct for collecting stats about log cleaning.
+ */
+public class CleanerStats {
+ long bytesRead = 0L;
+ long bytesWritten = 0L;
+ long mapBytesRead = 0L;
+ long messagesRead = 0L;
+ long invalidMessagesRead = 0L;
+ long messagesWritten = 0L;
+ double bufferUtilization = 0.0d;
+
+ private long mapCompleteTime = -1L;
+ private long endTime = -1L;
+ private long mapMessagesRead = 0L;
+
+ private final Time time;
+ private final long startTime;
+
+ public CleanerStats(Time time) {
+ this.time = time;
+ startTime = time.milliseconds();
+ }
+
+ public void readMessages(int messagesRead, int bytesRead) {
+ this.messagesRead += messagesRead;
+ this.bytesRead += bytesRead;
+ }
+
+ public void invalidMessage() {
+ invalidMessagesRead += 1;
+ }
+
+ public void recopyMessages(int messagesWritten, int bytesWritten) {
+ this.messagesWritten += messagesWritten;
+ this.bytesWritten += bytesWritten;
+ }
+
+ public void indexMessagesRead(int size) {
+ mapMessagesRead += size;
+ }
+
+ public void indexBytesRead(int size) {
+ mapBytesRead += size;
+ }
+
+ public void indexDone() {
+ mapCompleteTime = time.milliseconds();
+ }
+
+ public void allDone() {
+ endTime = time.milliseconds();
+ }
+
+ public double elapsedSecs() {
+ return (endTime - startTime) / 1000.0;
+ }
+
+ public double elapsedIndexSecs() {
+ return (mapCompleteTime - startTime) / 1000.0;
+ }
+
+ // Only for testing
+ public long startTime() {
+ return startTime;
+ }
+
+ // Only for testing
+ public long endTime() {
+ return endTime;
+ }
+
+ // Only for testing
+ public void setEndTime(long endTime) {
+ this.endTime = endTime;
+ }
+
+ // Only for testing
+ public long bytesRead() {
+ return bytesRead;
+ }
+
+ // Only for testing
+ public long bytesWritten() {
+ return bytesWritten;
+ }
+
+ // Only for testing
+ public long mapMessagesRead() {
+ return mapMessagesRead;
+ }
+
+ // Only for testing
+ public long messagesRead() {
+ return messagesRead;
+ }
+
+ // Only for testing
+ public long invalidMessagesRead() {
+ return invalidMessagesRead;
+ }
+
+ // Only for testing
+ public long messagesWritten() {
+ return messagesWritten;
+ }
+
+ // Only for testing
+ public double bufferUtilization() {
+ return bufferUtilization;
+ }
+
+ // Only for testing
+ public void setBufferUtilization(double bufferUtilization) {
+ this.bufferUtilization = bufferUtilization;
+ }
+}
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java
new file mode 100644
index 0000000000000..2c0086c4700a1
--- /dev/null
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java
@@ -0,0 +1,650 @@
+/*
+ * 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 org.apache.kafka.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.AbstractConfig;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.errors.KafkaStorageException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.config.BrokerReconfigurable;
+import org.apache.kafka.server.config.ServerConfigs;
+import org.apache.kafka.server.metrics.KafkaMetricsGroup;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.apache.kafka.storage.internals.utils.Throttler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.security.DigestException;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The cleaner is responsible for removing obsolete records from logs which have the "compact" retention strategy.
+ * A message with key K and offset O is obsolete if there exists a message with key K and offset O' such that O < O'.
+ *
+ * Each log can be thought of being split into two sections of segments: a "clean" section which has previously been cleaned followed by a
+ * "dirty" section that has not yet been cleaned. The dirty section is further divided into the "cleanable" section followed by an "uncleanable" section.
+ * The uncleanable section is excluded from cleaning. The active log segment is always uncleanable. If there is a
+ * compaction lag time set, segments whose largest message timestamp is within the compaction lag time of the cleaning operation are also uncleanable.
+ *
+ * The cleaning is carried out by a pool of background threads. Each thread chooses the dirtiest log that has the "compact" retention policy
+ * and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log.
+ *
+ * To clean a log the cleaner first builds a mapping of key=>last_offset for the dirty section of the log. See {@link OffsetMap} for details of
+ * the implementation of the mapping.
+ *
+ * Once the key=>last_offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a
+ * higher offset than what is found in the segment (i.e. messages with a key that appears in the dirty section of the log).
+ *
+ * To avoid segments shrinking to very small sizes with repeated cleanings we implement a rule by which if we will merge successive segments when
+ * doing a cleaning if their log and index size are less than the maximum log and index size prior to the clean beginning.
+ *
+ * Cleaned segments are swapped into the log as they become available.
+ *
+ * One nuance that the cleaner must handle is log truncation. If a log is truncated while it is being cleaned the cleaning of that log is aborted.
+ *
+ * Messages with null payload are treated as deletes for the purpose of log compaction. This means that they receive special treatment by the cleaner.
+ * The cleaner will only retain delete records for a period of time to avoid accumulating space indefinitely. This period of time is configurable on a per-topic
+ * basis and is measured from the time the segment enters the clean portion of the log (at which point any prior message with that key has been removed).
+ * Delete markers in the clean section of the log that are older than this time will not be retained when log segments are being recopied as part of cleaning.
+ * This time is tracked by setting the base timestamp of a record batch with delete markers when the batch is recopied in the first cleaning that encounters
+ * it. The relative timestamps of the records in the batch are also modified when recopied in this cleaning according to the new base timestamp of the batch.
+ *
+ * Note that cleaning is more complicated with the idempotent/transactional producer capabilities. The following
+ * are the key points:
+ *
+ *
+ * - In order to maintain sequence number continuity for active producers, we always retain the last batch
+ * from each producerId, even if all the records from the batch have been removed. The batch will be removed
+ * once the producer either writes a new batch or is expired due to inactivity.
+ * - We do not clean beyond the last stable offset. This ensures that all records observed by the cleaner have
+ * been decided (i.e. committed or aborted). In particular, this allows us to use the transaction index to
+ * collect the aborted transactions ahead of time.
+ * - Records from aborted transactions are removed by the cleaner immediately without regard to record keys.
+ * - Transaction markers are retained until all record batches from the same transaction have been removed and
+ * a sufficient amount of time has passed to reasonably ensure that an active consumer wouldn't consume any
+ * data from the transaction prior to reaching the offset of the marker. This follows the same logic used for
+ * tombstone deletion.
+ *
+ */
+public class LogCleaner implements BrokerReconfigurable {
+ private static final Logger LOG = LoggerFactory.getLogger(LogCleaner.class);
+
+ public static final Set RECONFIGURABLE_CONFIGS = Set.of(
+ CleanerConfig.LOG_CLEANER_THREADS_PROP,
+ CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP,
+ CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP,
+ CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP,
+ ServerConfigs.MESSAGE_MAX_BYTES_CONFIG,
+ CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP,
+ CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP
+ );
+
+ // Visible for test
+ public static final String MAX_BUFFER_UTILIZATION_PERCENT_METRIC_NAME = "max-buffer-utilization-percent";
+ public static final String MAX_CLEAN_TIME_METRIC_NAME = "max-clean-time-secs";
+ public static final String MAX_COMPACTION_DELAY_METRICS_NAME = "max-compaction-delay-secs";
+
+ private static final String CLEANER_RECOPY_PERCENT_METRIC_NAME = "cleaner-recopy-percent";
+ private static final String DEAD_THREAD_COUNT_METRIC_NAME = "DeadThreadCount";
+
+ // Visible for test
+ public static final Set METRIC_NAMES = Set.of(
+ MAX_BUFFER_UTILIZATION_PERCENT_METRIC_NAME,
+ CLEANER_RECOPY_PERCENT_METRIC_NAME,
+ MAX_CLEAN_TIME_METRIC_NAME,
+ MAX_COMPACTION_DELAY_METRICS_NAME,
+ DEAD_THREAD_COUNT_METRIC_NAME
+ );
+
+ // For compatibility, metrics are defined to be under `kafka.log.LogCleaner` class
+ private final KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.log", "LogCleaner");
+
+ /**
+ * For managing the state of partitions being cleaned.
+ */
+ private final LogCleanerManager cleanerManager;
+
+ /**
+ * A throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate.
+ */
+ private final Throttler throttler;
+ private final ConcurrentMap logs;
+ private final LogDirFailureChannel logDirFailureChannel;
+ private final Time time;
+ private final List cleaners = new ArrayList<>();
+
+ /**
+ * Log cleaner configuration which may be dynamically updated.
+ */
+ private volatile CleanerConfig config;
+
+ /**
+ * @param initialConfig Initial configuration parameters for the cleaner. Actual config may be dynamically updated.
+ * @param logDirs The directories where offset checkpoints reside
+ * @param logs The map of logs
+ * @param logDirFailureChannel The channel used to add offline log dirs that may be encountered when cleaning the log
+ * @param time A way to control the passage of time
+ */
+ @SuppressWarnings("this-escape")
+ public LogCleaner(CleanerConfig initialConfig,
+ List logDirs,
+ ConcurrentMap logs,
+ LogDirFailureChannel logDirFailureChannel,
+ Time time) {
+ config = initialConfig;
+ this.logs = logs;
+ this.logDirFailureChannel = logDirFailureChannel;
+ cleanerManager = new LogCleanerManager(logDirs, logs, this.logDirFailureChannel);
+ this.time = time;
+ throttler = new Throttler(config.maxIoBytesPerSecond, 300, "cleaner-io", "bytes", this.time);
+
+ registerMetrics();
+ }
+
+ private void registerMetrics() {
+ /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */
+ metricsGroup.newGauge(MAX_BUFFER_UTILIZATION_PERCENT_METRIC_NAME,
+ () -> (int) (maxOverCleanerThreads(t -> t.lastStats.bufferUtilization) * 100));
+
+ /* a metric to track the recopy rate of each thread's last cleaning */
+ metricsGroup.newGauge(CLEANER_RECOPY_PERCENT_METRIC_NAME, () -> {
+ List stats = cleaners.stream().map(t -> t.lastStats).toList();
+ double recopyRate = (double) stats.stream().mapToLong(stat -> stat.bytesWritten).sum() /
+ Math.max(stats.stream().mapToLong(stat -> stat.bytesRead).sum(), 1);
+ return (int) (100 * recopyRate);
+ });
+
+ /* a metric to track the maximum cleaning time for the last cleaning from each thread */
+ metricsGroup.newGauge(MAX_CLEAN_TIME_METRIC_NAME, () -> (int) maxOverCleanerThreads(t -> t.lastStats.elapsedSecs()));
+
+ // a metric to track delay between the time when a log is required to be compacted
+ // as determined by max compaction lag and the time of last cleaner run.
+ metricsGroup.newGauge(MAX_COMPACTION_DELAY_METRICS_NAME,
+ () -> (int) (maxOverCleanerThreads(t -> (double) t.lastPreCleanStats.maxCompactionDelayMs()) / 1000));
+
+ metricsGroup.newGauge(DEAD_THREAD_COUNT_METRIC_NAME, this::deadThreadCount);
+ }
+
+ /**
+ * Start the background cleaner threads.
+ */
+ public void startup() {
+ LOG.info("Starting the log cleaner");
+ IntStream.range(0, config.numThreads).forEach(i -> {
+ try {
+ CleanerThread cleaner = new CleanerThread(i);
+ cleaners.add(cleaner);
+ cleaner.start();
+ } catch (NoSuchAlgorithmException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ /**
+ * Stop the background cleaner threads.
+ */
+ private void shutdownCleaners() {
+ LOG.info("Shutting down the log cleaner.");
+ cleaners.forEach(thread -> {
+ try {
+ thread.shutdown();
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ cleaners.clear();
+ }
+
+ /**
+ * Stop the background cleaner threads.
+ */
+ public void shutdown() {
+ try {
+ shutdownCleaners();
+ } finally {
+ removeMetrics();
+ }
+ }
+
+ /**
+ * Remove metrics.
+ */
+ public void removeMetrics() {
+ METRIC_NAMES.forEach(metricsGroup::removeMetric);
+ cleanerManager.removeMetrics();
+ }
+
+ /**
+ * @return A set of configs that is reconfigurable in LogCleaner
+ */
+ @Override
+ public Set reconfigurableConfigs() {
+ return RECONFIGURABLE_CONFIGS;
+ }
+
+ /**
+ * Validate the new cleaner threads num is reasonable.
+ *
+ * @param newConfig A submitted new AbstractConfig instance that contains new cleaner config
+ */
+ @Override
+ public void validateReconfiguration(AbstractConfig newConfig) {
+ int numThreads = new CleanerConfig(newConfig).numThreads;
+ int currentThreads = config.numThreads;
+ if (numThreads < 1)
+ throw new ConfigException("Log cleaner threads should be at least 1");
+ if (numThreads < currentThreads / 2)
+ throw new ConfigException("Log cleaner threads cannot be reduced to less than half the current value " + currentThreads);
+ if (numThreads > currentThreads * 2)
+ throw new ConfigException("Log cleaner threads cannot be increased to more than double the current value " + currentThreads);
+
+ }
+
+ /**
+ * Reconfigure log clean config. The will:
+ *
+ * - update desiredRatePerSec in Throttler with logCleanerIoMaxBytesPerSecond, if necessary
+ * - stop current log cleaners and create new ones.
+ *
+ * That ensures that if any of the cleaners had failed, new cleaners are created to match the new config.
+ *
+ * @param oldConfig the old log cleaner config
+ * @param newConfig the new log cleaner config reconfigured
+ */
+ @Override
+ public void reconfigure(AbstractConfig oldConfig, AbstractConfig newConfig) {
+ config = new CleanerConfig(newConfig);
+
+ double maxIoBytesPerSecond = config.maxIoBytesPerSecond;
+ if (maxIoBytesPerSecond != oldConfig.getDouble(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP)) {
+ LOG.info("Updating logCleanerIoMaxBytesPerSecond: {}", maxIoBytesPerSecond);
+ throttler.updateDesiredRatePerSec(maxIoBytesPerSecond);
+ }
+ // call shutdownCleaners() instead of shutdown to avoid unnecessary deletion of metrics
+ shutdownCleaners();
+ startup();
+ }
+
+ /**
+ * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of
+ * the partition is aborted.
+ *
+ * @param topicPartition The topic and partition to abort cleaning
+ */
+ public void abortCleaning(TopicPartition topicPartition) {
+ cleanerManager.abortCleaning(topicPartition);
+ }
+
+ /**
+ * Update checkpoint file to remove partitions if necessary.
+ *
+ * @param dataDir The data dir to be updated if necessary
+ * @param partitionToRemove The topicPartition to be removed
+ */
+ public void updateCheckpoints(File dataDir, Optional partitionToRemove) {
+ cleanerManager.updateCheckpoints(dataDir, Optional.empty(), partitionToRemove);
+ }
+
+ /**
+ * Alter the checkpoint directory for the `topicPartition`, to remove the data in `sourceLogDir`, and add the data in `destLogDir`.
+ * Generally occurs when the disk balance ends and replaces the previous file with the future file.
+ *
+ * @param topicPartition The topic and partition to alter checkpoint
+ * @param sourceLogDir The source log dir to remove checkpoint
+ * @param destLogDir The dest log dir to remove checkpoint
+ */
+ public void alterCheckpointDir(TopicPartition topicPartition, File sourceLogDir, File destLogDir) {
+ cleanerManager.alterCheckpointDir(topicPartition, sourceLogDir, destLogDir);
+ }
+
+ /**
+ * Stop cleaning logs in the provided directory when handling log dir failure.
+ *
+ * @param dir the absolute path of the log dir
+ */
+ public void handleLogDirFailure(String dir) {
+ cleanerManager.handleLogDirFailure(dir);
+ }
+
+ /**
+ * Truncate cleaner offset checkpoint for the given partition if its checkpoint offset is larger than the given offset.
+ *
+ * @param dataDir The data dir to be truncated if necessary
+ * @param topicPartition The topic and partition to truncate checkpoint offset
+ * @param offset The given offset to be compared
+ */
+ public void maybeTruncateCheckpoint(File dataDir, TopicPartition topicPartition, long offset) {
+ cleanerManager.maybeTruncateCheckpoint(dataDir, topicPartition, offset);
+ }
+
+ /**
+ * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition.
+ * This call blocks until the cleaning of the partition is aborted and paused.
+ *
+ * @param topicPartition The topic and partition to abort and pause cleaning
+ */
+ public void abortAndPauseCleaning(TopicPartition topicPartition) {
+ cleanerManager.abortAndPauseCleaning(topicPartition);
+ }
+
+ /**
+ * Resume the cleaning of paused partitions.
+ *
+ * @param topicPartitions The collection of topicPartitions to be resumed cleaning
+ */
+ public void resumeCleaning(Set topicPartitions) {
+ cleanerManager.resumeCleaning(topicPartitions);
+ }
+
+ /**
+ * For testing, a way to know when work has completed. This method waits until the
+ * cleaner has processed up to the given offset on the specified topic/partition.
+ *
+ * @param topicPartition The topic and partition to be cleaned
+ * @param offset The first dirty offset that the cleaner doesn't have to clean
+ * @param maxWaitMs The maximum time in ms to wait for cleaner
+ *
+ * @return A boolean indicating whether the work has completed before timeout
+ */
+ public boolean awaitCleaned(TopicPartition topicPartition, long offset, long maxWaitMs) throws InterruptedException {
+ long remainingWaitMs = maxWaitMs;
+ while (!isCleaned(topicPartition, offset) && remainingWaitMs > 0) {
+ long sleepTime = Math.min(100, remainingWaitMs);
+ Thread.sleep(sleepTime);
+ remainingWaitMs -= sleepTime;
+ }
+ return isCleaned(topicPartition, offset);
+ }
+
+ private boolean isCleaned(TopicPartition topicPartition, long offset) {
+ return Optional.ofNullable(cleanerManager.allCleanerCheckpoints().get(topicPartition))
+ .map(checkpoint -> checkpoint >= offset)
+ .orElse(false);
+ }
+
+ /**
+ * To prevent race between retention and compaction,
+ * retention threads need to make this call to obtain:
+ *
+ * @return A map of log partitions that retention threads can safely work on
+ */
+ public Map pauseCleaningForNonCompactedPartitions() {
+ return cleanerManager.pauseCleaningForNonCompactedPartitions().stream()
+ .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+ }
+
+ /**
+ * @param f to compute the result
+ * @return the max value or 0 if there is no cleaner
+ */
+ public double maxOverCleanerThreads(Function f) {
+ return cleaners.stream()
+ .mapToDouble(f::apply)
+ .max()
+ .orElse(0.0d);
+ }
+
+ public int deadThreadCount() {
+ return (int) cleaners.stream()
+ .filter(ShutdownableThread::isThreadFailed)
+ .count();
+ }
+
+ // Only for testing
+ public LogCleanerManager cleanerManager() {
+ return cleanerManager;
+ }
+
+ // Only for testing
+ public Throttler throttler() {
+ return throttler;
+ }
+
+ // Only for testing
+ public ConcurrentMap logs() {
+ return logs;
+ }
+
+ // Only for testing
+ public List cleaners() {
+ return cleaners;
+ }
+
+ // Only for testing
+ public KafkaMetricsGroup metricsGroup() {
+ return metricsGroup;
+ }
+
+ // Only for testing
+ public CleanerConfig currentConfig() {
+ return config;
+ }
+
+ // Only for testing
+ public int cleanerCount() {
+ return cleaners.size();
+ }
+
+ /**
+ * The cleaner threads do the actual log cleaning. Each thread processes does its cleaning repeatedly by
+ * choosing the dirtiest log, cleaning it, and then swapping in the cleaned segments.
+ */
+ public class CleanerThread extends ShutdownableThread {
+ private final Logger logger = new LogContext(logPrefix).logger(CleanerThread.class);
+
+ private final Cleaner cleaner;
+
+ private volatile CleanerStats lastStats = new CleanerStats(Time.SYSTEM);
+ private volatile PreCleanStats lastPreCleanStats = new PreCleanStats();
+
+ @SuppressWarnings("this-escape")
+ public CleanerThread(int threadId) throws NoSuchAlgorithmException {
+ super("kafka-log-cleaner-thread-" + threadId, false);
+
+ cleaner = new Cleaner(
+ threadId,
+ new SkimpyOffsetMap((int) Math.min(config.dedupeBufferSize / config.numThreads, Integer.MAX_VALUE), config.hashAlgorithm()),
+ config.ioBufferSize / config.numThreads / 2,
+ config.maxMessageSize,
+ config.dedupeBufferLoadFactor,
+ throttler,
+ time,
+ this::checkDone
+ );
+
+ if (config.dedupeBufferSize / config.numThreads > Integer.MAX_VALUE) {
+ logger.warn("Cannot use more than 2G of cleaner buffer space per cleaner thread, ignoring excess buffer space...");
+ }
+ }
+
+ /**
+ * Check if the cleaning for a partition is aborted. If so, throw an exception.
+ *
+ * @param topicPartition The topic and partition to check
+ */
+ private void checkDone(TopicPartition topicPartition) {
+ if (!isRunning()) {
+ throw new ThreadShutdownException();
+ }
+
+ cleanerManager.checkCleaningAborted(topicPartition);
+ }
+
+ /**
+ * The main loop for the cleaner thread.
+ * Clean a log if there is a dirty log available, otherwise sleep for a bit.
+ */
+ @Override
+ public void doWork() {
+ boolean cleaned = tryCleanFilthiestLog();
+ if (!cleaned) {
+ try {
+ pause(config.backoffMs, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ cleanerManager.maintainUncleanablePartitions();
+ }
+
+ public CleanerStats lastStats() {
+ return lastStats;
+ }
+
+ public void setLastStats(CleanerStats lastStats) {
+ this.lastStats = lastStats;
+ }
+
+ public PreCleanStats lastPreCleanStats() {
+ return lastPreCleanStats;
+ }
+
+ /**
+ * Cleans a log if there is a dirty log available.
+ *
+ * @return whether a log was cleaned
+ */
+ private boolean tryCleanFilthiestLog() {
+ try {
+ return cleanFilthiestLog();
+ } catch (LogCleaningException e) {
+ logger.warn("Unexpected exception thrown when cleaning log {}. Marking its partition ({}) as uncleanable", e.log, e.log.topicPartition(), e);
+ cleanerManager.markPartitionUncleanable(e.log.parentDir(), e.log.topicPartition());
+
+ return false;
+ }
+ }
+
+ private boolean cleanFilthiestLog() throws LogCleaningException {
+ PreCleanStats preCleanStats = new PreCleanStats();
+ Optional ltc = cleanerManager.grabFilthiestCompactedLog(time, preCleanStats);
+ boolean cleaned;
+
+ if (ltc.isEmpty()) {
+ cleaned = false;
+ } else {
+ // there's a log, clean it
+ this.lastPreCleanStats = preCleanStats;
+ LogToClean cleanable = null;
+ try {
+ cleanable = ltc.get();
+ cleanLog(cleanable);
+ cleaned = true;
+ } catch (ThreadShutdownException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new LogCleaningException(cleanable != null ? cleanable.log() : null, e.getMessage(), e);
+ }
+ }
+
+ Map deletable = cleanerManager.deletableLogs();
+ try {
+ deletable.forEach((topicPartition, log) -> {
+ try {
+ log.deleteOldSegments();
+ } catch (ThreadShutdownException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new LogCleaningException(log, e.getMessage(), e);
+ }
+ });
+ } finally {
+ cleanerManager.doneDeleting(deletable.keySet().stream().toList());
+ }
+
+ return cleaned;
+ }
+
+ private void cleanLog(LogToClean cleanable) throws DigestException {
+ long startOffset = cleanable.firstDirtyOffset();
+ long endOffset = startOffset;
+ try {
+ Map.Entry entry = cleaner.clean(cleanable);
+ endOffset = entry.getKey();
+ recordStats(cleaner.id(), cleanable.log().name(), startOffset, endOffset, entry.getValue());
+ } catch (LogCleaningAbortedException ignored) {
+ // task can be aborted, let it go.
+ } catch (KafkaStorageException ignored) {
+ // partition is already offline. let it go.
+ } catch (IOException e) {
+ String logDirectory = cleanable.log().parentDir();
+ String msg = String.format("Failed to clean up log for %s in dir %s due to IOException", cleanable.topicPartition(), logDirectory);
+ logDirFailureChannel.maybeAddOfflineLogDir(logDirectory, msg, e);
+ } finally {
+ cleanerManager.doneCleaning(cleanable.topicPartition(), cleanable.log().parentDirFile(), endOffset);
+ }
+ }
+
+ /**
+ * Log out statistics on a single run of the cleaner.
+ *
+ * @param id The cleaner thread id
+ * @param name The cleaned log name
+ * @param from The cleaned offset that is the first dirty offset to begin
+ * @param to The cleaned offset that is the first not cleaned offset to end
+ * @param stats The statistics for this round of cleaning
+ */
+ private void recordStats(int id, String name, long from, long to, CleanerStats stats) {
+ this.lastStats = stats;
+ String message = String.format("%n\tLog cleaner thread %d cleaned log %s (dirty section = [%d, %d])%n", id, name, from, to) +
+ String.format("\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n", mb(stats.bytesRead),
+ stats.elapsedSecs(),
+ mb(stats.bytesRead / stats.elapsedSecs())) +
+ String.format("\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n", mb(stats.mapBytesRead),
+ stats.elapsedIndexSecs(),
+ mb(stats.mapBytesRead) / stats.elapsedIndexSecs(),
+ 100 * stats.elapsedIndexSecs() / stats.elapsedSecs()) +
+ String.format("\tBuffer utilization: %.1f%%%n", 100 * stats.bufferUtilization) +
+ String.format("\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n", mb(stats.bytesRead),
+ stats.elapsedSecs() - stats.elapsedIndexSecs(),
+ mb(stats.bytesRead) / (stats.elapsedSecs() - stats.elapsedIndexSecs()), 100 * (stats.elapsedSecs() - stats.elapsedIndexSecs()) / stats.elapsedSecs()) +
+ String.format("\tStart size: %,.1f MB (%,d messages)%n", mb(stats.bytesRead), stats.messagesRead) +
+ String.format("\tEnd size: %,.1f MB (%,d messages)%n", mb(stats.bytesWritten), stats.messagesWritten) +
+ String.format("\t%.1f%% size reduction (%.1f%% fewer messages)%n", 100.0 * (1.0 - Long.valueOf(stats.bytesWritten).doubleValue() / stats.bytesRead),
+ 100.0 * (1.0 - Long.valueOf(stats.messagesWritten).doubleValue() / stats.messagesRead));
+ logger.info(message);
+ if (lastPreCleanStats.delayedPartitions() > 0) {
+ logger.info("\tCleanable partitions: {}, Delayed partitions: {}, max delay: {}",
+ lastPreCleanStats.cleanablePartitions(), lastPreCleanStats.delayedPartitions(), lastPreCleanStats.maxCompactionDelayMs());
+ }
+ if (stats.invalidMessagesRead > 0) {
+ logger.warn("\tFound {} invalid messages during compaction.", stats.invalidMessagesRead);
+ }
+ }
+
+ private double mb(double bytes) {
+ return bytes / (1024 * 1024);
+ }
+ }
+}
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleanerManager.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleanerManager.java
index dfee694afefb4..0b8c2b398d039 100644
--- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleanerManager.java
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleanerManager.java
@@ -346,7 +346,7 @@ public Map deletableLogs() {
public void abortCleaning(TopicPartition topicPartition) {
inLock(lock, () -> {
abortAndPauseCleaning(topicPartition);
- resumeCleaning(List.of(topicPartition));
+ resumeCleaning(Set.of(topicPartition));
return null;
});
}
@@ -396,7 +396,7 @@ public void abortAndPauseCleaning(TopicPartition topicPartition) {
* Resume the cleaning of paused partitions.
* Each call of this function will undo one pause.
*/
- public void resumeCleaning(List topicPartitions) {
+ public void resumeCleaning(Set topicPartitions) {
inLock(lock, () -> {
topicPartitions.forEach(topicPartition -> {
LogCleaningState state = inProgress.get(topicPartition);
diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java
index 2e89cdee21d40..21c92cd84dff4 100644
--- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java
@@ -27,7 +27,6 @@
import org.apache.kafka.common.errors.InvalidConfigurationException;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.LegacyRecord;
-import org.apache.kafka.common.record.Records;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.ConfigUtils;
import org.apache.kafka.common.utils.Utils;
@@ -124,7 +123,6 @@ public Optional serverConfigName(String configName) {
// Visible for testing
public static final String SERVER_DEFAULT_HEADER_NAME = "Server Default Property";
- public static final int DEFAULT_MAX_MESSAGE_BYTES = 1024 * 1024 + Records.LOG_OVERHEAD;
public static final int DEFAULT_SEGMENT_BYTES = 1024 * 1024 * 1024;
public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L;
public static final long DEFAULT_SEGMENT_JITTER_MS = 0;
@@ -134,7 +132,6 @@ public Optional serverConfigName(String configName) {
public static final long DEFAULT_MAX_COMPACTION_LAG_MS = Long.MAX_VALUE;
public static final double DEFAULT_MIN_CLEANABLE_DIRTY_RATIO = 0.5;
public static final boolean DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE = false;
- public static final String DEFAULT_COMPRESSION_TYPE = BrokerCompressionType.PRODUCER.name;
public static final boolean DEFAULT_PREALLOCATE = false;
public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false;
@@ -210,7 +207,7 @@ public Optional serverConfigName(String configName) {
// can be negative. See kafka.log.LogManager.cleanupExpiredSegments
.define(TopicConfig.RETENTION_MS_CONFIG, LONG, DEFAULT_RETENTION_MS, atLeast(-1), MEDIUM,
TopicConfig.RETENTION_MS_DOC)
- .define(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, INT, DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), MEDIUM,
+ .define(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, INT, ServerLogConfigs.MAX_MESSAGE_BYTES_DEFAULT, atLeast(0), MEDIUM,
TopicConfig.MAX_MESSAGE_BYTES_DOC)
.define(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, INT, ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_DEFAULT, atLeast(0), MEDIUM,
TopicConfig.INDEX_INTERVAL_BYTES_DOC)
@@ -230,7 +227,7 @@ public Optional serverConfigName(String configName) {
MEDIUM, TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_DOC)
.define(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, INT, ServerLogConfigs.MIN_IN_SYNC_REPLICAS_DEFAULT, atLeast(1), MEDIUM,
TopicConfig.MIN_IN_SYNC_REPLICAS_DOC)
- .define(TopicConfig.COMPRESSION_TYPE_CONFIG, STRING, DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names().toArray(new String[0])),
+ .define(TopicConfig.COMPRESSION_TYPE_CONFIG, STRING, ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, in(BrokerCompressionType.names().toArray(new String[0])),
MEDIUM, TopicConfig.COMPRESSION_TYPE_DOC)
.define(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG, INT, CompressionType.GZIP.defaultLevel(),
CompressionType.GZIP.levelValidator(), MEDIUM, TopicConfig.COMPRESSION_GZIP_LEVEL_DOC)
diff --git a/tests/kafkatest/services/kafka/templates/log4j2.yaml b/tests/kafkatest/services/kafka/templates/log4j2.yaml
index 22e3f118f680d..ff2c01e4c1be5 100644
--- a/tests/kafkatest/services/kafka/templates/log4j2.yaml
+++ b/tests/kafkatest/services/kafka/templates/log4j2.yaml
@@ -254,7 +254,21 @@ Configuration:
- ref: ControllerInfoAppender
- ref: ControllerDebugAppender
- - name: kafka.log.LogCleaner
+ - name: org.apache.kafka.storage.internals.log.LogCleaner
+ level: {{ log_level|default("DEBUG") }}
+ additivity: false
+ AppenderRef:
+ - ref: CleanerInfoAppender
+ - ref: CleanerDebugAppender
+
+ - name: org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread
+ level: {{ log_level|default("DEBUG") }}
+ additivity: false
+ AppenderRef:
+ - ref: CleanerInfoAppender
+ - ref: CleanerDebugAppender
+
+ - name: org.apache.kafka.storage.internals.log.Cleaner
level: {{ log_level|default("DEBUG") }}
additivity: false
AppenderRef: