fix: stuck on the batch with zero records with long gap#3150
fix: stuck on the batch with zero records with long gap#3150Sergey-Belyakov wants to merge 1 commit intoIBM:mainfrom
Conversation
Signed-off-by: Belyakov Sergey Sergeevich <[email protected]>
252a40b to
b2a6cdf
Compare
|
@Sergey-Belyakov thanks for the PR, this is interesting! Will take a look to see what the Java client behaviour is in this scenario to ensure we align |
|
This bug has hit us multiple times in production - can it receive a review? (PR works for us when we apply it locally) |
|
@kgatv2 what is your backend cluster? We just need to be clear why we think the FetchResponse that isn't end-of-topic, has no error and no records – to ensure we're not just papering over some corruption in a third-party broker implementation and it is a genuine Apache Kafka broker behaviour (perhaps due to aborted transactions?) that causes these responses. I guess I'm nervous about tossing in |
|
We are hosting our kafka topics on Aiven - https://aiven.io/kafka. Using 4 partitions for the topic that got "corrupted". |
|
Hi, sorry for my English. I deleted my previous comment because I wasn't completely sure, but now I think I'm sure of what I'm saying. It looks like we've encountered issue KAFKA-5443 I think the problem is how exactly Let's see how it is defined: Line 153 in aa1a2c5 Now let's on recordsOffset function Lines 186 to 199 in aa1a2c5 That is, in LastRecordsBatchOffset we store first offset of the records batch. This is exactly the implementation in other libraries Let's see how it's done in the franz-go library: And now let see how it's done in the Java library: I think the I just got this bug on our cluster.
FirstOffset = 1273344 We keep sending the request with LastRecordsBatchOffset + 1 = 1273345 over and over, and we keep getting the same result every time. But if we modify the recordsOffset function the way I described above, we no longer get stuck at that part of the code. @dnwe What do you think about this? |
|
@sterligov thanks this is good analysis. I think you’re probably right and the original fix attempted under #2057 was put in the wrong place and should have used your delta increment rather than a +1 Are you happy to draft a PR up with your proposed fix (removing the old #2057 one) so we can run it through the FV? |
This pull request addresses the known issue described in [KAFKA-5443](https://issues.apache.org/jira/browse/KAFKA-5443), which was initially analyzed in [#2053](#2053). A prior attempt to fix it was made in [#2057](#2057), but the fix was applied incorrectly and did not fully resolve the root cause. I’ve provided a more detailed explanation of the underlying issue and the reasoning behind this fix in [this comment](#3150 (comment)). --------- Signed-off-by: Sterligov Denis <[email protected]> Signed-off-by: Dominic Evans <[email protected]> Co-authored-by: Sterligov Denis <[email protected]> Co-authored-by: Dominic Evans <[email protected]>
|
closing in favour of merged #3221 – thanks for raising the issue! |

Hi! We found a bug. If a repeated fetch with an increased offset has returned the void again, then fetching is constantly trying to request a batch with the same offset. It is always necessary to increase the offset in such cases.