-
Notifications
You must be signed in to change notification settings - Fork 224
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
Need help on backpressure and auto pause / resume during high load for reactive consumer #190
Comments
Which version of |
@simonbasle I have done some analysis (with 1.3.3-SNAPSHOT). Here is what I found... Disposable flux = receiver.receive()
.doOnNext(rec -> {
System.out.println(rec);
})
.subscribe(); If I set a breakpoint on the Disposable flux = receiver.receive()
.publishOn(Schedulers.newSingle("mightHang"))
.doOnNext(rec -> {
System.out.println(rec);
})
.subscribe(); Now, the polling thread continues to poll. When the requests count down to 0 (due to the back pressure), we correctly pause the consumer. However, due to this code.... reactor-kafka/src/main/java/reactor/kafka/receiver/internals/ConsumerEventLoop.java Lines 243 to 247 in 1c67718
...we don't schedule the It looks to me like we should always reschedule, even when r is 0. When I change it to unconditionally schedule, it works as expected (keep polling while paused, returning 0 records on each poll). When I release the breakpoint, we resume the consumer as expected (I was actually testing with my PR in place). However, there is one more oddity. Initially, After forcing the back pressure, it counts down to 0 and we pause the consumer. When I released the break point and we resumed the consumer, requests went to 767, counted down to 744; then it went to 936, counting down to 744, with the latter sequence repeated over and over. |
@simonbasle I'm using |
I will try to get a fix into 1.3.3, due in a couple of weeks. |
- enhance test to verify proper resumption after auto pause - continue to schedule polls while paused Now also resolves reactor#190
- enhance test to verify proper resumption after auto pause - continue to schedule polls while paused Now also resolves reactor#190
* Fix Partition Resumes `ConsumerEventLoop.PollEvent` should not resume partitions that it, itself, has not paused. Prior to d908b81 pause/resume due to backpressure was protected by a boolean but it still unconditionally resumed all partitions. - reintroduce the boolean and only resume if we paused - keep track of user-paused partitions and exclude them from resuming * Address PR Comments - enhance test to verify proper resumption after auto pause - continue to schedule polls while paused Now also resolves #190 * Fix memory leak. Fixes #198 Only one `PollTask` needs to be scheduled at a time. Also upgrade docker image. * Use AtomicBoolean for scheduled flag. - `onRequest()` can be called on a different thread. * Address PR Comments. * Fix copyrights. * Don't emit empty `ConsumerRecords`.
Hi All,
I have a reactive kafka consumer which consumes data from a topic and pushes it to an endpoint. However, these endpoints are flaky and are error prone. I have circuit breakers which trip on failures and I induce an artificial delay
Mono.delay
to slow down the pipeline. By theory, reactive backpressure should kick in and pause the kafka consumers right? I see that If i set my max.poll.interval.ms to low number, it does not pause the consumer and it results in rebalancing. How can I make sure that the Kafka consumer applies back-pressure and pauses the consumption until the downstream messages are done processing?Here's my consumer code:
processFluxStream has
Mono.delay
code to slow down incase of circuit breaker errors.How do I go about linking the reactive pipeline to pause and resume on these signals and also avoid rebalancing issues?
I read about an earlier issue which had the same problem but that does not have a solution.
Thanks.
The text was updated successfully, but these errors were encountered: