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


##########
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:
   Yes, this sounds like a useful metric to have. Thanks!



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/metrics/AsyncConsumerMetrics.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals.metrics;
+
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Value;
+
+import java.util.Arrays;
+
+import static 
org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRICS_SUFFIX;
+
+public class AsyncConsumerMetrics extends KafkaConsumerMetrics implements 
AutoCloseable {
+    private final Metrics metrics;
+
+    public static final String TIME_BETWEEN_NETWORK_THREAD_POLL_SENSOR_NAME = 
"time-between-network-thread-poll";
+    public static final String APPLICATION_EVENT_QUEUE_SIZE_SENSOR_NAME = 
"application-event-queue-size";
+    public static final String APPLICATION_EVENT_QUEUE_TIME_SENSOR_NAME = 
"application-event-queue-time";
+    public static final String 
APPLICATION_EVENT_QUEUE_PROCESSING_TIME_SENSOR_NAME = 
"application-event-queue-processing-time";
+    public static final String APPLICATION_EVENT_EXPIRED_SIZE_SENSOR_NAME = 
"application-event-expired-size";
+    public static final String BACKGROUND_EVENT_QUEUE_SIZE_SENSOR_NAME = 
"background-event-queue-size";
+    public static final String BACKGROUND_EVENT_QUEUE_TIME_SENSOR_NAME = 
"background-event-queue-time";
+    public static final String 
BACKGROUND_EVENT_QUEUE_PROCESSING_TIME_SENSOR_NAME = 
"background-event-queue-processing-time";
+    public static final String UNSENT_REQUESTS_QUEUE_SIZE_SENSOR_NAME = 
"unsent-requests-queue-size";
+    public static final String UNSENT_REQUESTS_QUEUE_TIME_SENSOR_NAME = 
"unsent-requests-queue-time";
+    private final Sensor timeBetweenNetworkThreadPollSensor;
+    private final Sensor applicationEventQueueSizeSensor;
+    private final Sensor applicationEventQueueTimeSensor;
+    private final Sensor applicationEventQueueProcessingTimeSensor;
+    private final Sensor applicationEventExpiredSizeSensor;
+    private final Sensor backgroundEventQueueSizeSensor;
+    private final Sensor backgroundEventQueueTimeSensor;
+    private final Sensor backgroundEventQueueProcessingTimeSensor;
+    private final Sensor unsentRequestsQueueSizeSensor;
+    private final Sensor unsentRequestsQueueTimeSensor;
+
+    public AsyncConsumerMetrics(Metrics metrics, String metricGrpPrefix) {

Review Comment:
   Nit: I made a quick pass and I don't see anywhere that we're passing in 
anything other than `CONSUMER_METRIC_GROUP_PREFIX` or `"consumer"`. Does it 
make sense to provide this parameter if the value is always the same?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/AsyncConsumerMetricsTest.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals.metrics;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.metrics.Metrics;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.HashSet;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class AsyncConsumerMetricsTest {
+    private static final long METRIC_VALUE = 123L;
+    private static final String CONSUMER_GROUP_PREFIX = "consumer";
+    private static final String CONSUMER_METRIC_GROUP = "consumer-metrics";

Review Comment:
   Aren't these constants already declared elsewhere?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/metrics/AsyncConsumerMetricsTest.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals.metrics;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.metrics.Metrics;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.HashSet;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class AsyncConsumerMetricsTest {
+    private static final long METRIC_VALUE = 123L;
+    private static final String CONSUMER_GROUP_PREFIX = "consumer";
+    private static final String CONSUMER_METRIC_GROUP = "consumer-metrics";
+
+    private final Metrics metrics = new Metrics();
+    private AsyncConsumerMetrics consumerMetrics;
+
+    @AfterEach
+    public void tearDown() {
+        if (consumerMetrics != null) {
+            consumerMetrics.close();
+        }
+        metrics.close();
+    }
+
+    @Test
+    public void shouldMetricNames() {
+        // create
+        consumerMetrics = new AsyncConsumerMetrics(metrics, 
CONSUMER_GROUP_PREFIX);
+        HashSet<MetricName> expectedMetrics = new HashSet<>(Arrays.asList(
+            metrics.metricName("last-poll-seconds-ago", CONSUMER_METRIC_GROUP),
+            metrics.metricName("time-between-poll-avg", CONSUMER_METRIC_GROUP),
+            metrics.metricName("time-between-poll-max", CONSUMER_METRIC_GROUP),
+            metrics.metricName("poll-idle-ratio-avg", CONSUMER_METRIC_GROUP),
+            metrics.metricName("commit-sync-time-ns-total", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("committed-time-ns-total", 
CONSUMER_METRIC_GROUP)
+        ));
+        expectedMetrics.forEach(metricName -> 
assertTrue(metrics.metrics().containsKey(metricName), "Missing metric: " + 
metricName));
+
+        HashSet<MetricName> expectedConsumerMetrics = new 
HashSet<>(Arrays.asList(
+            metrics.metricName("time-between-network-thread-poll-avg", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("time-between-network-thread-poll-max", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("application-event-queue-size", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("application-event-queue-time-avg", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("application-event-queue-time-max", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("application-event-queue-processing-time-avg", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("application-event-queue-processing-time-max", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("unsent-requests-queue-size", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("unsent-requests-queue-time-avg", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("unsent-requests-queue-time-max", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("background-event-queue-size", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("background-event-queue-time-avg", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("background-event-queue-time-max", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("background-event-queue-processing-time-avg", 
CONSUMER_METRIC_GROUP),
+            metrics.metricName("background-event-queue-processing-time-max", 
CONSUMER_METRIC_GROUP)
+        ));
+        expectedConsumerMetrics.forEach(metricName -> 
assertTrue(metrics.metrics().containsKey(metricName), "Missing metric: " + 
metricName));
+
+        // close
+        consumerMetrics.close();
+        expectedMetrics.forEach(metricName -> 
assertFalse(metrics.metrics().containsKey(metricName), "Missing metric: " + 
metricName));
+        expectedConsumerMetrics.forEach(metricName -> 
assertFalse(metrics.metrics().containsKey(metricName), "Missing metric: " + 
metricName));

Review Comment:
   The error message on test failure is incorrect here, right? Shouldn't it be:
   
   ```suggestion
           expectedMetrics.forEach(metricName -> 
assertFalse(metrics.metrics().containsKey(metricName), "Metric present after 
close: " + metricName));
           expectedConsumerMetrics.forEach(metricName -> 
assertFalse(metrics.metrics().containsKey(metricName), "Metric present after 
close: " + metricName));
   ```



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