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

CPU increase from 250% to 360% after upgrade from 0.11.6 to 1.1.0 #2450

Closed
3 of 7 tasks
eelyaj opened this issue Aug 2, 2019 · 37 comments
Closed
3 of 7 tasks

CPU increase from 250% to 360% after upgrade from 0.11.6 to 1.1.0 #2450

eelyaj opened this issue Aug 2, 2019 · 37 comments

Comments

@eelyaj
Copy link

eelyaj commented Aug 2, 2019

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

  1. install librdkafka 1.1.0
  2. clean and make my application
  3. run application

Checklist

Please provide the following information:

  • librdkafka version (release number or git tag): 0.11.6
  • Apache Kafka version: 1.1.1
  • librdkafka client configuration: "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"
  • Operating system: <Centos 7.2 (x64)>
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts
  • Critical issue
@edenhill
Copy link
Contributor

edenhill commented Aug 2, 2019

Can you provide more detailed profiling?

@eelyaj
Copy link
Author

eelyaj commented Aug 2, 2019

Can you provide more detailed profiling?

Sure. What should I do?

@edenhill
Copy link
Contributor

edenhill commented Aug 2, 2019

I dont know what profiler you are using, but see if it can give you per-line cpu usage of the top functions.

@eelyaj
Copy link
Author

eelyaj commented Aug 5, 2019

I compiled librdkafka with --disable-optimization. And I use 'perf top -g', I found something strange.
The call stack of 1.1.0 is like this:

 24.98% rd_kafka_broker_producer_serve                                                  
    18.66% rd_kafka_broker_produce_toppars                                              
       18.47% rd_kafka_toppar_producer_serve                                            
          6.21% rd_kafka_ProduceRequest                                                 
          5.11% pthread_mutex_lock                                                      
          2.68% rd_kafka_fatal_error_code                                               
             2.65% rd_atomic32_get                                                      
          1.54% rd_kafka_broker_outbufs_space                                           
             1.22% rd_atomic32_get                                                      
          0.94% pthread_mutex_unlock                                                    
          0.81% rd_kafka_msgq_insert_msgq                                               
             0.77% rd_kafka_msgq_overlap                                                
                0.60% rd_kafka_msgq_last                                                
    6.10% rd_kafka_broker_ops_io_serve                                                  
       4.94% rd_kafka_transport_io_serve                                                
          2.47% rd_kafka_transport_io_event                                             
             2.27% rd_kafka_recv                                                        
                2.22% rd_kafka_req_response                                             
                   2.22% rd_kafka_buf_callback                                          
                      2.20% rd_kafka_handle_Produce                                     
                         2.26% rd_kafka_msgbatch_handle_Produce_result                  
                            1.32% rd_kafka_dr_msgq                                      
                               1.12% rd_kafka_msgq_purge                                
                                  0.92% rd_kafka_msg_destroy                            
                            0.96% rd_kafka_msgq_set_metadata                            
          1.39% __GI___libc_poll                                                        
             1.30% system_call_fastpath                                                 
          0.90% 0xe6fd                                                                  
       0.57% __clock_gettime                                        					

But in the old version 0.11.6, is like this:

 9.82% rd_kafka_broker_producer_serve                                             
    6.91% rd_kafka_broker_produce_toppars                                         
       6.99% rd_kafka_toppar_producer_serve                                       
          6.03% rd_kafka_ProduceRequest                                           
             6.06% rd_kafka_msgset_create_ProduceRequest                          
                6.18% rd_kafka_msgset_writer_write_msgq                           
                   4.32% rd_kafka_msgset_writer_write_msg                         
                      4.22% rd_kafka_msgset_writer_write_msg_v2                   
                         2.75% rd_kafka_buf_write                                 
                            1.70% rd_buf_write                                    
                               0.69% rd_buf_get_writable0                         
                            1.03% __memcpy_ssse3                                  
                         1.09% rd_kafka_msgset_writer_write_msg_payload           
                            1.29% rd_kafka_buf_write                              
                               1.11% __memcpy_ssse3                               
                   1.08% rd_avg_add                                               
                   0.86% rd_kafka_msg_wire_size                                   
    2.91% rd_kafka_broker_serve                                                   
       2.78% rd_kafka_transport_io_serve                                          
          2.70% rd_kafka_transport_io_event                                       
             2.50% rd_kafka_recv                                                  
                2.45% rd_kafka_req_response                                       
                   2.45% rd_kafka_buf_callback                                    
                      2.44% rd_kafka_handle_Produce                    

