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

Shivsundar R commented on KAFKA-19804:
--------------------------------------

Hey [~brandboat] , thanks for the investigation. 
I too faced this issue of pollTimer in {color:#0747a6}ShareConsumerImpl{color}
 being 0 initially (the heartbeatInterval=0 makes the maximumTimeToWait=0).
{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.
{quote}
Yes, but the {color:#0747a6}pollTimeout{color} also is based on the timeout 
argument passed to poll(). We will take the minimum of the 
{color:#0747a6}applicationTimeout{color}
(in this case would be 30 seconds as we set 
{color:#0747a6}heartbeatInterval{color} to 30s) and the pollTimeout(which the 
user passes in poll).

Assuming the user passes in 30000ms in poll, the poll only blocks the 
application thread(until data is received), the request managers in the 
background thread meanwhile will still send the next heartbeat request and 
reconcile the assignment when the heartbeat response is received. The 
FetchRequestManager and ShareConsumeRequestManager would still poll() and fetch 
data once partitions are assigned.

So we really would not block the operations by setting the value to 30s too. 
What do you think?

> 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