[
https://issues.apache.org/jira/browse/KAFKA-19259?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17956632#comment-17956632
]
Lianet Magrans commented on KAFKA-19259:
----------------------------------------
Hey [~goyarpit], interesting! We had a kind of major change in this exact area
(the pace at which we send fetch requests), so I wonder if this has always been
the case or if this is a regression
-
https://github.com/apache/kafka/commit/9e424755d4d236442847b13863580f44f27e22a6
I haven't had the time to try to repro without that commit, but maybe worth
checking.
For the record, I did try a while back excluding the other major change we had
in fetch but still saw the same behaviour, so not related ( excluding this
https://github.com/apache/kafka/commit/057460e807a218b169c69ae0b849e0b3b4d27f42,
but same behviour so not related)
Sharing in case it helps! Keep me posted. Thanks!
> Async consumer fetch intermittent delays on console consumer
> ------------------------------------------------------------
>
> Key: KAFKA-19259
> URL: https://issues.apache.org/jira/browse/KAFKA-19259
> Project: Kafka
> Issue Type: Bug
> Components: clients, consumer
> Affects Versions: 4.0.0
> Reporter: Lianet Magrans
> Assignee: Arpit Goyal
> Priority: Major
> Fix For: 4.1.0
>
> Attachments: Screenshot 2025-05-31 at 10.44.29 PM.png,
> console-consumer-classic-vs-consumer.mov, consumer_KAFKA-19259.log, debug5.log
>
>
> We noticed that fetching with the kafka-console-consumer.sh tool using the
> new consumer shows some intermittent delays, that are not seen when running
> the same with the classic consumer. Note that I disabled auto-commit to
> isolate the delay, and from a first look seems to come from the
> fetchBuffer.awaitNonEmpty logic, that alternatively takes almost the full
> poll timeout (runs "fast", then "slow", and continues to alternate)
> [https://github.com/apache/kafka/blob/0b81d6c7802c1be55dc823ce51729f2c6a6071a7/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java#L1808]
>
> The difference in behaviour between the 2 consumers can be seen with this
> setup:
> * topic with 6 partitions (I tried with 1 partition first and didn't see the
> delay, then with 3 and 6 I could see it)
> * data populated in topic with producer sending generated uuids to the topic
> in while loop
> * run console consumer (asycn) no commit:
> bin/kafka-console-consumer.sh --topic t1 --bootstrap-server localhost:9092
> --consumer-property group.protocol=consumer --group cg1 --consumer-property
> enable.auto.commit=false
> Here we can notice the pattern that looks like batches, and custom logs on
> the awaitNonEmpty show it take the full poll timeout on alternate poll
> iterations.
> * run same but for classic consumer (consumer-property
> group.protocol=classic) -> not such pattern of intermittent delays
> Produce continuously (I used this)
> while sleep 1; do echo $(uuidgen); done | bin/kafka-console-producer.sh
> --bootstrap-server localhost:9092 --topic t1
> This needs more investigation to fully understand if it's indeed something in
> the fetch path or something else)
--
This message was sent by Atlassian Jira
(v8.20.10#820010)