These function appear in version 1.1.0
pthread_mutex_lock
rd_kafka_fatal_error_code
rd_kafka_broker_outbufs_space
pthread_mutex_unlock

@eelyaj
Copy link
Author

eelyaj commented Aug 5, 2019

version 1.0.0 also has the same issue. It seems something changed between 0.11.6 and 1.0.0.

@eelyaj
Copy link
Author

eelyaj commented Aug 9, 2019

@edenhill anything else I can do?

@edenhill
Copy link
Contributor

Thank you for the profiling breakdown.
Version 1.0.0 added support for idempotence which also affects the non-idempotence mode.
It seems like atomic ops are expensive on your machine, I'm looking into removing some of them when not needed.

@sarkanyi
Copy link

I'm all for removing locks and atomic ops if possible :D
Maybe even set a mode where there are no locks in producing if you are sure to call produce() from a single thread (using different instances) of rd_kafka_t for different threads. Hadn't had much time to think about it.

@eelyaj
Copy link
Author

eelyaj commented Oct 11, 2019

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
5.62% libc-2.17.so [.] __memcpy_ssse3
4.83% librdkafka.so.1 [.] rd_kafka_msgq_find_pos
4.12% liblz4.so.1.8.0 [.] LZ4_compress_fast_extState
3.24% UfdrUdpCollector [.] std::_Hashtable<UfdrJoinKey*, std::pair<UfdrJoinKey* const, S_UFDR_JOIN_CACHE*>, std::allocator<std::pair<UfdrJoinKey* const, S_UFDR_JOIN_CACHE*> >, std::__detail:
3.22% UfdrUdpCollector [.] PbMessage::serialize
1.95% [kernel] [k] _raw_spin_unlock_irqrestore
1.61% UfdrUdpCollector [.] PbMessage::setField
1.39% librdkafka.so.1 [.] rd_kafka_msg_partitioner
1.28% libc-2.17.so [.] __memset_sse2
1.26% libtcmalloc.so.4.4.3 [.] tc_deletearray_nothrow

@edenhill
Copy link
Contributor

Oh no.

Can you provide the perf tree with perf top -g?

What is your produce rate?
Are there any errors repprted?
How many topics, partitions and brokers are you producing to?

@eelyaj
Copy link
Author

eelyaj commented Oct 12, 2019

Oh no.

Can you provide the perf tree with perf top -g?

What is your produce rate?
Are there any errors repprted?
How many topics, partitions and brokers are you producing to?

The perf tree is

  • 47.26% 44.99% librdkafka.so.1 [.] rd_kafka_msg_cmp_msgid
  • 46.66% rd_kafka_msg_cmp_msgid
    • 1.65% xen_hvm_callback_vector
      xen_evtchn_do_upcall
      irq_exit
      do_softirq
      call_softirq
      • __do_softirq
  • 8.75% 0.00% [unknown] [k] 0000000000000000
  • 6.00% 0.00% libc-2.17.so [.] __libc_start_main
  • 5.93% 0.00% perf [.] 0xffffffffffc7edc3
  • 5.93% 0.00% perf [.] main

The produce rate is 350000 message/sec.
Only 1 topic with 200 partitions in 3 brokers.
The host has 8vcpu and 32GB.

