/
LogCleaner.scala
1330 lines (1190 loc) · 59.7 KB
/
LogCleaner.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.log
import java.io.{File, IOException}
import java.nio._
import java.util.Date
import java.util.concurrent.TimeUnit
import kafka.common._
import kafka.log.LogCleaner.{CleanerRecopyPercentMetricName, DeadThreadCountMetricName, MaxBufferUtilizationPercentMetricName, MaxCleanTimeMetricName, MaxCompactionDelayMetricsName}
import kafka.server.{BrokerReconfigurable, KafkaConfig}
import kafka.utils._
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.metrics.KafkaMetricsGroup
import org.apache.kafka.server.util.ShutdownableThread
import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, LastRecord, LogDirFailureChannel, OffsetMap, SkimpyOffsetMap, TransactionIndex}
import scala.jdk.CollectionConverters._
import scala.collection.mutable.ListBuffer
import scala.collection.{Iterable, Seq, Set, mutable}
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 pool 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: Pool[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, 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(desiredRatePerSec = config.maxIoBytesPerSecond,
checkIntervalMs = 300,
throttleDown = true,
"cleaner-io",
"bytes",
time = time)
private[log] val cleaners = mutable.ArrayBuffer[CleanerThread]()
/**
* scala 2.12 does not support maxOption so we handle the empty manually.
* @param f to compute the result
* @return the max value (int value) or 0 if there is no cleaner
*/
private def maxOverCleanerThreads(f: CleanerThread => Double): Int =
cleaners.foldLeft(0.0d)((max: Double, thread: CleanerThread) => math.max(max, f(thread))).toInt
/* a metric to track the maximum utilization of any thread's buffer in the last cleaning */
metricsGroup.newGauge(MaxBufferUtilizationPercentMetricName,
() => maxOverCleanerThreads(_.lastStats.bufferUtilization) * 100)
/* 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))
// 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)
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
*/
def shutdown(): Unit = {
info("Shutting down the log cleaner.")
try {
cleaners.foreach(_.shutdown())
cleaners.clear()
} 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)
}
shutdown()
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, partitionToRemove = 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
*/
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)
}
/**
* 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 = 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()
}
// 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 = 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)
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: Iterable[(TopicPartition, UnifiedLog)] = cleanerManager.deletableLogs()
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.map(_._1))
}
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
*/
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(
KafkaConfig.LogCleanerThreadsProp,
KafkaConfig.LogCleanerDedupeBufferSizeProp,
KafkaConfig.LogCleanerDedupeBufferLoadFactorProp,
KafkaConfig.LogCleanerIoBufferSizeProp,
KafkaConfig.MessageMaxBytesProp,
KafkaConfig.LogCleanerIoMaxBytesPerSecondProp,
KafkaConfig.LogCleanerBackoffMsProp
)
def cleanerConfig(config: KafkaConfig): CleanerConfig = {
new CleanerConfig(config.logCleanerThreads,
config.logCleanerDedupeBufferSize,
config.logCleanerDedupeBufferLoadFactor,
config.logCleanerIoBufferSize,
config.messageMaxBytes,
config.logCleanerIoMaxBytesPerSecond,
config.logCleanerBackoffMs,
config.logCleanerEnable)
}
private val MaxBufferUtilizationPercentMetricName = "max-buffer-utilization-percent"
private val CleanerRecopyPercentMetricName = "cleaner-recopy-percent"
private val MaxCleanTimeMetricName = "max-clean-time-secs"
private 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 = 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).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).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)
// 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
*/
private[log] def cleanSegments(log: UnifiedLog,
segments: Seq[LogSegment],
map: OffsetMap,
currentTime: Long,
stats: CleanerStats,
transactionMetadata: CleanedTransactionMetadata,
legacyDeleteHorizonMs: 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)
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, 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.lastModified = modified
// swap in new segment
info(s"Swapping in cleaned segment $cleaned for segment(s) $segments in log $log")
log.replaceSegments(List(cleaned), segments)
} 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 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: mutable.Map[Long, LastRecord],
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 <= 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.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 (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 = 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(topicPartition, logCleanerFilter, writeBuffer, maxLogMessageSize, 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(largestOffset = result.maxOffset,
largestTimestamp = result.maxTimestamp,
shallowOffsetOfMaxTimestamp = result.shallowOffsetOfMaxTimestamp,
records = 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
*/
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
*/
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: Iterable[LogSegment], maxSize: Int, maxIndexSize: Int, firstUncleanableOffset: Long): List[Seq[LogSegment]] = {
var grouped = List[List[LogSegment]]()
var segs = segments.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.