[ 
https://issues.apache.org/jira/browse/KAFKA-13953?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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: 100002 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 *.*.*.*:<port> 
> --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)

Reply via email to