Apache Kafka version: 1.1.1
librdkafka client configuration: "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"

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)
%7|1570841036.272|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [78] 253 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.272|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [84] 277 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.272|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [87] 248 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.272|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.281|RECV|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Received ProduceResponse (v7, 52 bytes, CorrId 4317, rtt 8.99ms)
%7|1570841036.281|MSGSET|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [42]: MessageSet with 365 message(s) (MsgId 0, BaseSeq -1) delivered
%7|1570841036.281|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [78] 253 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.282|PRODUCE|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [78]: Produce MessageSet with 253 message(s) (28837 bytes, ApiVersion 7, MsgVersion 2, MsgId 0, BaseSeq -1, PID{Invalid})
%7|1570841036.282|SEND|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Sent ProduceRequest (v7, 28892 bytes @ 0, CorrId 4318)
%7|1570841036.282|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [84] 277 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.282|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [87] 248 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.282|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.290|RECV|rdkafka#producer-1| [thrd:10.177.165.26:9092/bootstrap]: 10.177.165.26:9092/1: Received ProduceResponse (v7, 52 bytes, CorrId 4265, rtt 130.50ms)
%7|1570841036.290|RECV|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Received ProduceResponse (v7, 52 bytes, CorrId 4318, rtt 8.73ms)
%7|1570841036.290|MSGSET|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [78]: MessageSet with 253 message(s) (MsgId 0, BaseSeq -1) delivered
%7|1570841036.290|MSGSET|rdkafka#producer-1| [thrd:10.177.165.26:9092/bootstrap]: 10.177.165.26:9092/1: ugw_ufdr [32]: MessageSet with 444 message(s) (MsgId 0, BaseSeq -1) delivered
%7|1570841036.291|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [84] 277 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.291|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [87] 248 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.291|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.315|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [84] 277 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.315|PRODUCE|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [84]: Produce MessageSet with 277 message(s) (30517 bytes, ApiVersion 7, MsgVersion 2, MsgId 0, BaseSeq -1, PID{Invalid})
%7|1570841036.316|SEND|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Sent ProduceRequest (v7, 30572 bytes @ 0, CorrId 4319)
%7|1570841036.316|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [87] 248 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.316|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.324|RECV|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Received ProduceResponse (v7, 52 bytes, CorrId 4319, rtt 8.26ms)
%7|1570841036.324|MSGSET|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [84]: MessageSet with 277 message(s) (MsgId 0, BaseSeq -1) delivered
%7|1570841036.324|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [87] 248 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.324|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.338|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [87] 248 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.339|PRODUCE|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [87]: Produce MessageSet with 248 message(s) (28684 bytes, ApiVersion 7, MsgVersion 2, MsgId 0, BaseSeq -1, PID{Invalid})
%7|1570841036.339|SEND|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Sent ProduceRequest (v7, 28739 bytes @ 0, CorrId 4320)
%7|1570841036.339|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.348|RECV|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Received ProduceResponse (v7, 52 bytes, CorrId 4320, rtt 8.99ms)
%7|1570841036.348|MSGSET|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [87]: MessageSet with 248 message(s) (MsgId 0, BaseSeq -1) delivered
%7|1570841036.348|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.353|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)
%7|1570841036.354|PRODUCE|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141]: Produce MessageSet with 283 message(s) (29826 bytes, ApiVersion 7, MsgVersion 2, MsgId 0, BaseSeq -1, PID{Invalid})
%7|1570841036.354|SEND|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Sent ProduceRequest (v7, 29881 bytes @ 0, CorrId 4321)
%7|1570841036.363|RECV|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: Received ProduceResponse (v7, 52 bytes, CorrId 4321, rtt 8.41ms)
%7|1570841036.363|MSGSET|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141]: MessageSet with 283 message(s) (MsgId 0, BaseSeq -1) delivered

@edenhill
Copy link
Contributor

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.

