-
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
CPU increase from 250% to 360% after upgrade from 0.11.6 to 1.1.0 #2450
Comments
Can you provide more detailed profiling? |
Sure. What should I do? |
I dont know what profiler you are using, but see if it can give you per-line cpu usage of the top functions. |
I compiled librdkafka with --disable-optimization. And I use 'perf top -g', I found something strange.
But in the old version 0.11.6, is like this:
These function appear in version 1.1.0 |
version 1.0.0 also has the same issue. It seems something changed between 0.11.6 and 1.0.0. |
@edenhill anything else I can do? |
Thank you for the profiling breakdown. |
I'm all for removing locks and atomic ops if possible :D |
Hi @edenhill , I have upgraded to version 1.2.1. The CPU increase to 400%. The hot function changed to "rd_kafka_msg_cmp_msgid". 35.76% librdkafka.so.1 [.] rd_kafka_msg_cmp_msgid |
Oh no. Can you provide the perf tree with perf top -g? What is your produce rate? |
The perf tree is
The produce rate is 350000 message/sec. Apache Kafka version: 1.1.1 topic configuration:"request.required.acks": "1", "request.timeout.ms": "30000", "partitioner": "murmur2_random" I run my application with "debug":"all". The logs are all like follow, it seems normal to me. %7|1570841036.272|SEND|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Sent ProduceRequest (v7, 40523 bytes @ 0, CorrId 4317) |
Thank you, I'm trying to reproduce this but while the CPU usage is high I don't see the cmp_msgid topping the perf list. |
Is there any test I can do? |
I disabled optimization of librdkafka by setting --disable-optimization.
|
Thank you @eelyaj That callstack is https://github.com/edenhill/librdkafka/blob/master/src/rdkafka_broker.c#L3319 , which is moving messages from the partition's produce() queue to the partition's xmit queue. Specifically the |
hello @edenhill I found 'rd_kafka_msgq_find_pos' function was modified in 1.2.1. Maybe something wrong with it. |
Not all X are 0 in my log. ugw_ufdr [141] 983 message(s) in xmit queue (11 added from partition queue) |
I think I've found an issue, let me push a branch with a proposed fix which you can try out |
The common case where the source queue is simply appended to the destination queue was not optimized, causing O(n) scans to find the insert position. Issue introduced in v1.2.1
Can you try out the |
The 'rd_kafka_msg_cmp_msgid' issue is fixed, but the CPU usage is still a little bit high than 1.2.0, around 390%. The perf stack is as follow.
|
Are you producing 220k msgs/s, or are you producing at full speed and 220k msgs/s is as high as it gets? Is it consistently at 390% CPU while 1.2.0 was consistently at 360%? The perf stacks look okay (expected) now |
I tested 1.2.0 again. CPU usage is the same with issue2450 branch now, between 370%~410%. I only producing 220k msgs/s. It can reach 650k msgs/s in 0.11.6 and 500k msgs/s in 1.1.0. This is the first issue. There are still some lock in libradkafka 1.1.0+. |
Trying to reproduce your tests with v1.2.1 with rdkafka_performance I get around 5-15% CPU usage per librdkafka thread (across the 3 broker threads), and rdkafka_performance hogging one CPU at 100%, using the command: |
I tried this command in several version. Here is the result. 0.11.6 got the best performance. librdkafka-0.11.6CPU usagePID USER PR NI VIRT RES SHR S %CPU %MEM TIME+ COMMAND log% ... 160422 msgs/s and 16.04 MB/s, 0 produce failures, 117733 in queue, lz4 compression librdkafka-1.2.0CPU usagePID USER PR NI VIRT RES SHR S %CPU %MEM TIME+ COMMAND log% ... 98897 msgs/s and 9.89 MB/s, 0 produce failures, 75820 in queue, lz4 compression librdkafka-1.2.1CPU usagePID USER PR NI VIRT RES SHR S %CPU %MEM TIME+ COMMAND log% ... 114725 msgs/s and 11.47 MB/s, 0 produce failures, 71923 in queue, lz4 compression librdkafka-issue2450CPU usagePID USER PR NI VIRT RES SHR S %CPU %MEM TIME+ COMMAND log% ... 98409 msgs/s and 9.84 MB/s, 0 produce failures, 72880 in queue, lz4 compression |
@eelyaj performance runs:
CPU is per-broker-thread CPU @edenhill performance runs: Intel(R) Core(TM) i7-2600 CPU @ 3.40GHz, 8 cores
|
@eelyaj Since your producer can't even reach the given produce rate of 230k msgs/s (21MB/s), I think your system is just underpowered. And make sure to run your performance tests with optimizations enabled (no --disable-optimization) |
I am using virtual machine to run these tests. processor : 0 |
I use the default config to compile librdkafka. command is ./configure && make && make install |
Virtual machines are not suitable for performance testing |
My application is running on virtual machines. I need to test it on VM. |
Then it sounds like you need to beef up your machine specs |
Can you share your linux version, gcc version and libc version? |
|
I think the best test is checking the sustained speed using the same message size and params on the same machine. I haven't really tested 0.11 because we decided early that we want idempotence, but without it 1.x was clearly faster than with it for the obvious reasons. I did saw a slight perf drop after 1.2.1 but didn't had much time to investigate it as the machine was booked for other stuff, good thing that it will be fixed. |
On the other hand my method probably wouldn't have noticed the drop except when I pack multiple broker threads on the same core. |
Yeah, we'll push a new release soon |
The common case where the source queue is simply appended to the destination queue was not optimized, causing O(n) scans to find the insert position. Issue introduced in v1.2.1
The different performance between 0.11.6 and 1.1.0 caused by old version of GCC. I upgraded GCC from 4.8.5 to 7.3.0. Problem sovled. |
Description
I upgraded librdkafka from 0.11.6 to 1.1.0. I found that the CPU increased from 250% to 360%.
The 'perf top' shows the cpu of 'rd_kafka_toppar_producer_serve' is around 10%.
perf top result in librdkafka 1.1.0:
10.00% librdkafka.so.1 [.] rd_kafka_toppar_producer_serve
6.59% libc-2.17.so [.] __memcpy_ssse3
5.02% [kernel] [k] _raw_spin_unlock_irqrestore
4.90% libpthread-2.17.so [.] pthread_mutex_lock
4.54% UfdrUdpCollector [.] PbMessage::serialize
but in librdkafka 0.11.6, the perf top result shows:
9.42% libc-2.17.so [.] __memcpy_ssse3
6.72% UfdrUdpCollector [.] PbMessage::serialize
5.60% liblz4.so.1.8.0 [.] LZ4_compress_fast_extState
4.65% UfdrUdpCollector [.] std::_Hashtable<UfdrJoinKey, std::pair<UfdrJoinKey const...
2.88% [kernel] [k] _raw_spin_unlock_irqrestore
2.82% libc-2.17.so [.] __memset_sse2
Maybe there are something wrong with the function 'rd_kafka_toppar_producer_serve'.
How to reproduce
Checklist
Please provide the following information:
0.11.6
1.1.1
"queue.buffering.max.ms": "1000", "queue.buffering.max.messages": "1000000","queue.buffering.max.kbytes": "2048000","batch.num.messages": "10000","compression.codec": "lz4","socket.send.buffer.bytes": "3200000","socket.receive.buffer.bytes": "3200000","message.max.bytes": "209715200"
<Centos 7.2 (x64)>
debug=..
as necessary) from librdkafkaThe text was updated successfully, but these errors were encountered: