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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1893,25 +1901,30 @@ private void subscribeInternal(Collection<String> 
topics, Optional<ConsumerRebal
      * It is possible that {@link ErrorEvent an error}
      * could occur when processing the events. In such cases, the processor 
will take a reference to the first
      * error, continue to process the remaining events, and then throw the 
first error that occurred.
+     *
+     * Visible for testing.
      */
-    private boolean processBackgroundEvents() {
+    boolean processBackgroundEvents() {
         AtomicReference<KafkaException> firstError = new AtomicReference<>();
 
-        LinkedList<BackgroundEvent> events = new LinkedList<>();
-        backgroundEventQueue.drainTo(events);
-
-        for (BackgroundEvent event : events) {
-            try {
-                if (event instanceof CompletableEvent)
-                    backgroundEventReaper.add((CompletableEvent<?>) event);
-
-                backgroundEventProcessor.process(event);
-            } catch (Throwable t) {
-                KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t);
-
-                if (!firstError.compareAndSet(null, e))
-                    log.warn("An error occurred when processing the background 
event: {}", e.getMessage(), e);
+        List<BackgroundEvent> events = backgroundEventHandler.drainEvents();
+        if (!events.isEmpty()) {
+            long startMs = time.milliseconds();
+            for (BackgroundEvent event : events) {
+                
kafkaConsumerMetrics.recordBackgroundEventQueueTime(time.milliseconds() - 
event.enqueuedMs());
+                try {
+                    if (event instanceof CompletableEvent)
+                        backgroundEventReaper.add((CompletableEvent<?>) event);
+
+                    backgroundEventProcessor.process(event);
+                } catch (Throwable t) {
+                    KafkaException e = 
ConsumerUtils.maybeWrapAsKafkaException(t);
+
+                    if (!firstError.compareAndSet(null, e))
+                        log.warn("An error occurred when processing the 
background event: {}", e.getMessage(), e);
+                }
             }
+            
kafkaConsumerMetrics.recordBackgroundEventQueueProcessingTime(time.milliseconds()
 - startMs);
         }
 
         backgroundEventReaper.reap(time.milliseconds());

Review Comment:
   Agree that it's not applicable for background events because this callback 
needed is the only `CompletableBackgroundEvent`, and it's intentionally not 
expired (I don't think we need to change that)
   
   I would say that what might be interesting to know is expiration of 
Application events. 
   
https://github.com/apache/kafka/blob/ee4264439ddda7bdebcaa845752b824abba14161/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java#L184
   
   There we do have lots of events with deadline, I guess that's what 
@AndrewJSchofield had in mind maybe? (I notice now that the initial comment was 
here on the reap of background events).



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to