[ https://issues.apache.org/jira/browse/KAFKA-14817?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17719156#comment-17719156 ]
Sergey Ivanov commented on KAFKA-14817: --------------------------------------- Hi, We faced the same issue in Kafka 2.8.1. Some partitions for __consumer_offsets topic had exta-space size: {code:java} 9.7G /var/opt/kafka/data/1/__consumer_offsets-29 9.5G /var/opt/kafka/data/1/__consumer_offsets-40 6.1G /var/opt/kafka/data/1/__consumer_offsets-1 132M /var/opt/kafka/data/1/__consumer_offsets-37 129M /var/opt/kafka/data/1/__consumer_offsets-24 128M /var/opt/kafka/data/1/__consumer_offsets-5 ... 9.0M /var/opt/kafka/data/1/__consumer_offsets-6 7.9M /var/opt/kafka/data/1/__consumer_offsets-2 620K /var/opt/kafka/data/1/__consumer_offsets-35 {code} And when we check logs inside we found very old segments: {code:java} bash-5.1$ ls -lah /var/opt/kafka/data/1/__consumer_offsets-29 total 9.7G drwxrwsr-x 2 kafka kafka 24K May 3 04:29 . drwxrwsr-x 4583 kafka kafka 432K May 3 13:55 .. -rw-rw-r-- 1 kafka kafka 8 Feb 26 06:52 00000000000000000000.index -rw-rw-r-- 1 kafka kafka 19K Feb 26 06:52 00000000000000000000.log -rw-rw-r-- 1 kafka kafka 12 Feb 26 06:52 00000000000000000000.timeindex -rw-rw-r-- 1 kafka kafka 0 Feb 27 13:18 00000000000032953821.index -rw-rw-r-- 1 kafka kafka 7.8K Feb 27 13:18 00000000000032953821.log ... -rw-r--r-- 1 kafka kafka 34M May 3 13:55 00000000000069099848.log -rw-r--r-- 1 kafka kafka 10 May 3 02:37 00000000000069099848.snapshot -rw-r--r-- 1 kafka kafka 10M May 3 13:55 00000000000069099848.timeindex -rw-r--r-- 1 kafka kafka 803 May 3 04:29 leader-epoch-checkpoint -rw-rw-r-- 1 kafka kafka 43 Nov 18 12:12 partition.metadata {code} But this topic has standard retention is 7 days. And we also found log cleaner issue with this partition in logs: {code:java} [2023-04-22 12:10:06,410] WARN [kafka-log-cleaner-thread-0]: Unexpected exception thrown when cleaning log Log(dir=/var/opt/kafka/data/1/__consumer_offsets-29, topic=__consumer_offsets, partition=29, highWatermark=0, lastStableOffset=0, logStartOffset=0, logEndOffset=64827489). Marking its partition (__consumer_offsets-29) as uncleanable (kafka.log.LogCleaner) kafka.log.LogCleaningException: Batch size 176 < buffer size 1048588, but not processed for log segment /var/opt/kafka/data/1/__consumer_offsets-29/00000000000033563994.log at position 37515124 Caused by: java.lang.IllegalStateException: Batch size 176 < buffer size 1048588, but not processed for log segment /var/opt/kafka/data/1/__consumer_offsets-29/00000000000033563994.log at position 37515124 {code} As *WA* we changed "cleanup.policy" to "delete" and Kafka removed all old segments (including uncleanable partitions) and left only one week logs, after this we returned policy to "compact". This case is also considered in article [https://luppeng.wordpress.com/2022/08/21/possible-reasons-why-a-kafka-topic-is-not-being-compacted/] (point 3). But how to solve this issue in {*}permanent{*}? > LogCleaner mark some partitions of __consumer_offsets as uncleanable > -------------------------------------------------------------------- > > Key: KAFKA-14817 > URL: https://issues.apache.org/jira/browse/KAFKA-14817 > Project: Kafka > Issue Type: Bug > Components: core > Affects Versions: 2.8.1 > Reporter: ZhenChun Pan > Priority: Major > > We find some patitions of topic __consumer_offsets can't retention it's log > any more and takes up a lot of disk space. Then we found these patitions of > topic __consumer_offsets have to mark as uncleanable in log-cleaner.log. The > logs below: > [2023-03-17 17:53:46,655] INFO Starting the log cleaner (kafka.log.LogCleaner) > [2023-03-17 17:53:46,770] INFO [kafka-log-cleaner-thread-0]: Starting > (kafka.log.LogCleaner) > [2023-03-17 17:53:46,841] INFO Cleaner 0: Beginning cleaning of log > __consumer_offsets-24. (kafka.log.LogCleaner) > [2023-03-17 17:53:46,841] INFO Cleaner 0: Building offset map for > __consumer_offsets-24... (kafka.log.LogCleaner) > [2023-03-17 17:53:47,013] INFO Cleaner 0: Building offset map for log > __consumer_offsets-24 for 5 segments in offset range [0, 2360519). > (kafka.log.LogCleaner) > [2023-03-17 17:53:47,394] INFO Cleaner 0: Growing cleaner I/O buffers from > 262144 bytes to 524288 bytes. (kafka.log.LogCleaner) > [2023-03-17 17:53:47,395] INFO Cleaner 0: Growing cleaner I/O buffers from > 524288 bytes to 1048576 bytes. (kafka.log.LogCleaner) > [2023-03-17 17:53:47,396] INFO Cleaner 0: Growing cleaner I/O buffers from > 1048576 bytes to 2097152 bytes. (kafka.log.LogCleaner) > [2023-03-17 17:53:47,401] INFO Cleaner 0: Growing cleaner I/O buffers from > 2097152 bytes to 4194304 bytes. (kafka.log.LogCleaner) > [2023-03-17 17:53:47,409] INFO Cleaner 0: Growing cleaner I/O buffers from > 4194304 bytes to 8388608 bytes. (kafka.log.LogCleaner) > [2023-03-17 17:53:47,434] INFO Cleaner 0: Growing cleaner I/O buffers from > 8388608 bytes to 10485772 bytes. (kafka.log.LogCleaner) > [2023-03-17 17:53:47,465] WARN [kafka-log-cleaner-thread-0]: Unexpected > exception thrown when cleaning log > Log(dir=/opt/kafka-service/data/__consumer_offsets-24, > topic=__consumer_offsets, partition=24, highWatermark=0, lastStableOffset=0, > logStartOffset=0, logEndOffset=2759760). Marking its partition > (__consumer_offsets-24) as uncleanable (kafka.log.LogCleaner) > kafka.log.LogCleaningException: Batch size 223 < buffer size 10485772, but > not processed for log segment > /opt/kafka-service/data/__consumer_offsets-24/00000000000000000000.log at > position 31457091 > at > kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:356) > at > kafka.log.LogCleaner$CleanerThread.tryCleanFilthiestLog(LogCleaner.scala:332) > at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:321) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) > Caused by: java.lang.IllegalStateException: Batch size 223 < buffer size > 10485772, but not processed for log segment > /opt/kafka-service/data/__consumer_offsets-24/00000000000000000000.log at > position 31457091 > at kafka.log.Cleaner.growBuffersOrFail(LogCleaner.scala:745) > at kafka.log.Cleaner.buildOffsetMapForSegment(LogCleaner.scala:983) > at kafka.log.Cleaner.$anonfun$buildOffsetMap$5(LogCleaner.scala:908) > at > kafka.log.Cleaner.$anonfun$buildOffsetMap$5$adapted(LogCleaner.scala:904) > at > scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:985) > at > scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at > scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:984) > at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:904) > at kafka.log.Cleaner.doClean(LogCleaner.scala:523) > at kafka.log.Cleaner.clean(LogCleaner.scala:511) > at kafka.log.LogCleaner$CleanerThread.cleanLog(LogCleaner.scala:380) > at > kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:352) > ... 3 more > [2023-03-17 17:54:02,477] INFO Cleaner 0: Beginning cleaning of log > __consumer_offsets-33. (kafka.log.LogCleaner) > [2023-03-17 17:54:02,477] INFO Cleaner 0: Building offset map for > __consumer_offsets-33... (kafka.log.LogCleaner) > [2023-03-17 17:54:02,511] INFO Cleaner 0: Building offset map for log > __consumer_offsets-33 for 2361 segments in offset range [2946663171, > 7343698202). (kafka.log.LogCleaner) > [2023-03-17 17:54:03,235] WARN [kafka-log-cleaner-thread-0]: Unexpected > exception thrown when cleaning log > Log(dir=/opt/kafka-service/data/__consumer_offsets-33, > topic=__consumer_offsets, partition=33, highWatermark=7344703164, > lastStableOffset=7344703164, logStartOffset=0, logEndOffset=7344703164). > Marking its partition (__consumer_offsets-33) as uncleanable > (kafka.log.LogCleaner) > kafka.log.LogCleaningException: Batch size 529 < buffer size 10485772, but > not processed for log segment > /opt/kafka-service/data/__consumer_offsets-33/00000000002946663171.log at > position 70025136 > at > kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:356) > at > kafka.log.LogCleaner$CleanerThread.tryCleanFilthiestLog(LogCleaner.scala:332) > at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:321) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) > Caused by: java.lang.IllegalStateException: Batch size 529 < buffer size > 10485772, but not processed for log segment > /opt/kafka-service/data/__consumer_offsets-33/00000000002946663171.log at > position 70025136 > at kafka.log.Cleaner.growBuffersOrFail(LogCleaner.scala:745) > at kafka.log.Cleaner.buildOffsetMapForSegment(LogCleaner.scala:983) > at kafka.log.Cleaner.$anonfun$buildOffsetMap$5(LogCleaner.scala:908) > at > kafka.log.Cleaner.$anonfun$buildOffsetMap$5$adapted(LogCleaner.scala:904) > at > scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:985) > at > scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at > scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:984) > at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:904) > at kafka.log.Cleaner.doClean(LogCleaner.scala:523) > at kafka.log.Cleaner.clean(LogCleaner.scala:511) > at kafka.log.LogCleaner$CleanerThread.cleanLog(LogCleaner.scala:380) > at > kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:352) > ... 3 more > We want to know why they marked as uncleanable and how to resume. -- This message was sent by Atlassian Jira (v8.20.10#820010)