-
Notifications
You must be signed in to change notification settings - Fork 101
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
Stuck partitions - question about the pause/resume logic #1230
Comments
Just curious because we are having a similar issue, which version of fs2-kafka are you on? We are on 2.6.1, wanted to know if this was also happening with 3.x |
Just witnessed the issue on v3.0.1 |
Thanks for the reply. That's unfortunate. I was hoping upgrading would be sufficient. |
We operate an internal service that runs a few thousand streams and we have also experienced the "lost partition" issue. The issue seems to be a race condition, and happens reliably for topics with higher volume of traffic: we see a couple of "partition lost" events every day in topics with higher volume of traffic. We haven't so far identified the source of this race condition. After spending some time trying to identify the root cause, we settled on a workaround where we monitor consumer offset lag for each stream, and recreate/restart consumers that get stuck for a few minutes. This worked out well in practice, and allowed us to ignore this issue for a while. Mainly triggered by the reporting of the current issue, as I was trying to gather some data to share, and getting ready to dive once more in search for the race condition, a colleague pointed to some comments (I don't have a reference handy) that relate the lost partition issue to the use of Are you using We ran some experiments in our staging environment, removing From our side, we already have a stream that mixes clock ticks and message chunks so we're looking at replacing |
Thank you for the reply. Unfortunately no use of |
Do you use |
Yes! I will look into this, thank you for the lead |
Thanks for the hint @biochimia. Since we seemed to have encountered this issue when moving from FS2 Kafka 2 to 3, I've just been doing some sleuthing. This change was made to groupWithin a couple of years ago. Before that was this issue which has an interesting comment from @SystemFw
I’m not convinced I understand how behaviour around commits would prevent fetching new records though, my naive model has them as entirely separate "threads", and committing offsets just minimizes the amount of records you re-process when starting up a stream again - you could in theory run a stream for hours/days not making any commits. Anyway… I was thinking about a similar workaround that involves these steps:
|
Regarding
Tangentially, the release notes for cats-effect 3.5.0 mention an interface/behaviour change where the legacy behaviour could "result in an effect which could be canceled unconditionally, without the invocation of any finalizer". There's no smoking gun or indication that this addresses the issues we have seen, but somehow it seems like it could be related. We're looking into testing this too, but for us it involves updates of various dependencies across various components. https://github.com/typelevel/cats-effect/releases/tag/v3.5.0 |
I don't think we tried the
|
Unfortunately, our current workaround is getting entangled in the problem as we are seeing some errored streams getting stuck and not being able to recover. This may be more related to the cancelation/finaliser changes in cats-effect 3.5, and changes being made by various libraries to adapt to it... Anyway, at this point I'm happy to share the information we have in case it helps someone make a connection. |
Thanks. Wasn't aware of this one. |
An update from our side:
Unfortunately, we still see dropped partitions with the updated stack. The only good news is that we seem to be reliably restarting those again, so the updated dependencies seem to have addressed the issue where our restart mechanism would get itself stuck. This could be related to the fix in typelevel/cats-effect#3444. I can't rule out that there are other bugs in our code and in the stack, but perhaps the main trigger here could be CPU starvation or a similar bad behaviour by our application such as "expensive" tasks that work against the cats-effect scheduler. At this moment, I no longer have an indication about a specific construct being the trigger for this issue. |
Update from us. We tried implementing the timed pull approach (described here: https://medium.com/@ivovk/fs2-streams-detecting-stream-idleness-aec12af06936). Unfortunately, this didn't work. It appears that when the partition becomes "stuck" the stream stops being pulled (as our timer never fires again). |
Hey. Just as an update, we've not seen stuck partitions for a month now. We have since updated various libraries (Cats, FS2, FS2-Kafka) so our assumption is one of these fixed the issue - or it was due to some issue at the broker side which has since been resolved. |
That's awesome to hear. Just curious, which version of Kafka is your broker on? Would love to be able to compare once we do some upgrades. |
Just to say, we've had reoccurrences of this. Luckily, they're rare, and mostly in non-prod environments, but still. @jarrodcodes our Kafka broker version is 3.4 (soon to be 3.5 apparently, so will be interested to see if that changes anything) |
I'm having the same issue. Chunks pulled from kafka reports as size 0 after some time. It is not occuring in our test env with 8 partitions/4 consumers, but is happening quite frequently in production on 24 partitions/4 consumer replicas. Metrics are reported downstream in forked threads (based on |
I've read the whole thread and I must say that this is really unfortunate 😞 I mean the fact that you folks have been workaround this by restarting the consumers... I'd like to investigate this bug in depth. Just to be sure, these partitions stuck happen without a rebalance, am I right? If that's the case, it will be easier to trace the issue 🙏🏽 |
Thanks @aartigao, in our case this mostly happens in test environments at some point after a deployment (we've actually moved to using sticky partition assignor to mitigate other issues, but doesn't seem to prevent this issue). What we see happening is some time after a deployment, when some messages finally start arriving, the lag starts going up - and never goes down. Usually only for one of the partitions (we're consuming from one topic with 50 partitions), sometimes more. As described above if we simply restart the services it comes back to life. |
Hey.
We have a topic with 50 partitions, with consumers scaled out to a relatively small number of hosts (3). Sporadically, we get a situation where one particular partition gets stuck and does not commit any new offsets. As soon as we restart the hosts, it kicks back into life and immediately catches up.
We see this in the logs very frequently after it becomes stuck:
Skipping fetching records for assigned partition <mypartition> because it is paused
The last couple of FS2 Kafka logs mentioning the stuck partition, before it was stuck, say this:
Completed fetches with records for partitions [ <mypartition> -> { first: 45677489, last: 45677495 }, .... ]
Followed by logs that never mention the partition again:
Current state [State(fetches = Map(... a number of other partitions, but not <mypartition> ... ), ...)]
In trying to figure out what's going on, I ended up looking at this bit of the code:
Filling in all the possible combinations, I think we get these possible outcomes:
I've highlighted the state that I think we're in, and, given that the java kafka library is saying it will skip fetching for paused partitions, I'm a little confused how the partition is then expected to resume.
If I pull down the FS2 Kafka codebase, and replace the
intersect
withunion
the KafkaConsumerSpec tests still pass, and it would cause our highlighted scenario to be resumed (i think), but I don't know what the consequences of that would be!It feels like this should be a more widespread issue given how central this code is, and how long it's been like this for, so I bet I'm missing something.
Thanks
The text was updated successfully, but these errors were encountered: