Skip to content
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

Closed
3 of 7 tasks
shanson7 opened this issue Sep 10, 2021 · 11 comments
Closed
3 of 7 tasks

Significant performance regression between 1.6.1 and 1.7.0 #3538

shanson7 opened this issue Sep 10, 2021 · 11 comments
Milestone

Comments

@shanson7
Copy link
Contributor

shanson7 commented Sep 10, 2021

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 to 1.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:

  • librdkafka version (release number or git tag): 1.7.0
  • Apache Kafka version: <REPLACE with e.g., 0.10.2.3>
  • librdkafka client configuration: queue.buffering.max.messages=1024000, queue.buffering.max.kbytes=1024000 , queue.buffering.max.ms=1000
  • Operating system: RHEL 7
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue
@shanson7
Copy link
Contributor Author

I built both branches via:

git checkout v<version>
./configure
make

And ran

time ./examples/rdkafka_performance -P -t <topic> -b <remote broker> -X 'queue.buffering.max.messages=1024000' -X 'queue.buffering.max.kbytes=1024000' -X 'queue.buffering.max.ms=1000' -z 'snappy' -s 80 -c 1000000

to replicate our production. The results were:

Branch Wall time User Time System Time msgs/sec
v1.6.1 32.5s 2.5s 29.6s 30.8k
v1.7.0 31.6s 2.8s 41.6s 31.65k

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.

@edenhill
Copy link
Contributor

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.

@shanson7
Copy link
Contributor Author

Did this miss the v1.9.0 release?

@tgruben
Copy link

tgruben commented Mar 17, 2022

has v1.9.0 been released?

@edenhill
Copy link
Contributor

edenhill commented Apr 4, 2022

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

Branch Wall time User Time System Time msgs/sec
v1.6.1 26s 35s 0.5s 1.9M
v1.8.2 28s 43s 7s 1.72M
qlatency branch 22s 34s 0.6s 2.23M

I'll push the branch soon, it'd be great if you could all try it out on your workloads.

@edenhill
Copy link
Contributor

edenhill commented Apr 5, 2022

Please see #2912 (comment)

@edenhill
Copy link
Contributor

edenhill commented Apr 7, 2022

@shanson7 Please try out the qlatency branch, if possible. Thanks

@edenhill edenhill closed this as completed Apr 8, 2022
@shanson7
Copy link
Contributor Author

shanson7 commented Apr 8, 2022

@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!

@shanson7
Copy link
Contributor Author

shanson7 commented Apr 8, 2022

Ran rdkafka_performance a few times with the various branches (against a real kafka cluster) and I get fairly consistent results with below averages:

Branch Wall time User Time System Time msgs/sec
v1.6.1 19.3s 18.4s 1.59s 1.09 Million
v1.7.0 16.2s 20.7 2.8s 1.24 Million
qlatency 17.4s 19.3s 0.96s 1.22 Million

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.

@edenhill
Copy link
Contributor

edenhill commented Apr 8, 2022

Thank you @shanson7 , much appreciated! (and relieving)

@shanson7
Copy link
Contributor Author

shanson7 commented Apr 8, 2022

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

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

3 participants