fapaul commented on a change in pull request #16838:
URL: https://github.com/apache/flink/pull/16838#discussion_r691968413



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetrics.java
##########
@@ -150,6 +171,104 @@ public void recordFailedCommit() {
         commitsFailed.inc();
     }
 
+    /**
+     * Register {@link MetricNames#IO_NUM_BYTES_IN}.
+     *
+     * @param consumer Kafka consumer
+     */
+    public void registerNumBytesIn(KafkaConsumer<?, ?> consumer) {
+        try {
+            this.bytesConsumedTotalMetric =
+                    KafkaConsumerMetricUtils.getKafkaConsumerMetric(
+                            consumer,
+                            CONSUMER_FETCH_MANAGER_GROUP,
+                            BYTES_CONSUMED_TOTAL,
+                            (metric) -> 
!metric.metricName().tags().containsKey("topic"));
+        } catch (IllegalStateException e) {
+            LOG.warn(
+                    String.format(
+                            "Error when getting Kafka consumer metric \"%s\". "
+                                    + "I/O metric \"%s\" will not be reported. 
",
+                            BYTES_CONSUMED_TOTAL, MetricNames.IO_NUM_BYTES_IN),
+                    e);
+        }
+    }
+
+    /**
+     * Add a partition's records-lag metric to tracking list if this partition 
never appears before.
+     *
+     * <p>This method also lazily register {@link
+     * org.apache.flink.runtime.metrics.MetricNames#PENDING_RECORDS} in {@link
+     * SourceReaderMetricGroup}
+     *
+     * @param consumer Kafka consumer
+     * @param tp Topic partition
+     */
+    public void maybeAddRecordsLagMetric(KafkaConsumer<?, ?> consumer, 
TopicPartition tp) {
+        // Lazily register pendingRecords

Review comment:
       Why is this done lazily?

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaConsumerMetricUtils.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.flink.connector.kafka.source.metrics;
+
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/** Utilities for handling metrics in Kafka consumer. */
+public class KafkaConsumerMetricUtils {
+
+    /**
+     * Get the specific Kafka consumer metric matching given group, name, and 
predicate for
+     * filtering.
+     *
+     * @param consumer Kafka consumer
+     * @param group Group name of the metric
+     * @param name Name of the metric
+     * @param predicate Predicate for filtering metrics with same group and 
name
+     * @return Matching Kafka metric
+     * @throws IllegalStateException if no metric or multiple metrics match 
the given condition
+     */
+    public static Metric getKafkaConsumerMetric(
+            KafkaConsumer<?, ?> consumer, String group, String name, 
Predicate<Metric> predicate) {
+        final List<? extends Metric> matchingMetrics =
+                getKafkaConsumerMetrics(consumer, group, name, predicate);
+        if (matchingMetrics.isEmpty()) {
+            throw new IllegalStateException(
+                    "No Kafka consumer metric matches the provided condition");
+        }
+        if (matchingMetrics.size() > 1) {
+            throw new IllegalStateException(
+                    "Multiple Kafka consumer metrics match the provided 
condition");
+        }
+        return matchingMetrics.get(0);
+    }
+
+    /**
+     * Get a list of Kafka consumer metric matching given group, name, and 
predicate for filtering.
+     *
+     * @param consumer Kafka consumer
+     * @param group Group name of the metric
+     * @param name Name of the metric
+     * @param predicate Predicate for filtering metrics with same group and 
name
+     * @return Matching Kafka metric
+     */
+    public static List<? extends Metric> getKafkaConsumerMetrics(

Review comment:
       Probably should be `private`

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaConsumerMetricUtils.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.flink.connector.kafka.source.metrics;
+
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/** Utilities for handling metrics in Kafka consumer. */
+public class KafkaConsumerMetricUtils {

Review comment:
       WDYT about introducing a more general tool like 
https://github.com/fapaul/flink/commit/5181be4fa5157535ebc1277fda6a97cc36d60c53#diff-245045b28fefe52d71fe29bb70ccdbe4fd64d3df0600db813843b2047ea9bd6a
 for KafkaSink and KafkaSource. In general, I do not see a reason why the util 
should only work with a consumer.

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetrics.java
##########
@@ -150,6 +171,104 @@ public void recordFailedCommit() {
         commitsFailed.inc();
     }
 
+    /**
+     * Register {@link MetricNames#IO_NUM_BYTES_IN}.
+     *
+     * @param consumer Kafka consumer
+     */
+    public void registerNumBytesIn(KafkaConsumer<?, ?> consumer) {
+        try {
+            this.bytesConsumedTotalMetric =
+                    KafkaConsumerMetricUtils.getKafkaConsumerMetric(
+                            consumer,
+                            CONSUMER_FETCH_MANAGER_GROUP,
+                            BYTES_CONSUMED_TOTAL,
+                            (metric) -> 
!metric.metricName().tags().containsKey("topic"));
+        } catch (IllegalStateException e) {
+            LOG.warn(
+                    String.format(
+                            "Error when getting Kafka consumer metric \"%s\". "
+                                    + "I/O metric \"%s\" will not be reported. 
",
+                            BYTES_CONSUMED_TOTAL, MetricNames.IO_NUM_BYTES_IN),
+                    e);
+        }
+    }
+
+    /**
+     * Add a partition's records-lag metric to tracking list if this partition 
never appears before.
+     *
+     * <p>This method also lazily register {@link
+     * org.apache.flink.runtime.metrics.MetricNames#PENDING_RECORDS} in {@link
+     * SourceReaderMetricGroup}
+     *
+     * @param consumer Kafka consumer
+     * @param tp Topic partition
+     */
+    public void maybeAddRecordsLagMetric(KafkaConsumer<?, ?> consumer, 
TopicPartition tp) {
+        // Lazily register pendingRecords
+        if (recordsLagMetrics == null) {
+            this.recordsLagMetrics = new ConcurrentHashMap<>();
+            this.sourceReaderMetricGroup.setPendingRecordsGauge(
+                    () -> {
+                        long pendingRecordsTotal = 0;
+                        for (Metric recordsLagMetric : 
this.recordsLagMetrics.values()) {
+                            pendingRecordsTotal +=
+                                    ((Double) 
recordsLagMetric.metricValue()).longValue();

Review comment:
       If I understand this code correctly we sum the lag of all assigned 
partitions. What is the difference between doing so and just using 
`records-lag-max`?

##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java
##########
@@ -115,6 +130,76 @@ public void testWakeUp() throws Exception {
         assertNull(error.get());
     }
 
+    @Test
+    public void testNumBytesInCounter() throws Exception {
+        // Create a task metric group for intercepting numBytesOut
+        final InterceptingTaskMetricGroup taskMetricGroup = new 
InterceptingTaskMetricGroup();
+        final OperatorMetricGroup operatorMetricGroup =
+                taskMetricGroup.getOrAddOperator(new OperatorID(0, 0), 
"fakeOperator");
+        KafkaPartitionSplitReader<Integer> reader =
+                createReader(
+                        new Properties(),
+                        
InternalSourceReaderMetricGroup.wrap(operatorMetricGroup));

Review comment:
       I think it is better to introduce a new method 
`InternalSourceReaderMetricGroup#mock(MetricGroup metricGroup, 
OperatorIOMetricGroup opMetricGroup)` like I did here 
https://github.com/fapaul/flink/commit/01de3e6db4c68d316d02bda35279e163673aa2bc 
and not use the task metric.

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaConsumerMetricUtils.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.flink.connector.kafka.source.metrics;
+
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/** Utilities for handling metrics in Kafka consumer. */
+public class KafkaConsumerMetricUtils {

Review comment:
       Add `@Internal`




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