[ https://issues.apache.org/jira/browse/KAFKA-19259?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17959621#comment-17959621 ]
Kirk True commented on KAFKA-19259: ----------------------------------- tl;dr There’s a difference in the two consumers’ {{pollForFetches()}} methods in this case: {{ClassicKafkaConsumer}} doesn't block for {{{}pollTimeout{}}}, but AsyncKafkaConsumer does. At the beginning of {{{}ClassicKafkaConsumer{}}}, {{pollForFetches()}} calculates the value of {{{}pollTimeout{}}}. When {{pollForFetches()}} determines it needs to fetch more data, a {{FETCH}} request is enqueued into the {{NetworkClient}} within {{{}sendFetches(){}}}. Then {{ClassicKafkaConsumer}} calls {{ConsumerNetworkClient.poll()}} which will call {{{}NetworkClient.poll(){}}}. This will then send out the request over the network. Even though {{pollForFetches()}} has a value of 3000 for {{{}pollTimeout{}}}, the underlying call to {{NetworkClient.poll()}} returns _almost instantaneously_ because it successfully sent the {{FETCH}} request. So even though the {{pollTimeout}} value is 3000, the call to {{ConsumerNetworkClient.poll()}} doesn't block (in this case). {{{}AsyncKafkaConsumer{}}}, {{pollForFetches()}} starts the same way as {{{}ClassicKafkaConsumer{}}}, by calculating {{{}pollTimeout{}}}. When {{pollForFetches()}} determines it needs to fetch more data, a {{FETCH}} request is created (but not necessarily immediately enqueued into the {{NetworkClient}} within {{{}sendFetches(){}}}). Because the {{FETCH}} response is empty, there's no new data in {{{}FetchBuffer{}}}. {{AsyncKafkaConsumer}} then calls {{FetchBuffer.awaitNotEmpty()}} and proceeds to block there for the full length of {{{}pollTimeout{}}}. Because it's blocked, it's not sending out any more {{FETCH}} requests, causing the long pauses in the console consumer. > 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, consumer11_KAFKA-19259.log, > 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)