kirktrue commented on code in PR #20521:
URL: https://github.com/apache/kafka/pull/20521#discussion_r2418088765


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -217,35 +235,109 @@ public void process(ApplicationEvent event) {
         }
     }
 
-    private void process(final PollEvent event) {
-        // Trigger a reconciliation that can safely commit offsets if needed 
to rebalance,
-        // as we're processing before any new fetching starts in the app thread
-        
requestManagers.consumerMembershipManager.ifPresent(consumerMembershipManager ->
-            consumerMembershipManager.maybeReconcile(true));
-        if (requestManagers.commitRequestManager.isPresent()) {
-            CommitRequestManager commitRequestManager = 
requestManagers.commitRequestManager.get();
-            commitRequestManager.updateTimerAndMaybeCommit(event.pollTimeMs());
-            // all commit request generation points have been passed,
-            // so it's safe to notify the app thread could proceed and start 
fetching
-            event.markReconcileAndAutoCommitComplete();
-            requestManagers.consumerHeartbeatRequestManager.ifPresent(hrm -> {
-                hrm.membershipManager().onConsumerPoll();
-                hrm.resetPollTimer(event.pollTimeMs());
-            });
-            requestManagers.streamsGroupHeartbeatRequestManager.ifPresent(hrm 
-> {
-                hrm.membershipManager().onConsumerPoll();
-                hrm.resetPollTimer(event.pollTimeMs());
-            });
-        } else {
-            // safe to unblock - no auto-commit risk here:
-            // 1. commitRequestManager is not present
-            // 2. shareConsumer has no auto-commit mechanism
-            event.markReconcileAndAutoCommitComplete();
-            requestManagers.shareHeartbeatRequestManager.ifPresent(hrm -> {
-                hrm.membershipManager().onConsumerPoll();
-                hrm.resetPollTimer(event.pollTimeMs());
+    private void process(final CompositePollEvent event) {
+        if (maybeFailCompositePoll(event) || maybePauseCompositePoll(event, 
ApplicationEvent.Type.POLL))
+            return;
+
+        ApplicationEvent.Type nextEventType = event.nextEventType();
+
+        if (nextEventType == ApplicationEvent.Type.POLL) {
+            log.debug("Processing {} logic for {}", nextEventType, event);
+            processPollEvent(event.pollTimeMs());
+            nextEventType = ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA;
+
+            if (maybeFailCompositePoll(event) || 
maybePauseCompositePoll(event, nextEventType))
+                return;
+        }
+
+        if (nextEventType == 
ApplicationEvent.Type.UPDATE_SUBSCRIPTION_METADATA) {
+            log.debug("Processing {} logic for {}", nextEventType, event);
+            processUpdatePatternSubscriptionEvent();
+            nextEventType = ApplicationEvent.Type.CHECK_AND_UPDATE_POSITIONS;
+
+            if (maybeFailCompositePoll(event) || 
maybePauseCompositePoll(event, nextEventType))
+                return;
+        }

Review Comment:
   I made some changes in this area. Let me know if this comment thread can be 
marked as resolved.



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