@eelyaj
Copy link
Author

eelyaj commented Oct 15, 2019

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 tried both rd_kafka_produce_batch and rd_kafka_produce api. No difference.

@eelyaj
Copy link
Author

eelyaj commented Oct 15, 2019

I disabled optimization of librdkafka by setting --disable-optimization.
The call stack is as follow.

  • 65.06% 0.02% librdkafka.so.1 [.] rd_kafka_broker_producer_serve
  • 6.95% rd_kafka_broker_producer_serve
    • 6.65% rd_kafka_broker_produce_toppars
      • 6.74% rd_kafka_toppar_producer_serve
        • 10.17% rd_kafka_msgq_insert_msgq
          • 57.62% rd_kafka_msgq_find_pos
            • 53.72% rd_kafka_msg_cmp_msgid
  • 62.57% 0.08% librdkafka.so.1 [.] rd_kafka_toppar_producer_serve
  • 6.66% rd_kafka_toppar_producer_serve
  • 62.29% 0.01% librdkafka.so.1 [.] rd_kafka_broker_produce_toppars
  • 60.19% 0.01% librdkafka.so.1 [.] rd_kafka_msgq_insert_msgq
  • 59.68% 3.91% librdkafka.so.1 [.] rd_kafka_msgq_find_pos
  • 56.10% 54.25% librdkafka.so.1 [.] rd_kafka_msg_cmp_msgid
  • 9.02% 0.02% perf [.] hist_entry_iter__add
  • 7.47% 0.00% perf [.] cmd_top
  • 7.41% 0.00% libpthread-2.17.so [.] start_thread
  • 6.90% 0.00% librdkafka.so.1 [.] _thrd_wrapper_function
  • 6.90% 0.00% librdkafka.so.1 [.] rd_kafka_broker_serve
  • 6.90% 0.00% librdkafka.so.1 [.] rd_kafka_broker_thread_main
  • 6.15% 0.01% [kernel] [k] system_call_fastpath
  • 5.44% 0.15% [kernel] [k] __do_softirq
  • 5.26% 0.02% [kernel] [k] net_rx_action

@edenhill
Copy link
Contributor

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.
And the number of messages moved is printed in this log message:
%7|1570841036.348|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)

Specifically the X added from partition queue part, but your log only has 0 added, which means the cmp_msgid part of your callstack would not have been called.

@eelyaj
Copy link
Author

eelyaj commented Oct 15, 2019

hello @edenhill
The 'rd_kafka_msg_cmp_msgid' issue only appears in version 1.2.1. I've tested 1.2.0, it seems ok.
The CPU rate of each version is as follow.
0.11.6 -- 250%
1.1.0 -- 360%
1.2.0-RC4 -- 360%
1.2.0 -- 360%
1.2.1 -- 460%
Produce rate is around 220000 message/second.

I found 'rd_kafka_msgq_find_pos' function was modified in 1.2.1. Maybe something wrong with it.

@eelyaj
Copy link
Author

eelyaj commented Oct 15, 2019

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.
And the number of messages moved is printed in this log message:
%7|1570841036.348|TOPPAR|rdkafka#producer-1| [thrd:10.184.154.248:9092/bootstrap]: 10.184.154.248:9092/2: ugw_ufdr [141] 283 message(s) in xmit queue (0 added from partition queue)

Specifically the X added from partition queue part, but your log only has 0 added, which means the cmp_msgid part of your callstack would not have been called.

Not all X are 0 in my log.

