[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17569354#comment-17569354 ] Doguscan Namal commented on KAFKA-13953: I tried to reproduce the issue by sending a negative length for a record on the produce request. I overrode the DefaultRecord.java with the following, to corrupt the fifth record in the middle of a batch. However, Broker detected that record was not valid `org.apache.kafka.common.InvalidRecordException: This record has failed the validation on broker and hence will be rejected.` Code that I changed to reproduce: ``` public static int writeTo(DataOutputStream out, int offsetDelta, long timestampDelta, ByteBuffer key, ByteBuffer value, Header[] headers) throws IOException { int sizeInBytes = sizeOfBodyInBytes(offsetDelta, timestampDelta, key, value, headers); int sizeOfSize; if ( offsetDelta == 5 ) { ByteUtils.writeVarint(-155493822, out); sizeOfSize = ByteUtils.sizeOfVarint(-155493822); } else { ByteUtils.writeVarint(sizeInBytes, out); sizeOfSize = ByteUtils.sizeOfVarint(sizeInBytes); } ... return sizeOfSize + sizeInBytes; } ``` > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17569104#comment-17569104 ] Doguscan Namal commented on KAFKA-13953: [~junrao] 1) The DumpLogSegment tool fails to output the data because it is corrupted. But here is the batch level info: baseOffset: 88062356025 lastOffset: 88062429966 count: 73942 baseSequence: 39997558 lastSequence: 40071499 producerId: 12 producerEpoch: 0 partitionLeaderEpoch: 70 isTransactional: false isControl: false position: 86385503 CreateTime: 1656027642522 size: 2543176 magic: 2 compresscodec: ZSTD crc: 2665675442 isvalid: true One more point is that, actually consumer does not receive CorruptRecordException. Broker actually doesn't give anything back to the consumer at all. The upstream message comes from a 2.8.1 KafkaCluster and data is read by a MirrorMaker 2.5.1 and written to a 2.5.1 cluster. Only one of the "sink" clusters ends up with the corrupted data. 2) DumpLogSegment is failing to provide record level information because there is a corrupted record. However, I am able to read all of the records up to the corrupted one, with negative size. So here is the structure of the log file: So all the batches are valid up to the one which contains the corrupted batch. All records in the corrupted batch is valid, until the one which is corrupted. readVarint() return -155493822 from the bytestream. ``` batch0 | record 0 | record 1 | ... batch1 | record 0 | record 1 | ... ... batchX | record 0 | record 1 | record n | bytes (-155493822) bytes (recordStart:0 attributes: 0 timestampDelta: 391 timestamp 1656027641475 offset: 88062375700 sequence:40017233 key: java.nio.HeapByteBuffer[pos=0 lim=25 cap=1199] value: java.nio.HeapByteBuffer[pos=0 lim=961 cap=1149] numHeaders: 5) ... ... NOT REACHABLE POSSIBLY NOT CORRPUTED BUT NO WAY TO VERIFY... batchNOTReachable | record... | record... ``` Given that crc check is valid, any idea on the next steps in debugging? > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17566412#comment-17566412 ] Jun Rao commented on KAFKA-13953: - [~doguscan] : It's possible that a Kafka bug caused the corruption. The CorruptRecordException in the description comes from [https://github.com/apache/kafka/blob/2.5.1/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java#L73]. This indicates a 0 size of a record batch, which is different from the negative record size in your analysis. So, we need to verify the first byte where the corruption happens. As for next steps, could you (1) run DumpLogSegment tool on the corrupted file to double check the CRC is valid? (2) attach the section of the log file including bytes from the beginning of a record batch to the corruption point? > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17566253#comment-17566253 ] Doguscan Namal commented on KAFKA-13953: * Changing the leaders to the toher replicas did not resolve the issue. Therefore I am right to assume that data is corrupted in their disk too right? ** Given that data being corrupted in the disk and replicated to the other brokers, I believe that is very unlikely to happen. ** Therefore, the data coming from the ProduceRequest, or the in memory representation of it must be corrupted. *** It verifies the batch level CRC is correct, but if the Producer set that CRC after the batch is already corrupted, then broker would not find it out right? *** Other option is that, if we convert the incoming ProduceRequest, like message down conversion, then the in memory representation of the request might have been corrupted right? * There are two corrupted areas, one is the message size field of the record, and the other one is the valueSize of the 3rd header. Bytes in between are fine. ** -155493822 is the message size ** 1991988702 as the headerValue size ** I couldn't find anything special about these numbers, what shall I check? Any idea for the next steps or how to reproduce? > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17565138#comment-17565138 ] Jun Rao commented on KAFKA-13953: - [~doguscan] : The broker verifies the batch level CRC before appending the batch to the log. So, it's more likely that the corruption happened at the storage level. Byte wise, does the corrupted area have any patterns (e.g. consecutive 0s)? Next time if this happens again, it would be useful to compare the bytes across replicas to see if the corrupted bytes are identical across replicas. > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17564575#comment-17564575 ] Doguscan Namal commented on KAFKA-13953: I put the part of the data that I was able to read. Changing the leadership of the partition did not help, so I assume that all of them are corrupted. (data retention is expired now, so unfortunately I could not verify it) * It did not corrupted in the RecordBatch but rather in the record level. I could read all of the data in that batch up to the problematic offset. * It fails in the following line [https://github.com/apache/kafka/blob/2.5.1/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java#L296-L297] . sizeOfBodyInBytes is read as -155493822, which should have been 1156. * I overrode that value for this offset and it was able to read the record data up to its 3rd header among its 5 headers, up to here [https://github.com/apache/kafka/blob/2.5.1/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java#L545] * It failed because headerValueSize was read as 1991988702, which should have been 13. I realized that `ByteUtils.readVarint` reading 5 bytes of data instead of 1. * Even overriding this headerValueSize did not help after this point. On next read headerKeySize was read as -58 and processing failed. This makes me think that Kafka only has the partial record for this offset. And although there are more records in this batch after this point none of them are accessible because this particular offset is corrupted. Q1) Could it be possible for producer to send a corrupted batch? Due to zero-copy maybe it just copied the received content to the data log file? Q2) I also see `ProduceMessageConversionsPerSec` metric for this topic. Could it be related to it i.e. message conversion? * Topic is configured to use zstd compression * Kafka version 2.5.1 Here is a data from my println when this record is being read: recordStart:0 attributes: 0 timestampDelta: 391 timestamp 1656027641475 offset: 88062375700 sequence:40017233 key: java.nio.HeapByteBuffer[pos=0 lim=25 cap=1199] value: java.nio.HeapByteBuffer[pos=0 lim=961 cap=1149] numHeaders: 5 headerValueSize: 12 capacity: 147 headerValueSize: 8 capacity: 122 headerValueSize: 1991988702 capacity: 90 > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17564386#comment-17564386 ] Jun Rao commented on KAFKA-13953: - [~doguscan] : Interesting. Do all replicas have the identical corrupted bytes? Would it be possible for you to attach the corrupted log segment for further analysis? If not, our record batch format is documented in DefaultRecordBatch. It would be great if you could help identify where the corrupted field is (e.g. at batch or record level). Thanks, > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17564370#comment-17564370 ] Doguscan Namal commented on KAFKA-13953: [~junrao] , in my case all of the replicas are corrupted. I investigated the logs using DumpLogSegments and it failed so I patched it and identified the issue is related to a corrupted record for a given offset with large sizeOfBodyBytes. - Could this be a producer issue? Does Kafka broker make any checks for the incoming records on ProduceRequest? I checked it out but only validation I found was around versioning. > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17546196#comment-17546196 ] Jun Rao commented on KAFKA-13953: - [~abasilbr] : If you have multiple replicas, it's unlikely all replicas are corrupted. You can just delete the corrupted replica without losing data. Also, is this transient? The corruption could also be caused during network transfer. > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17545837#comment-17545837 ] Aldan Brito commented on KAFKA-13953: - hi [~junrao] , there is no evidence of data corruption on the kafka broker logs, is this expected, is deleting the log segment the only way to recover the system ? – this would mean data loss right. > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17545575#comment-17545575 ] Jun Rao commented on KAFKA-13953: - [~abasilbr] : This could be caused by data corruption on the broker. You might want to use the DumpLogSegments tool on the log files in the broker to see if there is any data corruption. > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server *.*.*.*: > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (KAFKA-13953) kafka Console consumer fails with CorruptRecordException
[ https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17545271#comment-17545271 ] Aldan Brito commented on KAFKA-13953: - hi [~junrao] hi [~ijuma], could you please have a look at this issue. > kafka Console consumer fails with CorruptRecordException > - > > Key: KAFKA-13953 > URL: https://issues.apache.org/jira/browse/KAFKA-13953 > Project: Kafka > Issue Type: Bug > Components: consumer, controller, core >Affects Versions: 2.7.0 >Reporter: Aldan Brito >Priority: Blocker > > Kafka consumer fails with corrupt record exception. > {code:java} > opt/kafka/bin/kafka-console-consumer.sh --bootstrap-server 192.168.7.93:28104 > --topic BQR-PULL-DEFAULT --from-beginning > > /opt/nokia/kafka-zookeeper-clustering/kafka/topic-data/tmpsdh/dumptest > [{*}2022-05-15 18:34:15,146]{*} ERROR Error processing message, terminating > consumer process: (kafka.tools.ConsoleConsumer$) > org.apache.kafka.common.KafkaException: Received exception when fetching the > next record from BQR-PULL-DEFAULT-30. If needed, please seek past the record > to continue consumption. > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.fetchRecords(Fetcher.java:1577) > at > org.apache.kafka.clients.consumer.internals.Fetcher$CompletedFetch.access$1700(Fetcher.java:1432) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchRecords(Fetcher.java:684) > at > org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:635) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1276) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1237) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) > at > kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:438) > at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:104) > at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:78) > at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:55) > at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) > Caused by: org.apache.kafka.common.errors.CorruptRecordException: Record size > 0 is less than the minimum record overhead (14) > Processed a total of 15765197 messages {code} -- This message was sent by Atlassian Jira (v8.20.7#820007)