zentol commented on a change in pull request #16770: URL: https://github.com/apache/flink/pull/16770#discussion_r686173496
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/InternalSourceReaderMetricGroup.java ########## @@ -0,0 +1,143 @@ +/* + * 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.SettableGauge; +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 final OperatorIOMetricGroup operatorIOMetricGroup; + private final Clock clock; + private final Counter numRecordsInErrors; + // only if source emits at least one watermark + private SettableGauge<Long> watermarkGauge; + // only if records with event timestamp are emitted + private SettableGauge<Long> eventTimeGauge; + 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_GAUGE, + () -> 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; + } + + public void idlingStarted() { + if (!isIdling()) { + idleStartTime = clock.absoluteTimeMillis(); + } + } + + public void recordEmitted() { + idleStartTime = ACTIVE; + } + + @Override + public Counter getNumRecordsInErrorsCounter() { + return numRecordsInErrors; + } + + public void watermarkEmitted(long watermark) { + // iff a respective source emits a watermark, Flink can provide the watermark lag + if (watermarkGauge == null) { + watermarkGauge = new SettableGauge<>(watermark); + parentMetricGroup.gauge( + MetricNames.WATERMARK_LAG, + () -> clock.absoluteTimeMillis() - watermarkGauge.getValue()); + } else { + watermarkGauge.setValue(watermark); + } + } + + public void eventTimeEmitted(long timestamp) { + // iff a respective source emits a timestamp, Flink can provide the event lag + if (eventTimeGauge == null) { + eventTimeGauge = new SettableGauge<>(timestamp); Review comment: What exactly does the SettableGauge provide us here opposed to a plain `long` that defaults to -1? -- 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]
