[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16510050#comment-16510050 ] Eugen Feller commented on KAFKA-5630: - Looks like we are running into a similar issue usingĀ 0.10.2.1 broker and kafka streams client 0.11.0.1. Wonder if this fix helps only if broker is also on 0.11.01? This is the my relatedĀ JIRA (https://issues.apache.org/jira/browse/KAFKA-6977) Thanks. > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin >Assignee: Jiangjie Qin >Priority: Critical > Labels: regression, reliability > Fix For: 0.11.0.1, 1.0.0 > > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098990#comment-16098990 ] Jiangjie Qin commented on KAFKA-5630: - [~vmaurin_glispa] Thanks for reporting the issue. It looks that the issue here is that we assumed the InvalidRecordException will only be thrown when we explicitly validate the record in the Fetcher.PartitionRecords. But it can actually be thrown from the iterator as well. I'll fix that. > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin >Priority: Critical > Labels: regression, reliability > Fix For: 0.11.0.1 > > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098279#comment-16098279 ] Ismael Juma commented on KAFKA-5630: Oh, I see. That's definitely a bug, you should be getting an error. cc [~lindong], [~becket_qin] [~hachikuji] who contributed or reviewed changes in this area. > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin > Labels: reliability > Fix For: 0.11.0.1 > > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098276#comment-16098276 ] Vincent Maurin commented on KAFKA-5630: --- And I haven't noticed any other issues so far. After a check with the DumpLogSegments tool, it appears that 2 partitions where impacted both on the same topic. I had log cleaner errors for these two partitions (same as the consumer one). > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098263#comment-16098263 ] Vincent Maurin commented on KAFKA-5630: --- [~ijuma] thank you for your feedback. Regarding consumer, I have test with version 0.10.2.1 and it is actually throwing the error if calling "poll". Then it sounds fair enough to skip the record with seek. But with 0.11, I don't get any error, a call to poll just returns the same record duplicated max.poll.record. The logic then to seek for the next offsets is more complicated than reacting to the exception, it sounds for me that I have to compare records returned by poll and advance my offset if they are all equals ? Or am I misusing the client ? (It is a manual assigned partition use case, without committing offsets to kafka, I have tried to follow the recommendations in the KafkaConsumer javadoc for that) > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098254#comment-16098254 ] Ismael Juma commented on KAFKA-5630: Have you had any other issues since the rolling upgrade? > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098253#comment-16098253 ] Ismael Juma commented on KAFKA-5630: [~vmaurin_glispa], the consumer behaviour is as expected. The application should decide whether it wants to skip the bad record (via `seek`) or not. However, we should figure out if the corruption is due to a bug in Kafka. And fix it, if that's the case. > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098154#comment-16098154 ] Vincent Maurin commented on KAFKA-5630: --- It is ``` offset: 210648 position: 172156054 CreateTime: 1499416798791 isvalid: true size: 610 magic: 1 compresscodec: NONE crc: 1846714374 offset: 210649 position: 172156664 CreateTime: 1499416798796 isvalid: true size: 586 magic: 1 compresscodec: NONE crc: 3995473502 offset: 210650 position: 172157250 CreateTime: 1499416798798 isvalid: true size: 641 magic: 1 compresscodec: NONE crc: 2352501239 Exception in thread "main" org.apache.kafka.common.errors.CorruptRecordException: Record size is smaller than minimum record overhead (14). ``` > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098134#comment-16098134 ] huxihx commented on KAFKA-5630: --- Seems it's a duplicate of [KAFKA-5431|https://issues.apache.org/jira/browse/KAFKA-5431]. Could you run "bin/kafka-run-class.sh kafka.tools.DumpLogSegments" to see verify that the underlying log file does contain the corrupted records? > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098129#comment-16098129 ] Vincent Maurin commented on KAFKA-5630: --- A rolling upgrade from 0.10.2.0 has also been performed a couple of week ago. Could be a reason for the corruption problem ? > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-5630) Consumer poll loop over the same record after a CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-5630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098117#comment-16098117 ] huxihx commented on KAFKA-5630: --- Did you enable `preallocate` for the topic? > Consumer poll loop over the same record after a CorruptRecordException > -- > > Key: KAFKA-5630 > URL: https://issues.apache.org/jira/browse/KAFKA-5630 > Project: Kafka > Issue Type: Bug > Components: consumer >Affects Versions: 0.11.0.0 >Reporter: Vincent Maurin > > Hello > While consuming a topic with log compaction enabled, I am getting an infinite > consumption loop of the same record, i.e, each call to poll is returning to > me 500 times one record (500 is my max.poll.records). I am using the java > client 0.11.0.0. > Running the code with the debugger, the initial problem come from > `Fetcher.PartitionRecords,fetchRecords()`. > Here I get a `org.apache.kafka.common.errors.CorruptRecordException: Record > size is less than the minimum record overhead (14)` > Then the boolean `hasExceptionInLastFetch` is set to true, resulting the test > block in `Fetcher.PartitionRecords.nextFetchedRecord()` to always return the > last record. > I guess the corruption problem is similar too > https://issues.apache.org/jira/browse/KAFKA-5582 but this behavior of the > client is probably not the expected one -- This message was sent by Atlassian JIRA (v6.4.14#64029)