From 11514a59fd1d62de9913b39e7bd91f25e63e0eb3 Mon Sep 17 00:00:00 2001 From: Dmitry Werner Date: Thu, 27 Mar 2025 12:33:44 +0500 Subject: [PATCH 1/5] KAFKA-14485: Move LogCleaner to storage module --- build.gradle | 1 + checkstyle/import-control-server-common.xml | 1 + .../transaction/TransactionCoordinator.scala | 13 +- .../src/main/scala/kafka/log/LogCleaner.scala | 1307 ----------------- .../src/main/scala/kafka/log/LogManager.scala | 20 +- .../server/AutoTopicCreationManager.scala | 6 +- .../kafka/server/DynamicBrokerConfig.scala | 8 +- .../main/scala/kafka/server/KafkaConfig.scala | 3 +- .../scala/kafka/server/ReplicaManager.scala | 4 +- .../metadata/BrokerMetadataPublisher.scala | 4 +- .../api/GroupCoordinatorIntegrationTest.scala | 2 +- .../api/PlaintextAdminIntegrationTest.scala | 12 +- .../kafka/api/PlaintextProducerSendTest.scala | 4 +- .../api/SaslSslAdminIntegrationTest.scala | 5 +- .../TransactionStateManagerTest.scala | 3 +- .../AbstractLogCleanerIntegrationTest.scala | 10 +- .../kafka/log/LogCleanerIntegrationTest.scala | 8 +- .../log/LogCleanerLagIntegrationTest.scala | 2 +- ...gCleanerParameterizedIntegrationTest.scala | 8 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 289 ++-- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 18 +- .../server/DynamicBrokerConfigTest.scala | 2 +- .../kafka/server/ReplicaManagerTest.scala | 6 +- .../server/config/BrokerReconfigurable.java | 12 +- .../kafka/server/config/ServerConfigs.java | 5 +- .../kafka/server/config/ServerLogConfigs.java | 5 + .../server/config/AbstractKafkaConfig.java | 2 - .../DynamicProducerStateManagerConfig.java | 9 +- .../log/CleanedTransactionMetadata.java | 160 ++ .../kafka/storage/internals/log/Cleaner.java | 762 ++++++++++ .../storage/internals/log/CleanerConfig.java | 18 + .../storage/internals/log/CleanerStats.java | 137 ++ .../storage/internals/log/LogCleaner.java | 652 ++++++++ .../storage/internals/log/LogConfig.java | 7 +- 35 files changed, 1972 insertions(+), 1535 deletions(-) delete mode 100644 core/src/main/scala/kafka/log/LogCleaner.scala rename {server => server-common}/src/main/java/org/apache/kafka/server/config/BrokerReconfigurable.java (89%) rename {server => server-common}/src/main/java/org/apache/kafka/server/config/ServerConfigs.java (97%) create mode 100644 storage/src/main/java/org/apache/kafka/storage/internals/log/CleanedTransactionMetadata.java create mode 100644 storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java create mode 100644 storage/src/main/java/org/apache/kafka/storage/internals/log/CleanerStats.java create mode 100644 storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java diff --git a/build.gradle b/build.gradle index 4753be6ccc6af..195a4ef0c2cc3 100644 --- a/build.gradle +++ b/build.gradle @@ -3737,6 +3737,7 @@ project(':connect:mirror') { testImplementation project(':core') testImplementation project(':test-common:test-common-runtime') testImplementation project(':server') + testImplementation project(':server-common') testImplementation project(':server-common').sourceSets.test.output diff --git a/checkstyle/import-control-server-common.xml b/checkstyle/import-control-server-common.xml index 20f6e32fc9a3b..9bd778453e857 100644 --- a/checkstyle/import-control-server-common.xml +++ b/checkstyle/import-control-server-common.xml @@ -130,6 +130,7 @@ + diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 697d6a3014459..ed8140e596f56 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{AddPartitionsToTxnResponse, TransactionResult} import org.apache.kafka.common.utils.{LogContext, ProducerIdAndEpoch, Time} -import org.apache.kafka.coordinator.transaction.ProducerIdManager +import org.apache.kafka.coordinator.transaction.{ProducerIdManager, TransactionLogConfig} import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.server.common.{RequestLocal, TransactionVersion} import org.apache.kafka.server.util.Scheduler @@ -46,13 +46,14 @@ object TransactionCoordinator { metadataCache: MetadataCache, time: Time): TransactionCoordinator = { + val transactionLogConfig = new TransactionLogConfig(config) val txnConfig = TransactionConfig(config.transactionStateManagerConfig.transactionalIdExpirationMs, config.transactionStateManagerConfig.transactionMaxTimeoutMs, - config.transactionLogConfig.transactionTopicPartitions, - config.transactionLogConfig.transactionTopicReplicationFactor, - config.transactionLogConfig.transactionTopicSegmentBytes, - config.transactionLogConfig.transactionLoadBufferSize, - config.transactionLogConfig.transactionTopicMinISR, + transactionLogConfig.transactionTopicPartitions, + transactionLogConfig.transactionTopicReplicationFactor, + transactionLogConfig.transactionTopicSegmentBytes, + transactionLogConfig.transactionLoadBufferSize, + transactionLogConfig.transactionTopicMinISR, config.transactionStateManagerConfig.transactionAbortTimedOutTransactionCleanupIntervalMs, config.transactionStateManagerConfig.transactionRemoveExpiredTransactionalIdCleanupIntervalMs, config.transactionStateManagerConfig.transaction2PCEnabled, 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 2bafe735283cf..a468bd58551ad 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -29,6 +29,7 @@ import kafka.utils.{CoreUtils, Logging, Pool} import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.utils.{Exit, KafkaThread, Time, Utils} import org.apache.kafka.common.errors.{InconsistentTopicIdException, KafkaStorageException, LogDirNotFoundException} +import org.apache.kafka.coordinator.transaction.TransactionLogConfig import scala.jdk.CollectionConverters._ import scala.collection._ @@ -41,7 +42,7 @@ 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 @@ -628,7 +629,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() } } @@ -893,7 +894,7 @@ class LogManager(logDirs: Seq[File], */ private def resumeCleaning(topicPartition: TopicPartition): Unit = { if (cleaner != null) { - cleaner.resumeCleaning(Seq(topicPartition)) + cleaner.resumeCleaning(util.List.of(topicPartition)) info(s"Cleaning for partition $topicPartition is resumed") } } @@ -1285,7 +1286,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) { @@ -1343,7 +1344,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) @@ -1384,7 +1385,7 @@ class LogManager(logDirs: Seq[File], val deletableLogs = { if (cleaner != null) { // prevent cleaner from working on same partitions when changing cleanup policy - cleaner.pauseCleaningForNonCompactedPartitions() + cleaner.pauseCleaningForNonCompactedPartitions().asScala.map(entry => (entry.getKey, entry.getValue)) } else { currentLogs.asScala.filter { case (_, log) => !log.config.compact @@ -1407,7 +1408,7 @@ class LogManager(logDirs: Seq[File], } } finally { if (cleaner != null) { - cleaner.resumeCleaning(deletableLogs.map(_._1)) + cleaner.resumeCleaning(deletableLogs.map(_._1).toList.asJava) } } @@ -1548,6 +1549,7 @@ object LogManager { val defaultLogConfig = new LogConfig(defaultProps) val cleanerConfig = LogCleaner.cleanerConfig(config) + val transactionLogConfig = new TransactionLogConfig(config) new LogManager(logDirs = config.logDirs.map(new File(_).getAbsoluteFile), initialOfflineDirs = initialOfflineDirs.map(new File(_).getAbsoluteFile), @@ -1560,8 +1562,8 @@ object LogManager { flushStartOffsetCheckpointMs = config.logFlushStartOffsetCheckpointIntervalMs, retentionCheckMs = config.logCleanupIntervalMs, maxTransactionTimeoutMs = config.transactionStateManagerConfig.transactionMaxTimeoutMs, - producerStateManagerConfig = new ProducerStateManagerConfig(config.transactionLogConfig.producerIdExpirationMs, config.transactionLogConfig.transactionPartitionVerificationEnable), - producerIdExpirationCheckIntervalMs = config.transactionLogConfig.producerIdExpirationCheckIntervalMs, + producerStateManagerConfig = new ProducerStateManagerConfig(transactionLogConfig.producerIdExpirationMs, transactionLogConfig.transactionPartitionVerificationEnable), + producerIdExpirationCheckIntervalMs = transactionLogConfig.producerIdExpirationCheckIntervalMs, scheduler = kafkaScheduler, brokerTopicStats = brokerTopicStats, logDirFailureChannel = logDirFailureChannel, diff --git a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala index 85308473bf6c7..a2c2bd4d80b35 100644 --- a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala +++ b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala @@ -32,6 +32,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{CreateTopicsRequest, RequestContext, RequestHeader} import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.coordinator.share.ShareCoordinator +import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import scala.collection.{Map, Seq, Set, mutable} @@ -189,10 +190,11 @@ class DefaultAutoTopicCreationManager( .setReplicationFactor(config.groupCoordinatorConfig.offsetsTopicReplicationFactor) .setConfigs(convertToTopicConfigCollections(groupCoordinator.groupMetadataTopicConfigs)) case TRANSACTION_STATE_TOPIC_NAME => + val transactionLogConfig = new TransactionLogConfig(config) new CreatableTopic() .setName(topic) - .setNumPartitions(config.transactionLogConfig.transactionTopicPartitions) - .setReplicationFactor(config.transactionLogConfig.transactionTopicReplicationFactor) + .setNumPartitions(transactionLogConfig.transactionTopicPartitions) + .setReplicationFactor(transactionLogConfig.transactionTopicReplicationFactor) .setConfigs(convertToTopicConfigCollections( txnCoordinator.transactionTopicConfigs)) case SHARE_GROUP_STATE_TOPIC_NAME => diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 29a3971d37f36..826710f05010e 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._ @@ -38,14 +38,14 @@ import org.apache.kafka.common.utils.{BufferSupplier, ConfigUtils, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.KafkaRaftClient -import org.apache.kafka.server.{ProcessRole, DynamicThreadPool} +import org.apache.kafka.server.{DynamicThreadPool, ProcessRole} import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.server.config.{DynamicProducerStateManagerConfig, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms} 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._ @@ -88,7 +88,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/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 2b8be8518b50e..87bfd9cd21f3c 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -37,7 +37,7 @@ import org.apache.kafka.coordinator.group.Group.GroupType import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig import org.apache.kafka.coordinator.group.{GroupConfig, GroupCoordinatorConfig} import org.apache.kafka.coordinator.share.ShareCoordinatorConfig -import org.apache.kafka.coordinator.transaction.{AddPartitionsToTxnConfig, TransactionLogConfig, TransactionStateManagerConfig} +import org.apache.kafka.coordinator.transaction.{AddPartitionsToTxnConfig, TransactionStateManagerConfig} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.security.authorizer.AuthorizerUtils @@ -204,7 +204,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) private val _shareCoordinatorConfig = new ShareCoordinatorConfig(this) def shareCoordinatorConfig: ShareCoordinatorConfig = _shareCoordinatorConfig - override val transactionLogConfig = new TransactionLogConfig(this) private val _transactionStateManagerConfig = new TransactionStateManagerConfig(this) private val _addPartitionsToTxnConfig = new AddPartitionsToTxnConfig(this) def transactionStateManagerConfig: TransactionStateManagerConfig = _transactionStateManagerConfig diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 9b5c01a85c9c5..ea79cd4c69ef3 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -47,6 +47,7 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.{Exit, Time, Utils} +import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.metadata.MetadataCache @@ -1038,9 +1039,10 @@ class ReplicaManager(val config: KafkaConfig, callback: ((Map[TopicPartition, Errors], Map[TopicPartition, VerificationGuard])) => Unit, transactionSupportedOperation: TransactionSupportedOperation ): Unit = { + val transactionLogConfig = new TransactionLogConfig(config) // Skip verification if the request is not transactional or transaction verification is disabled. if (transactionalId == null || - (!config.transactionLogConfig.transactionPartitionVerificationEnable && !transactionSupportedOperation.supportsEpochBump) + (!transactionLogConfig.transactionPartitionVerificationEnable && !transactionSupportedOperation.supportsEpochBump) || addPartitionsToTxnManager.isEmpty ) { callback((Map.empty[TopicPartition, Errors], Map.empty[TopicPartition, VerificationGuard])) diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index ead385e7447b2..de8f16e1e5808 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.internals.Topic import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.coordinator.share.ShareCoordinator +import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.image.loader.LoaderManifest import org.apache.kafka.image.publisher.MetadataPublisher import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} @@ -331,9 +332,10 @@ class BrokerMetadataPublisher( case t: Throwable => fatalFaultHandler.handleFault("Error starting GroupCoordinator", t) } try { + val transactionLogConfig = new TransactionLogConfig(config) // Start the transaction coordinator. txnCoordinator.startup(() => metadataCache.numPartitions( - Topic.TRANSACTION_STATE_TOPIC_NAME).orElse(config.transactionLogConfig.transactionTopicPartitions)) + Topic.TRANSACTION_STATE_TOPIC_NAME).orElse(transactionLogConfig.transactionTopicPartitions)) } catch { case t: Throwable => fatalFaultHandler.handleFault("Error starting TransactionCoordinator", t) } 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..095781993f4ad 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -28,7 +28,7 @@ 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} @@ -133,10 +133,10 @@ abstract class AbstractLogCleanerIntegrationTest { backoffMs, true) new LogCleaner(cleanerConfig, - logDirs = Array(logDir), - logs = logMap, - logDirFailureChannel = new LogDirFailureChannel(1), - time = time) + java.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/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..98280a01f0813 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) @@ -2010,10 +2011,10 @@ class LogCleanerTest extends Logging { 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) { + 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 816752b270665..d13bc6a993bac 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/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 5ae1262df40ab..a8e760c7b7a5b 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -2523,7 +2523,8 @@ class ReplicaManagerTest { val props = new Properties() props.put(TransactionLogConfig.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG, "true") config.dynamicConfig.updateBrokerConfig(config.brokerId, props) - TestUtils.waitUntilTrue(() => config.transactionLogConfig.transactionPartitionVerificationEnable, "Config did not dynamically update.") + val transactionLogConfig = new TransactionLogConfig(config) + TestUtils.waitUntilTrue(() => transactionLogConfig.transactionPartitionVerificationEnable, "Config did not dynamically update.") // Try to append more records. We don't need to send a request since the transaction is already ongoing. val moreTransactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence + 1, @@ -2575,7 +2576,8 @@ class ReplicaManagerTest { val props = new Properties() props.put(TransactionLogConfig.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG, "false") config.dynamicConfig.updateBrokerConfig(config.brokerId, props) - TestUtils.waitUntilTrue(() => !config.transactionLogConfig.transactionPartitionVerificationEnable, "Config did not dynamically update.") + val transactionLogConfig = new TransactionLogConfig(config) + TestUtils.waitUntilTrue(() => !transactionLogConfig.transactionPartitionVerificationEnable, "Config did not dynamically update.") // Confirm we did not write to the log and instead returned error. val callback: AddPartitionsToTxnManager.AppendCallback = appendCallback.getValue diff --git a/server/src/main/java/org/apache/kafka/server/config/BrokerReconfigurable.java b/server-common/src/main/java/org/apache/kafka/server/config/BrokerReconfigurable.java similarity index 89% rename from server/src/main/java/org/apache/kafka/server/config/BrokerReconfigurable.java rename to server-common/src/main/java/org/apache/kafka/server/config/BrokerReconfigurable.java index 2b3fe9dcf34a8..dd44015d07023 100644 --- a/server/src/main/java/org/apache/kafka/server/config/BrokerReconfigurable.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/BrokerReconfigurable.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.server.config; +import org.apache.kafka.common.config.AbstractConfig; + import java.util.Set; /** @@ -27,13 +29,13 @@ * The reconfiguration process follows three steps: *
    *
  1. Determining which configurations can be dynamically updated via {@link #reconfigurableConfigs()}
  2. - *
  3. Validating the new configuration before applying it via {@link #validateReconfiguration(AbstractKafkaConfig)}
  4. - *
  5. Applying the new configuration via {@link #reconfigure(AbstractKafkaConfig, AbstractKafkaConfig)}
  6. + *
  7. Validating the new configuration before applying it via {@link #validateReconfiguration(AbstractConfig)}
  8. + *
  9. Applying the new configuration via {@link #reconfigure(AbstractConfig, AbstractConfig)}
  10. *
* Note: Since Kafka is eliminating Scala, developers should implement this interface instead of {@link kafka.server.BrokerReconfigurable} * * - * @see AbstractKafkaConfig + * @see AbstractConfig */ public interface BrokerReconfigurable { /** @@ -55,7 +57,7 @@ public interface BrokerReconfigurable { * * @param newConfig the new configuration to validate */ - void validateReconfiguration(AbstractKafkaConfig newConfig); + void validateReconfiguration(AbstractConfig newConfig); /** * Applies the new configuration. @@ -65,5 +67,5 @@ public interface BrokerReconfigurable { * @param oldConfig the previous configuration * @param newConfig the new configuration to apply */ - void reconfigure(AbstractKafkaConfig oldConfig, AbstractKafkaConfig newConfig); + void reconfigure(AbstractConfig oldConfig, AbstractConfig newConfig); } 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/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index fc6906b96d36e..87bf18a412f50 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -83,6 +83,4 @@ public int numRecoveryThreadsPerDataDir() { public int backgroundThreads() { return getInt(ServerConfigs.BACKGROUND_THREADS_CONFIG); } - - public abstract TransactionLogConfig transactionLogConfig(); } diff --git a/server/src/main/java/org/apache/kafka/server/config/DynamicProducerStateManagerConfig.java b/server/src/main/java/org/apache/kafka/server/config/DynamicProducerStateManagerConfig.java index 4158194f00060..a09e12d951da5 100644 --- a/server/src/main/java/org/apache/kafka/server/config/DynamicProducerStateManagerConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/DynamicProducerStateManagerConfig.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.server.config; +import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.coordinator.transaction.TransactionLogConfig; import org.apache.kafka.storage.internals.log.ProducerStateManagerConfig; @@ -39,16 +40,16 @@ public Set reconfigurableConfigs() { } @Override - public void validateReconfiguration(AbstractKafkaConfig newConfig) { - TransactionLogConfig transactionLogConfig = newConfig.transactionLogConfig(); + public void validateReconfiguration(AbstractConfig newConfig) { + TransactionLogConfig transactionLogConfig = new TransactionLogConfig(newConfig); if (transactionLogConfig.producerIdExpirationMs() < 0) throw new ConfigException(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_CONFIG + "cannot be less than 0, current value is " + producerStateManagerConfig.producerIdExpirationMs() + ", and new value is " + transactionLogConfig.producerIdExpirationMs()); } @Override - public void reconfigure(AbstractKafkaConfig oldConfig, AbstractKafkaConfig newConfig) { - TransactionLogConfig transactionLogConfig = newConfig.transactionLogConfig(); + public void reconfigure(AbstractConfig oldConfig, AbstractConfig newConfig) { + TransactionLogConfig transactionLogConfig = new TransactionLogConfig(newConfig); if (producerStateManagerConfig.producerIdExpirationMs() != transactionLogConfig.producerIdExpirationMs()) { log.info("Reconfigure {} from {} to {}", TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_CONFIG, 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..7168b6e5257ed --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/CleanedTransactionMetadata.java @@ -0,0 +1,160 @@ +/* + * 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 + */ + Optional cleanedIndex = Optional.empty(); + + /** + * 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; + } + + @Override + public String toString() { + return "(txn: " + abortedTxn + ", lastOffset: " + lastObservedBatchOffset + ")"; + } + } +} 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..5adf85baa3cc0 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java @@ -0,0 +1,762 @@ +/* + * 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.Collection; +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(LogCleaner.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 { + logger.info("Beginning cleaning of log {}", 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 + long legacyDeleteHorizonMs = cleanable.log().logSegments(0, cleanable.firstDirtyOffset()).stream() + .reduce((first, second) -> second) + .map(segment -> segment.lastModified() - cleanable.log().config().deleteRetentionMs) + .orElse(0L); + + UnifiedLog log = cleanable.log(); + CleanerStats stats = new CleanerStats(Time.SYSTEM); + + // build the offset map + logger.info("Building offset map for {}...", cleanable.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 + long cleanableHorizonMs = log.logSegments(0, cleanable.firstUncleanableOffset()).stream() + .reduce((first, second) -> second) + .map(LogSegment::lastModified) + .orElse(0L); + + // 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.cleanedIndex = 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: + * 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 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 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 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 has 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(Collection segments, int maxSize, int maxIndexSize, long firstUncleanableOffset) throws IOException { + List> grouped = new ArrayList<>(); + List segs = new ArrayList<>(segments); + + while (!segs.isEmpty()) { + List group = new ArrayList<>(); + group.add(segs.get(0)); + + long logSize = segs.get(0).size(); + long indexSize = segs.get(0).offsetIndex().sizeInBytes(); + long timeIndexSize = segs.get(0).timeIndex().sizeInBytes(); + + segs = segs.subList(1, segs.size()); + + while (!segs.isEmpty() && + logSize + segs.get(0).size() <= maxSize && + indexSize + segs.get(0).offsetIndex().sizeInBytes() <= maxIndexSize && + timeIndexSize + segs.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. + (segs.get(0).size() == 0 || + lastOffsetForFirstSegment(segs, firstUncleanableOffset) - group.get(group.size() - 1).baseOffset() <= Integer.MAX_VALUE)) { + group.add(0, segs.get(0)); + logSize += segs.get(0).size(); + indexSize += segs.get(0).offsetIndex().sizeInBytes(); + timeIndexSize += segs.get(0).timeIndex().sizeInBytes(); + segs = segs.subList(1, segs.size()); + } + + List reversedGroup = new ArrayList<>(group); + Collections.reverse(reversedGroup); + grouped.add(0, reversedGroup); + } + + List> reversedGrouped = new ArrayList<>(grouped); + Collections.reverse(reversedGrouped); + + return reversedGrouped; + } + + /** + * 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..d2cca592bbba7 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java @@ -0,0 +1,652 @@ +/* + * 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.server.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.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: + *

+ *

    + *
  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. + *
  3. 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.
  4. + *
  5. Records from aborted transactions are removed by the cleaner immediately without regard to record keys.
  6. + *
  7. 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.
  8. + *
+ */ +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); + } + + public static CleanerConfig cleanerConfig(AbstractConfig config) { + return new CleanerConfig(config); + } + + /** + * 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 = LogCleaner.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: + * 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 + public void reconfigure(AbstractConfig oldConfig, AbstractConfig newConfig) { + config = LogCleaner.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, default none + */ + 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(List 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 list of log partitions that retention threads can safely work on + */ + public List> pauseCleaningForNonCompactedPartitions() { + return cleanerManager.pauseCleaningForNonCompactedPartitions(); + } + + /** + * @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(LogCleaner.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/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) From fff6118eb285387c152aa165a3cc9c4faddd8720 Mon Sep 17 00:00:00 2001 From: Dmitry Werner Date: Tue, 8 Apr 2025 11:16:52 +0500 Subject: [PATCH 2/5] KAFKA-14485: Move LogCleaner to storage module address review comments --- build.gradle | 1 - checkstyle/import-control-storage.xml | 1 + core/src/main/scala/kafka/log/LogManager.scala | 16 ++++++++++------ .../log/AbstractLogCleanerIntegrationTest.scala | 11 ++++++----- .../kafka/config/BrokerReconfigurable.java | 8 -------- .../log/CleanedTransactionMetadata.java | 15 +++++---------- .../kafka/storage/internals/log/LogCleaner.java | 14 +++++++------- 7 files changed, 29 insertions(+), 37 deletions(-) diff --git a/build.gradle b/build.gradle index 195a4ef0c2cc3..2320291e32a24 100644 --- a/build.gradle +++ b/build.gradle @@ -3738,7 +3738,6 @@ project(':connect:mirror') { testImplementation project(':test-common:test-common-runtime') testImplementation project(':server') testImplementation project(':server-common') - testImplementation project(':server-common').sourceSets.test.output testRuntimeOnly project(':connect:runtime') 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/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index a468bd58551ad..dffbdf233761d 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -47,6 +47,7 @@ import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, 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. @@ -1385,16 +1386,19 @@ class LogManager(logDirs: Seq[File], val deletableLogs = { if (cleaner != null) { // prevent cleaner from working on same partitions when changing cleanup policy - cleaner.pauseCleaningForNonCompactedPartitions().asScala.map(entry => (entry.getKey, entry.getValue)) + 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).toList.asJava) + cleaner.resumeCleaning(deletableLogs.keySet().stream().collect(Collectors.toList[TopicPartition])) } } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 095781993f4ad..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 @@ -32,6 +29,10 @@ import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogCon 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,7 +134,7 @@ abstract class AbstractLogCleanerIntegrationTest { backoffMs, true) new LogCleaner(cleanerConfig, - java.util.List.of(logDir), + util.List.of(logDir), logMap, new LogDirFailureChannel(1), time) diff --git a/server-common/src/main/java/org/apache/kafka/config/BrokerReconfigurable.java b/server-common/src/main/java/org/apache/kafka/config/BrokerReconfigurable.java index 0ded1866a5ab2..b7076f05ef312 100644 --- a/server-common/src/main/java/org/apache/kafka/config/BrokerReconfigurable.java +++ b/server-common/src/main/java/org/apache/kafka/config/BrokerReconfigurable.java @@ -18,8 +18,6 @@ import org.apache.kafka.common.config.AbstractConfig; -import org.apache.kafka.common.config.AbstractConfig; - import java.util.Set; /** @@ -35,12 +33,6 @@ *
  • Applying the new configuration via {@link #reconfigure(AbstractConfig, AbstractConfig)}
  • * * Note: Since Kafka is eliminating Scala, developers should implement this interface instead of {@link kafka.server.BrokerReconfigurable} -<<<<<<<< HEAD:server-common/src/main/java/org/apache/kafka/server/config/BrokerReconfigurable.java - * - * - * @see AbstractConfig -======== ->>>>>>>> refs/heads/trunk:server-common/src/main/java/org/apache/kafka/config/BrokerReconfigurable.java */ public interface BrokerReconfigurable { /** 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 index 7168b6e5257ed..efd184663dfa4 100644 --- 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 @@ -38,6 +38,11 @@ * for updating the cleaned transaction index accordingly. */ public class CleanedTransactionMetadata { + /** + * Output cleaned index to write retained aborted transactions + */ + Optional cleanedIndex = Optional.empty(); + private final Set ongoingCommittedTxns = new HashSet<>(); private final Map ongoingAbortedTxns = new HashMap<>(); @@ -48,11 +53,6 @@ public class CleanedTransactionMetadata { Comparator.comparingLong(AbortedTxn::firstOffset) ); - /** - * Output cleaned index to write retained aborted transactions - */ - Optional cleanedIndex = Optional.empty(); - /** * Update the cleaned transaction state with the new found aborted transactions that has just been traversed. * @@ -151,10 +151,5 @@ private static class AbortedTransactionMetadata { public AbortedTransactionMetadata(AbortedTxn abortedTxn) { this.abortedTxn = abortedTxn; } - - @Override - public String toString() { - return "(txn: " + abortedTxn + ", lastOffset: " + lastObservedBatchOffset + ")"; - } } } 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 index d2cca592bbba7..9fa303f9afbbe 100644 --- 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 @@ -22,7 +22,7 @@ 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.server.config.BrokerReconfigurable; +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; @@ -43,6 +43,7 @@ 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; /** @@ -95,7 +96,7 @@ * */ public class LogCleaner implements BrokerReconfigurable { - private static final Logger LOG = LoggerFactory.getLogger(LogCleaner.class); + private static final Logger LOG = LoggerFactory.getLogger("kafka.log.LogCleaner"); public static final Set RECONFIGURABLE_CONFIGS = Set.of( CleanerConfig.LOG_CLEANER_THREADS_PROP, @@ -136,11 +137,9 @@ public class LogCleaner implements BrokerReconfigurable { * 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<>(); /** @@ -399,10 +398,11 @@ private boolean isCleaned(TopicPartition topicPartition, long offset) { * 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 + * @return A map of log partitions that retention threads can safely work on */ - public List> pauseCleaningForNonCompactedPartitions() { - return cleanerManager.pauseCleaningForNonCompactedPartitions(); + public Map pauseCleaningForNonCompactedPartitions() { + return cleanerManager.pauseCleaningForNonCompactedPartitions().stream() + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } /** From 2211a6b5936e44944dabb3534a1d01bda4fe14e7 Mon Sep 17 00:00:00 2001 From: Dmitry Werner Date: Wed, 9 Apr 2025 11:19:17 +0500 Subject: [PATCH 3/5] KAFKA-14485: Move LogCleaner to storage module address review comments --- config/log4j2.yaml | 2 +- .../src/main/scala/kafka/log/LogManager.scala | 8 +- .../kafka/log/LogCleanerManagerTest.scala | 8 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 2 +- .../log/CleanedTransactionMetadata.java | 19 ++- .../kafka/storage/internals/log/Cleaner.java | 112 ++++++++++-------- .../storage/internals/log/LogCleaner.java | 26 ++-- .../internals/log/LogCleanerManager.java | 4 +- .../services/kafka/templates/log4j2.yaml | 2 +- 9 files changed, 99 insertions(+), 84 deletions(-) diff --git a/config/log4j2.yaml b/config/log4j2.yaml index 7ee6f001e18ea..e24a72cedc6a8 100644 --- a/config/log4j2.yaml +++ b/config/log4j2.yaml @@ -133,7 +133,7 @@ Configuration: AppenderRef: ref: ControllerAppender # LogCleaner logger - - name: kafka.log.LogCleaner + - name: org.apache.kafka.storage.internals.log.LogCleaner level: INFO additivity: false AppenderRef: diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index dffbdf233761d..fc5b15e8fc6d0 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -895,7 +895,7 @@ class LogManager(logDirs: Seq[File], */ private def resumeCleaning(topicPartition: TopicPartition): Unit = { if (cleaner != null) { - cleaner.resumeCleaning(util.List.of(topicPartition)) + cleaner.resumeCleaning(util.Set.of(topicPartition)) info(s"Cleaning for partition $topicPartition is resumed") } } @@ -1383,7 +1383,7 @@ 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() @@ -1412,7 +1412,7 @@ class LogManager(logDirs: Seq[File], } } finally { if (cleaner != null) { - cleaner.resumeCleaning(deletableLogs.keySet().stream().collect(Collectors.toList[TopicPartition])) + cleaner.resumeCleaning(deletableLogs.keySet()) } } @@ -1552,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/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/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 98280a01f0813..09a9d1c40f140 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -2010,7 +2010,7 @@ 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)), + val logCleaner = new LogCleaner(new CleanerConfig(new KafkaConfig(oldKafkaProps)), util.List.of(TestUtils.tempDir()), new util.concurrent.ConcurrentHashMap[TopicPartition, UnifiedLog](), new LogDirFailureChannel(1), 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 index efd184663dfa4..d6e9642e59950 100644 --- 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 @@ -38,11 +38,6 @@ * for updating the cleaned transaction index accordingly. */ public class CleanedTransactionMetadata { - /** - * Output cleaned index to write retained aborted transactions - */ - Optional cleanedIndex = Optional.empty(); - private final Set ongoingCommittedTxns = new HashSet<>(); private final Map ongoingAbortedTxns = new HashMap<>(); @@ -53,6 +48,20 @@ public class CleanedTransactionMetadata { 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. * 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 index 5adf85baa3cc0..52e117555a8d4 100644 --- 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 @@ -36,14 +36,13 @@ import java.nio.ByteBuffer; import java.security.DigestException; import java.util.ArrayList; -import java.util.Collection; -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; +import java.util.stream.IntStream; /** * This class holds the actual logic for cleaning a log. @@ -61,10 +60,14 @@ public class Cleaner { private final Time time; private final Consumer checkDone; - /* buffer used for read i/o */ + /** + * Buffer used for read i/o + */ private ByteBuffer readBuffer; - /* buffer used for write i/o */ + /** + * Buffer used for write i/o + */ private ByteBuffer writeBuffer; /** @@ -114,7 +117,7 @@ public OffsetMap offsetMap() { } /** - * Clean the given log + * Clean the given log. * * @param cleanable The log to be cleaned * @@ -125,7 +128,7 @@ public Map.Entry clean(LogToClean cleanable) throws IOExcept } /** - * Clean the given log + * Clean the given log. * * @param cleanable The log to be cleaned * @param currentTime The current timestamp for doing cleaning @@ -133,21 +136,22 @@ public Map.Entry clean(LogToClean cleanable) throws IOExcept * @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 { - logger.info("Beginning cleaning of log {}", cleanable.log().name()); + 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 - long legacyDeleteHorizonMs = cleanable.log().logSegments(0, cleanable.firstDirtyOffset()).stream() - .reduce((first, second) -> second) - .map(segment -> segment.lastModified() - cleanable.log().config().deleteRetentionMs) - .orElse(0L); + List segments = log.logSegments(0, cleanable.firstDirtyOffset()); + long legacyDeleteHorizonMs = segments.isEmpty() + ? 0L + : segments.get(segments.size() - 1).lastModified() - log.config().deleteRetentionMs; - UnifiedLog log = cleanable.log(); CleanerStats stats = new CleanerStats(Time.SYSTEM); // build the offset map - logger.info("Building offset map for {}...", cleanable.log().name()); + logger.info("Building offset map for {}...", log.name()); long upperBoundOffset = cleanable.firstUncleanableOffset(); buildOffsetMap(log, cleanable.firstDirtyOffset(), upperBoundOffset, offsetMap, stats); long endOffset = offsetMap.latestOffset() + 1; @@ -185,7 +189,7 @@ public Map.Entry doClean(LogToClean cleanable, long currentT } /** - * Clean a group of segments into a single replacement segment + * Clean a group of segments into a single replacement segment. * * @param log The log being cleaned * @param segments The group of segments being cleaned @@ -208,7 +212,7 @@ public void cleanSegments(UnifiedLog log, 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.cleanedIndex = Optional.of(cleaned.txnIndex()); + transactionMetadata.setCleanedIndex(Optional.of(cleaned.txnIndex())); try { // clean segments into the new destination segment @@ -284,7 +288,7 @@ public void cleanSegments(UnifiedLog log, /** * Clean the given source log segment into the destination segment using the key=>offset mapping - * provided + * provided. * * @param topicPartition The topic and partition of the log segment to clean * @param sourceRecords The dirty log segment @@ -415,8 +419,10 @@ public boolean shouldRetainRecord(RecordBatch batch, Record record) { * 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 + *
      + *
    1. A compacted topic using compression may contain a message set slightly larger than max.message.bytes
    2. + *
    3. max.message.bytes of a topic could have been reduced after writing larger messages
    4. + *
    * In these cases, grow the buffer to hold the next batch. * * @param sourceRecords The dirty log segment records to process @@ -456,7 +462,7 @@ private void growBuffersOrFail(FileRecords sourceRecords, } /** - * Check if a batch should be discard by cleaned transaction state. + * 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 @@ -472,7 +478,7 @@ private boolean shouldDiscardBatch(RecordBatch batch, } /** - * Check if a record should be retained + * 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 @@ -500,7 +506,7 @@ private boolean shouldRetainRecord(OffsetMap map, /* 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. + * 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; @@ -521,7 +527,7 @@ private boolean shouldRetainRecord(OffsetMap map, } /** - * Double the I/O buffer capacity + * Double the I/O buffer capacity. * * @param maxLogMessageSize The maximum record size in bytes allowed */ @@ -536,7 +542,7 @@ private void growBuffers(int maxLogMessageSize) { } /** - * Restore the I/O buffer capacity to its original size + * Restore the I/O buffer capacity to its original size. */ private void restoreBuffers() { if (readBuffer.capacity() > ioBufferSize) @@ -547,8 +553,8 @@ private void restoreBuffers() { /** * 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. + * 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 @@ -557,44 +563,45 @@ private void restoreBuffers() { * * @return A list of grouped segments */ - public List> groupSegmentsBySize(Collection segments, int maxSize, int maxIndexSize, long firstUncleanableOffset) throws IOException { + public List> groupSegmentsBySize(List segments, int maxSize, int maxIndexSize, long firstUncleanableOffset) throws IOException { List> grouped = new ArrayList<>(); - List segs = new ArrayList<>(segments); - while (!segs.isEmpty()) { + while (!segments.isEmpty()) { List group = new ArrayList<>(); - group.add(segs.get(0)); + group.add(segments.get(0)); - long logSize = segs.get(0).size(); - long indexSize = segs.get(0).offsetIndex().sizeInBytes(); - long timeIndexSize = segs.get(0).timeIndex().sizeInBytes(); + long logSize = segments.get(0).size(); + long indexSize = segments.get(0).offsetIndex().sizeInBytes(); + long timeIndexSize = segments.get(0).timeIndex().sizeInBytes(); - segs = segs.subList(1, segs.size()); + segments = segments.subList(1, segments.size()); - while (!segs.isEmpty() && - logSize + segs.get(0).size() <= maxSize && - indexSize + segs.get(0).offsetIndex().sizeInBytes() <= maxIndexSize && - timeIndexSize + segs.get(0).timeIndex().sizeInBytes() <= maxIndexSize && + 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. - (segs.get(0).size() == 0 || - lastOffsetForFirstSegment(segs, firstUncleanableOffset) - group.get(group.size() - 1).baseOffset() <= Integer.MAX_VALUE)) { - group.add(0, segs.get(0)); - logSize += segs.get(0).size(); - indexSize += segs.get(0).offsetIndex().sizeInBytes(); - timeIndexSize += segs.get(0).timeIndex().sizeInBytes(); - segs = segs.subList(1, segs.size()); + (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()); } - List reversedGroup = new ArrayList<>(group); - Collections.reverse(reversedGroup); + List reversedGroup = IntStream.range(0, group.size()) + .map(i -> group.size() - 1 - i) + .mapToObj(group::get) + .toList(); grouped.add(0, reversedGroup); } - List> reversedGrouped = new ArrayList<>(grouped); - Collections.reverse(reversedGrouped); - - return reversedGrouped; + return IntStream.range(0, grouped.size()) + .map(i -> grouped.size() - 1 - i) + .mapToObj(grouped::get) + .toList(); } /** @@ -605,7 +612,7 @@ public List> groupSegmentsBySize(Collection segment * 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 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 */ @@ -622,6 +629,7 @@ private long lastOffsetForFirstSegment(List segs, long firstUncleana /** * 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 @@ -677,7 +685,7 @@ public void buildOffsetMap(UnifiedLog log, } /** - * Add the messages in the given segment to the offset map + * 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 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 index 9fa303f9afbbe..ef4ad79d48847 100644 --- 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 @@ -96,7 +96,7 @@ * */ public class LogCleaner implements BrokerReconfigurable { - private static final Logger LOG = LoggerFactory.getLogger("kafka.log.LogCleaner"); + private static final Logger LOG = LoggerFactory.getLogger(LogCleaner.class); public static final Set RECONFIGURABLE_CONFIGS = Set.of( CleanerConfig.LOG_CLEANER_THREADS_PROP, @@ -194,10 +194,6 @@ private void registerMetrics() { metricsGroup.newGauge(DEAD_THREAD_COUNT_METRIC_NAME, this::deadThreadCount); } - public static CleanerConfig cleanerConfig(AbstractConfig config) { - return new CleanerConfig(config); - } - /** * Start the background cleaner threads. */ @@ -263,7 +259,7 @@ public Set reconfigurableConfigs() { */ @Override public void validateReconfiguration(AbstractConfig newConfig) { - int numThreads = LogCleaner.cleanerConfig(newConfig).numThreads; + int numThreads = new CleanerConfig(newConfig).numThreads; int currentThreads = config.numThreads; if (numThreads < 1) throw new ConfigException("Log cleaner threads should be at least 1"); @@ -276,8 +272,10 @@ public void validateReconfiguration(AbstractConfig newConfig) { /** * Reconfigure log clean config. The will: - * 1. update desiredRatePerSec in Throttler with logCleanerIoMaxBytesPerSecond, if necessary - * 2. stop current log cleaners and create new ones. + *
      + *
    1. update desiredRatePerSec in Throttler with logCleanerIoMaxBytesPerSecond, if necessary
    2. + *
    3. stop current log cleaners and create new ones.
    4. + *
    * 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 @@ -285,7 +283,7 @@ public void validateReconfiguration(AbstractConfig newConfig) { */ @Override public void reconfigure(AbstractConfig oldConfig, AbstractConfig newConfig) { - config = LogCleaner.cleanerConfig(newConfig); + config = new CleanerConfig(newConfig); double maxIoBytesPerSecond = config.maxIoBytesPerSecond; if (maxIoBytesPerSecond != oldConfig.getDouble(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP)) { @@ -311,15 +309,15 @@ public void abortCleaning(TopicPartition 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 + * @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 + * 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 @@ -364,13 +362,13 @@ public void abortAndPauseCleaning(TopicPartition topicPartition) { * * @param topicPartitions The collection of topicPartitions to be resumed cleaning */ - public void resumeCleaning(List topicPartitions) { + 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 + * 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 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/tests/kafkatest/services/kafka/templates/log4j2.yaml b/tests/kafkatest/services/kafka/templates/log4j2.yaml index 22e3f118f680d..f120aff1f7388 100644 --- a/tests/kafkatest/services/kafka/templates/log4j2.yaml +++ b/tests/kafkatest/services/kafka/templates/log4j2.yaml @@ -254,7 +254,7 @@ 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: From 6c465668520634d50b564b7bdc738908e5db8ecb Mon Sep 17 00:00:00 2001 From: Dmitry Werner Date: Wed, 9 Apr 2025 17:54:52 +0500 Subject: [PATCH 4/5] KAFKA-14485: Move LogCleaner to storage module add a logger note to the upgrade.html --- docs/upgrade.html | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index 3ffa47f0b0242..afb9c2014299e 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -24,6 +24,13 @@

    Upgrading to 4.1.0

    Upgrading Servers to 4.1.0 from any version 3.3.x through 4.0.x
    Notable changes in 4.1.0