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



##########
File path: 
flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/groups/UnregisteredMetricsGroup.java
##########
@@ -89,4 +89,47 @@ public String getMetricIdentifier(String metricName) {
     public String getMetricIdentifier(String metricName, CharacterFilter 
filter) {
         return metricName;
     }
+
+    public static OperatorMetricGroup createUnregisteredOperatorMetricGroup() {

Review comment:
       Nit: Maybe `createUnregisteredOperatorMetricGroup` -> 
`createOperatorMetricGroup` since unregistered is already in the class name.

##########
File path: 
flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/TestReporter.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.metrics.reporter;
+
+import org.apache.flink.metrics.LogicalScopeProvider;
+import org.apache.flink.metrics.Metric;
+import org.apache.flink.metrics.MetricConfig;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorMetricGroup;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A mock implementation that makes all reported metrics available for tests. 
Metrics remain
+ * registered even after the job finishes for easier assertions.
+ */
+public class TestReporter implements MetricReporter {

Review comment:
       I do not really understand why a mock implementation is part of the 
production and not some test package.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalOperatorIOMetricGroup.java
##########
@@ -21,21 +21,26 @@
 import org.apache.flink.metrics.Counter;
 import org.apache.flink.metrics.Meter;
 import org.apache.flink.metrics.MeterView;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
 import org.apache.flink.runtime.metrics.MetricNames;
 
 /**
  * Metric group that contains shareable pre-defined IO-related metrics. The 
metrics registration is
  * forwarded to the parent operator metric group.
  */
-public class OperatorIOMetricGroup extends 
ProxyMetricGroup<OperatorMetricGroup> {
+public class InternalOperatorIOMetricGroup extends 
ProxyMetricGroup<InternalOperatorMetricGroup>

Review comment:
       `@Internal`?

##########
File path: 
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java
##########
@@ -143,7 +143,11 @@ public void 
testMultipleSplitsWithDifferentFinishingMoments() throws Exception {
                         .setBlockingFetch(false)
                         .build();
         MockSourceReader reader =
-                new MockSourceReader(elementsQueue, () -> mockSplitReader, 
getConfig(), null);
+                new MockSourceReader(
+                        elementsQueue,
+                        () -> mockSplitReader,
+                        getConfig(),
+                        new TestingReaderContext());

Review comment:
       Why did you change it from `null` to a value?

##########
File path: 
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceITCase.java
##########
@@ -60,6 +65,20 @@ public void testMultipleSources() throws Exception {
         executeAndVerify(env, stream1.union(stream2), 40);
     }
 
+    @Test
+    public void testMetrics() throws Exception {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        MockBaseSource source = new MockBaseSource(2, 10, Boundedness.BOUNDED);
+        DataStream<Integer> stream =
+                env.fromSource(source, WatermarkStrategy.noWatermarks(), 
"TestingSource");
+        executeAndVerify(env, stream, 20);
+        Map<String, Metric> metrics =
+                
miniClusterResource.getMetricReporter().getMetrics().entrySet().stream()
+                        .filter(m -> m.getKey().contains("TestingSource"))
+                        .collect(Collectors.toMap(e -> e.getKey(), e -> 
e.getValue()));
+        assertThat(metrics.entrySet(), Matchers.contains(7));

Review comment:
       Nit: Can you use a variable for the `7` because it is hard to understand 
what this should describe.

##########
File path: 
flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/metrics/testutils/MetricMatchers.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.metrics.testutils;
+
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.Metric;
+
+import org.hamcrest.Description;
+import org.hamcrest.DiagnosingMatcher;
+import org.hamcrest.Matcher;
+import org.hamcrest.core.IsInstanceOf;
+
+import java.util.function.Function;
+
+/**
+ * Provides hamcrest Matchers for metric assertions.
+ *
+ * @see org.apache.flink.metrics.reporter.TestReporter
+ */
+public enum MetricMatchers {
+    ;

Review comment:
       Nit: Remove `;`?

##########
File path: 
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SinkITCase.java
##########
@@ -256,6 +275,95 @@ public void writerAndGlobalCommitterExecuteInBatchMode() 
throws Exception {
                 
containsInAnyOrder(EXPECTED_GLOBAL_COMMITTED_DATA_IN_BATCH_MODE.toArray()));
     }
 
+    @Test
+    public void testMetrics() throws Exception {

Review comment:
       The code for the source metric tests looks very similar can you try to 
generify this a bit?

##########
File path: 
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceITCase.java
##########
@@ -19,30 +19,175 @@
 package org.apache.flink.connector.base.source.reader;
 
 import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner;
 import org.apache.flink.api.common.eventtime.WatermarkStrategy;
 import org.apache.flink.api.connector.source.Boundedness;
 import org.apache.flink.api.connector.source.Source;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.connector.base.source.reader.mocks.MockBaseSource;
+import org.apache.flink.connector.base.source.reader.mocks.MockRecordEmitter;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.metrics.Gauge;
 import org.apache.flink.metrics.Metric;
+import org.apache.flink.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.MetricNames;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.testutils.junit.SharedReference;
 
-import org.hamcrest.Matchers;
+import org.junit.Rule;
 import org.junit.Test;
 
+import java.time.Duration;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
+import java.util.concurrent.CyclicBarrier;
 
+import static org.apache.flink.metrics.testutils.MetricMatchers.isCounter;
+import static org.apache.flink.metrics.testutils.MetricMatchers.isGauge;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.Matchers.both;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.lessThan;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
 /** IT case for the {@link Source} with a coordinator. */
 public class CoordinatedSourceITCase extends AbstractTestBase {
+    @Rule public final SharedObjects sharedObjects = SharedObjects.create();
+
+    // since integration tests depend on wall clock time, use huge lags
+    private static final long EVENTTIME_LAG = Duration.ofDays(100).toMillis();
+    private static final long WATERMARK_LAG = Duration.ofDays(1).toMillis();
+    private static final long EVENTTIME_EPSILON = 
Duration.ofDays(20).toMillis();
+    // this basically is the time a build is allowed to be frozen before the 
test fails
+    private static final long WATERMARK_EPSILON = 
Duration.ofHours(6).toMillis();
+
+    @Test
+    public void testMetrics() throws Exception {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        int numSplits = 2;
+        env.setParallelism(numSplits + 2);
+        env.getConfig().setAutoWatermarkInterval(1L);
+
+        int numRecordsPerSplit = 10;
+        MockBaseSource source =
+                new MockBaseSource(numSplits, numRecordsPerSplit, 
Boundedness.BOUNDED);
+
+        long baseTime = System.currentTimeMillis() - EVENTTIME_LAG;
+        WatermarkStrategy<Integer> strategy =
+                WatermarkStrategy.<Integer>forBoundedOutOfOrderness(
+                                Duration.ofMillis(WATERMARK_LAG))
+                        .withTimestampAssigner(new 
LaggingTimestampAssigner(baseTime));
+
+        // make sure all parallel instances have processed the same amount of 
records before
+        // validating metrics
+        SharedReference<CyclicBarrier> beforeBarrier =
+                sharedObjects.add(new CyclicBarrier(numSplits + 1));
+        SharedReference<CyclicBarrier> afterBarrier =
+                sharedObjects.add(new CyclicBarrier(numSplits + 1));
+        int stopAtRecord1 = 3;
+        int stopAtRecord2 = numRecordsPerSplit - 1;
+        DataStream<Integer> stream =
+                env.fromSource(source, strategy, "TestingSource")
+                        .map(
+                                i -> {
+                                    if (i % numRecordsPerSplit == stopAtRecord1
+                                            || i % numRecordsPerSplit == 
stopAtRecord2) {
+                                        beforeBarrier.get().await();
+                                        afterBarrier.get().await();
+                                    }
+                                    return i;
+                                });
+        stream.addSink(new DiscardingSink<>());
+        JobClient jobClient = env.executeAsync();
+
+        beforeBarrier.get().await();
+        assertSourceMetrics(stopAtRecord1 + 1, numRecordsPerSplit, 
env.getParallelism(), numSplits);
+        afterBarrier.get().await();
+
+        beforeBarrier.get().await();
+        assertSourceMetrics(stopAtRecord2 + 1, numRecordsPerSplit, 
env.getParallelism(), numSplits);
+        afterBarrier.get().await();
+
+        jobClient.getJobExecutionResult().get();
+    }
+
+    private void assertSourceMetrics(
+            long processedRecordsPerSubtask,
+            long numTotalPerSubtask,
+            int parallelism,
+            int numSplits) {
+        List<OperatorMetricGroup> groups =
+                
miniClusterResource.getMetricReporter().findOperatorMetricGroups("TestingSource");
+        assertThat(groups, hasSize(parallelism));
+
+        int subtaskWithMetrics = 0;
+        for (OperatorMetricGroup group : groups) {
+            Map<String, Metric> metrics =
+                    
miniClusterResource.getMetricReporter().getMetricsByGroup(group);
+            // there are only 2 splits assigned; so two groups will not update 
metrics
+            if (group.getIOMetricGroup().getNumRecordsInCounter().getCount() 
== 0) {
+                // assert that optional metrics are not initialized when no 
split assigned
+                
assertThat(metrics.get(MetricNames.CURRENT_EMIT_EVENT_TIME_LAG), nullValue());
+                assertThat(metrics.get(MetricNames.WATERMARK_LAG), 
nullValue());
+                continue;
+            }
+            subtaskWithMetrics++;
+            // I/O metrics
+            assertThat(
+                    group.getIOMetricGroup().getNumRecordsInCounter(),
+                    isCounter(equalTo(processedRecordsPerSubtask)));
+            assertThat(
+                    group.getIOMetricGroup().getNumBytesInCounter(),
+                    isCounter(
+                            equalTo(
+                                    processedRecordsPerSubtask
+                                            * 
MockRecordEmitter.RECORD_SIZE_IN_BYTES)));
+            // MockRecordEmitter is just incrementing errors every even record
+            assertThat(
+                    metrics.get(MetricNames.NUM_RECORDS_IN_ERRORS),
+                    isCounter(equalTo(processedRecordsPerSubtask / 2)));
+            // Timestamp assigner subtracting EVENTTIME_LAG from wall clock, 
so expect that lag
+            assertThat(
+                    metrics.get(MetricNames.CURRENT_EMIT_EVENT_TIME_LAG),
+                    isGauge(
+                            both(greaterThan(EVENTTIME_LAG - 
EVENTTIME_EPSILON))
+                                    .and(lessThan(EVENTTIME_LAG + 
EVENTTIME_EPSILON))));

Review comment:
       Ni: extract a function for this functionality

##########
File path: 
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceITCase.java
##########
@@ -19,30 +19,175 @@
 package org.apache.flink.connector.base.source.reader;
 
 import org.apache.flink.api.common.accumulators.ListAccumulator;
+import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner;
 import org.apache.flink.api.common.eventtime.WatermarkStrategy;
 import org.apache.flink.api.connector.source.Boundedness;
 import org.apache.flink.api.connector.source.Source;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.connector.base.source.reader.mocks.MockBaseSource;
+import org.apache.flink.connector.base.source.reader.mocks.MockRecordEmitter;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.metrics.Gauge;
 import org.apache.flink.metrics.Metric;
+import org.apache.flink.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.MetricNames;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.testutils.junit.SharedReference;
 
-import org.hamcrest.Matchers;
+import org.junit.Rule;
 import org.junit.Test;
 
+import java.time.Duration;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
+import java.util.concurrent.CyclicBarrier;
 
+import static org.apache.flink.metrics.testutils.MetricMatchers.isCounter;
+import static org.apache.flink.metrics.testutils.MetricMatchers.isGauge;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.Matchers.both;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.lessThan;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
 /** IT case for the {@link Source} with a coordinator. */
 public class CoordinatedSourceITCase extends AbstractTestBase {
+    @Rule public final SharedObjects sharedObjects = SharedObjects.create();
+
+    // since integration tests depend on wall clock time, use huge lags
+    private static final long EVENTTIME_LAG = Duration.ofDays(100).toMillis();
+    private static final long WATERMARK_LAG = Duration.ofDays(1).toMillis();
+    private static final long EVENTTIME_EPSILON = 
Duration.ofDays(20).toMillis();
+    // this basically is the time a build is allowed to be frozen before the 
test fails
+    private static final long WATERMARK_EPSILON = 
Duration.ofHours(6).toMillis();
+
+    @Test
+    public void testMetrics() throws Exception {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        int numSplits = 2;
+        env.setParallelism(numSplits + 2);
+        env.getConfig().setAutoWatermarkInterval(1L);
+
+        int numRecordsPerSplit = 10;
+        MockBaseSource source =
+                new MockBaseSource(numSplits, numRecordsPerSplit, 
Boundedness.BOUNDED);
+
+        long baseTime = System.currentTimeMillis() - EVENTTIME_LAG;
+        WatermarkStrategy<Integer> strategy =
+                WatermarkStrategy.<Integer>forBoundedOutOfOrderness(
+                                Duration.ofMillis(WATERMARK_LAG))
+                        .withTimestampAssigner(new 
LaggingTimestampAssigner(baseTime));
+
+        // make sure all parallel instances have processed the same amount of 
records before
+        // validating metrics
+        SharedReference<CyclicBarrier> beforeBarrier =
+                sharedObjects.add(new CyclicBarrier(numSplits + 1));
+        SharedReference<CyclicBarrier> afterBarrier =
+                sharedObjects.add(new CyclicBarrier(numSplits + 1));
+        int stopAtRecord1 = 3;
+        int stopAtRecord2 = numRecordsPerSplit - 1;
+        DataStream<Integer> stream =
+                env.fromSource(source, strategy, "TestingSource")
+                        .map(
+                                i -> {
+                                    if (i % numRecordsPerSplit == stopAtRecord1
+                                            || i % numRecordsPerSplit == 
stopAtRecord2) {
+                                        beforeBarrier.get().await();
+                                        afterBarrier.get().await();
+                                    }
+                                    return i;
+                                });
+        stream.addSink(new DiscardingSink<>());
+        JobClient jobClient = env.executeAsync();
+
+        beforeBarrier.get().await();
+        assertSourceMetrics(stopAtRecord1 + 1, numRecordsPerSplit, 
env.getParallelism(), numSplits);
+        afterBarrier.get().await();
+
+        beforeBarrier.get().await();
+        assertSourceMetrics(stopAtRecord2 + 1, numRecordsPerSplit, 
env.getParallelism(), numSplits);
+        afterBarrier.get().await();
+
+        jobClient.getJobExecutionResult().get();
+    }
+
+    private void assertSourceMetrics(
+            long processedRecordsPerSubtask,
+            long numTotalPerSubtask,
+            int parallelism,
+            int numSplits) {
+        List<OperatorMetricGroup> groups =
+                
miniClusterResource.getMetricReporter().findOperatorMetricGroups("TestingSource");
+        assertThat(groups, hasSize(parallelism));
+
+        int subtaskWithMetrics = 0;
+        for (OperatorMetricGroup group : groups) {
+            Map<String, Metric> metrics =
+                    
miniClusterResource.getMetricReporter().getMetricsByGroup(group);
+            // there are only 2 splits assigned; so two groups will not update 
metrics
+            if (group.getIOMetricGroup().getNumRecordsInCounter().getCount() 
== 0) {
+                // assert that optional metrics are not initialized when no 
split assigned
+                
assertThat(metrics.get(MetricNames.CURRENT_EMIT_EVENT_TIME_LAG), nullValue());
+                assertThat(metrics.get(MetricNames.WATERMARK_LAG), 
nullValue());
+                continue;
+            }
+            subtaskWithMetrics++;
+            // I/O metrics
+            assertThat(
+                    group.getIOMetricGroup().getNumRecordsInCounter(),
+                    isCounter(equalTo(processedRecordsPerSubtask)));
+            assertThat(
+                    group.getIOMetricGroup().getNumBytesInCounter(),
+                    isCounter(
+                            equalTo(
+                                    processedRecordsPerSubtask
+                                            * 
MockRecordEmitter.RECORD_SIZE_IN_BYTES)));
+            // MockRecordEmitter is just incrementing errors every even record
+            assertThat(
+                    metrics.get(MetricNames.NUM_RECORDS_IN_ERRORS),
+                    isCounter(equalTo(processedRecordsPerSubtask / 2)));
+            // Timestamp assigner subtracting EVENTTIME_LAG from wall clock, 
so expect that lag

Review comment:
       Comment incomplete?

##########
File path: 
flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/groups/SinkWriterMetricGroup.java
##########
@@ -41,5 +41,5 @@
      * For example, a `volatile` field could be set in the async thread and 
lazily read in the
      * gauge.
      */
-    <G extends Gauge<Long>> G addCurrentSendTimeGauge(G currentSendTime);
+    void setCurrentSendTimeGauge(Gauge<Long> currentSendTimeGauge);

Review comment:
       I do not know if the signature changes belong to this commit. IMO they 
should be part of the core commit.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSourceReaderMetricGroup.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.runtime.metrics.groups;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
+import org.apache.flink.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.metrics.groups.SourceReaderMetricGroup;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.util.clock.Clock;
+import org.apache.flink.util.clock.SystemClock;
+
+/** Special {@link org.apache.flink.metrics.MetricGroup} representing an 
Operator. */
+@Internal
+public class InternalSourceReaderMetricGroup extends 
ProxyMetricGroup<MetricGroup>
+        implements SourceReaderMetricGroup {
+
+    public static final long ACTIVE = Long.MAX_VALUE;
+    private static final long UNDEFINED = Long.MIN_VALUE;
+
+    private final OperatorIOMetricGroup operatorIOMetricGroup;
+    private final Clock clock;
+    private final Counter numRecordsInErrors;
+    private long lastWatermark = UNDEFINED;
+    private long lastEventTime = UNDEFINED;
+    private long idleStartTime = ACTIVE;
+
+    private InternalSourceReaderMetricGroup(
+            MetricGroup parentMetricGroup,
+            OperatorIOMetricGroup operatorIOMetricGroup,
+            Clock clock) {
+        super(parentMetricGroup);
+        numRecordsInErrors = 
parentMetricGroup.counter(MetricNames.NUM_RECORDS_IN_ERRORS);
+        this.operatorIOMetricGroup = operatorIOMetricGroup;
+        this.clock = clock;
+        parentMetricGroup.gauge(
+                MetricNames.SOURCE_IDLE_TIME,
+                () -> isIdling() ? this.clock.absoluteTimeMillis() - 
idleStartTime : 0);
+    }
+
+    public static InternalSourceReaderMetricGroup wrap(OperatorMetricGroup 
operatorMetricGroup) {
+        return new InternalSourceReaderMetricGroup(
+                operatorMetricGroup,
+                operatorMetricGroup.getIOMetricGroup(),
+                SystemClock.getInstance());
+    }
+
+    @VisibleForTesting
+    public static InternalSourceReaderMetricGroup mock(MetricGroup 
metricGroup) {
+        return new InternalSourceReaderMetricGroup(
+                metricGroup,
+                
UnregisteredMetricsGroup.createUnregisteredOperatorIOMetricGroup(),
+                SystemClock.getInstance());
+    }
+
+    private boolean isIdling() {
+        return idleStartTime != ACTIVE;

Review comment:
       It seems strange that if the source reader was idle once the idleTime 
will always increase. I would like to reset it somehow to 0.




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