ugw_ufdr [141] 983 message(s) in xmit queue (11 added from partition queue)
ugw_ufdr [141] 991 message(s) in xmit queue (8 added from partition queue)
ugw_ufdr [141] 1007 message(s) in xmit queue (16 added from partition queue)
ugw_ufdr [141] 1040 message(s) in xmit queue (33 added from partition queue)
ugw_ufdr [141] 1068 message(s) in xmit queue (28 added from partition queue)
ugw_ufdr [141] 1132 message(s) in xmit queue (64 added from partition queue)
ugw_ufdr [141] 1138 message(s) in xmit queue (6 added from partition queue)
ugw_ufdr [141]: Produce MessageSet with 1138 message(s) (123759 bytes, ApiVersion 7, MsgVersion 2, MsgId 0, BaseSeq -1, PID{Invalid})
ugw_ufdr [141] 11 message(s) in xmit queue (11 added from partition queue)
ugw_ufdr [141]: MessageSet with 1138 message(s) (MsgId 0, BaseSeq -1) delivered
ugw_ufdr [141] 24 message(s) in xmit queue (13 added from partition queue)
ugw_ufdr [141] 24 message(s) in xmit queue (0 added from partition queue)
ugw_ufdr [141] 34 message(s) in xmit queue (10 added from partition queue)
ugw_ufdr [141] 49 message(s) in xmit queue (15 added from partition queue)
ugw_ufdr [141] 51 message(s) in xmit queue (2 added from partition queue)
ugw_ufdr [141] 51 message(s) in xmit queue (0 added from partition queue)
ugw_ufdr [141] 61 message(s) in xmit queue (10 added from partition queue)
ugw_ufdr [141] 76 message(s) in xmit queue (15 added from partition queue)
ugw_ufdr [141] 76 message(s) in xmit queue (0 added from partition queue)
ugw_ufdr [141] 78 message(s) in xmit queue (2 added from partition queue)
ugw_ufdr [141] 95 message(s) in xmit queue (17 added from partition queue)
ugw_ufdr [141] 99 message(s) in xmit queue (4 added from partition queue)
ugw_ufdr [141] 99 message(s) in xmit queue (0 added from partition queue)
ugw_ufdr [141] 99 message(s) in xmit queue (0 added from partition queue)
ugw_ufdr [141] 109 message(s) in xmit queue (10 added from partition queue)
ugw_ufdr [141] 115 message(s) in xmit queue (6 added from partition queue)

@edenhill
Copy link
Contributor

I think I've found an issue, let me push a branch with a proposed fix which you can try out

edenhill added a commit that referenced this issue Oct 15, 2019
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
@edenhill
Copy link
Contributor

Can you try out the issue2450 branch and report CPU usage and perf stack (if necessary)?

@eelyaj
Copy link
Author

