ShivsundarR commented on code in PR #16727:
URL: https://github.com/apache/kafka/pull/16727#discussion_r1705259783


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManager.java:
##########
@@ -215,37 +215,20 @@ public void fetch(Map<TopicIdPartition, Acknowledgements> 
acknowledgementsMap) {
      */
     private PollResult processAcknowledgements(long currentTimeMs) {
         List<UnsentRequest> unsentRequests = new ArrayList<>();
-        Iterator<AcknowledgeRequestState> iterator = 
acknowledgeRequestStates.iterator();
-        while (iterator.hasNext()) {
-            AcknowledgeRequestState acknowledgeRequestState = iterator.next();
-            if (acknowledgeRequestState.isProcessed()) {
-                iterator.remove();
-            } else if (!acknowledgeRequestState.maybeExpire()) {
-                if 
(nodesWithPendingRequests.contains(acknowledgeRequestState.nodeId)) {
-                    log.trace("Skipping acknowledge request because previous 
request to {} has not been processed", acknowledgeRequestState.nodeId);
-                } else {
-                    if (acknowledgeRequestState.canSendRequest(currentTimeMs)) 
{
-                        acknowledgeRequestState.onSendAttempt(currentTimeMs);
-                        UnsentRequest request = 
acknowledgeRequestState.buildRequest(currentTimeMs);
-                        if (request != null) {
-                            unsentRequests.add(request);
-                        }
-                    }
-                }
-            } else {
-                // Fill in TimeoutException
-                for (TopicIdPartition tip : 
acknowledgeRequestState.acknowledgementsMap.keySet()) {
-                    
metricsManager.recordFailedAcknowledgements(acknowledgeRequestState.getAcknowledgementsCount(tip));
-                    acknowledgeRequestState.handleAcknowledgeErrorCode(tip, 
Errors.REQUEST_TIMED_OUT);
-                }
-                iterator.remove();
-            }
+        for (Pair<AcknowledgeRequestState> requestStates : 
acknowledgeRequestStates.values()) {
+            isAsyncDone = false;
+            // For commitAsync
+            maybeBuildRequest(requestStates.getAsyncRequest(), currentTimeMs, 
true).ifPresent(unsentRequests::add);
+
+            // Check to ensure we start processing commitSync/close only if 
there are no commitAsync requests left to process.
+            if (isAsyncDone)
+                maybeBuildRequest(requestStates.getSyncRequest(), 
currentTimeMs, false).ifPresent(unsentRequests::add);
         }
 
         PollResult pollResult = null;
         if (!unsentRequests.isEmpty()) {
             pollResult = new PollResult(unsentRequests);
-        } else if (!acknowledgeRequestStates.isEmpty()) {
+        } else if (areAnyAcknowledgementsLeft()) {

Review Comment:
   Right yeah, we do need to prune the acknowledgements here, I have renamed 
the method name to make it clearer.



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