kafka-jira mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Philippe Laflamme (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (KAFKA-5413) Log cleaner fails due to large offset in segment file
Date Thu, 11 Jan 2018 19:41:00 GMT

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

Philippe Laflamme commented on KAFKA-5413:
------------------------------------------

Yes, that's what happens when the log cleaner dies for any reason (the compacted logs are
no longer compacted and will grow unbounded). The original issue has no workaround, meaning
that the log cleaner will die again after restarting the broker...

FWIW, we ended up upgrading to 0.11.x since there was clearly no intention on releasing 0.10.2.2.

> Log cleaner fails due to large offset in segment file
> -----------------------------------------------------
>
>                 Key: KAFKA-5413
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5413
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 0.10.2.0, 0.10.2.1
>         Environment: Ubuntu 14.04 LTS, Oracle Java 8u92, kafka_2.11-0.10.2.0
>            Reporter: Nicholas Ngorok
>            Assignee: Kelvin Rutt
>            Priority: Critical
>              Labels: reliability
>             Fix For: 0.10.2.2, 0.11.0.0
>
>         Attachments: 00000000000000000000.index.cleaned, 00000000000000000000.log, 00000000000000000000.log.cleaned,
00000000000000000000.timeindex.cleaned, 00000000002147422683.log, kafka-5413.patch
>
>
> The log cleaner thread in our brokers is failing with the trace below
> {noformat}
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: Cleaning segment
0 in log __consumer_offsets-12 (largest timestamp Thu Jun 08 15:48:59 PDT 2017) into 0, retaining
deletes. (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: Cleaning segment
2147343575 in log __consumer_offsets-12 (largest timestamp Thu Jun 08 15:49:06 PDT 2017) into
0, retaining deletes. (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,834] ERROR {kafka-log-cleaner-thread-0} [kafka-log-cleaner-thread-0],
Error due to  (kafka.log.LogCleaner)
> java.lang.IllegalArgumentException: requirement failed: largest offset in message set
can not be safely converted to relative offset.
>         at scala.Predef$.require(Predef.scala:224)
>         at kafka.log.LogSegment.append(LogSegment.scala:109)
>         at kafka.log.Cleaner.cleanInto(LogCleaner.scala:478)
>         at kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:405)
>         at kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:401)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:401)
>         at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:363)
>         at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:362)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at kafka.log.Cleaner.clean(LogCleaner.scala:362)
>         at kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:241)
>         at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:220)
>         at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> [2017-06-08 15:49:54,835] INFO {kafka-log-cleaner-thread-0} [kafka-log-cleaner-thread-0],
Stopped  (kafka.log.LogCleaner)
> {noformat}
> This seems to point at the specific line [here| https://github.com/apache/kafka/blob/0.11.0/core/src/main/scala/kafka/log/LogSegment.scala#L92]
in the kafka src where the difference is actually larger than MAXINT as both baseOffset and
offset are of type long. It was introduced in this [pr| https://github.com/apache/kafka/pull/2210/files/56d1f8196b77a47b176b7bbd1e4220a3be827631]
> These were the outputs of dumping the first two log segments
> {noformat}
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration --files /kafka-logs/__consumer_offsets-12/000
> 00000000000000000.log
> Dumping /kafka-logs/__consumer_offsets-12/00000000000000000000.log
> Starting offset: 0
> offset: 1810054758 position: 0 NoTimestampType: -1 isvalid: true payloadsize: -1 magic:
0 compresscodec: NONE crc: 3127861909 keysize: 34
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration --files /kafka-logs/__consumer_offsets-12/000
> 00000002147343575.log
> Dumping /kafka-logs/__consumer_offsets-12/00000000002147343575.log
> Starting offset: 2147343575
> offset: 2147539884 position: 0 NoTimestampType: -1 isvalid: true paylo
> adsize: -1 magic: 0 compresscodec: NONE crc: 2282192097 keysize: 34
> {noformat}
> My guess is that since 2147539884 is larger than MAXINT, we are hitting this exception.
Was there a specific reason, this check was added in 0.10.2?
> E.g. if the first offset is a key = "key 0" and then we have MAXINT + 1 of "key 1" following,
wouldn't we run into this situation whenever the log cleaner runs?



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message