eelyaj commented Oct 15, 2019

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.

  • 23.63% 0.02% librdkafka.so.1 [.] rd_kafka_broker_producer_serve
    • 20.00% rd_kafka_broker_producer_serve
      • 12.68% rd_kafka_broker_produce_toppars
        • 12.33% rd_kafka_toppar_producer_serve
          • 5.74% rd_kafka_ProduceRequest
            • 5.75% rd_kafka_msgset_create_ProduceRequest
              • 5.70% rd_kafka_msgset_writer_write_msgq
                • 3.97% rd_kafka_msgset_writer_write_msg
                  • 3.82% rd_kafka_msgset_writer_write_msg_v2
                    • 2.23% rd_kafka_buf_write
                      1.40% rd_buf_write
                      0.77% __memcpy_ssse3
                    • 1.03% rd_kafka_msgset_writer_write_msg_payload
                    • 1.04% rd_kafka_buf_write
                      0.89% __memcpy_ssse3
                • 0.78% rd_avg_add
                  0.70% pthread_mutex_lock
                  0.62% rd_kafka_msg_wire_size
                  1.55% pthread_mutex_lock
          • 1.26% rd_kafka_broker_outbufs_space
            0.83% rd_atomic32_get
            1.07% pthread_mutex_unlock
            0.82% rd_kafka_msgq_insert_msgq
      • 7.28% rd_kafka_broker_ops_io_serve
        • 5.96% rd_kafka_transport_io_serve
          • 2.69% rd_kafka_transport_io_event
            • 2.46% rd_kafka_recv
              • 2.39% rd_kafka_req_response
                • 2.39% rd_kafka_buf_callback
                  • 2.37% rd_kafka_handle_Produce
                    • 2.52% rd_kafka_msgbatch_handle_Produce_result
                    • 1.42% rd_kafka_dr_msgq
                    • 1.25% rd_kafka_msgq_purge
                      1.04% rd_kafka_msg_destroy
                      1.16% rd_kafka_msgq_set_metadata
          • 1.88% __GI___libc_poll
            • 1.68% system_call_fastpath
              • 1.69% sys_poll
                • 1.58% do_sys_poll
                  • 0.68% poll_schedule_timeout
                    • 0.60% schedule_hrtimeout_range
                    • 0.60% schedule_hrtimeout_range_clock
          • 1.13% 0xe6fd
            • 0.87% system_call_fastpath
              • 0.88% sys_read
                • 0.81% vfs_read
                  0.70% do_sync_read
        • 0.55% rd_kafka_broker_ops_serve
          • 0.54% rd_kafka_q_pop
            0.57% rd_kafka_q_pop_serve
  • 21.42% 0.00% libpthread-2.17.so [.] start_thread
  • 19.90% 0.00% librdkafka.so.1 [.] _thrd_wrapper_function
  • 19.88% 0.00% librdkafka.so.1 [.] rd_kafka_broker_serve
  • 19.88% 0.00% librdkafka.so.1 [.] rd_kafka_broker_thread_main
  • 14.59% 0.56% librdkafka.so.1 [.] rd_kafka_broker_produce_toppars
  • 14.11% 0.00% [unknown] [.] 0000000000000000
  • 13.97% 0.02% [kernel] [k] system_call_fastpath

@edenhill
Copy link
Contributor

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

@eelyaj
Copy link
Author

eelyaj commented Oct 15, 2019

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+.

@edenhill
Copy link
Contributor

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:
examples/rdkafka_performance -b $BROKERS -P -t issue2450 -r 230000 -s 100 -c 6000000 -X statistics.interval.ms=1000 -X linger.ms=1000 -X queue.buffering.max.messages=1000000 -X queue.buffering.max.kbytes=2048000 -X batch.num.messages=10000 -z lz4 -X socket.send.buffer.bytes=3200000 -X socket.receive.buffer.bytes=3200000 -X message.max.bytes=209715200

@eelyaj
Copy link
Author

eelyaj commented Oct 16, 2019

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:
examples/rdkafka_performance -b $BROKERS -P -t issue2450 -r 230000 -s 100 -c 6000000 -X statistics.interval.ms=1000 -X linger.ms=1000 -X queue.buffering.max.messages=1000000 -X queue.buffering.max.kbytes=2048000 -X batch.num.messages=10000 -z lz4 -X socket.send.buffer.bytes=3200000 -X socket.receive.buffer.bytes=3200000 -X message.max.bytes=209715200

I tried this command in several version. Here is the result. 0.11.6 got the best performance.

librdkafka-0.11.6

CPU usage

PID USER PR NI VIRT RES SHR S %CPU %MEM TIME+ COMMAND
97911 root 20 0 385196 37652 2576 R 99.9 0.1 0:25.56 rdkafka_perform
97915 root 20 0 385196 37652 2576 S 7.9 0.1 0:01.73 rdk:broker3
97916 root 20 0 385196 37652 2576 S 7.6 0.1 0:01.71 rdk:broker2
97914 root 20 0 385196 37652 2576 S 7.3 0.1 0:01.69 rdk:broker1

log

% ... 160422 msgs/s and 16.04 MB/s, 0 produce failures, 117733 in queue, lz4 compression
% ... 160737 msgs/s and 16.07 MB/s, 0 produce failures, 120129 in queue, lz4 compression
% ... 161037 msgs/s and 16.10 MB/s, 0 produce failures, 111042 in queue, lz4 compression

