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



##########
File path: 
flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/metrics/testutils/MetricListener.java
##########
@@ -75,14 +75,13 @@ public MetricGroup getMetricGroup() {
      * metric group can be reached by identifier ("myGroup", "myMetric")
      *
      * @param identifier identifier relative to the root metric group
-     * @return Registered metric
+     * @return Registered metric, or null if the metric doesn't exist

Review comment:
       Use Optional?

##########
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:
       Mockito is banned for new stuff since we got burnt several times in the 
past.
   #16875 now managed to just use the existing stuff, so please check if you 
can use that.

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetrics.java
##########
@@ -150,6 +172,103 @@ public void recordFailedCommit() {
         commitsFailed.inc();
     }
 
+    /**
+     * Register {@link MetricNames#IO_NUM_BYTES_IN}.
+     *
+     * @param consumer Kafka consumer
+     */
+    public void registerNumBytesIn(KafkaConsumer<?, ?> consumer) {
+        try {
+            this.bytesConsumedTotalMetric =
+                    KafkaMetricUtils.getKafkaMetric(
+                            consumer.metrics(),
+                            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();
+                        }
+                        return pendingRecordsTotal;
+                    });
+        }
+
+        recordsLagMetrics.computeIfAbsent(
+                tp,
+                (ignored) -> {
+                    try {
+                        return KafkaMetricUtils.getKafkaMetric(
+                                consumer.metrics(),
+                                CONSUMER_FETCH_MANAGER_GROUP,
+                                RECORDS_LAG,
+                                (metric) -> {
+                                    final Map<String, String> tags = 
metric.metricName().tags();
+                                    return tags.containsKey("topic")
+                                            && 
tags.get("topic").equals(tp.topic())
+                                            && tags.containsKey("partition")
+                                            && tags.get("partition")
+                                                    
.equals(String.valueOf(tp.partition()));
+                                });
+                    } catch (IllegalStateException e) {
+                        LOG.warn(
+                                String.format(
+                                        "Error when getting Kafka consumer 
metric \"%s\" "
+                                                + "for partition \"%s\". "
+                                                + "Metric \"%s\" may not be 
reported correctly. ",
+                                        RECORDS_LAG, tp, 
MetricNames.PENDING_BYTES),
+                                e);
+                        return null;
+                    }
+                });

Review comment:
       Can we please avoid such large lambdas by pulling out method lambdas?

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetrics.java
##########
@@ -150,6 +172,103 @@ public void recordFailedCommit() {
         commitsFailed.inc();
     }
 
+    /**
+     * Register {@link MetricNames#IO_NUM_BYTES_IN}.
+     *
+     * @param consumer Kafka consumer
+     */
+    public void registerNumBytesIn(KafkaConsumer<?, ?> consumer) {
+        try {
+            this.bytesConsumedTotalMetric =
+                    KafkaMetricUtils.getKafkaMetric(
+                            consumer.metrics(),
+                            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();
+                        }
+                        return pendingRecordsTotal;
+                    });
+        }
+
+        recordsLagMetrics.computeIfAbsent(
+                tp,
+                (ignored) -> {
+                    try {
+                        return KafkaMetricUtils.getKafkaMetric(
+                                consumer.metrics(),
+                                CONSUMER_FETCH_MANAGER_GROUP,
+                                RECORDS_LAG,
+                                (metric) -> {
+                                    final Map<String, String> tags = 
metric.metricName().tags();
+                                    return tags.containsKey("topic")
+                                            && 
tags.get("topic").equals(tp.topic())
+                                            && tags.containsKey("partition")
+                                            && tags.get("partition")
+                                                    
.equals(String.valueOf(tp.partition()));
+                                });
+                    } catch (IllegalStateException e) {
+                        LOG.warn(
+                                String.format(
+                                        "Error when getting Kafka consumer 
metric \"%s\" "
+                                                + "for partition \"%s\". "
+                                                + "Metric \"%s\" may not be 
reported correctly. ",
+                                        RECORDS_LAG, tp, 
MetricNames.PENDING_BYTES),
+                                e);
+                        return null;
+                    }
+                });
+    }
+
+    /**
+     * Remove a partition's records-lag metric from tracking list.
+     *
+     * @param tp Unassigned topic partition
+     */
+    public void removeRecordsLagMetric(TopicPartition tp) {
+        if (recordsLagMetrics != null) {
+            recordsLagMetrics.remove(tp);
+        }
+    }
+
+    /** Update {@link 
org.apache.flink.runtime.metrics.MetricNames#IO_NUM_BYTES_IN}. */
+    public void updateNumBytesInCounter() {
+        if (this.bytesConsumedTotalMetric != null) {
+            KafkaMetricUtils.sync(
+                    this.bytesConsumedTotalMetric,
+                    
this.sourceReaderMetricGroup.getIOMetricGroup().getNumBytesInCounter());

Review comment:
       I think we also need sum up the metrics per split, right? Currently, if 
we have two splits and consumers, they would overwrite each other.




-- 
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: issues-unsubscr...@flink.apache.org

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


Reply via email to