kafka-jira mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Ari Uka (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (KAFKA-6679) Random corruption (CRC validation issues)
Date Mon, 19 Mar 2018 21:27:00 GMT

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

Ari Uka edited comment on KAFKA-6679 at 3/19/18 9:26 PM:
---------------------------------------------------------

So the records seem to be v2, in this case, there were 5 records. This is what the header
looked like:

`
 baseOffset: 17360912 lastOffset: 17360916 baseSequence: -1 lastSequence: -1 producerId: -1
producerEpoch: -1 partitionLeaderEpoch: 27 isTransactional: false position: 267886748 CreateTime:
-1 isvalid: true size: 762 magic: 2 compresscodec: NONE crc: 3599616919`

so I dumped this via `hd`, the hex dump of the header looks like this:

 `hd -s 267886748 -n 762 -C 00000000000016325357.log`

 

{{0ff78cb2 00 00 00 00 01 08 e7 ef 00 00 00 c6 00 00 00 1b |................|}}
{{ 0ff78cb2 00 00 00 00 01 08 e7 ef 00 00 00 c6 00 00 00 1b |................|}}
{{ 0ff78cc2 02 6e c1 b7 8d 00 00 00 00 00 00 ff ff ff ff ff |.n..............|}}
{{ 0ff78cc2 02 6e c1 b7 8d 00 00 00 00 00 00 ff ff ff ff ff |.n..............|}}

Is it normal for the CRC and magic portion to be duplicated like that? 

 

 


was (Author: ari6123):
So the records seem to be v2, in this case, there were 5 records. This is what the header
looked like:

`
 baseOffset: 17360912 lastOffset: 17360916 baseSequence: -1 lastSequence: -1 producerId: -1
producerEpoch: -1 partitionLeaderEpoch: 27 isTransactional: false position: 267886748 CreateTime:
-1 isvalid: true size: 762 magic: 2 compresscodec: NONE crc: 3599616919`

so I dumped this via `hd`, the hex dump of the header looks like this:

 `hd -s 267886748 -n 762 -C 00000000000016325357.log`

 

0ff78cb2 00 00 00 00 01 08 e7 ef 00 00 00 c6 00 00 00 1b |................|
 0ff78cb2 00 00 00 00 01 08 e7 ef 00 00 00 c6 00 00 00 1b |................|
 0ff78cc2 02 6e c1 b7 8d 00 00 00 00 00 00 ff ff ff ff ff |.n..............|
 0ff78cc2 02 6e c1 b7 8d 00 00 00 00 00 00 ff ff ff ff ff |.n..............|

Is it normal for the CRC and magic portion to be duplicated like that? 

 

 

> Random corruption (CRC validation issues) 
> ------------------------------------------
>
>                 Key: KAFKA-6679
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6679
>             Project: Kafka
>          Issue Type: Bug
>          Components: consumer, replication
>    Affects Versions: 0.10.2.0, 1.0.1
>         Environment: FreeBSD 11.0-RELEASE-p8
>            Reporter: Ari Uka
>            Priority: Major
>
> I'm running into a really strange issue on production. I have 3 brokers and randomly
consumers will start to fail with an error message saying the CRC does not match. The brokers
are all on 1.0.1, but the issue started on 0.10.2 with the hope that upgrading would help
fix the issue.
> On the kafka side, I see errors related to this across all 3 brokers:
> ```
> [2018-03-17 20:59:58,967] ERROR [ReplicaFetcher replicaId=3, leaderId=1, fetcherId=0]
Error for partition topic-a-0 to broker 1:org.apache.kafka.common.errors.CorruptRecordException:
This message has failed its CRC checksum, exceeds the valid size, or is otherwise corrupt.
(kafka.server.ReplicaFetcherThread)
> [2018-03-17 20:59:59,411] ERROR [ReplicaManager broker=3] Error processing fetch operation
on partition topic-b-0, offset 23848795 (kafka.server.ReplicaManager)
> org.apache.kafka.common.errors.CorruptRecordException: Record size is smaller than minimum
record overhead (14).
> [2018-03-17 20:59:59,411] ERROR [ReplicaManager broker=3] Error processing fetch operation
on partition topic-b-0, offset 23848795 (kafka.server.ReplicaManager)
> org.apache.kafka.common.errors.CorruptRecordException: Record size is smaller than minimum
record overhead (14)
> [2018-03-17 20:59:59,490] ERROR [ReplicaFetcher replicaId=3, leaderId=2, fetcherId=0]
Error for partition topic-c-2 to broker 2:org.apache.kafka.common.errors.CorruptRecordException:
This message has failed its CRC checksum, exceeds the valid size, or is otherwise corrupt.
(kafka.server.ReplicaFetcherThread)
> ```
>  
> To fix this, I have to use the kafka-consumer-groups.sh command line tool and do a binary
search until I can find a non corrupt message and push the offsets forward. It's annoying
because I can't actually push to a specific date because kafka-consumer-groups.sh starts
to emit the same error, ErrInvalidMessage, CRC does not match.
> The error popped up again the next day after fixing it tho, so I'm trying to find the
root cause. 
> I'm using the Go consumer [https://github.com/Shopify/sarama] and [https://github.com/bsm/sarama-cluster]. 
> At first, I thought it could be the consumer libraries, but the error happens with kafka-console-consumer.sh
as well when a specific message is corrupted in Kafka. I don't think it's possible for Kafka
producers to actually push corrupt messages to Kafka and then cause all consumers to break
right? I assume Kafka would reject corrupt messages, so I'm not sure what's going on here.
> Should I just re-create the cluster, I don't think it's hardware failure across the 3
machines tho.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message