rust-rdkafka
rust-rdkafka copied to clipboard
Polling for messages takes twice the time it should due to double poll
The following commit: https://github.com/fede1024/rust-rdkafka/commit/24a28bbad94a385929cd3a1b7761c7e11f83f70a introduced polling two queues instead of one on message polling calls:
https://github.com/fede1024/rust-rdkafka/blob/35177be201eef182a66b2bdc3af695354bcc1e8b/src/consumer/base_consumer.rs#L125-L132
As it turns out, this has a very significant perf. impact, as polling a queue takes a long amount of time (1us, or about 1 000 CPU instructions per message), and this doubles that time (so on a single thread you're going from 300 000 msg/s to 150 000 msg/s):
We can see that each call to BaseConsumer::poll_raw
spends half its time in rd_kafka_poll
, and half in rd_kafka_consumer_poll
. Each of these calls represents 1us, or about 1 000 CPU instructions per message, which is very non neglictable.
The double-poll here looks like it's a quick-and-dirty place to put the rd_kafka_poll
, to not have a separate thread to poll for results, as is otherwise done in the producer.
In addition the way it is implemented introduces a timeout of main_queue_min_poll_interval
to the poll, which causes librdkafka to make clock calls to respect that timeout, which themselves appear to be a good part of what takes time in these functions.
=> It would probably be best to either bring the redirect back, or do something similar to what is done in the StreamConsumer
, or rd_kafka_poll
for events independently (as is done with the ThreadedProducer
). At least we probably don't want polling for messages on the critical path to actually perform several poll
s, as these take a very significant amount of time.
The problem with moving that rd_kafka_poll
to a separate thread is that it breaks max.poll.interval.ms
, IIRC, and that makes some folks very upset. See #264 for a similar discussion about the StreamConsumer
.
That said, a 2x regression here is rough! You should feel free to take a stab at fixing it, but I'm thoroughly burned out on trying to balance the competing interests.
Thanks! I see. That doesn't seem to be the case with queue redirection (before https://github.com/fede1024/rust-rdkafka/commit/24a28bbad94a385929cd3a1b7761c7e11f83f70a) though, no? (Queue redirection is what's done with the StreamConsumer)
The commit message mentions:
This makes it possible to receive more targeted notifications when there is activity on just one queue.
But I'm having a hard time understanding what that means.
Still waiting for an explanation about what 24a28bbad94a385929cd3a1b7761c7e11f83f70a was trying to achieve to see whether I can ~revert it to fix this issue.
IIRC, 24a28bb makes it so that message_queue_nonempty_callback
is only invoked when there is a new message available—i.e., the next call to poll
returns Some
. Otherwise, message_queue_nonempty_callback
may be invoked because there is a pending callback or timer or something like that, and the next call to poll
returns None
.
I see. At first sight that doesn't seem worth the perf loss. What's the use-case?
I'm sorry, I'm afraid I don't remember details beyond what I mentioned in my last comment.