-
Notifications
You must be signed in to change notification settings - Fork 798
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
Panic when reading message from start of compacted topic #874
Comments
Hello @GFriedrich, thanks for reporting the issue! This is not something we have observed before but it seems like a legit issue from what you reported. If you are able to next time, it would be really useful if you are able to provide us with the state of the partition that triggered the bug (I understand this might be difficult depending on the kind of data that you store in Kafka). |
hi @achille-roussel, |
Thanks for pointing out #551, this seems like a mistake indeed, we'll address 🙇 |
Thanks @achille-roussel |
That's entirely possible, addressing the panic there seems like the right thing to do tho (we shouldn't trust data coming from the network), and not crashing the application might give us more opportunities to figure out the conditions that triggered the issue. |
We have also run in to this panic on a compacted topic, though using a consumer group, and are able to reliably have it happen. The reader gets to offset 3330 in this case and then panics. This is from a
every offset before this was successfully fetched/committed. This does not seem to necessarily stop the consumer from being able to consume from a later offset though. I had checked a bunch of offsets between 3300 and 3692 to see if it was ever able to continue, and it did at 3392 (unsure if there are more in between, but it was panicing at 3691). The following are the offsets around that 3692:
Unsure if this is helpful information, but if there is something else I can get to help debug this, let me know. |
@achille-roussel today I was finally able to debug the situation on one of our Kafka clusters. The application is still on version 0.4.30 - so all the line numbers you'll see, are matching to this specific version: As said before: The wild thing is that due to the Important to note here: I found that empty batches are only available from Kafka message v2 on. You can find the corresponding Kafka part at https://github.com/apache/kafka/blob/a3a4323a5a2ea4ba1820da58a8c31532e8864f0d/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java#L219 So what to do: I think if these points are tackled, it should be possible to fix all of the previously mentioned deserialization issues. Let me know if you need any more details - I've kept all the binary response data in case it is needed once again. Unfortunately I can't release it here, as it contains some private data, but I hope the given information is already enough to reproduce the situation. |
Please, fix this bug ASAP! |
@GFriedrich thanks for the detailed bug report. @sergeylarionov We always welcome contributions, would you have free cycles to help submit a pull request which fixes the bug? |
We had a similar issue (not the first time sadly), but this debug is very very good. I was checking on the code a little bit, but I am not that familiar with every detail so this might be wrong, but a possible fix could be just to add a check here https://github.com/segmentio/kafka-go/blob/v0.4.30/message_reader.go#L247 on the I am not sure, but I think that should go back to the lib's caller which can handle it however it wants? |
Here is a stack trace from v0.4.39
|
With magic v2 and above, it's possible to have empty record batches. This was previously not considered and could lead to panics because the code relied on the record count being non-zero in multiple places. Fixes segmentio#874
With magic v2 and above, it's possible to have empty record batches. This was previously not considered and could lead to panics because the code relied on the record count being non-zero in multiple places. Fixes segmentio#874
With magic v2 and above, it's possible to have empty record batches. This was previously not considered and could lead to panics because the code relied on the record count being non-zero in multiple places. Fixes segmentio#874
Hi, I've started using kafka-go package and this error keeps occuring. To which version should I downgrade to temporarily get around it until this gets fixed ? |
With magic v2 and above, it's possible to have empty record batches. This was previously not considered and could lead to panics because the code relied on the record count being non-zero in multiple places. Fixes segmentio#874
we are also hitting this. Any progress? This is almost 9 months old. |
@PaulForgey-Discovery I've migrated to sarama and so far all good! |
I've migrated to the official Go client from Confluent |
Describe the bug
When trying to read from a compacted topic using the
FirstOffset
configuration without a consumer group, the library panics with the following stacktrace.Kafka Version
2.4.0
To Reproduce
Sadly I'm unable to reproduce the issue. But maybe you've seen the issue already in the past or you can point me to a place what I could check.
To be exact I found the application to panic even when restarting it over and over again. My only solution was to truncate the topic, which then brought the consumer back to life. But these messages were not special at all because I've exported them and reimported them to a different cluster and couldn't make the application fail with the other cluster. So it must be (additionally?!) connected to some internal state of the Kafka cluster.
Expected behavior
No panic at all should happen.
Additional context
Used version of the library is 0.4.30
The text was updated successfully, but these errors were encountered: