[ 
https://issues.apache.org/jira/browse/KAFKA-14817?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17719250#comment-17719250
 ] 

Luke Chen edited comment on KAFKA-14817 at 5/4/23 10:11 AM:
------------------------------------------------------------

[~polaris.alioth] [~mrMigles] , thanks for reporting this issue. The error 
means we cannot recognize the batch in the log position (maybe corrupted), so 
that we cannot proceed with compaction for this partition logs. Like pointed 
out in this article  
[https://luppeng.wordpress.com/2022/08/21/possible-reasons-why-a-kafka-topic-is-not-being-compacted/]
 (point 3), letting Kafka do log recovery by itself should be able to fix it.

About how we can fix this permanently, I have to think about it.

But before that, if it's possible to help verify if the log segments appeared 
in the log is indeed corrupted?

You can verify it by running this command to dump the uncleanable logs?:
{code:java}
./bin/kafka-dump-log.sh --files 
/tmp/kraft-combined-logs/quickstart-events-0/00000000000000000000.log{code}
Of course, if it's allowed, it'll be better you upload the uncleanable logs for 
me to investigation. But I can understand if it's not permitted.

 

Thanks.


was (Author: showuon):
[~polaris.alioth] [~mrMigles] , thanks for reporting this issue. The error 
means we cannot recognize the batch in the log position (maybe corrupted), so 
that we cannot proceed with compaction for this partition logs. Like 
[~mrMigles] pointed out, letting Kafka do log recovery by itself should be able 
to fix it. 

About how we can fix this permanently, I have to think about it.

But before that, if it's possible to help verify if the log segments appeared 
in the log is indeed corrupted?

You can verify it by running this command to dump the uncleanable logs?:
{code:java}
./bin/kafka-dump-log.sh --files 
/tmp/kraft-combined-logs/quickstart-events-0/00000000000000000000.log{code}
Of course, if it's allowed, it'll be better you upload the uncleanable logs for 
me to investigation. But I can understand if it's not permitted.

 

Thanks.

> 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)

Reply via email to