-
Notifications
You must be signed in to change notification settings - Fork 3.2k
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
Significant performance regression between 1.6.1 and 1.7.0 #3538
Comments
I built both branches via:
And ran
to replicate our production. The results were:
So while in wall time the throughput is higher in 1.7.0 the total CPU usage for delivering 1 million messages goes from 32.1s to 44.4s (~38% increase). I assume this is only exacerbated in a multi-threaded environment. |
Huhm, yes, the fix in #2912 was primarily aimed at low linger.ms use-cases and I think I can see how the removal of the rate-limiting is less than optimal for high ligner.ms use.cases. I think we need to bring back the wakeup timestamp-based rate-limiter, but instead of using the final target queue's rate (rkb_ops queue), the timestamp must be maintained per input queue (partition message queue). I have some ideas I need to explore. |
Did this miss the |
has v1.9.0 been released? |
I've revamped the q wakeups on a private branch and these are the current perf numbers compared to v1.8.2 and 1.6.1. Instead of using a real cluster it is utilizing the builtin mock cluster to cut out broker IO noise. time ./examples/rdkafka_performance -P -t test -X test.mock.num.brokers=2 -b boo -X 'queue.buffering.max.messages=1024000' -X 'queue.buffering.max.kbytes=1024000' -X 'queue.buffering.max.ms=1000' -z 'snappy' -s 80 -c 50000000 -Y 'jq .brokers[].wakeups' -T 1000 50M messages, 4 partitions, 2 mock brokers
I'll push the branch soon, it'd be great if you could all try it out on your workloads. |
Please see #2912 (comment) |
@shanson7 Please try out the qlatency branch, if possible. Thanks |
Awesome! I'll try to run a test against our brokers today. This looks really promising, thanks! |
Ran
These tests aren't as dramatic as my previous runs (unfortunately I don't have access to the same hardware) so I'm not entirely sure we can rely on them. However, it does seem like a considerable improvement over the 1.6.1 branch in terms of throughput and a reasonable improvement over 1.7.0 in terms of CPU. |
Thank you @shanson7 , much appreciated! (and relieving) |
I should also note that 1.6.1 also consistently had a few hundred produce errors and small backpressures that I didn't see in the qlatency branch |
Read the FAQ first: https://github.com/edenhill/librdkafka/wiki/FAQ
Description
We have a producer that calls
produce
about 250k times/sec with very small messages (about 60-80 bytes). Going from 1.6.1 to 1.7.0 we saw CPU usage jump by about 90% (nearly doubled!) and produce throughput drop significantly (likely due to CPU limits on the process). I believe this is due to the wakeup changes in #2912. This change would mean we need to double the number of CPUs ffor our producer (which is a non-trivial number!). For now, we are pinning back to1.6.1
.How to reproduce
Still working on simple repro cases, however I think that increased wakeups are expected and so CPU increases. However this workload is CPU sensitive and latency being in the hundreds to low thousands of milliseconds is acceptable so long as throughput is high and CPU is reasonable.
Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
<REPLACE with e.g., 0.10.2.3>
queue.buffering.max.messages=1024000
,queue.buffering.max.kbytes=1024000
,queue.buffering.max.ms=1000
RHEL 7
debug=..
as necessary) from librdkafkaThe text was updated successfully, but these errors were encountered: