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

Kuan Po Tseng commented on KAFKA-19804:
---------------------------------------

Hi [~lianetm],
I took a closer look at this issue and wanted to share a few thoughts.
{quote}
That 0 ms initial interval causes the HB manager poll to run continuously in a 
tight loop, executing logic that may not really be needed—it mostly just waits 
for a response or failure.
{quote}
This actually shouldn’t happen because we already check whether there’s any 
in-flight heartbeat request before sending a new one:
https://github.com/apache/kafka/blob/1330870efbb4efd9fd394ec5cb8a0fecf8e69b24/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestState.java#L96
https://github.com/apache/kafka/blob/80f31224aad543dbfc892bce1ad73b6bb693855a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestState.java#L79

{quote}
We should review and consider setting a non-zero initial interval (while we 
wait for the actual interval from the broker). One option to consider would be 
using the request timeout maybe (just a first thought)
{quote}

It’s also worth noting that if we set a non-zero initial interval, the first 
poll() call will block until the timeout because no partitions are assigned 
yet. That means no Fetch requests are sent and the fetchBuffer remains empty 
until the poll timeout expires. This "tight loop" behavior on the application 
thread is actually intentional per PR 
https://github.com/apache/kafka/pull/14835 to mitigate the case where 
Consumer.poll(Duration timeout) would otherwise block for the entire duration.

I’m thinking we could set a small non-zero initial heartbeat interval, what 
about 1s?  to slightly loosen the tight loop. Of course, this means the first 
poll will still block for up to 1s. Using request.timeout.ms (default 30 s) 
with pollTimeout (e.g., 15 s) would make the first poll take too long before 
doing any useful work.

> Improve heartbeat request manager initial HB interval 
> ------------------------------------------------------
>
>                 Key: KAFKA-19804
>                 URL: https://issues.apache.org/jira/browse/KAFKA-19804
>             Project: Kafka
>          Issue Type: Task
>          Components: clients, consumer
>            Reporter: Lianet Magrans
>            Assignee: Kuan Po Tseng
>            Priority: Major
>             Fix For: 4.2.0
>
>
> With KIP-848, consumer HB interval config moved to the broker, so currently, 
> the consumer HB request manager starts with a 0ms interval (mainly to send a 
> first HB right away after the consumer subscribe + poll). Once a response is 
> received, the consumer takes the interval from the response and starts using 
> it. 
> That 0ms initial interval makes that the HB mgr poll continuously executes 
> logic on a tight loop that may not really be needed. It mostly has to wait 
> for a response (or a failure).
> Probably worse than this, is the impact on the app thread, given that 
> pollTimeout takes into account the maxTimeToWait from the network thread, 
> that is directly impacted by the timeToNextHeartbeat
>  * 
> [https://github.com/apache/kafka/blob/388739f5d847d7a16e389d9891f806547f023476/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java#L1764-L1766]
>  * 
> [https://github.com/apache/kafka/blob/781bc7a54b8c4f7c86f0d6bb9ef8399d86d0735e/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java#L255]
> We should review and consider setting a non-zero initial interval (while we 
> wait for the actual interval from the broker). One option to consider would 
> be using the request timeout maybe (just a first thought)
> High level goals here would be to:
>  * maintain the behaviour of sending a first HB without delay 
>  * ensure no unneeded activity on the HB mgr poll in the background, in tight 
> loop, while we're just waiting for the first HB response with an interval
>  * ensure the app thread poll timeout is not affected



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to