lianetm commented on code in PR #18475:
URL: https://github.com/apache/kafka/pull/18475#discussion_r1910555977


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java:
##########
@@ -167,6 +169,8 @@ private PendingFetchCommittedRequest(final 
Set<TopicPartition> requestedPartitio
     @Override
     public NetworkClientDelegate.PollResult poll(final long currentTimeMs) {
         // Copy the outgoing request list and clear it.
+        maybeRetryListOffsetsRequests(false);

Review Comment:
   I'm not sure we're achieving the same as the classic consumer with this, 
this is my reasoning: The classic does `client.awaitMetadataUpdate(timer)` if 
the request completes with a retriable error (and totally agree that the new 
consumer is not requesting a metadata update in that case, gap)
   
   But note that the classic does not retry until it receives the metadata 
response (awaitMetadata blocks until it gets a response or the timer expires, 
so only if it gets a metadata update in time it will loop back into the 
do-while that sends the requests, correct?)
   
   If my understanding is right, then to achieve the same in the new consumer 
isn't the missing part simply to call `metadata.requestUpdate` when the request 
fails with a retriable? Here maybe?
   
https://github.com/apache/kafka/blob/cd061c8039d615f6584eab522c19b407317ba031/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java#L583-L584
   
   This, without changing the part of WHEN requests are retried, which is when 
a metadata response is received (what awaitMetadataUpdate call achieved in the 
Class, and the retry on `onUpdate` achieved in the AsyncC).
   
   Also concerning that by retrying here on every poll iteration, seems we have 
no backoff in place for fetchOffsets and would be storming the broker? I expect 
the retry pace was intended to be driven by the metadata responses received, 
but we loose that by retrying on every poll iteration.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to