[ https://issues.apache.org/jira/browse/KAFKA-5413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16044555#comment-16044555 ]
Federico Giraud commented on KAFKA-5413: ---------------------------------------- More details (I'm investigating the same issue in the same company): the problem seems to be exactly what is described in [this comment on the PR|https://github.com/apache/kafka/pull/2210/files/56d1f8196b77a47b176b7bbd1e4220a3be827631#r91192806]. Debugger output: {code} kafka-log-cleaner-thread-0[7] where [7] kafka.log.Cleaner.cleanSegments (LogCleaner.scala:401) [8] kafka.log.Cleaner$$anonfun$clean$4.apply (LogCleaner.scala:363) [9] kafka.log.Cleaner$$anonfun$clean$4.apply (LogCleaner.scala:362) [10] scala.collection.immutable.List.foreach (List.scala:381) [11] kafka.log.Cleaner.clean (LogCleaner.scala:362) [12] kafka.log.LogCleaner$CleanerThread.cleanOrSleep (LogCleaner.scala:241) [13] kafka.log.LogCleaner$CleanerThread.doWork (LogCleaner.scala:220) [14] kafka.utils.ShutdownableThread.run (ShutdownableThread.scala:63) kafka-log-cleaner-thread-0[7] print segments segments = "List(LogSegment(baseOffset=0, size=18527), LogSegment(baseOffset=2147343575, size=60))" {code} > 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 > > 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.3.15#6346)