librdkafka-1.2.0

CPU usage

PID USER PR NI VIRT RES SHR S %CPU %MEM TIME+ COMMAND
95819 root 20 0 373952 24008 2680 R 99.7 0.1 0:18.02 rdkafka_perform
95823 root 20 0 373952 24008 2680 R 47.2 0.1 0:08.51 rdk:broker3
95824 root 20 0 373952 24008 2680 S 47.2 0.1 0:08.53 rdk:broker2
95822 root 20 0 373952 24008 2680 S 46.5 0.1 0:08.38 rdk:broker1

log

% ... 98897 msgs/s and 9.89 MB/s, 0 produce failures, 75820 in queue, lz4 compression
% ... 98855 msgs/s and 9.89 MB/s, 0 produce failures, 78910 in queue, lz4 compression
% ... 98861 msgs/s and 9.89 MB/s, 0 produce failures, 76663 in queue, lz4 compression
% ... 98839 msgs/s and 9.88 MB/s, 0 produce failures, 77977 in queue, lz4 compression

librdkafka-1.2.1

CPU usage

PID USER PR NI VIRT RES SHR S %CPU %MEM TIME+ COMMAND
101373 root 20 0 378448 34296 2672 S 98.3 0.1 0:17.71 rdkafka_perform
101378 root 20 0 378448 34296 2672 R 78.2 0.1 0:13.62 rdk:broker2
101376 root 20 0 378448 34296 2672 R 77.9 0.1 0:13.51 rdk:broker1
101377 root 20 0 378448 34296 2672 R 77.6 0.1 0:13.51 rdk:broker3

log

% ... 114725 msgs/s and 11.47 MB/s, 0 produce failures, 71923 in queue, lz4 compression
% ... 114344 msgs/s and 11.43 MB/s, 0 produce failures, 62631 in queue, lz4 compression
% ... 113985 msgs/s and 11.40 MB/s, 0 produce failures, 71621 in queue, lz4 compression

librdkafka-issue2450

CPU usage

PID USER PR NI VIRT RES SHR S %CPU %MEM TIME+ COMMAND
103562 root 20 0 373432 23156 2680 R 99.9 0.1 0:06.30 rdkafka_perform
103567 root 20 0 373432 23156 2680 S 46.7 0.1 0:02.95 rdk:broker2
103566 root 20 0 373432 23156 2680 R 46.4 0.1 0:02.91 rdk:broker3
103565 root 20 0 373432 23156 2680 S 45.7 0.1 0:02.91 rdk:broker1

log

% ... 98409 msgs/s and 9.84 MB/s, 0 produce failures, 72880 in queue, lz4 compression
% ... 98376 msgs/s and 9.84 MB/s, 0 produce failures, 73805 in queue, lz4 compression
% ... 98373 msgs/s and 9.84 MB/s, 0 produce failures, 76653 in queue, lz4 compression

@edenhill
Copy link
Contributor

@eelyaj performance runs:

Version CPU Throughput
v0.11.6 7% 16 MB/s
v1.2.0 47% 10 MB/s
v1.2.1 78% 11 MB/s
issue2450 46% 10 MB/s

CPU is per-broker-thread CPU

@edenhill performance runs:

Intel(R) Core(TM) i7-2600 CPU @ 3.40GHz, 8 cores

Version CPU Throughput
v0.11.6 1.6% 21 MB/s
v1.1.0 48% 15 MB/s
v1.2.0 1.3% 21 MB/s
v1.2.1 9% 21 MB/s
issue2450 1.3% 21 MB/s

@edenhill
Copy link
Contributor

edenhill commented Oct 16, 2019

@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.
Can you share your system specs?

And make sure to run your performance tests with optimizations enabled (no --disable-optimization)

@eelyaj
Copy link
Author

eelyaj commented Oct 16, 2019

