-
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
Random latency spikes when producing messages #2912
Comments
I also ran a java producer against the same server and topic with below config and did see spikes of upto 150ms, but nothing close to 900ms.
|
How many partitions does test-topic have, how many brokers in your cluster, and what is the replication-factor of the topic? |
Partitions: 4, Cluster Size: 12, Replication Factor: 6 I was also able to reproduce the issue on my local mac with: |
See if the latency spikes can be correlated with disk flush/writes or some other broker activity by increasing the broker log level and inspecting the logs. |
I tried changing broker log.flush config parameters to be 600seconds, 60 seconds and 1 second. However, it is hard to find any clear pattern of the latency spike. Also, even after enabling debug log level in kafka/log4j.properties, i do not see any logs on the server related to disk flush:
Below are the broker parameters I changed in kafka/server.properties:
|
This is happening only when |
Can you add a debug printf to (or similar) to see what the poll timeout parameter is set to when you get the latency spikes? |
It is being set to 10 at normal times, but set to 999 during these spikes:
This is the debug log i added:
|
Interesting. |
Here is the full log file - https://gist.github.com/meetsitaram/19fbc055b41f2d9abc2ebd9facd10296 sample log at latency spike time:
|
From this log, it appears the delay is happening even before the produce call. |
Any suggestions on how to troubleshoot further? |
would be helpful if anyone can give more clues on what to debug. I would like to troubleshoot this issue, but not sure how to proceed further. thx. |
@edenhill I think i found the bug. Here is what i understood so far. The broker thread loops constantly with
However, before that logic if reached, there are multiple conditions in rd_kafka_toppar_producer_serve() that return without honouring that check. One of them is if the message count is 0, it returns without updating the next_wakeup time.
This results in invoking rd_kafka_transport_poll() with almost 1 second timeout. If the poll() doesn't return until the timeout, there are messages getting queued up during this 1 second interval in a separate thread. The queue.buffering.max.ms condition will fail for all these queued up messages and could result in P99 latency of close to a second for the messages in that batch. I was able to replicate the issue and resolve it my making the below change, although i haven't done any thorough testing.
Please let me know if this makes sense so i can create a PR for this. |
Great investigation @meetsitaram ! The rktp_xmit_msgq is the broker's local queue (without locks) that any new messages on the (locked) rktp_msgq are moved to in toppar_serve. When a message is appended to rktp_msgq and the queue was previously empty an IO-based wakeup is triggered on the broker thread, waking it up from its blocking IO poll. Setting the next_wakeup to buffering_max_us (linger.ms) is not a good solution because with a low linger.ms it will cause a lot of unwarranted wakeups. |
@edenhill Hitting the IO-wakeup rate-limiter when a message is appended to a previously empty rktp_msgq is also a problem. The rate-limit is only for linger.ms, but once the first message is enqueued, it will never trigger a wakeup again even after enqueuing subsequent messages and hence broker thread is blocked until the poll times out after 1 second. Looks like this is what is happening. rd_kafka_toppar_enq_msg() triggers a wakeup only after first msg is enqueued, but with rate_limit set to true:
rd_kafka_q_io_event(rkq, rate_limit) then checks for rate_limit and will not wake up as there is still time left:
No further wakeup calls are made upon enqueuing further messages. I am not familiar with how a forwarded-to-queue plays role in here. Can we atleast invoke rd_kafka_q_yield without rate_limit from rd_kafka_toppar_enq_msg() |
Rate-limiting was introduced to limit CPU wakeups and thus CPU usage: So we need to be careful not to reintroduce that problem. |
@shanson7 Would it be possible for you to try out the suggested fix by toggling the |
I spoke with @meetsitaram and told him the steps I used to test before. Hopefully he can reproduce. I basically just used |
As suspected, removing the rate-limit does impact performance severely. Below are the performance results for varying linger.ms, with existing code, fix with removing rate-limit, and fix with next-wakeup updated to linger.ms. Though performance was severly hit with removing rate-limit, updating next-wakeup looks promising. This approach also makes the wakeup logic, rate-limited or not, unnecessary.
Your concerns about too many unnecessary wakeups, when updating next-wakeup with linger.ms, are highlighted below. when linger.ms set to 0, and msg rate is minimal, the fix results in excessive cpu usage. We need to somehow eliminate this scenario. However, when set to 1ms, the increase in cpu usage is minimal. If users set a low linger.ms, isn't it reasonable for them to expect a little increase in cpu usage for the improved latencies?
I also tested and made sure the latency spikes are resolved
|
@edenhill any comments? |
@edenhill | @meetsitaram |
@edenhill hope this gets picked soon! |
Thanks for a great analysis and testing of the different approaches, @meetsitaram , and sorry for not picking this up sooner.
I don't think that is an acceptable assumption, e.g., a low rate but low latency producer should not have high CPU usage. A problem summary: So what we have is:
But their combined logic does not cover the case where the produce interval is < linger.ms but we still want to adhere to linger.ms. For that we would need the rate-limiter on msg enqueue to say "don't wake it up now but in linger.ms", but there is no way to accomplish that with today's io-evented queues. We arrive in this state when:
Optimally we want to allow a maximum of one wakeup-events per wakeup, anything more than that is a waste of resources. Edit: An even simpler implementation would be to just have a bool to track if a wakeup has been performed or not. Saves us from calling rd_clock()., What do you all think? P.S. The long term proper solution is to split out IO into its own thread and only do condvar-triggerd queues internally, but that's a big undertaking and not something we have scheduled. |
…of based on rate (#2912) The IO-based wakeup (rkq_qio) now has a `sent` boolean that tracks whether a wakeup has been sent, this boolean is reset by the queue reader each time it polls the queue - effectively allowing only one wakeup-event per non-polling period and voiding the need for rate-limiting the wakeups.
|
…of based on rate (#2912) The IO-based wakeup (rkq_qio) now has a `sent` boolean that tracks whether a wakeup has been sent, this boolean is reset by the queue reader each time it polls the queue - effectively allowing only one wakeup-event per non-polling period and voiding the need for rate-limiting the wakeups.
…of based on rate (#2912) The IO-based wakeup (rkq_qio) now has a `sent` boolean that tracks whether a wakeup has been sent, this boolean is reset by the queue reader each time it polls the queue - effectively allowing only one wakeup-event per non-polling period and voiding the need for rate-limiting the wakeups.
Merged to master, so try master instead. |
Were you able to verify the fixes on master? |
Ran some tests using latencies
peak performance
resource usage at low rates
resource usage at peak rates
Will integrate these changes with our app and update this issue if the spikes still happen. |
Thanks for the fix. Works like magic. |
Can you take a look at increase in system time at peak rates? It is now around |
Thank you, again, for verifying this fix! As for the increased system times, isn't that expected since without the rate-limiter we now do a lot more wakeups? |
The wakeup rate-limiting discussed here and in #3538 that was introduced in v1.7.0 proved to be problematic for certain produce patterns, so I redesigned the wakeup signalling and made it more formal (https://github.com/edenhill/librdkafka/blob/qlatency/src/rdkafka_msg.c#L1671-L1813). From my testing we're now back at v1.6 levels of thruput, or even higher, with lower CPU usage, and the wakeup signalling should also fix the corner-cases that v1.7.0 addressed as well as the ones that v1.7.0 added. It would be great if you could try out the qlatency branch with your workloads and see how it behaves. |
This is now merged to master. |
Thanks for the fix. Overall, i do not see any issues with our use case. Below are some test results with various scenarios. latencies
peak performance
resource usage at low rates
resource usage at peak rates
|
Description
I have been running latency tests for my application using librdkafka and noticed some random latency spikes of upto 1 second when producing messages. This happens even when my queue.buffering.max.ms is only set to 10 milliseconds. I am trying to investigate if the issue is within librdkafka, or something specific to my kafka configuration or at the server.
How to reproduce
The issue can be reproduced by running kafka server locally on Mac, and running examples/rdkakfa_performance, setting two extra configurations:
This issue can be reproduced more often by producing at a rate of 1k msgs/sec. As this happens at random, it has to be running for a few minutes before we see any spikes in latency.
Checklist
1.4.2
2.5.0
queue.buffering.max.ms=10, request.required.acks=1
macOs Mojave
debug=..
as necessary) from librdkafkaBelow are some sample logs from running examples/rdkafka_performance. For a produce rate of 1k/sec, and queue buffering of 10ms, we should be seeing max of 10 messages in queue. However, some times, the queue has 100s of messages.
Per message top latencies reach above 900 milliseconds:
I also verified that this is not just because server is not able to handle the load, by running at peak rate:
Debugging internals
I tried debugging the rdkafka code, and noticed the major lag is happening at https://github.com/edenhill/librdkafka/blob/master/src/rdkafka_transport.c#L965 in poll(). However, I do not understand what that means, and also if that is the main cause of delay.
The text was updated successfully, but these errors were encountered: