-
Notifications
You must be signed in to change notification settings - Fork 1.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
High Rate Producer Corruption With Large Messages #962
Comments
This is weird. The error message isn't the greatest, but it looks like Kafka is sending us back a newer message format than we can handle. I'm not sure if this is a Kafka bug or if we're somehow indicating that we support that version accidentally, or... What's even weirder is that this version switch seems to be triggered by message size. Is there any difference in the broker-side logs between the two cases? |
I'm not an expert Kafka log reader, but I think the logs are the same. I've attached them below. What's also weird is that adding the Sleep is another way of fixing the problem. Sending messages of size 3000:
Sending messages of size 2000:
|
The logs look the same to me too. The fact that it can be fixed by reducing the size or the frequency makes me think it's some sort of volume-of-bytes trigger. If Kafka accumulates enough data in a short enough period of time maybe it is falling back to a different internal format and that's leaking out in the API somehow? I'm kind of guessing. To figure out why Kafka is changing the format I think you'll have to ask upstream. The obvious solution to this right now is for Sarama to implement the new message format (documented at https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-Messagesets) so that if Kafka sends us that we can just read it. I know @wladh mentioned he was working on it (#901 (comment)) but I don't know how far he got. |
Thanks for your response. I did some more digging (I didn't know about the new message format introduced in 0.11.0) and I have found a work-around. Add the following line to server.properties:
And it all seems to work. |
Good to know. The other ticket is tracking the implementation of the message format in Sarama, so I don't think there's anything else to do here. |
Just wanted to note that the Kafka 0.11 support in #901 doesn't seem to fix this. I was getting the same error reported here, first with sarama c765017, and again with sarama 1.14. For some reason, v1.11.0-44-g3efb95d (which is what I've been using up to now) doesn't seem to be affected.
I noticed #990, which looked related, so I tried the master version of sarama. With master, it's still not working, but the error message has changed:
Please let me know if I should file a new issue to debug this. The Kafka version is 0.11.0.1. I don't have direct access to the broker's configuration, but I can dig it up if necessary. cc @wladh |
This sounds like the message is corrupted somewhere. |
I managed to capture the packet causing this. I can't share the full packet, but I can share the header. This is the base64 representation:
It's followed by 32661 bytes of payload. In case it's significant, it looks like the payload is not a full message (it's in JSON format and the JSON object isn't complete). |
I find the hex-dump version easier to read. Hopefully this converted from base64 properly:
|
(FWIW, Wireshark does understand the kafka protocol. If you have the original pcap and could share a screenshot of the decoded header that might save us some headache manually lining up the bytes) |
Assuming these are |
I am not explicitly setting a max fetch limit anywhere, so it should be using the defaults. |
What Kafka version are you using? I've tried producing a large (80KB) message on Kafka 0.11.0.1 and it doesn't even need to retry because despite having the 32KB per partition limit the broker sends the entire message. |
I am also using Kafka 0.11.0.1. Does it looks like the root cause of the problem is the broker truncating the message to fit in the 32 kb limit? Would it be useful to try setting the It's curious that this problem doesn't occur with v1.11.0-44-g3efb95d. When I switch back to that version and try to consume the same topic, it seems to work fine. |
This is what I could infer from the packet fragment and seems to match the fact that you saw that JSON object was truncated. You can try increasing One change that looks relevant between |
Setting I am setting |
Setting |
Given you are actually using kafka 0.11, why are you not just setting |
I'm using Until now, I didn't know there was any value in changing this default. The reason I used 0.10.2.0 in my workaround is that sarama-cluster hasn't updated its dependencies for sarama 1.14. I can manually copy in sarama 1.14 for testing, but |
But it seems there's something fishy about having version |
@wladh even if your client sets an earlier version, kafka can send back v2 records in protocol versions before 0.10.1.0, unless your fetch size is set to larger than the record batch. This is noted in http://kafka.apache.org/documentation.html#upgrade_11_message_format |
There's a change in behaviour in between different API version in Kafka. So, for v0 of The sarama version you said doesn't have the issue is not checking that version of message is less than 2, so it will try to decode the record batch as a message set and will likely end up trying to interpret last byte of last offset delta (which would be always 0) and first 3 bytes of first timestamp (which in your config is -1) as key and will try to read 0x00FFFFFF bytes of data as the key which would fail with insufficient data and the retry mechanism will start and eventually the request limit will be big enough that the whole message will go through and will be downgraded correctly and then correctly consumed by sarama. PR #940 introduced the check that magic is < 2 (before support for records was introduced). In (edit: I just saw that @tcrayford mentioned the limitation as well) |
Versions
Please specify real version numbers or git SHAs, not just "Latest" since that changes fairly regularly.
Sarama Version: 1.13.0
Kafka Version: kafka_2.11-0.11.0.1
Go Version: 1.9
Configuration
What configuration values are you using for Sarama and Kafka?
Default installation for Kafka, except that
delete.topic.enable=true
. One instance.Problem Description
Run the test producer at https://gist.github.com/wingedpig/c635e77fcd42d95cfa7923a541724f6b. It creates a test partition and fills it with 15 messages. Then run the test consumer, at https://gist.github.com/wingedpig/6d4999591fc305b683b9fd7cbeaed240, which tries to read the partition. It fails with the following errors:
If you delete the test partition, uncomment the Sleep line in the test producer, and re-run the test, the consumer is able to read the messages without issue.
If you delete the test partition, change the size of the message in the producer (change the 3000 to 2000 in line 50), and re-run the test, the consumer is able to read the messages without issue.
I am new to Kafka, so I wouldn't be surprised if I'm doing something wrong. But I haven't been able to figure out what.
The text was updated successfully, but these errors were encountered: