[ https://issues.apache.org/jira/browse/KAFKA-12868?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Yuneng Xie updated KAFKA-12868: ------------------------------- Description: our broker spent too long loading the offset partition. ``` [2021-05-30 03:18:20,505] INFO [GroupMetadataManager brokerId=2] Finished loading offsets and group metadata from __consumer_offsets-47 in 1236029 milliseconds. (kafka.coordinator.group.GroupMetadataManager) ``` so i checked the partition data , it's too big ``` 106G /var/lib/kafka/data/__consumer_offsets-47 ``` the retention time is 7 days ``` log.retention.hours=168 ``` but i found many segments data that's supposed to be cleaned in /var/lib/kafka/data/__consumer_offsets-47. and i found error log about log cleaner: ``` [2021-04-09 00:41:48,327] INFO Cleaner 0: Beginning cleaning of log __consumer_offsets-47. (kafka.log.LogCleaner) [2021-04-09 00:41:48,327] INFO Cleaner 0: Building offset map for __consumer_offsets-47... (kafka.log.LogCleaner) [2021-04-09 00:41:48,353] INFO Cleaner 0: Building offset map for log __consumer_offsets-47 for 1 segments in offset range [1010155298, 1010757586). (kafka.log.LogCleaner) [2021-04-09 13:02:09,425] INFO Cleaner 0: Offset map for log __consumer_offsets-47 complete. (kafka.log.LogCleaner) [2021-04-09 13:02:09,426] INFO Cleaner 0: Cleaning log __consumer_offsets-47 (cleaning prior to Fri Apr 09 00:41:34 SRET 2021, discarding tombstones prior to Wed Apr 07 23:19:50 SRET 2021)... (kafka.log.LogCleaner) [2021-04-09 13:02:09,427] INFO Cleaner 0: Cleaning LogSegment(baseOffset=0, size=51681, lastModifiedTime=1617890796000, largestTime=1617890765851) in log __consumer_offsets-47 into 0 with deletion horizon 1617808790000, retaining deletes. (kafka.log.LogCleaner) [2021-04-09 13:02:27,573] INFO Cleaner 0: Cleaning LogSegment(baseOffset=1009555589, size=65218, lastModifiedTime=1617895190000, largestTime=1617895190754) in log __consumer_offsets-47 into 0 with deletion horizon 1617808790000, retaining deletes. (kafka.log.LogCleaner) [2021-04-09 13:02:52,991] INFO Cleaner 0: Swapping in cleaned segment LogSegment(baseOffset=0, size=53871, lastModifiedTime=1617895190000, largestTime=1617894932233) for segment(s) List(LogSegment(baseOffset=0, size=51681, lastModifiedTime=1617890796000, largestTime=1617890765851), LogSegment(baseOffset=1009555589, size=65218, lastModifiedTime=1617895190000, largestTime=1617895190754)) in log Log(dir=/var/lib/kafka/data/__consumer_offsets-47, topic=__consumer_offsets, partition=47, highWatermark=1015632787, lastStableOffset=1015632787, logStartOffset=0, logEndOffset=1015632789) (kafka.log.LogCleaner) [2021-04-09 13:02:52,992] INFO Cleaner 0: Cleaning LogSegment(baseOffset=1010155298, size=104857448, lastModifiedTime=1617900094000, largestTime=1617900094618) in log __consumer_offsets-47 into 1010155298 with deletion horizon 1617808790000, retaining deletes. (kafka.log.LogCleaner) [2021-04-10 00:28:24,476] INFO Cleaner 0: Swapping in cleaned segment LogSegment(baseOffset=1010155298, size=63028, lastModifiedTime=1617900094000, largestTime=1617900094618) for segment(s) List(LogSegment(baseOffset=1010155298, size=104857448, lastModifiedTime=1617900094000, largestTime=1617900094618)) in log Log(dir=/var/lib/kafka/data/__consumer_offsets-47, topic=__consumer_offsets, partition=47, highWatermark=1021404662, lastStableOffset=1021404662, logStartOffset=0, logEndOffset=1021404663) (kafka.log.LogCleaner) [2021-04-10 00:28:24,492] INFO [kafka-log-cleaner-thread-0]: Log cleaner thread 0 cleaned log __consumer_offsets-47 (dirty section = [1010155298, 1010155298]) 100.1 MB of log processed in 85,596.2 seconds (0.0 MB/sec). Indexed 100.0 MB in 44421.1 seconds (0.0 Mb/sec, 51.9% of total time) Buffer utilization: 0.0% Cleaned 100.1 MB in 41175.1 seconds (0.0 Mb/sec, 48.1% of total time) Start size: 100.1 MB (602,913 messages) End size: 0.1 MB (625 messages) 99.9% size reduction (99.9% fewer messages) (kafka.log.LogCleaner) [2021-04-10 00:28:24,507] INFO Cleaner 0: Beginning cleaning of log __consumer_offsets-47. (kafka.log.LogCleaner) [2021-04-10 00:28:24,507] INFO Cleaner 0: Building offset map for __consumer_offsets-47... (kafka.log.LogCleaner) [2021-04-10 00:28:24,526] INFO Cleaner 0: Building offset map for log __consumer_offsets-47 for 17 segments in offset range [1010757586, 1020967392). (kafka.log.LogCleaner) [2021-04-10 00:29:22,669] WARN [kafka-log-cleaner-thread-0]: Unexpected exception thrown when cleaning log Log(dir=/var/lib/kafka/data/__consumer_offsets-47, topic=__consumer_offsets, partition=47, highWatermark=1021411007, lastStableOffset=1021411007, logStartOffset=0, logEndOffset=1021411007). Marking its partition (__consumer_offsets-47) as uncleanable (kafka.log.LogCleaner) kafka.log.LogCleaningException: -2147483648 at kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:348) at kafka.log.LogCleaner$CleanerThread.tryCleanFilthiestLog(LogCleaner.scala:324) at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:313) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) Caused by: java.lang.ArrayIndexOutOfBoundsException: -2147483648 at kafka.utils.CoreUtils$.readInt(CoreUtils.scala:241) at kafka.log.SkimpyOffsetMap.positionOf(OffsetMap.scala:183) at kafka.log.SkimpyOffsetMap.put(OffsetMap.scala:101) at kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$2(LogCleaner.scala:947) at kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$2$adapted(LogCleaner.scala:944) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$1(LogCleaner.scala:944) at kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$1$adapted(LogCleaner.scala:933) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at kafka.log.Cleaner.buildOffsetMapForSegment(LogCleaner.scala:933) at kafka.log.Cleaner.$anonfun$buildOffsetMap$3(LogCleaner.scala:894) at kafka.log.Cleaner.$anonfun$buildOffsetMap$3$adapted(LogCleaner.scala:890) at scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:877) 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:876) at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:890) at kafka.log.Cleaner.doClean(LogCleaner.scala:514) at kafka.log.Cleaner.clean(LogCleaner.scala:502) at kafka.log.LogCleaner$CleanerThread.cleanLog(LogCleaner.scala:371) at kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:344) ... 3 more ``` seems like the cleaner failed to compact __consumer_offsets-47 any suggestion on this? was: our broker spent too long loading the offset partition. ``` [2021-05-30 03:18:20,505] INFO [GroupMetadataManager brokerId=2] Finished loading offsets and group metadata from __consumer_offsets-47 in 1236029 milliseconds. (kafka.coordinator.group.GroupMetadataManager) ``` so i checked the partition data , it's too big ``` 106G /var/lib/kafka/data/__consumer_offsets-47 ``` the retention time is 7 days ``` log.retention.hours=168 ``` but i found many segments data that's supposed to be cleaned in /var/lib/kafka/data/__consumer_offsets-47. and i found error log about log cleaner: ``` [2021-04-10 00:28:24,507] INFO Cleaner 0: Beginning cleaning of log __consumer_offsets-47. (kafka.log.LogCleaner) [2021-04-10 00:28:24,507] INFO Cleaner 0: Building offset map for __consumer_offsets-47... (kafka.log.LogCleaner) [2021-04-10 00:28:24,526] INFO Cleaner 0: Building offset map for log __consumer_offsets-47 for 17 segments in offset range [1010757586, 1020967392). (kafka.log.LogCleaner) [2021-04-10 00:29:22,669] WARN [kafka-log-cleaner-thread-0]: Unexpected exception thrown when cleaning log Log(dir=/var/lib/kafka/data/__consumer_offsets-47, topic=__consumer_offsets, partition=47, highWatermark=1021411007, lastStableOffset=1021411007, logStartOffset=0, logEndOffset=1021411007). Marking its partition (__consumer_offsets-47) as uncleanable (kafka.log.LogCleaner) kafka.log.LogCleaningException: -2147483648 at kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:348) at kafka.log.LogCleaner$CleanerThread.tryCleanFilthiestLog(LogCleaner.scala:324) at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:313) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) Caused by: java.lang.ArrayIndexOutOfBoundsException: -2147483648 at kafka.utils.CoreUtils$.readInt(CoreUtils.scala:241) at kafka.log.SkimpyOffsetMap.positionOf(OffsetMap.scala:183) at kafka.log.SkimpyOffsetMap.put(OffsetMap.scala:101) at kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$2(LogCleaner.scala:947) at kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$2$adapted(LogCleaner.scala:944) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$1(LogCleaner.scala:944) at kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$1$adapted(LogCleaner.scala:933) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at kafka.log.Cleaner.buildOffsetMapForSegment(LogCleaner.scala:933) at kafka.log.Cleaner.$anonfun$buildOffsetMap$3(LogCleaner.scala:894) at kafka.log.Cleaner.$anonfun$buildOffsetMap$3$adapted(LogCleaner.scala:890) at scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:877) 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:876) at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:890) at kafka.log.Cleaner.doClean(LogCleaner.scala:514) at kafka.log.Cleaner.clean(LogCleaner.scala:502) at kafka.log.LogCleaner$CleanerThread.cleanLog(LogCleaner.scala:371) at kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:344) ... 3 more ``` seems like the cleaner failed to compact __consumer_offsets-47 any suggestion on this? > log cleaner failed with java.lang.ArrayIndexOutOfBoundsException: -2147483648 > ----------------------------------------------------------------------------- > > Key: KAFKA-12868 > URL: https://issues.apache.org/jira/browse/KAFKA-12868 > Project: Kafka > Issue Type: Bug > Components: log cleaner > Affects Versions: 2.4.0 > Reporter: Yuneng Xie > Priority: Major > > our broker spent too long loading the offset partition. > ``` > [2021-05-30 03:18:20,505] INFO [GroupMetadataManager brokerId=2] Finished > loading offsets and group metadata from __consumer_offsets-47 in 1236029 > milliseconds. (kafka.coordinator.group.GroupMetadataManager) > ``` > > so i checked the partition data , it's too big > ``` > 106G /var/lib/kafka/data/__consumer_offsets-47 > ``` > > the retention time is 7 days > ``` > log.retention.hours=168 > ``` > but i found many segments data that's supposed to be cleaned in > /var/lib/kafka/data/__consumer_offsets-47. > > and i found error log about log cleaner: > ``` > > [2021-04-09 00:41:48,327] INFO Cleaner 0: Beginning cleaning of log > __consumer_offsets-47. (kafka.log.LogCleaner) > [2021-04-09 00:41:48,327] INFO Cleaner 0: Building offset map for > __consumer_offsets-47... (kafka.log.LogCleaner) > [2021-04-09 00:41:48,353] INFO Cleaner 0: Building offset map for log > __consumer_offsets-47 for 1 segments in offset range [1010155298, > 1010757586). (kafka.log.LogCleaner) > [2021-04-09 13:02:09,425] INFO Cleaner 0: Offset map for log > __consumer_offsets-47 complete. (kafka.log.LogCleaner) > [2021-04-09 13:02:09,426] INFO Cleaner 0: Cleaning log __consumer_offsets-47 > (cleaning prior to Fri Apr 09 00:41:34 SRET 2021, discarding tombstones prior > to Wed Apr 07 23:19:50 SRET 2021)... (kafka.log.LogCleaner) > [2021-04-09 13:02:09,427] INFO Cleaner 0: Cleaning LogSegment(baseOffset=0, > size=51681, lastModifiedTime=1617890796000, largestTime=1617890765851) in log > __consumer_offsets-47 into 0 with deletion horizon 1617808790000, retaining > deletes. (kafka.log.LogCleaner) > [2021-04-09 13:02:27,573] INFO Cleaner 0: Cleaning > LogSegment(baseOffset=1009555589, size=65218, lastModifiedTime=1617895190000, > largestTime=1617895190754) in log __consumer_offsets-47 into 0 with deletion > horizon 1617808790000, retaining deletes. (kafka.log.LogCleaner) > [2021-04-09 13:02:52,991] INFO Cleaner 0: Swapping in cleaned segment > LogSegment(baseOffset=0, size=53871, lastModifiedTime=1617895190000, > largestTime=1617894932233) for segment(s) List(LogSegment(baseOffset=0, > size=51681, lastModifiedTime=1617890796000, largestTime=1617890765851), > LogSegment(baseOffset=1009555589, size=65218, lastModifiedTime=1617895190000, > largestTime=1617895190754)) in log > Log(dir=/var/lib/kafka/data/__consumer_offsets-47, topic=__consumer_offsets, > partition=47, highWatermark=1015632787, lastStableOffset=1015632787, > logStartOffset=0, logEndOffset=1015632789) (kafka.log.LogCleaner) > [2021-04-09 13:02:52,992] INFO Cleaner 0: Cleaning > LogSegment(baseOffset=1010155298, size=104857448, > lastModifiedTime=1617900094000, largestTime=1617900094618) in log > __consumer_offsets-47 into 1010155298 with deletion horizon 1617808790000, > retaining deletes. (kafka.log.LogCleaner) > [2021-04-10 00:28:24,476] INFO Cleaner 0: Swapping in cleaned segment > LogSegment(baseOffset=1010155298, size=63028, lastModifiedTime=1617900094000, > largestTime=1617900094618) for segment(s) > List(LogSegment(baseOffset=1010155298, size=104857448, > lastModifiedTime=1617900094000, largestTime=1617900094618)) in log > Log(dir=/var/lib/kafka/data/__consumer_offsets-47, topic=__consumer_offsets, > partition=47, highWatermark=1021404662, lastStableOffset=1021404662, > logStartOffset=0, logEndOffset=1021404663) (kafka.log.LogCleaner) > [2021-04-10 00:28:24,492] INFO [kafka-log-cleaner-thread-0]: > Log cleaner thread 0 cleaned log __consumer_offsets-47 (dirty section = > [1010155298, 1010155298]) > 100.1 MB of log processed in 85,596.2 seconds (0.0 MB/sec). > Indexed 100.0 MB in 44421.1 seconds (0.0 Mb/sec, 51.9% of total time) > Buffer utilization: 0.0% > Cleaned 100.1 MB in 41175.1 seconds (0.0 Mb/sec, 48.1% of total time) > Start size: 100.1 MB (602,913 messages) > End size: 0.1 MB (625 messages) > 99.9% size reduction (99.9% fewer messages) > (kafka.log.LogCleaner) > [2021-04-10 00:28:24,507] INFO Cleaner 0: Beginning cleaning of log > __consumer_offsets-47. (kafka.log.LogCleaner) > [2021-04-10 00:28:24,507] INFO Cleaner 0: Building offset map for > __consumer_offsets-47... (kafka.log.LogCleaner) > [2021-04-10 00:28:24,526] INFO Cleaner 0: Building offset map for log > __consumer_offsets-47 for 17 segments in offset range [1010757586, > 1020967392). (kafka.log.LogCleaner) > [2021-04-10 00:29:22,669] WARN [kafka-log-cleaner-thread-0]: Unexpected > exception thrown when cleaning log > Log(dir=/var/lib/kafka/data/__consumer_offsets-47, topic=__consumer_offsets, > partition=47, highWatermark=1021411007, lastStableOffset=1021411007, > logStartOffset=0, logEndOffset=1021411007). Marking its partition > (__consumer_offsets-47) as uncleanable (kafka.log.LogCleaner) > kafka.log.LogCleaningException: -2147483648 > at kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:348) > at > kafka.log.LogCleaner$CleanerThread.tryCleanFilthiestLog(LogCleaner.scala:324) > at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:313) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) > Caused by: java.lang.ArrayIndexOutOfBoundsException: -2147483648 > at kafka.utils.CoreUtils$.readInt(CoreUtils.scala:241) > at kafka.log.SkimpyOffsetMap.positionOf(OffsetMap.scala:183) > at kafka.log.SkimpyOffsetMap.put(OffsetMap.scala:101) > at > kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$2(LogCleaner.scala:947) > at > kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$2$adapted(LogCleaner.scala:944) > at scala.collection.Iterator.foreach(Iterator.scala:941) > at scala.collection.Iterator.foreach$(Iterator.scala:941) > at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) > at scala.collection.IterableLike.foreach(IterableLike.scala:74) > at scala.collection.IterableLike.foreach$(IterableLike.scala:73) > at scala.collection.AbstractIterable.foreach(Iterable.scala:56) > at > kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$1(LogCleaner.scala:944) > at > kafka.log.Cleaner.$anonfun$buildOffsetMapForSegment$1$adapted(LogCleaner.scala:933) > at scala.collection.Iterator.foreach(Iterator.scala:941) > at scala.collection.Iterator.foreach$(Iterator.scala:941) > at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) > at scala.collection.IterableLike.foreach(IterableLike.scala:74) > at scala.collection.IterableLike.foreach$(IterableLike.scala:73) > at scala.collection.AbstractIterable.foreach(Iterable.scala:56) > at kafka.log.Cleaner.buildOffsetMapForSegment(LogCleaner.scala:933) > at kafka.log.Cleaner.$anonfun$buildOffsetMap$3(LogCleaner.scala:894) > at kafka.log.Cleaner.$anonfun$buildOffsetMap$3$adapted(LogCleaner.scala:890) > at > scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:877) > 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:876) > at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:890) > at kafka.log.Cleaner.doClean(LogCleaner.scala:514) > at kafka.log.Cleaner.clean(LogCleaner.scala:502) > at kafka.log.LogCleaner$CleanerThread.cleanLog(LogCleaner.scala:371) > at kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:344) > ... 3 more > ``` > > seems like the cleaner failed to compact __consumer_offsets-47 > any suggestion on this? > -- This message was sent by Atlassian Jira (v8.3.4#803005)