I am using virtual machine to run these tests.
cat cpu info I got this.

processor : 0
vendor_id : GenuineIntel
cpu family : 6
model : 63
model name : Intel(R) Xeon(R) CPU E5-2680 v3 @ 2.50GHz
stepping : 2
microcode : 0x38
cpu MHz : 2494.237
cache size : 30720 KB
physical id : 0
siblings : 8
core id : 0
cpu cores : 8
apicid : 0
initial apicid : 0
fpu : yes
fpu_exception : yes
cpuid level : 13
wp : yes
flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush mmx fxsr sse sse2 ht syscall nx rdtscp lm constant_tsc rep_good nopl eagerfpu pni pclmulqdq ssse3 fma cx16 pcid sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand hypervisor lahf_lm fsgsbase bmi1 avx2 smep bmi2 erms invpcid xsaveopt
bogomips : 4988.61
clflush size : 64
cache_alignment : 64
address sizes : 46 bits physical, 48 bits virtual
power management:

@eelyaj
Copy link
Author

eelyaj commented Oct 16, 2019

@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.
Can you share your system specs?

And make sure to run your performance tests with optimizations enabled (no --disable-optimization)

I use the default config to compile librdkafka. command is ./configure && make && make install

@edenhill
Copy link
Contributor

I am using virtual machine to run these tests.

Virtual machines are not suitable for performance testing

@edenhill edenhill reopened this Oct 16, 2019
@eelyaj
Copy link
Author

eelyaj commented Oct 17, 2019

I am using virtual machine to run these tests.

Virtual machines are not suitable for performance testing

My application is running on virtual machines. I need to test it on VM.

@edenhill
Copy link
Contributor

Then it sounds like you need to beef up your machine specs

@eelyaj
Copy link
Author

eelyaj commented Oct 17, 2019

Then it sounds like you need to beef up your machine specs

Can you share your linux version, gcc version and libc version?

@edenhill
Copy link
Contributor

edenhill commented Oct 17, 2019

# dpkg -l libc6
Önskat=Okänd(U)/Installera(I)/Radera(R)/Rensa(P)/Håll(H)
| Status.=Ej inst.(N)/(I)nst./Konffil.(C)/(U)ppack./Halvkonf.(F)/(H)alvinst.
| /       Vänt.utl(W)/Föresl.utl(T)
|/ Fel?Inget(=)/Ominstallera(R)/Båda(X) (Status,Fel: versaler=illa)
||/ Namn           Version       Arkitektur   Beskrivning
+++-==============-=============-============-=================================
ii  libc6:amd64    2.29-0ubuntu2 amd64        GNU C Library: Shared libraries
ii  libc6:i386     2.29-0ubuntu2 i386         GNU C Library: Shared libraries
# lsb_release -a
No LSB modules are available.
Distributor ID:	Ubuntu
Description:	Ubuntu 19.04
Release:	19.04
Codename:	disco
# uname -a
Linux eden 5.0.0-31-generic #33-Ubuntu SMP Mon Sep 30 18:51:59 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux
# grep Intel /proc/cpuinfo | head -2
vendor_id	: GenuineIntel
model name	: Intel(R) Core(TM) i7-2600 CPU @ 3.40GHz

#
gcc version 8.3.0 (Ubuntu 8.3.0-6ubuntu1)

@sarkanyi
Copy link

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.
@edenhill I'm assuming the fix will land in 1.2.2 and won't have to wait for 1.3.0, right?

@sarkanyi
Copy link

On the other hand my method probably wouldn't have noticed the drop except when I pack multiple broker threads on the same core.

@edenhill
Copy link
Contributor

Yeah, we'll push a new release soon

edenhill added a commit that referenced this issue Oct 17, 2019
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
@eelyaj
Copy link
Author

eelyaj commented Aug 19, 2020

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.

@eelyaj eelyaj closed this as completed Aug 19, 2020
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

3 participants