[ 
https://issues.apache.org/jira/browse/KAFKA-19259?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17956545#comment-17956545
 ] 

Arpit Goyal commented on KAFKA-19259:
-------------------------------------

Hi [~lianetm] 
Update on the above issue:
It seems I figured out the issue. It seems we are not sending immediate fetches 
in the new consumer compared to the classic one. Check this logs , I added 
debug logs in awaitNotEmpty function. Ideally the consumer should be making 
multiple fetch requests during the 5-second poll period, especially since each 
fetch request has maxWaitMs=500ms. Instead, it seems to be waiting the full 
poll timeout after receiving just one empty response. 
As a next step I will be checking  part of the code which is preventing fetches.


{code:java}
[2025-06-06 13:07:26,720] DEBUG [Consumer clientId=console-consumer, 
groupId=cg2] Entering awaitNotEmpty with timer remaining: 4949ms 
(org.apache.kafka.clients.consumer.internals.FetchBuffer)
[2025-06-06 13:07:26,720] DEBUG [Consumer clientId=console-consumer, 
groupId=cg2] Buffer is empty, will wait. Timer remaining: 4949ms, wokenup: 
false (org.apache.kafka.clients.consumer.internals.FetchBuffer)
[2025-06-06 13:07:26,720] DEBUG [Consumer clientId=console-consumer, 
groupId=cg2] Starting wait at 1749195446720 for 4949ms 
(org.apache.kafka.clients.consumer.internals.FetchBuffer)
[2025-06-06 13:07:27,230] DEBUG [Consumer clientId=console-consumer, 
groupId=cg2] Received FETCH response from node 1 for request with header 
RequestHeader(apiKey=FETCH, apiVersion=17, clientId=console-consumer, 
correlationId=44, headerVersion=2): FetchResponseData(throttleTimeMs=0, 
errorCode=0, sessionId=963412284, responses=[], nodeEndpoints=[]) 
(org.apache.kafka.clients.NetworkClient)
[2025-06-06 13:07:27,325] DEBUG [Consumer clientId=console-consumer, 
groupId=cg2] Node 1 sent an incremental fetch response with throttleTimeMs = 0 
for session 963412284 with 0 response partition(s), 6 implied partition(s) 
(org.apache.kafka.clients.FetchSessionHandler)
[2025-06-06 13:07:27,325] DEBUG [Consumer clientId=console-consumer, 
groupId=cg2] Removing pending request for fetch session: 963412284 for node: 
localhost:9092 (id: 1 rack: null isFenced: false) 
(org.apache.kafka.clients.consumer.internals.AbstractFetch)
[2025-06-06 13:07:31,678] DEBUG [Consumer clientId=console-consumer, 
groupId=cg2] Wait timed out after 4955ms 
(org.apache.kafka.clients.consumer.internals.FetchBuffer)
{code}



> 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)

Reply via email to