Closed shanson7 closed 4 months ago
Code-wise, the only thing that different rd_kafka_t instances share, regardless of producer or consumer, is initialization of SSL - which is only done once. But it is unlikely that shared access to OpenSSL would be a problem, so if I had to guess it might be something with the CPU core contention, seeing how librdkafka makes use of a number of threads and if the number of (v)cores is too low there might be stalls.
Do you have any debug logs?
Do you have any debug logs?
I took some debug logs for 5 minutes(very verbose), about 4.7GB. Any particular categories or patterns I can use to narrow it down?
The bit leading up to the timeouts, for the producer instance. It'd be interesting to see what it does, what requests it is sending, what the response RTTs are, when things are working good and when things are timing out, for comparison.
Here is a (very) stripped down version of the logs. I removed the individual failure lines like:
07APR2022_15:28:19.187813 PRODUCER KAFKA LOG: severity = 7 fac = REQERR event = [thrd:tpubpp-pw-703.bloomberg.com:9092/45]: tpubpp-pw-703:9092/45: ProduceRequest failed: Local: Timed out: explicit actions Retry,MsgPossiblyPersisted
07APR2022_15:28:19.187835 PRODUCER KAFKA LOG: severity = 7 fac = MSGSET event = [thrd:tpubpp-pw-703:9092/45]: tpubpp-pw-703:9092/45: metrictank [1755]: MessageSet with 7 message(s) (MsgId 0, BaseSeq -1) encountered error: Local: Timed out (actions Retry,MsgPossiblyPersisted)
as well as delivery / produce notifications as there were millions of lines like this.
But what I'm getting is that this isn't a problem you've heard of before and is probably related to some sort of congestion broker/client side? The clients are running in k8s and are not being CPU throttled (limits are set very high). We looked into network congestion but we removed a lot of network load and still see the issue. We also have a large number of other consumers with the same source/dest that are unaffected during this period (where if it were resource congestion I would expect noisy neighbor problems).
If I'm reading your stats correctly you have a single topic with 2000 partitions spread across 28 of the 57 brokers., with about 71 partitions per broker.
That's a lot of partitions.
Your stats show quite high .brokers[].int_latency
, which is the time between a message is produce()d and it being written to a MessageSet in the broker thread. See https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md#latency-measurement
Each librdkafka broker thread will, for each wakeup, scan its list of partitions to check if any messages need timing out or sending, Depending on linger.ms (you have this set to 100ms, which is fine), batch.size and the produce rate, this might result in quite a lot of wakeups and scans. There are also certain produce patterns in librdkafka that causes too many wakeups, as you're aware from the other issues we've communicated on.
So.. my thoughts are:
Thanks for this! This definitely gives us some things to thing about.
If I'm reading your stats correctly you have a single topic with 2000 partitions spread across 28 of the 57 brokers., with about 71 partitions per broker.
That is correct. A salient point might be that we are in the process of migrating datacenters and the 28 brokers are the "new" brokers.
Your stats show quite high
.brokers[].int_latency
Hmm, that is true. If I look at the first few stats message (emitted before the consumer rebalance), the int_latency
looks fine. I'm not sure why int_latency
would spike due to the consumer group rebalance unless this causes enough additional context switching to introduce latency.
It does seem like the system is a tad overloaded - be it CPU, cache, core contention, memory pressure, network load. What's the system load? (e.g., cpu&io wait factor)
The system load is not very high during these times and not much higher than during "normal" periods where we don't experience timeouts. Additionally we don't see other consumers/producers running on the same machine experiencing the same issues. We will look into this more though.
It would be very interesting to see how master behaves, given the recent producer queue performance optimizations.
I agree. Unfortunately the master branch is not yet approved to run on production and we haven't managed to reproduce this on our QA clusters. Worst case scenario, we will be able to run 1.9.0 when it's released/approved.
the partition count and message rate may be pushing the current producer scheduling design.
This is an interesting point. We use a large number of partitions to support a large number of consumers. That being said, we've used 2000 partitions for several years now and this is a new issue for us. Additionally, the number of messages has actually been reduced about 30 fold (we moved to a client side batching scheme to make fewer, larger messages). Given that this used to work with a notably larger workload I'm not sure that we've hit that limit.
You mention that you have other clients on the same machine which are not affected by the rebalance? Does that include other producers as well? And if so, are they producing to the same set of brokers?
Do other clients, on the same machine or other, see any increase in latencies/response-times from the same set of brokers when this happens?
If the rebalance only affects the producer in the same process then that limits the scope further..
Is there any per-process throttling? - eg per-process CPU/memory/ctxswitch/networking/etc constraints configured?
Some shared state in OpenSSL that's used across different SSL contexts that could cause blockage?
Or are the clients in the same process perhaps sharing the same broker-side quota?
You mention that you have other clients on the same machine which are not affected by the rebalance? Does that include other producers as well? And if so, are they producing to the same set of brokers?
Yes there are other producers to the same brokers, but a different set of topics.
Do other clients, on the same machine or other, see any increase in latencies/response-times from the same set of brokers when this happens?
We haven't noticed any change in latency on other clients producing to the same brokers when this happens.
Is there any per-process throttling? - eg per-process CPU/memory/ctxswitch/networking/etc constraints configured?
We have limits imposed in kubernetes, but one of the first things we tried was raising the CPU limits so that no CPU throttling is occurring. So, at this moment, there are effectively no limit imposed.
Some shared state in OpenSSL that's used across different SSL contexts that could cause blockage?
We don't use SSL configurations for these brokers.
Or are the clients in the same process perhaps sharing the same broker-side quota?
How would we tell if this is the case? Each process has a single producer and a single consumer.
Is there any per-process throttling? - eg per-process CPU/memory/ctxswitch/networking/etc constraints configured?
We have limits imposed in kubernetes, but one of the first things we tried was raising the CPU limits so that no CPU throttling is occurring. So, at this moment, there are effectively no limit imposed.
Is there any limit to the number of vCPUs/cores/threads it can utilitize at any given time?
Or are the clients in the same process perhaps sharing the same broker-side quota?
How would we tell if this is the case? Each process has a single producer and a single consumer.
You would be seeing throttling. https://github.com/edenhill/librdkafka/blob/master/INTRODUCTION.md#latency-measurement I would monitor and compare all the metrics mentioned there.
Thanks for that! We migrated the output topic back to the original datacenter and see no issues. This leads us to believe it's some issue with the link between the datacenters or perhaps something with the brokers.
Although, that would seem to rule out CPU contention in the client process. That doesn't explain why int_latency
shoots up during consumer rebalance. The only thing tying this together is the brokers in the new datacenter.
int_latency will probably increase if the broker is congested, part of the backpressure.
Closing this issue as there's been no discussion after Magnus' comment, and it seems like the cause was the link between the datacenters. Please reopen if this still happens.
Description
We have a consumer group (call it
CG1
) that consumes from a set of topics and produces to a single output topic. Historically this has worked well, however we moved our kafka cluster to a separate datacenter and now every time the consumer group rebalances we start to see producer timeouts. We aren't sure why this would happen as the producer should be completely separate from the consumer group. We have a separate consumer group that consumes from a different kafka cluster and pushes to the same topic and that consumer group (call itCG1_x
) doesn't have any issues during the same period.We spent some time looking at network utilization numbers, but it seems like this is only an issue during consumer group rebalance and only effects the consumer group being rebalanced.
The timeouts last for about 5-10 minutes until eventually the broker state of each broker is reset. By reset I mean we actually see the state change from UP->DOWN (which doesn't always seem to happen on "timed out: disconnect" events?
How to reproduce
We can reliably reproduce in our environment but are unsure how to reproduce generically. We cannot test a newer version of librdkafka due to https://github.com/edenhill/librdkafka/issues/3538
Checklist
IMPORTANT: We will close issues where the checklist has not been completed.
Please provide the following information:
1.6.1
2.6.0
queue.buffering.max.messages=1024000, queue.buffering.max.kbytes=1024000, queue.buffering.max.ms=100
RHEL7.6
debug=..
as necessary) from librdkafka