From 7bc13f9f8dcb814fa88b746c1839511a639b234e Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Feb 2018 11:11:56 +0100 Subject: [PATCH 1/3] [FLINK-6053][metrics] Add new Number-/StringGauge metric types --- .../java/org/apache/flink/metrics/Gauge.java | 3 + .../org/apache/flink/metrics/MetricGroup.java | 20 +++++ .../org/apache/flink/metrics/NumberGauge.java | 26 ++++++ .../org/apache/flink/metrics/StringGauge.java | 26 ++++++ .../groups/UnregisteredMetricsGroup.java | 12 +++ .../metrics/reporter/AbstractReporter.java | 3 + .../metrics/reporter/AbstractReporterV2.java | 89 +++++++++++++++++++ .../metrics/groups/AbstractMetricGroup.java | 89 ++++++++++++++++++- .../metrics/groups/ProxyMetricGroup.java | 14 +++ .../groups/MetricGroupRegistrationTest.java | 11 +-- .../metrics/groups/MetricGroupTest.java | 8 +- .../flink/api/scala/metrics/ScalaGauge.scala | 1 + 12 files changed, 286 insertions(+), 16 deletions(-) create mode 100644 flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/NumberGauge.java create mode 100644 flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/StringGauge.java create mode 100644 flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporterV2.java diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/Gauge.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/Gauge.java index eb7c40ffd0e0e..4df447a07fd4e 100644 --- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/Gauge.java +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/Gauge.java @@ -20,7 +20,10 @@ /** * A Gauge is a {@link Metric} that calculates a specific value at a point in time. + * + * @deprecated use {@link NumberGauge} or {@link StringGauge} instead */ +@Deprecated public interface Gauge extends Metric { /** diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricGroup.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricGroup.java index f1f19813b1658..c847aac7f4617 100644 --- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricGroup.java +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricGroup.java @@ -77,7 +77,9 @@ public interface MetricGroup { * @param gauge gauge to register * @param return type of the gauge * @return the given gauge + * @deprecated use {@link #register(String, NumberGauge)} or {@link #register(String, StringGauge)} instead */ + @Deprecated > G gauge(int name, G gauge); /** @@ -87,9 +89,27 @@ public interface MetricGroup { * @param gauge gauge to register * @param return type of the gauge * @return the given gauge + * @deprecated use {@link #register(String, NumberGauge)} or {@link #register(String, StringGauge)} instead */ + @Deprecated > G gauge(String name, G gauge); + /** + * Registers a new {@link org.apache.flink.metrics.NumberGauge} with Flink. + * + * @param name name of the gauge + * @param gauge gauge to register + */ + void register(String name, NumberGauge gauge); + + /** + * Registers a new {@link org.apache.flink.metrics.StringGauge} with Flink. + * + * @param name name of the gauge + * @param gauge gauge to register + */ + void register(String name, StringGauge gauge); + /** * Registers a new {@link Histogram} with Flink. * diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/NumberGauge.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/NumberGauge.java new file mode 100644 index 0000000000000..af32fbd8d9517 --- /dev/null +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/NumberGauge.java @@ -0,0 +1,26 @@ +/* + * 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; + +/** + * A NumberGauge is a {@link Metric} that calculates a specific {@link Number} value at a point in time. + */ +public interface NumberGauge extends Metric { + Number getNumberValue(); +} diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/StringGauge.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/StringGauge.java new file mode 100644 index 0000000000000..a51c093199553 --- /dev/null +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/StringGauge.java @@ -0,0 +1,26 @@ +/* + * 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; + +/** + * A StringGauge is a {@link Metric} that calculates a specific {@link String} value at a point in time. + */ +public interface StringGauge extends Metric { + String getStringValue(); +} diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/groups/UnregisteredMetricsGroup.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/groups/UnregisteredMetricsGroup.java index e004124f49183..ba558fccd0dca 100644 --- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/groups/UnregisteredMetricsGroup.java +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/groups/UnregisteredMetricsGroup.java @@ -24,7 +24,9 @@ import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.metrics.StringGauge; import java.util.Collections; import java.util.Map; @@ -56,15 +58,25 @@ public C counter(String name, C counter) { } @Override + @Deprecated public > G gauge(int name, G gauge) { return gauge; } @Override + @Deprecated public > G gauge(String name, G gauge) { return gauge; } + @Override + public void register(String name, NumberGauge gauge) { + } + + @Override + public void register(String name, StringGauge gauge) { + } + @Override public H histogram(int name, H histogram) { return histogram; diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java index c0aeb4bc4eb32..aaf83ed495a54 100644 --- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java @@ -34,7 +34,10 @@ /** * Base interface for custom metric reporters. + * + * @deprecated use {@link AbstractReporterV2} instead */ +@Deprecated public abstract class AbstractReporter implements MetricReporter, CharacterFilter { protected final Logger log = LoggerFactory.getLogger(getClass()); diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporterV2.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporterV2.java new file mode 100644 index 0000000000000..a4786fbfcd26e --- /dev/null +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporterV2.java @@ -0,0 +1,89 @@ +/* + * 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.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +/** + * Base interface for custom metric reporters. + */ +public abstract class AbstractReporterV2 implements MetricReporter, CharacterFilter { + protected final Logger log = LoggerFactory.getLogger(getClass()); + + protected final Map stringGauges = new HashMap<>(); + protected final Map numberGauges = new HashMap<>(); + protected final Map counters = new HashMap<>(); + protected final Map histograms = new HashMap<>(); + protected final Map meters = new HashMap<>(); + + @Override + public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group) { + final String name = group.getMetricIdentifier(metricName, this); + + synchronized (this) { + if (metric instanceof Counter) { + counters.put((Counter) metric, name); + } else if (metric instanceof StringGauge) { + stringGauges.put((StringGauge) metric, name); + } else if (metric instanceof NumberGauge) { + numberGauges.put((NumberGauge) metric, name); + } else if (metric instanceof Histogram) { + histograms.put((Histogram) metric, name); + } else if (metric instanceof Meter) { + meters.put((Meter) metric, name); + } else { + log.warn("Cannot add unknown metric type {}. This indicates that the reporter " + + "does not support this metric type.", metric.getClass().getName()); + } + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group) { + synchronized (this) { + if (metric instanceof Counter) { + counters.remove(metric); + } else if (metric instanceof StringGauge) { + stringGauges.remove(metric); + } else if (metric instanceof NumberGauge) { + numberGauges.remove(metric); + } else if (metric instanceof Histogram) { + histograms.remove(metric); + } else if (metric instanceof Meter) { + meters.remove(metric); + } else { + log.warn("Cannot remove unknown metric type {}. This indicates that the reporter " + + "does not support this metric type.", metric.getClass().getName()); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java index 6d9c7d95a3476..2b4d6b949a421 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java @@ -26,7 +26,9 @@ import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.scope.ScopeFormat; @@ -313,16 +315,28 @@ public C counter(String name, C counter) { } @Override + @Deprecated public > G gauge(int name, G gauge) { return gauge(String.valueOf(name), gauge); } @Override + @Deprecated public > G gauge(String name, G gauge) { - addMetric(name, gauge); + addMetric(name, new LegacyGaugeWrapper<>(gauge)); return gauge; } + @Override + public void register(String name, StringGauge gauge) { + addMetric(name, new StringGaugeWrapper(gauge)); + } + + @Override + public void register(String name, NumberGauge gauge) { + addMetric(name, new NumberGaugeWrapper(gauge)); + } + @Override public H histogram(int name, H histogram) { return histogram(String.valueOf(name), histogram); @@ -460,4 +474,77 @@ protected enum ChildType { VALUE, GENERIC } + + /** + * This class wraps a legacy {@link Gauge} to ensure that legacy metrics are not ignored by reporters that only work + * against the {@link StringGauge} and {@link NumberGauge} interfaces. + * + * @param type of the gauge + */ + private static class LegacyGaugeWrapper implements StringGauge, Gauge { + private final Gauge legacyGauge; + + private LegacyGaugeWrapper(Gauge legacyGauge) { + this.legacyGauge = legacyGauge; + } + + @Override + public String getStringValue() { + T value = legacyGauge.getValue(); + if (value == null) { + return null; + } else { + return value.toString(); + } + } + + @Override + public T getValue() { + return legacyGauge.getValue(); + } + } + + /** + * This class wraps a {@link NumberGauge} for backwards compatibility with reporters that are only aware of + * {@link Gauge}s. + */ + private static class NumberGaugeWrapper implements NumberGauge, Gauge { + private final NumberGauge gauge; + + private NumberGaugeWrapper(NumberGauge gauge) { + this.gauge = gauge; + } + + @Override + public Number getNumberValue() { + return gauge.getNumberValue(); + } + + @Override + public Number getValue() { + return getNumberValue(); + } + } + + /** + * This class wraps a {@link StringGauge} for backwards compatibility with reporters that are only aware of + * {@link Gauge}s. + */ + private static class StringGaugeWrapper implements StringGauge, Gauge { + private final StringGauge gauge; + + private StringGaugeWrapper(StringGauge gauge) { + this.gauge = gauge; + } + + @Override + public String getStringValue() { + return gauge.getStringValue(); + } + + @Override + public String getValue() { + return getStringValue(); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/ProxyMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/ProxyMetricGroup.java index ea1ba418b1af8..d0c65347eea27 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/ProxyMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/ProxyMetricGroup.java @@ -24,6 +24,8 @@ import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; import java.util.Map; @@ -63,15 +65,27 @@ public final C counter(String name, C counter) { } @Override + @Deprecated public final > G gauge(int name, G gauge) { return parentMetricGroup.gauge(name, gauge); } @Override + @Deprecated public final > G gauge(String name, G gauge) { return parentMetricGroup.gauge(name, gauge); } + @Override + public void register(String name, NumberGauge gauge) { + parentMetricGroup.register(name, gauge); + } + + @Override + public void register(String name, StringGauge gauge) { + parentMetricGroup.register(name, gauge); + } + @Override public final H histogram(String name, H histogram) { return parentMetricGroup.histogram(name, histogram); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java index bcdcd63ac6dc9..597508faebd0e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java @@ -26,6 +26,7 @@ import org.apache.flink.metrics.HistogramStatistics; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.util.TestReporter; @@ -56,14 +57,8 @@ public void testMetricInstantiation() { assertEquals(counter, TestReporter1.lastPassedMetric); assertEquals("counter", TestReporter1.lastPassedName); - Gauge gauge = root.gauge("gauge", new Gauge() { - @Override - public Object getValue() { - return null; - } - }); - - Assert.assertEquals(gauge, TestReporter1.lastPassedMetric); + root.register("gauge", () -> 0); + Assert.assertTrue(TestReporter1.lastPassedMetric instanceof NumberGauge); assertEquals("gauge", TestReporter1.lastPassedName); Histogram histogram = root.histogram("histogram", new Histogram() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java index 4dc5edf8a17b6..15b125ddc5b56 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.metrics.CharacterFilter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -227,12 +226,7 @@ public void closedGroupDoesNotRegisterMetrics() { // these will fail is the registration is propagated group.counter("testcounter"); - group.gauge("testgauge", new Gauge() { - @Override - public Object getValue() { - return null; - } - }); + group.register("testgauge", () -> 0); } @Test diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/metrics/ScalaGauge.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/metrics/ScalaGauge.scala index e2f9ebfbc0ddd..98248974fbabd 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/metrics/ScalaGauge.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/metrics/ScalaGauge.scala @@ -23,6 +23,7 @@ import org.apache.flink.metrics.Gauge /** * This class allows the concise definition of a gauge from Scala using function references. */ +@deprecated("use NumberGauge or StringGauge instead") class ScalaGauge[T](func: () => T) extends Gauge[T] { override def getValue: T = { func() From 85a48c02fbd58edc822949cf9f9177c9c36661cc Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Feb 2018 11:34:50 +0100 Subject: [PATCH 2/3] update reporters --- .../apache/flink/metrics/datadog/DGauge.java | 10 ++-- .../metrics/datadog/DatadogHttpReporter.java | 11 ++-- .../datadog/DatadogHttpClientTest.java | 15 +---- .../ScheduledDropwizardReporter.java | 35 ++++++++---- .../dropwizard/metrics/FlinkGaugeWrapper.java | 6 +- .../metrics/FlinkNumberGaugeWrapper.java | 42 ++++++++++++++ .../metrics/FlinkStringGaugeWrapper.java | 42 ++++++++++++++ .../ScheduledDropwizardReporterTest.java | 27 +++++---- .../apache/flink/metrics/jmx/JMXReporter.java | 46 +++++++++++---- .../flink/metrics/jmx/JMXReporterTest.java | 30 ++-------- .../prometheus/PrometheusReporter.java | 25 +++----- .../PrometheusReporterTaskScopeTest.java | 19 +------ .../prometheus/PrometheusReporterTest.java | 57 ++----------------- .../flink/metrics/slf4j/Slf4jReporter.java | 25 +++++--- .../metrics/slf4j/Slf4jReporterTest.java | 11 ++-- .../flink/metrics/statsd/StatsDReporter.java | 26 +++++++-- .../runtime/metrics/util/TestReporter.java | 4 +- 17 files changed, 241 insertions(+), 190 deletions(-) create mode 100644 flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkNumberGaugeWrapper.java create mode 100644 flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkStringGaugeWrapper.java diff --git a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DGauge.java b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DGauge.java index ba97d59327b51..2f9d848449b01 100644 --- a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DGauge.java +++ b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DGauge.java @@ -18,17 +18,17 @@ package org.apache.flink.metrics.datadog; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import java.util.List; /** - * Mapping of gauge between Flink and Datadog. + * Mapping of number gauge between Flink and Datadog. */ public class DGauge extends DMetric { - private final Gauge gauge; + private final NumberGauge gauge; - public DGauge(Gauge g, String metricName, String host, List tags) { + public DGauge(NumberGauge g, String metricName, String host, List tags) { super(MetricType.gauge, metricName, host, tags); gauge = g; } @@ -39,6 +39,6 @@ public DGauge(Gauge g, String metricName, String host, List tags */ @Override public Number getMetricValue() { - return gauge.getValue(); + return gauge.getNumberValue(); } } diff --git a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java index b7e1c24fa6034..2b5366873822c 100644 --- a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java +++ b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java @@ -25,6 +25,7 @@ import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.Scheduled; @@ -48,7 +49,7 @@ public class DatadogHttpReporter implements MetricReporter, Scheduled { private static final String HOST_VARIABLE = ""; // Both Flink's Gauge and Meter values are taken as gauge in Datadog - private final Map gauges = new ConcurrentHashMap<>(); + private final Map gauges = new ConcurrentHashMap<>(); private final Map counters = new ConcurrentHashMap<>(); private final Map meters = new ConcurrentHashMap<>(); @@ -69,8 +70,8 @@ public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup gr if (metric instanceof Counter) { Counter c = (Counter) metric; counters.put(c, new DCounter(c, name, host, tags)); - } else if (metric instanceof Gauge) { - Gauge g = (Gauge) metric; + } else if (metric instanceof NumberGauge) { + NumberGauge g = (NumberGauge) metric; gauges.put(g, new DGauge(g, name, host, tags)); } else if (metric instanceof Meter) { Meter m = (Meter) metric; @@ -88,7 +89,7 @@ public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup gr public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group) { if (metric instanceof Counter) { counters.remove(metric); - } else if (metric instanceof Gauge) { + } else if (metric instanceof NumberGauge) { gauges.remove(metric); } else if (metric instanceof Meter) { meters.remove(metric); @@ -118,7 +119,7 @@ public void close() { public void report() { DatadogHttpRequest request = new DatadogHttpRequest(); - for (Map.Entry entry : gauges.entrySet()) { + for (Map.Entry entry : gauges.entrySet()) { DGauge g = entry.getValue(); try { // Will throw exception if the Gauge is not of Number type diff --git a/flink-metrics/flink-metrics-datadog/src/test/java/org/apache/flink/metrics/datadog/DatadogHttpClientTest.java b/flink-metrics/flink-metrics-datadog/src/test/java/org/apache/flink/metrics/datadog/DatadogHttpClientTest.java index 89e83bd9be37c..ef4cbc64ccf4a 100644 --- a/flink-metrics/flink-metrics-datadog/src/test/java/org/apache/flink/metrics/datadog/DatadogHttpClientTest.java +++ b/flink-metrics/flink-metrics-datadog/src/test/java/org/apache/flink/metrics/datadog/DatadogHttpClientTest.java @@ -19,7 +19,6 @@ package org.apache.flink.metrics.datadog; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Meter; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; @@ -66,12 +65,7 @@ public void testClientWithNullKey() { @Test public void serializeGauge() throws JsonProcessingException { - DGauge g = new DGauge(new Gauge() { - @Override - public Number getValue() { - return 1; - } - }, "testCounter", "localhost", tags); + DGauge g = new DGauge(() -> 1, "testCounter", "localhost", tags); assertEquals( "{\"metric\":\"testCounter\",\"type\":\"gauge\",\"host\":\"localhost\",\"tags\":[\"tag1\",\"tag2\"],\"points\":[[123,1]]}", @@ -81,12 +75,7 @@ public Number getValue() { @Test public void serializeGaugeWithoutHost() throws JsonProcessingException { - DGauge g = new DGauge(new Gauge() { - @Override - public Number getValue() { - return 1; - } - }, "testCounter", null, tags); + DGauge g = new DGauge(() -> 1, "testCounter", null, tags); assertEquals( "{\"metric\":\"testCounter\",\"type\":\"gauge\",\"tags\":[\"tag1\",\"tag2\"],\"points\":[[123,1]]}", diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java index e3184dafc0fce..1f25a6fdfc0a3 100644 --- a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java @@ -23,17 +23,19 @@ import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; import org.apache.flink.dropwizard.metrics.DropwizardMeterWrapper; import org.apache.flink.dropwizard.metrics.FlinkCounterWrapper; -import org.apache.flink.dropwizard.metrics.FlinkGaugeWrapper; import org.apache.flink.dropwizard.metrics.FlinkHistogramWrapper; import org.apache.flink.dropwizard.metrics.FlinkMeterWrapper; +import org.apache.flink.dropwizard.metrics.FlinkNumberGaugeWrapper; +import org.apache.flink.dropwizard.metrics.FlinkStringGaugeWrapper; import org.apache.flink.metrics.CharacterFilter; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.Scheduled; @@ -68,7 +70,8 @@ public abstract class ScheduledDropwizardReporter implements MetricReporter, Sch protected ScheduledReporter reporter; - private final Map, String> gauges = new HashMap<>(); + private final Map stringGauges = new HashMap<>(); + private final Map numberGauges = new HashMap<>(); private final Map counters = new HashMap<>(); private final Map histograms = new HashMap<>(); private final Map meters = new HashMap<>(); @@ -94,8 +97,13 @@ Map getMeters() { } @VisibleForTesting - Map, String> getGauges() { - return gauges; + Map getNumberGauges() { + return numberGauges; + } + + @VisibleForTesting + Map getStringGauges() { + return stringGauges; } @VisibleForTesting @@ -130,9 +138,12 @@ public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup gr counters.put((Counter) metric, fullName); registry.register(fullName, new FlinkCounterWrapper((Counter) metric)); } - else if (metric instanceof Gauge) { - gauges.put((Gauge) metric, fullName); - registry.register(fullName, FlinkGaugeWrapper.fromGauge((Gauge) metric)); + else if (metric instanceof NumberGauge) { + numberGauges.put((NumberGauge) metric, fullName); + registry.register(fullName, FlinkNumberGaugeWrapper.fromGauge((NumberGauge) metric)); + } else if (metric instanceof StringGauge) { + stringGauges.put((StringGauge) metric, fullName); + registry.register(fullName, FlinkStringGaugeWrapper.fromGauge((StringGauge) metric)); } else if (metric instanceof Histogram) { Histogram histogram = (Histogram) metric; histograms.put(histogram, fullName); @@ -165,9 +176,11 @@ public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup if (metric instanceof Counter) { fullName = counters.remove(metric); - } else if (metric instanceof Gauge) { - fullName = gauges.remove(metric); - } else if (metric instanceof Histogram) { + } else if (metric instanceof NumberGauge) { + fullName = numberGauges.remove(metric); + } else if (metric instanceof StringGauge) { + fullName = stringGauges.remove(metric); + } else if (metric instanceof Histogram) { fullName = histograms.remove(metric); } else if (metric instanceof Meter) { fullName = meters.remove(metric); diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkGaugeWrapper.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkGaugeWrapper.java index 0d53a9eb97c70..42cc5d4a232ba 100644 --- a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkGaugeWrapper.java +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkGaugeWrapper.java @@ -21,8 +21,11 @@ import org.apache.flink.metrics.Gauge; /** - * A wrapper that allows a Flink gauge to be used as a DropWizard gauge. + * A wrapper that allows a Flink {@link Gauge} to be used as a DropWizard number. + * + * @deprecated Use {@link FlinkNumberGaugeWrapper} or {@link FlinkStringGaugeWrapper} instead. */ +@Deprecated public class FlinkGaugeWrapper implements com.codahale.metrics.Gauge { private final Gauge gauge; @@ -36,6 +39,7 @@ public T getValue() { return this.gauge.getValue(); } + @Deprecated public static FlinkGaugeWrapper fromGauge(Gauge gauge) { @SuppressWarnings("unchecked") Gauge typedGauge = (Gauge) gauge; diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkNumberGaugeWrapper.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkNumberGaugeWrapper.java new file mode 100644 index 0000000000000..31ba902631ef9 --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkNumberGaugeWrapper.java @@ -0,0 +1,42 @@ +/* + * 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.dropwizard.metrics; + +import org.apache.flink.metrics.NumberGauge; + +/** + * A wrapper that allows a Flink {@link NumberGauge} to be used as a DropWizard number. + */ +public class FlinkNumberGaugeWrapper implements com.codahale.metrics.Gauge { + + private final NumberGauge gauge; + + private FlinkNumberGaugeWrapper(NumberGauge gauge) { + this.gauge = gauge; + } + + @Override + public Number getValue() { + return this.gauge.getNumberValue(); + } + + public static FlinkNumberGaugeWrapper fromGauge(NumberGauge gauge) { + return new FlinkNumberGaugeWrapper(gauge); + } +} diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkStringGaugeWrapper.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkStringGaugeWrapper.java new file mode 100644 index 0000000000000..a7ca487732066 --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkStringGaugeWrapper.java @@ -0,0 +1,42 @@ +/* + * 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.dropwizard.metrics; + +import org.apache.flink.metrics.StringGauge; + +/** + * A wrapper that allows a Flink {@link StringGauge} to be used as a DropWizard number. + */ +public class FlinkStringGaugeWrapper implements com.codahale.metrics.Gauge { + + private final StringGauge gauge; + + private FlinkStringGaugeWrapper(StringGauge gauge) { + this.gauge = gauge; + } + + @Override + public String getValue() { + return this.gauge.getStringValue(); + } + + public static FlinkStringGaugeWrapper fromGauge(StringGauge gauge) { + return new FlinkStringGaugeWrapper(gauge); + } +} diff --git a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java index 4a2ca3ab34d54..e8474804410b7 100644 --- a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java +++ b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java @@ -24,13 +24,14 @@ import org.apache.flink.configuration.MetricOptions; import org.apache.flink.dropwizard.metrics.DropwizardMeterWrapper; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.HistogramStatistics; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -180,12 +181,8 @@ public HistogramStatistics getStatistics() { return null; } }; - Gauge g = new Gauge() { - @Override - public Object getValue() { - return null; - } - }; + NumberGauge ng = () -> 0; + StringGauge sg = () -> "hello"; rep.notifyOfAddedMetric(c, "counter", mp); assertEquals(1, rep.getCounters().size()); @@ -199,10 +196,14 @@ public Object getValue() { assertEquals(1, rep.getHistograms().size()); assertEquals(1, rep.registry.getHistograms().size()); - rep.notifyOfAddedMetric(g, "gauge", mp); - assertEquals(1, rep.getGauges().size()); + rep.notifyOfAddedMetric(ng, "numbergauge", mp); + assertEquals(1, rep.getNumberGauges().size()); assertEquals(1, rep.registry.getGauges().size()); + rep.notifyOfAddedMetric(sg, "stringgauge", mp); + assertEquals(1, rep.getStringGauges().size()); + assertEquals(2, rep.registry.getGauges().size()); + rep.notifyOfRemovedMetric(c, "counter", mp); assertEquals(0, rep.getCounters().size()); assertEquals(0, rep.registry.getCounters().size()); @@ -215,8 +216,12 @@ public Object getValue() { assertEquals(0, rep.getHistograms().size()); assertEquals(0, rep.registry.getHistograms().size()); - rep.notifyOfRemovedMetric(g, "gauge", mp); - assertEquals(0, rep.getGauges().size()); + rep.notifyOfRemovedMetric(ng, "numbergauge", mp); + assertEquals(0, rep.getNumberGauges().size()); + assertEquals(1, rep.registry.getGauges().size()); + + rep.notifyOfRemovedMetric(sg, "stringgauge", mp); + assertEquals(0, rep.getStringGauges().size()); assertEquals(0, rep.registry.getGauges().size()); } diff --git a/flink-metrics/flink-metrics-jmx/src/main/java/org/apache/flink/metrics/jmx/JMXReporter.java b/flink-metrics/flink-metrics-jmx/src/main/java/org/apache/flink/metrics/jmx/JMXReporter.java index 461f1dc31cbf0..254db4b4675ff 100644 --- a/flink-metrics/flink-metrics-jmx/src/main/java/org/apache/flink/metrics/jmx/JMXReporter.java +++ b/flink-metrics/flink-metrics-jmx/src/main/java/org/apache/flink/metrics/jmx/JMXReporter.java @@ -20,12 +20,13 @@ import org.apache.flink.metrics.CharacterFilter; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; @@ -169,8 +170,10 @@ public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup gr return; } - if (metric instanceof Gauge) { - jmxMetric = new JmxGauge((Gauge) metric); + if (metric instanceof NumberGauge) { + jmxMetric = new JmxNumberGauge((NumberGauge) metric); + } else if (metric instanceof StringGauge) { + jmxMetric = new JmxStringGauge((StringGauge) metric); } else if (metric instanceof Counter) { jmxMetric = new JmxCounter((Counter) metric); } else if (metric instanceof Histogram) { @@ -327,23 +330,44 @@ public long getCount() { } /** - * The MBean interface for an exposed gauge. + * The MBean interface for an exposed {@link NumberGauge}. */ - public interface JmxGaugeMBean extends MetricMBean { - Object getValue(); + public interface JmxNumberGaugeMBean extends MetricMBean { + Number getValue(); } - private static class JmxGauge extends AbstractBean implements JmxGaugeMBean { + private static class JmxNumberGauge extends AbstractBean implements JmxNumberGaugeMBean { - private final Gauge gauge; + private final NumberGauge gauge; - JmxGauge(Gauge gauge) { + JmxNumberGauge(NumberGauge gauge) { this.gauge = gauge; } @Override - public Object getValue() { - return gauge.getValue(); + public Number getValue() { + return gauge.getNumberValue(); + } + } + + /** + * The MBean interface for an exposed {@link StringGauge}. + */ + public interface JmxStringGaugeMBean extends MetricMBean { + String getValue(); + } + + private static class JmxStringGauge extends AbstractBean implements JmxStringGaugeMBean { + + private final StringGauge gauge; + + JmxStringGauge(StringGauge gauge) { + this.gauge = gauge; + } + + @Override + public String getValue() { + return gauge.getStringValue(); } } diff --git a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java index 40b7f158639fb..c49d36a99a01c 100644 --- a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java +++ b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java @@ -20,7 +20,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.util.TestHistogram; import org.apache.flink.metrics.util.TestMeter; @@ -117,18 +117,8 @@ public void testPortConflictHandling() throws Exception { MetricReporter rep1 = reporters.get(0); MetricReporter rep2 = reporters.get(1); - Gauge g1 = new Gauge() { - @Override - public Integer getValue() { - return 1; - } - }; - Gauge g2 = new Gauge() { - @Override - public Integer getValue() { - return 2; - } - }; + NumberGauge g1 = () -> 1; + NumberGauge g2 = () -> 2; rep1.notifyOfAddedMetric(g1, "rep1", new FrontMetricGroup<>(0, new TaskManagerMetricGroup(reg, "host", "tm"))); rep2.notifyOfAddedMetric(g2, "rep2", new FrontMetricGroup<>(0, new TaskManagerMetricGroup(reg, "host", "tm"))); @@ -175,18 +165,8 @@ public void testJMXAvailability() throws Exception { MetricReporter rep1 = reporters.get(0); MetricReporter rep2 = reporters.get(1); - Gauge g1 = new Gauge() { - @Override - public Integer getValue() { - return 1; - } - }; - Gauge g2 = new Gauge() { - @Override - public Integer getValue() { - return 2; - } - }; + NumberGauge g1 = () -> 1; + NumberGauge g2 = () -> 2; rep1.notifyOfAddedMetric(g1, "rep1", new FrontMetricGroup<>(0, new TaskManagerMetricGroup(reg, "host", "tm"))); diff --git a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java index fad3ced58c1a2..6c6326d5ca0d7 100644 --- a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java +++ b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java @@ -22,12 +22,12 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.CharacterFilter; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; @@ -160,7 +160,7 @@ private static String getScopedName(String metricName, MetricGroup group) { private static Collector createCollector(Metric metric, List dimensionKeys, List dimensionValues, String scopedMetricName, String helpString) { Collector collector; - if (metric instanceof Gauge || metric instanceof Counter || metric instanceof Meter) { + if (metric instanceof NumberGauge || metric instanceof Counter || metric instanceof Meter) { collector = io.prometheus.client.Gauge .build() .name(scopedMetricName) @@ -178,8 +178,8 @@ private static Collector createCollector(Metric metric, List dimensionKe } private static void addMetric(Metric metric, List dimensionValues, Collector collector) { - if (metric instanceof Gauge) { - ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Gauge) metric), toArray(dimensionValues)); + if (metric instanceof NumberGauge) { + ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((NumberGauge) metric), toArray(dimensionValues)); } else if (metric instanceof Counter) { ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Counter) metric), toArray(dimensionValues)); } else if (metric instanceof Meter) { @@ -218,27 +218,16 @@ private static String getLogicalScope(MetricGroup group) { } @VisibleForTesting - static io.prometheus.client.Gauge.Child gaugeFrom(Gauge gauge) { + static io.prometheus.client.Gauge.Child gaugeFrom(NumberGauge gauge) { return new io.prometheus.client.Gauge.Child() { @Override public double get() { - final Object value = gauge.getValue(); + final Number value = gauge.getNumberValue(); if (value == null) { LOG.debug("Gauge {} is null-valued, defaulting to 0.", gauge); return 0; } - if (value instanceof Double) { - return (double) value; - } - if (value instanceof Number) { - return ((Number) value).doubleValue(); - } - if (value instanceof Boolean) { - return ((Boolean) value) ? 1 : 0; - } - LOG.debug("Invalid type for Gauge {}: {}, only number types and booleans are supported by this reporter.", - gauge, value.getClass().getName()); - return 0; + return value.doubleValue(); } }; } diff --git a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java index d4ad1f9ea0d14..024ea84e56a3f 100644 --- a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java +++ b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java @@ -20,9 +20,9 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.metrics.util.TestHistogram; import org.apache.flink.metrics.util.TestMeter; @@ -114,21 +114,8 @@ public void countersCanBeAddedSeveralTimesIfTheyDifferInLabels() throws UnirestE @Test public void gaugesCanBeAddedSeveralTimesIfTheyDifferInLabels() throws UnirestException { - Gauge gauge1 = new Gauge() { - @Override - public Integer getValue() { - return 3; - } - }; - Gauge gauge2 = new Gauge() { - @Override - public Integer getValue() { - return 4; - } - }; - - taskMetricGroup1.gauge("my_gauge", gauge1); - taskMetricGroup2.gauge("my_gauge", gauge2); + taskMetricGroup1.register("my_gauge", () -> 3); + taskMetricGroup2.register("my_gauge", () -> 4); assertThat(CollectorRegistry.defaultRegistry.getSampleValue("flink_taskmanager_job_task_my_gauge", LABEL_NAMES, labelValues1), equalTo(3.)); diff --git a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java index 39cf4be3fbd2a..6f14a11523570 100644 --- a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java +++ b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java @@ -21,10 +21,10 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.metrics.util.TestHistogram; import org.apache.flink.metrics.util.TestMeter; @@ -101,26 +101,12 @@ public void counterIsReportedAsPrometheusGauge() throws UnirestException { @Test public void gaugeIsReportedAsPrometheusGauge() throws UnirestException { - Gauge testGauge = new Gauge() { - @Override - public Integer getValue() { - return 1; - } - }; - - assertThatGaugeIsExported(testGauge, "testGauge", "1.0"); + assertThatGaugeIsExported((NumberGauge) () -> 1, "testGauge", "1.0"); } @Test public void nullGaugeDoesNotBreakReporter() throws UnirestException { - Gauge testGauge = new Gauge() { - @Override - public Integer getValue() { - return null; - } - }; - - assertThatGaugeIsExported(testGauge, "testGauge", "0.0"); + assertThatGaugeIsExported((NumberGauge) () -> null, "testGauge", "0.0"); } @Test @@ -183,45 +169,12 @@ public void invalidCharactersAreReplacedWithUnderscore() { @Test public void doubleGaugeIsConvertedCorrectly() { - assertThat(PrometheusReporter.gaugeFrom(new Gauge() { - @Override - public Double getValue() { - return 3.14; - } - }).get(), equalTo(3.14)); + assertThat(PrometheusReporter.gaugeFrom(() -> 3.14).get(), equalTo(3.14)); } @Test public void shortGaugeIsConvertedCorrectly() { - assertThat(PrometheusReporter.gaugeFrom(new Gauge() { - @Override - public Short getValue() { - return 13; - } - }).get(), equalTo(13.)); - } - - @Test - public void booleanGaugeIsConvertedCorrectly() { - assertThat(PrometheusReporter.gaugeFrom(new Gauge() { - @Override - public Boolean getValue() { - return true; - } - }).get(), equalTo(1.)); - } - - /** - * Prometheus only supports numbers, so report non-numeric gauges as 0. - */ - @Test - public void stringGaugeCannotBeConverted() { - assertThat(PrometheusReporter.gaugeFrom(new Gauge() { - @Override - public String getValue() { - return "I am not a number"; - } - }).get(), equalTo(0.)); + assertThat(PrometheusReporter.gaugeFrom(() -> 13).get(), equalTo(13.)); } @Test diff --git a/flink-metrics/flink-metrics-slf4j/src/main/java/org/apache/flink/metrics/slf4j/Slf4jReporter.java b/flink-metrics/flink-metrics-slf4j/src/main/java/org/apache/flink/metrics/slf4j/Slf4jReporter.java index 5efac30637f03..b91d0cb3f4e27 100644 --- a/flink-metrics/flink-metrics-slf4j/src/main/java/org/apache/flink/metrics/slf4j/Slf4jReporter.java +++ b/flink-metrics/flink-metrics-slf4j/src/main/java/org/apache/flink/metrics/slf4j/Slf4jReporter.java @@ -20,13 +20,14 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.HistogramStatistics; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.reporter.AbstractReporter; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; +import org.apache.flink.metrics.reporter.AbstractReporterV2; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.Scheduled; @@ -38,13 +39,18 @@ /** * {@link MetricReporter} that exports {@link Metric Metrics} via SLF4J {@link Logger}. */ -public class Slf4jReporter extends AbstractReporter implements Scheduled { +public class Slf4jReporter extends AbstractReporterV2 implements Scheduled { private static final Logger LOG = LoggerFactory.getLogger(Slf4jReporter.class); private static final String lineSeparator = System.lineSeparator(); @VisibleForTesting - Map, String> getGauges() { - return gauges; + Map getNumberGauges() { + return numberGauges; + } + + @VisibleForTesting + Map getStringGauges() { + return stringGauges; } @VisibleForTesting @@ -92,9 +98,14 @@ public void report() { .append(lineSeparator) .append("-- Gauges ---------------------------------------------------------------------") .append(lineSeparator); - for (Map.Entry, String> metric : gauges.entrySet()) { + for (Map.Entry metric : numberGauges.entrySet()) { + builder + .append(metric.getValue()).append(": ").append(metric.getKey().getNumberValue()) + .append(lineSeparator); + } + for (Map.Entry metric : stringGauges.entrySet()) { builder - .append(metric.getValue()).append(": ").append(metric.getKey().getValue()) + .append(metric.getValue()).append(": ").append(metric.getKey().getStringValue()) .append(lineSeparator); } diff --git a/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java b/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java index b344f45bca608..8a7e917b7a5b2 100644 --- a/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java +++ b/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java @@ -22,10 +22,10 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.metrics.util.TestHistogram; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -101,12 +101,9 @@ public void testAddCounter() throws Exception { public void testAddGauge() throws Exception { String gaugeName = "gauge"; - taskMetricGroup.gauge(gaugeName, null); - assertTrue(reporter.getGauges().isEmpty()); - - Gauge gauge = () -> null; - taskMetricGroup.gauge(gaugeName, gauge); - assertTrue(reporter.getGauges().containsKey(gauge)); + NumberGauge gauge = () -> null; + taskMetricGroup.register(gaugeName, gauge); + assertTrue(reporter.getNumberGauges().containsKey(gauge)); String expectedGaugeReport = reporter.filterCharacters(HOST_NAME) + delimiter + reporter.filterCharacters(TASK_MANAGER_ID) + delimiter + reporter.filterCharacters(JOB_NAME) + delimiter diff --git a/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java b/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java index 527f9c1076152..4668f2a9b963b 100644 --- a/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java +++ b/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java @@ -20,12 +20,13 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.HistogramStatistics; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.reporter.AbstractReporter; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; +import org.apache.flink.metrics.reporter.AbstractReporterV2; import org.apache.flink.metrics.reporter.Scheduled; import org.slf4j.Logger; @@ -49,7 +50,7 @@ *

Ported since it was not present in maven central. */ @PublicEvolving -public class StatsDReporter extends AbstractReporter implements Scheduled { +public class StatsDReporter extends AbstractReporterV2 implements Scheduled { private static final Logger LOG = LoggerFactory.getLogger(StatsDReporter.class); @@ -96,7 +97,13 @@ public void report() { // we do this to prevent holding the lock for very long and blocking // operator creation and shutdown try { - for (Map.Entry, String> entry : gauges.entrySet()) { + for (Map.Entry entry : numberGauges.entrySet()) { + if (closed) { + return; + } + reportGauge(entry.getValue(), entry.getKey()); + } + for (Map.Entry entry : stringGauges.entrySet()) { if (closed) { return; } @@ -130,8 +137,15 @@ private void reportCounter(final String name, final Counter counter) { send(name, String.valueOf(counter.getCount())); } - private void reportGauge(final String name, final Gauge gauge) { - Object value = gauge.getValue(); + private void reportGauge(final String name, final StringGauge gauge) { + String value = gauge.getStringValue(); + if (value != null) { + send(name, value); + } + } + + private void reportGauge(final String name, final NumberGauge gauge) { + Number value = gauge.getNumberValue(); if (value != null) { send(name, value.toString()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/TestReporter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/TestReporter.java index e5e09d4a5beb8..bcd1012dd1090 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/TestReporter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/TestReporter.java @@ -21,12 +21,12 @@ import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.reporter.AbstractReporter; +import org.apache.flink.metrics.reporter.AbstractReporterV2; /** * No-op reporter implementation. */ -public class TestReporter extends AbstractReporter { +public class TestReporter extends AbstractReporterV2 { @Override public void open(MetricConfig config) {} From 2046d3ee79adeb0528169686c3fddf980232614d Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 14 Feb 2018 15:26:04 +0100 Subject: [PATCH 3/3] replace all usages of now deprecated Gauge --- .../kafka/FlinkKafkaProducer011.java | 2 +- .../metrics/KafkaMetricMuttableWrapper.java | 6 +- .../kafka/internal/KafkaConsumerThread.java | 4 +- .../kafka/FlinkKafkaProducerBase.java | 2 +- .../kafka/internals/AbstractFetcher.java | 14 +- .../internals/metrics/KafkaMetricWrapper.java | 6 +- .../checkpoint/CheckpointStatsTracker.java | 69 +++------ .../executiongraph/ExecutionGraphBuilder.java | 8 +- .../executiongraph/metrics/DownTimeGauge.java | 8 +- .../metrics/NumberOfFullRestartsGauge.java | 8 +- .../metrics/RestartTimeGauge.java | 8 +- .../executiongraph/metrics/UpTimeGauge.java | 8 +- .../partition/ResultPartitionMetrics.java | 49 +------ .../partition/consumer/InputGateMetrics.java | 49 +------ .../runtime/metrics/dump/MetricDump.java | 2 +- .../metrics/dump/MetricDumpSerialization.java | 37 ++++- .../metrics/dump/MetricQueryService.java | 20 ++- .../metrics/groups/TaskIOMetricGroup.java | 26 ++-- .../runtime/metrics/util/MetricUtils.java | 131 ++++-------------- .../flink/runtime/jobmanager/JobManager.scala | 20 +-- .../CheckpointStatsTrackerTest.java | 115 ++++++++------- .../ExecutionGraphMetricsTest.java | 22 +-- .../dump/MetricDumpSerializerTest.java | 102 ++++++++------ .../metrics/dump/MetricQueryServiceTest.java | 9 +- .../groups/MetricGroupRegistrationTest.java | 1 - .../legacy/metrics/MetricFetcherTest.java | 19 ++- .../collector/selector/DirectedOutput.java | 4 +- .../runtime/io/RecordWriterOutput.java | 4 +- .../runtime/io/StreamInputProcessor.java | 2 +- .../runtime/io/StreamTwoInputProcessor.java | 2 +- .../runtime/metrics/MinWatermarkGauge.java | 10 +- .../runtime/metrics/WatermarkGauge.java | 8 +- .../runtime/tasks/OneInputStreamTask.java | 2 +- .../runtime/tasks/OperatorChain.java | 14 +- .../runtime/tasks/TwoInputStreamTask.java | 6 +- .../metrics/MinWatermarkGaugeTest.java | 8 +- .../runtime/metrics/WatermarkGaugeTest.java | 4 +- .../runtime/tasks/OneInputStreamTaskTest.java | 34 ++--- .../runtime/tasks/TwoInputStreamTaskTest.java | 62 ++++----- 39 files changed, 387 insertions(+), 518 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index 3fe4bc6b015f2..c964cdbfb488f 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -981,7 +981,7 @@ private FlinkKafkaProducer initProducer(boolean registerMetrics) // TODO: somehow merge metrics from all active producers? wrapper = new KafkaMetricMuttableWrapper(metric); previouslyCreatedMetrics.put(name, wrapper); - kafkaMetricGroup.gauge(name, wrapper); + kafkaMetricGroup.register(name, wrapper); } } } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/metrics/KafkaMetricMuttableWrapper.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/metrics/KafkaMetricMuttableWrapper.java index 3ff63636432d9..45cac00f82adc 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/metrics/KafkaMetricMuttableWrapper.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/metrics/KafkaMetricMuttableWrapper.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.kafka.internal.metrics; import org.apache.flink.annotation.Internal; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.kafka.common.Metric; @@ -27,7 +27,7 @@ * Gauge for getting the current value of a Kafka metric. */ @Internal -public class KafkaMetricMuttableWrapper implements Gauge { +public class KafkaMetricMuttableWrapper implements NumberGauge { private org.apache.kafka.common.Metric kafkaMetric; public KafkaMetricMuttableWrapper(org.apache.kafka.common.Metric metric) { @@ -35,7 +35,7 @@ public KafkaMetricMuttableWrapper(org.apache.kafka.common.Metric metric) { } @Override - public Double getValue() { + public Double getNumberValue() { return kafkaMetric.value(); } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java index 38e8a41d474cf..2b66c9189b6f6 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java @@ -187,10 +187,10 @@ public void run() { } else { // we have Kafka metrics, register them for (Map.Entry metric: metrics.entrySet()) { - consumerMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); + consumerMetricGroup.register(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); // TODO this metric is kept for compatibility purposes; should remove in the future - subtaskMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); + subtaskMetricGroup.register(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); } } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java index a4437d4cb4b84..7ef45d40aa9ef 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java @@ -237,7 +237,7 @@ public void open(Configuration configuration) { } else { final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer"); for (Map.Entry metric: metrics.entrySet()) { - kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); + kafkaMetricGroup.register(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); } } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java index 6f5d02a7b1395..c49f2d949f09b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java @@ -19,8 +19,8 @@ package org.apache.flink.streaming.connectors.kafka.internals; import org.apache.flink.annotation.Internal; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; @@ -618,11 +618,11 @@ private void registerOffsetMetrics( .addGroup(OFFSETS_BY_TOPIC_METRICS_GROUP, ktp.getTopic()) .addGroup(OFFSETS_BY_PARTITION_METRICS_GROUP, Integer.toString(ktp.getPartition())); - topicPartitionGroup.gauge(CURRENT_OFFSETS_METRICS_GAUGE, new OffsetGauge(ktp, OffsetGaugeType.CURRENT_OFFSET)); - topicPartitionGroup.gauge(COMMITTED_OFFSETS_METRICS_GAUGE, new OffsetGauge(ktp, OffsetGaugeType.COMMITTED_OFFSET)); + topicPartitionGroup.register(CURRENT_OFFSETS_METRICS_GAUGE, new OffsetGauge(ktp, OffsetGaugeType.CURRENT_OFFSET)); + topicPartitionGroup.register(COMMITTED_OFFSETS_METRICS_GAUGE, new OffsetGauge(ktp, OffsetGaugeType.COMMITTED_OFFSET)); - legacyCurrentOffsetsMetricGroup.gauge(getLegacyOffsetsMetricsGaugeName(ktp), new OffsetGauge(ktp, OffsetGaugeType.CURRENT_OFFSET)); - legacyCommittedOffsetsMetricGroup.gauge(getLegacyOffsetsMetricsGaugeName(ktp), new OffsetGauge(ktp, OffsetGaugeType.COMMITTED_OFFSET)); + legacyCurrentOffsetsMetricGroup.register(getLegacyOffsetsMetricsGaugeName(ktp), new OffsetGauge(ktp, OffsetGaugeType.CURRENT_OFFSET)); + legacyCommittedOffsetsMetricGroup.register(getLegacyOffsetsMetricsGaugeName(ktp), new OffsetGauge(ktp, OffsetGaugeType.COMMITTED_OFFSET)); } } @@ -641,7 +641,7 @@ private enum OffsetGaugeType { /** * Gauge for getting the offset of a KafkaTopicPartitionState. */ - private static class OffsetGauge implements Gauge { + private static class OffsetGauge implements NumberGauge { private final KafkaTopicPartitionState ktp; private final OffsetGaugeType gaugeType; @@ -652,7 +652,7 @@ private static class OffsetGauge implements Gauge { } @Override - public Long getValue() { + public Long getNumberValue() { switch(gaugeType) { case COMMITTED_OFFSET: return ktp.getCommittedOffset(); diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java index 4a5fb9d8390d7..9bd5ca3e90183 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java @@ -19,13 +19,13 @@ package org.apache.flink.streaming.connectors.kafka.internals.metrics; import org.apache.flink.annotation.Internal; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; /** * Gauge for getting the current value of a Kafka metric. */ @Internal -public class KafkaMetricWrapper implements Gauge { +public class KafkaMetricWrapper implements NumberGauge { private final org.apache.kafka.common.Metric kafkaMetric; public KafkaMetricWrapper(org.apache.kafka.common.Metric metric) { @@ -33,7 +33,7 @@ public KafkaMetricWrapper(org.apache.kafka.common.Metric metric) { } @Override - public Double getValue() { + public Double getNumberValue() { return kafkaMetric.value(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java index e6386ad74d624..716f212afad94 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java @@ -19,9 +19,10 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; @@ -353,48 +354,20 @@ void reportFailedCheckpoint(FailedCheckpointStats failed) { * @param metricGroup Metric group to use for the metrics. */ private void registerMetrics(MetricGroup metricGroup) { - metricGroup.gauge(NUMBER_OF_CHECKPOINTS_METRIC, new CheckpointsCounter()); - metricGroup.gauge(NUMBER_OF_IN_PROGRESS_CHECKPOINTS_METRIC, new InProgressCheckpointsCounter()); - metricGroup.gauge(NUMBER_OF_COMPLETED_CHECKPOINTS_METRIC, new CompletedCheckpointsCounter()); - metricGroup.gauge(NUMBER_OF_FAILED_CHECKPOINTS_METRIC, new FailedCheckpointsCounter()); - metricGroup.gauge(LATEST_RESTORED_CHECKPOINT_TIMESTAMP_METRIC, new LatestRestoredCheckpointTimestampGauge()); - metricGroup.gauge(LATEST_COMPLETED_CHECKPOINT_SIZE_METRIC, new LatestCompletedCheckpointSizeGauge()); - metricGroup.gauge(LATEST_COMPLETED_CHECKPOINT_DURATION_METRIC, new LatestCompletedCheckpointDurationGauge()); - metricGroup.gauge(LATEST_COMPLETED_CHECKPOINT_ALIGNMENT_BUFFERED_METRIC, new LatestCompletedCheckpointAlignmentBufferedGauge()); - metricGroup.gauge(LATEST_COMPLETED_CHECKPOINT_EXTERNAL_PATH_METRIC, new LatestCompletedCheckpointExternalPathGauge()); + metricGroup.register(NUMBER_OF_CHECKPOINTS_METRIC, counts::getTotalNumberOfCheckpoints); + metricGroup.register(NUMBER_OF_IN_PROGRESS_CHECKPOINTS_METRIC, counts::getNumberOfInProgressCheckpoints); + metricGroup.register(NUMBER_OF_COMPLETED_CHECKPOINTS_METRIC, counts::getNumberOfCompletedCheckpoints); + metricGroup.register(NUMBER_OF_FAILED_CHECKPOINTS_METRIC, counts::getNumberOfFailedCheckpoints); + metricGroup.register(LATEST_RESTORED_CHECKPOINT_TIMESTAMP_METRIC, new LatestRestoredCheckpointTimestampGauge()); + metricGroup.register(LATEST_COMPLETED_CHECKPOINT_SIZE_METRIC, new LatestCompletedCheckpointSizeGauge()); + metricGroup.register(LATEST_COMPLETED_CHECKPOINT_DURATION_METRIC, new LatestCompletedCheckpointDurationGauge()); + metricGroup.register(LATEST_COMPLETED_CHECKPOINT_ALIGNMENT_BUFFERED_METRIC, new LatestCompletedCheckpointAlignmentBufferedGauge()); + metricGroup.register(LATEST_COMPLETED_CHECKPOINT_EXTERNAL_PATH_METRIC, new LatestCompletedCheckpointExternalPathGauge()); } - private class CheckpointsCounter implements Gauge { + private class LatestRestoredCheckpointTimestampGauge implements NumberGauge { @Override - public Long getValue() { - return counts.getTotalNumberOfCheckpoints(); - } - } - - private class InProgressCheckpointsCounter implements Gauge { - @Override - public Integer getValue() { - return counts.getNumberOfInProgressCheckpoints(); - } - } - - private class CompletedCheckpointsCounter implements Gauge { - @Override - public Long getValue() { - return counts.getNumberOfCompletedCheckpoints(); - } - } - - private class FailedCheckpointsCounter implements Gauge { - @Override - public Long getValue() { - return counts.getNumberOfFailedCheckpoints(); - } - } - - private class LatestRestoredCheckpointTimestampGauge implements Gauge { - @Override - public Long getValue() { + public Long getNumberValue() { RestoredCheckpointStats restored = latestRestoredCheckpoint; if (restored != null) { return restored.getRestoreTimestamp(); @@ -404,9 +377,9 @@ public Long getValue() { } } - private class LatestCompletedCheckpointSizeGauge implements Gauge { + private class LatestCompletedCheckpointSizeGauge implements NumberGauge { @Override - public Long getValue() { + public Long getNumberValue() { CompletedCheckpointStats completed = latestCompletedCheckpoint; if (completed != null) { return completed.getStateSize(); @@ -416,9 +389,9 @@ public Long getValue() { } } - private class LatestCompletedCheckpointDurationGauge implements Gauge { + private class LatestCompletedCheckpointDurationGauge implements NumberGauge { @Override - public Long getValue() { + public Long getNumberValue() { CompletedCheckpointStats completed = latestCompletedCheckpoint; if (completed != null) { return completed.getEndToEndDuration(); @@ -428,9 +401,9 @@ public Long getValue() { } } - private class LatestCompletedCheckpointAlignmentBufferedGauge implements Gauge { + private class LatestCompletedCheckpointAlignmentBufferedGauge implements NumberGauge { @Override - public Long getValue() { + public Long getNumberValue() { CompletedCheckpointStats completed = latestCompletedCheckpoint; if (completed != null) { return completed.getAlignmentBuffered(); @@ -440,9 +413,9 @@ public Long getValue() { } } - private class LatestCompletedCheckpointExternalPathGauge implements Gauge { + private class LatestCompletedCheckpointExternalPathGauge implements StringGauge { @Override - public String getValue() { + public String getStringValue() { CompletedCheckpointStats completed = latestCompletedCheckpoint; if (completed != null && completed.getExternalPath() != null) { return completed.getExternalPath(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java index cd719bea98561..9f356fd7b5532 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java @@ -308,10 +308,10 @@ public static ExecutionGraph buildGraph( // create all the metrics for the Execution Graph - metrics.gauge(RestartTimeGauge.METRIC_NAME, new RestartTimeGauge(executionGraph)); - metrics.gauge(DownTimeGauge.METRIC_NAME, new DownTimeGauge(executionGraph)); - metrics.gauge(UpTimeGauge.METRIC_NAME, new UpTimeGauge(executionGraph)); - metrics.gauge(NumberOfFullRestartsGauge.METRIC_NAME, new NumberOfFullRestartsGauge(executionGraph)); + metrics.register(RestartTimeGauge.METRIC_NAME, new RestartTimeGauge(executionGraph)); + metrics.register(DownTimeGauge.METRIC_NAME, new DownTimeGauge(executionGraph)); + metrics.register(UpTimeGauge.METRIC_NAME, new UpTimeGauge(executionGraph)); + metrics.register(NumberOfFullRestartsGauge.METRIC_NAME, new NumberOfFullRestartsGauge(executionGraph)); executionGraph.getFailoverStrategy().registerMetrics(metrics); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/DownTimeGauge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/DownTimeGauge.java index 5f24587f9921a..6796f3d6c6230 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/DownTimeGauge.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/DownTimeGauge.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.executiongraph.metrics; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.jobgraph.JobStatus; @@ -33,7 +33,7 @@ * {@value NOT_YET_RUNNING}, and for jobs that are not running any more, it returns * {@value NO_LONGER_RUNNING}. */ -public class DownTimeGauge implements Gauge { +public class DownTimeGauge implements NumberGauge { public static final String METRIC_NAME = "downtime"; @@ -52,7 +52,7 @@ public DownTimeGauge(ExecutionGraph executionGraph) { // ------------------------------------------------------------------------ @Override - public Long getValue() { + public Long getNumberValue() { final JobStatus status = eg.getState(); if (status == JobStatus.RUNNING) { @@ -76,4 +76,4 @@ else if (status.isTerminalState()) { } } } -} \ No newline at end of file +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java index 05a641468e38c..c08b921426580 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.executiongraph.metrics; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -26,7 +26,7 @@ /** * Gauge which returns the number of full restarts. */ -public class NumberOfFullRestartsGauge implements Gauge { +public class NumberOfFullRestartsGauge implements NumberGauge { public static final String METRIC_NAME = "fullRestarts"; @@ -41,7 +41,7 @@ public NumberOfFullRestartsGauge(ExecutionGraph executionGraph) { // ------------------------------------------------------------------------ @Override - public Long getValue() { + public Long getNumberValue() { return eg.getNumberOfFullRestarts(); } -} \ No newline at end of file +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/RestartTimeGauge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/RestartTimeGauge.java index e0a22e31a6da1..62fb97facaba9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/RestartTimeGauge.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/RestartTimeGauge.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.executiongraph.metrics; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.jobgraph.JobStatus; @@ -33,7 +33,7 @@ *

If the job has not yet reached either of these states, then the time is measured since reaching * {@link JobStatus#RESTARTING}. If it is still the initial job execution, then the gauge will return 0. */ -public class RestartTimeGauge implements Gauge { +public class RestartTimeGauge implements NumberGauge { public static final String METRIC_NAME = "restartingTime"; @@ -48,7 +48,7 @@ public RestartTimeGauge(ExecutionGraph executionGraph) { // ------------------------------------------------------------------------ @Override - public Long getValue() { + public Long getNumberValue() { final JobStatus status = eg.getState(); final long restartingTimestamp = eg.getStatusTimestamp(JobStatus.RESTARTING); @@ -77,4 +77,4 @@ else if (status.isTerminalState()) { // we guard this with 'Math.max' to avoid negative timestamps when clocks re-sync return Math.max(lastRestartTime, 0); } -} \ No newline at end of file +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/UpTimeGauge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/UpTimeGauge.java index d3f6224cbe8c5..95a8d8e9aa077 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/UpTimeGauge.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/UpTimeGauge.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.executiongraph.metrics; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.jobgraph.JobStatus; @@ -29,7 +29,7 @@ * *

For jobs that are not running any more, it returns {@value NO_LONGER_RUNNING}. */ -public class UpTimeGauge implements Gauge { +public class UpTimeGauge implements NumberGauge { public static final String METRIC_NAME = "uptime"; @@ -46,7 +46,7 @@ public UpTimeGauge(ExecutionGraph executionGraph) { // ------------------------------------------------------------------------ @Override - public Long getValue() { + public Long getNumberValue() { final JobStatus status = eg.getState(); if (status == JobStatus.RUNNING) { @@ -64,4 +64,4 @@ else if (status.isTerminalState()) { return 0L; } } -} \ No newline at end of file +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionMetrics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionMetrics.java index fde2ebd56e573..36f1ad1a4d077 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionMetrics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionMetrics.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.io.network.partition; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -114,46 +113,6 @@ float refreshAndGetAvg() { return total / (float) allPartitions.length; } - // ------------------------------------------------------------------------ - // Gauges to access the stats - // ------------------------------------------------------------------------ - - private Gauge getTotalQueueLenGauge() { - return new Gauge() { - @Override - public Long getValue() { - return refreshAndGetTotal(); - } - }; - } - - private Gauge getMinQueueLenGauge() { - return new Gauge() { - @Override - public Integer getValue() { - return refreshAndGetMin(); - } - }; - } - - private Gauge getMaxQueueLenGauge() { - return new Gauge() { - @Override - public Integer getValue() { - return refreshAndGetMax(); - } - }; - } - - private Gauge getAvgQueueLenGauge() { - return new Gauge() { - @Override - public Float getValue() { - return refreshAndGetAvg(); - } - }; - } - // ------------------------------------------------------------------------ // Static access // ------------------------------------------------------------------------ @@ -161,9 +120,9 @@ public Float getValue() { public static void registerQueueLengthMetrics(MetricGroup group, ResultPartition partition) { ResultPartitionMetrics metrics = new ResultPartitionMetrics(partition); - group.gauge("totalQueueLen", metrics.getTotalQueueLenGauge()); - group.gauge("minQueueLen", metrics.getMinQueueLenGauge()); - group.gauge("maxQueueLen", metrics.getMaxQueueLenGauge()); - group.gauge("avgQueueLen", metrics.getAvgQueueLenGauge()); + group.register("totalQueueLen", metrics::refreshAndGetTotal); + group.register("minQueueLen", metrics::refreshAndGetMin); + group.register("maxQueueLen", metrics::refreshAndGetMax); + group.register("avgQueueLen", metrics::refreshAndGetAvg); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateMetrics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateMetrics.java index 69af4553fe4ff..bacd911271cb6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateMetrics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateMetrics.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; import java.util.Collection; @@ -132,46 +131,6 @@ float refreshAndGetAvg() { return count == 0 ? 0 : total / (float) count; } - // ------------------------------------------------------------------------ - // Gauges to access the stats - // ------------------------------------------------------------------------ - - private Gauge getTotalQueueLenGauge() { - return new Gauge() { - @Override - public Long getValue() { - return refreshAndGetTotal(); - } - }; - } - - private Gauge getMinQueueLenGauge() { - return new Gauge() { - @Override - public Integer getValue() { - return refreshAndGetMin(); - } - }; - } - - private Gauge getMaxQueueLenGauge() { - return new Gauge() { - @Override - public Integer getValue() { - return refreshAndGetMax(); - } - }; - } - - private Gauge getAvgQueueLenGauge() { - return new Gauge() { - @Override - public Float getValue() { - return refreshAndGetAvg(); - } - }; - } - // ------------------------------------------------------------------------ // Static access // ------------------------------------------------------------------------ @@ -179,9 +138,9 @@ public Float getValue() { public static void registerQueueLengthMetrics(MetricGroup group, SingleInputGate gate) { InputGateMetrics metrics = new InputGateMetrics(gate); - group.gauge("totalQueueLen", metrics.getTotalQueueLenGauge()); - group.gauge("minQueueLen", metrics.getMinQueueLenGauge()); - group.gauge("maxQueueLen", metrics.getMaxQueueLenGauge()); - group.gauge("avgQueueLen", metrics.getAvgQueueLenGauge()); + group.register("totalQueueLen", metrics::refreshAndGetTotal); + group.register("minQueueLen", metrics::refreshAndGetMin); + group.register("maxQueueLen", metrics::refreshAndGetMax); + group.register("avgQueueLen", metrics::refreshAndGetAvg); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDump.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDump.java index c2d1eea3c133d..2592e963c0525 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDump.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDump.java @@ -74,7 +74,7 @@ public byte getCategory() { } /** - * Container for the value of a {@link org.apache.flink.metrics.Gauge} as a string. + * Container for the value of a {@link org.apache.flink.metrics.NumberGauge} or {@link org.apache.flink.metrics.StringGauge} as a string. */ public static class GaugeDump extends MetricDump { public final String value; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerialization.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerialization.java index d821e0fdd6989..6ce5592593e3f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerialization.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerialization.java @@ -23,11 +23,12 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.HistogramStatistics; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -116,13 +117,16 @@ public static class MetricDumpSerializer { * fully serialized before the failure will be returned. * * @param counters counters to serialize - * @param gauges gauges to serialize + * @param numberGauges number gauges to serialize + * @param stringGauges string gauges to serialize * @param histograms histograms to serialize + * @param meters meters to serialize * @return MetricSerializationResult containing the serialized metrics and the count of each metric type */ public MetricSerializationResult serialize( Map> counters, - Map, Tuple2> gauges, + Map> numberGauges, + Map> stringGauges, Map> histograms, Map> meters) { @@ -139,7 +143,15 @@ public MetricSerializationResult serialize( } int numGauges = 0; - for (Map.Entry, Tuple2> entry : gauges.entrySet()) { + for (Map.Entry> entry : numberGauges.entrySet()) { + try { + serializeGauge(buffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); + numGauges++; + } catch (Exception e) { + LOG.debug("Failed to serialize gauge.", e); + } + } + for (Map.Entry> entry : stringGauges.entrySet()) { try { serializeGauge(buffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); numGauges++; @@ -215,12 +227,12 @@ private static void serializeCounter(DataOutput out, QueryScopeInfo info, String out.writeLong(count); } - private static void serializeGauge(DataOutput out, QueryScopeInfo info, String name, Gauge gauge) throws IOException { - Object value = gauge.getValue(); + private static void serializeGauge(DataOutput out, QueryScopeInfo info, String name, NumberGauge gauge) throws IOException { + Object value = gauge.getNumberValue(); if (value == null) { throw new NullPointerException("Value returned by gauge " + name + " was null."); } - String stringValue = gauge.getValue().toString(); + String stringValue = gauge.getNumberValue().toString(); if (stringValue == null) { throw new NullPointerException("toString() of the value returned by gauge " + name + " returned null."); } @@ -230,6 +242,17 @@ private static void serializeGauge(DataOutput out, QueryScopeInfo info, String n out.writeUTF(stringValue); } + private static void serializeGauge(DataOutput out, QueryScopeInfo info, String name, StringGauge gauge) throws IOException { + String value = gauge.getStringValue(); + if (value == null) { + throw new NullPointerException("Value returned by gauge " + name + " was null."); + } + + serializeMetricInfo(out, info); + out.writeUTF(name); + out.writeUTF(value); + } + private static void serializeHistogram(DataOutput out, QueryScopeInfo info, String name, Histogram histogram) throws IOException { HistogramStatistics stat = histogram.getStatistics(); long min = stat.getMin(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java index 8821e0d9f4a14..0876afaf3701e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java @@ -21,10 +21,11 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.metrics.CharacterFilter; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; @@ -65,7 +66,8 @@ public String filterCharacters(String input) { private final MetricDumpSerializer serializer = new MetricDumpSerializer(); - private final Map, Tuple2> gauges = new HashMap<>(); + private final Map> numbergGauges = new HashMap<>(); + private final Map> stringGauges = new HashMap<>(); private final Map> counters = new HashMap<>(); private final Map> histograms = new HashMap<>(); private final Map> meters = new HashMap<>(); @@ -89,8 +91,10 @@ public void onReceive(Object message) { if (metric instanceof Counter) { counters.put((Counter) metric, new Tuple2<>(info, FILTER.filterCharacters(metricName))); - } else if (metric instanceof Gauge) { - gauges.put((Gauge) metric, new Tuple2<>(info, FILTER.filterCharacters(metricName))); + } else if (metric instanceof NumberGauge) { + numbergGauges.put((NumberGauge) metric, new Tuple2<>(info, FILTER.filterCharacters(metricName))); + } else if (metric instanceof StringGauge) { + stringGauges.put((StringGauge) metric, new Tuple2<>(info, FILTER.filterCharacters(metricName))); } else if (metric instanceof Histogram) { histograms.put((Histogram) metric, new Tuple2<>(info, FILTER.filterCharacters(metricName))); } else if (metric instanceof Meter) { @@ -100,15 +104,17 @@ public void onReceive(Object message) { Metric metric = (((RemoveMetric) message).metric); if (metric instanceof Counter) { this.counters.remove(metric); - } else if (metric instanceof Gauge) { - this.gauges.remove(metric); + } else if (metric instanceof NumberGauge) { + this.numbergGauges.remove(metric); + } else if (metric instanceof StringGauge) { + this.stringGauges.remove(metric); } else if (metric instanceof Histogram) { this.histograms.remove(metric); } else if (metric instanceof Meter) { this.meters.remove(metric); } } else if (message instanceof CreateDump) { - MetricDumpSerialization.MetricSerializationResult dump = serializer.serialize(counters, gauges, histograms, meters); + MetricDumpSerialization.MetricSerializationResult dump = serializer.serialize(counters, numbergGauges, stringGauges, histograms, meters); getSender().tell(dump, getSelf()); } else { LOG.warn("MetricQueryServiceActor received an invalid message. " + message.toString()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java index e12ecd7d25c02..370cd1444ea43 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java @@ -19,10 +19,10 @@ package org.apache.flink.runtime.metrics.groups; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.MeterView; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.executiongraph.IOMetrics; import org.apache.flink.runtime.io.network.partition.ResultPartition; @@ -114,16 +114,16 @@ public Meter getNumBytesOutRateMeter() { */ public void initializeBufferMetrics(Task task) { final MetricGroup buffers = addGroup("buffers"); - buffers.gauge("inputQueueLength", new InputBuffersGauge(task)); - buffers.gauge("outputQueueLength", new OutputBuffersGauge(task)); - buffers.gauge("inPoolUsage", new InputBufferPoolUsageGauge(task)); - buffers.gauge("outPoolUsage", new OutputBufferPoolUsageGauge(task)); + buffers.register("inputQueueLength", new InputBuffersGauge(task)); + buffers.register("outputQueueLength", new OutputBuffersGauge(task)); + buffers.register("inPoolUsage", new InputBufferPoolUsageGauge(task)); + buffers.register("outPoolUsage", new OutputBufferPoolUsageGauge(task)); } /** * Gauge measuring the number of queued input buffers of a task. */ - private static final class InputBuffersGauge implements Gauge { + private static final class InputBuffersGauge implements NumberGauge { private final Task task; @@ -132,7 +132,7 @@ public InputBuffersGauge(Task task) { } @Override - public Integer getValue() { + public Integer getNumberValue() { int totalBuffers = 0; for (SingleInputGate inputGate : task.getAllInputGates()) { @@ -146,7 +146,7 @@ public Integer getValue() { /** * Gauge measuring the number of queued output buffers of a task. */ - private static final class OutputBuffersGauge implements Gauge { + private static final class OutputBuffersGauge implements NumberGauge { private final Task task; @@ -155,7 +155,7 @@ public OutputBuffersGauge(Task task) { } @Override - public Integer getValue() { + public Integer getNumberValue() { int totalBuffers = 0; for (ResultPartition producedPartition : task.getProducedPartitions()) { @@ -169,7 +169,7 @@ public Integer getValue() { /** * Gauge measuring the input buffer pool usage gauge of a task. */ - private static final class InputBufferPoolUsageGauge implements Gauge { + private static final class InputBufferPoolUsageGauge implements NumberGauge { private final Task task; @@ -178,7 +178,7 @@ public InputBufferPoolUsageGauge(Task task) { } @Override - public Float getValue() { + public Float getNumberValue() { int usedBuffers = 0; int bufferPoolSize = 0; @@ -198,7 +198,7 @@ public Float getValue() { /** * Gauge measuring the output buffer pool usage gauge of a task. */ - private static final class OutputBufferPoolUsageGauge implements Gauge { + private static final class OutputBufferPoolUsageGauge implements NumberGauge { private final Task task; @@ -207,7 +207,7 @@ public OutputBufferPoolUsageGauge(Task task) { } @Override - public Float getValue() { + public Float getNumberValue() { int usedBuffers = 0; int bufferPoolSize = 0; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java index 3fd268a1aeb7a..1e6875e99ed3b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java @@ -18,8 +18,8 @@ package org.apache.flink.runtime.metrics.util; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; @@ -105,37 +105,17 @@ public static void instantiateStatusMetrics( private static void instantiateNetworkMetrics( MetricGroup metrics, final NetworkEnvironment network) { - metrics.>gauge("TotalMemorySegments", new Gauge () { - @Override - public Long getValue() { - return (long) network.getNetworkBufferPool().getTotalNumberOfMemorySegments(); - } - }); + metrics.register("TotalMemorySegments", () -> network.getNetworkBufferPool().getTotalNumberOfMemorySegments()); - metrics.>gauge("AvailableMemorySegments", new Gauge () { - @Override - public Long getValue() { - return (long) network.getNetworkBufferPool().getNumberOfAvailableMemorySegments(); - } - }); + metrics.register("AvailableMemorySegments", () -> network.getNetworkBufferPool().getNumberOfAvailableMemorySegments()); } private static void instantiateClassLoaderMetrics(MetricGroup metrics) { final ClassLoadingMXBean mxBean = ManagementFactory.getClassLoadingMXBean(); - metrics.>gauge("ClassesLoaded", new Gauge () { - @Override - public Long getValue() { - return mxBean.getTotalLoadedClassCount(); - } - }); + metrics.register("ClassesLoaded", mxBean::getTotalLoadedClassCount); - metrics.>gauge("ClassesUnloaded", new Gauge () { - @Override - public Long getValue() { - return mxBean.getUnloadedClassCount(); - } - }); + metrics.register("ClassesUnloaded", mxBean::getUnloadedClassCount); } private static void instantiateGarbageCollectorMetrics(MetricGroup metrics) { @@ -144,19 +124,9 @@ private static void instantiateGarbageCollectorMetrics(MetricGroup metrics) { for (final GarbageCollectorMXBean garbageCollector: garbageCollectors) { MetricGroup gcGroup = metrics.addGroup(garbageCollector.getName()); - gcGroup.>gauge("Count", new Gauge () { - @Override - public Long getValue() { - return garbageCollector.getCollectionCount(); - } - }); - - gcGroup.>gauge("Time", new Gauge () { - @Override - public Long getValue() { - return garbageCollector.getCollectionTime(); - } - }); + gcGroup.register("Count", garbageCollector::getCollectionCount); + + gcGroup.register("Time", garbageCollector::getCollectionTime); } } @@ -165,45 +135,15 @@ private static void instantiateMemoryMetrics(MetricGroup metrics) { MetricGroup heap = metrics.addGroup("Heap"); - heap.>gauge("Used", new Gauge () { - @Override - public Long getValue() { - return mxBean.getHeapMemoryUsage().getUsed(); - } - }); - heap.>gauge("Committed", new Gauge () { - @Override - public Long getValue() { - return mxBean.getHeapMemoryUsage().getCommitted(); - } - }); - heap.>gauge("Max", new Gauge () { - @Override - public Long getValue() { - return mxBean.getHeapMemoryUsage().getMax(); - } - }); + heap.register("Used", mxBean.getHeapMemoryUsage()::getUsed); + heap.register("Committed", mxBean.getHeapMemoryUsage()::getCommitted); + heap.register("Max", mxBean.getHeapMemoryUsage()::getMax); MetricGroup nonHeap = metrics.addGroup("NonHeap"); - nonHeap.>gauge("Used", new Gauge () { - @Override - public Long getValue() { - return mxBean.getNonHeapMemoryUsage().getUsed(); - } - }); - nonHeap.>gauge("Committed", new Gauge () { - @Override - public Long getValue() { - return mxBean.getNonHeapMemoryUsage().getCommitted(); - } - }); - nonHeap.>gauge("Max", new Gauge () { - @Override - public Long getValue() { - return mxBean.getNonHeapMemoryUsage().getMax(); - } - }); + nonHeap.register("Used", mxBean.getNonHeapMemoryUsage()::getUsed); + nonHeap.register("Committed", mxBean.getNonHeapMemoryUsage()::getCommitted); + nonHeap.register("Max", mxBean.getNonHeapMemoryUsage()::getMax); final MBeanServer con = ManagementFactory.getPlatformMBeanServer(); @@ -214,9 +154,9 @@ public Long getValue() { MetricGroup direct = metrics.addGroup("Direct"); - direct.>gauge("Count", new AttributeGauge<>(con, directObjectName, "Count", -1L)); - direct.>gauge("MemoryUsed", new AttributeGauge<>(con, directObjectName, "MemoryUsed", -1L)); - direct.>gauge("TotalCapacity", new AttributeGauge<>(con, directObjectName, "TotalCapacity", -1L)); + direct.register("Count", new AttributeGauge(con, directObjectName, "Count", -1L)); + direct.register("MemoryUsed", new AttributeGauge(con, directObjectName, "MemoryUsed", -1L)); + direct.register("TotalCapacity", new AttributeGauge(con, directObjectName, "TotalCapacity", -1L)); } catch (MalformedObjectNameException e) { LOG.warn("Could not create object name {}.", directBufferPoolName, e); } @@ -228,9 +168,9 @@ public Long getValue() { MetricGroup mapped = metrics.addGroup("Mapped"); - mapped.>gauge("Count", new AttributeGauge<>(con, mappedObjectName, "Count", -1L)); - mapped.>gauge("MemoryUsed", new AttributeGauge<>(con, mappedObjectName, "MemoryUsed", -1L)); - mapped.>gauge("TotalCapacity", new AttributeGauge<>(con, mappedObjectName, "TotalCapacity", -1L)); + mapped.register("Count", new AttributeGauge(con, mappedObjectName, "Count", -1L)); + mapped.register("MemoryUsed", new AttributeGauge(con, mappedObjectName, "MemoryUsed", -1L)); + mapped.register("TotalCapacity", new AttributeGauge(con, mappedObjectName, "TotalCapacity", -1L)); } catch (MalformedObjectNameException e) { LOG.warn("Could not create object name {}.", mappedBufferPoolName, e); } @@ -239,43 +179,28 @@ public Long getValue() { private static void instantiateThreadMetrics(MetricGroup metrics) { final ThreadMXBean mxBean = ManagementFactory.getThreadMXBean(); - metrics.>gauge("Count", new Gauge () { - @Override - public Integer getValue() { - return mxBean.getThreadCount(); - } - }); + metrics.register("Count", mxBean::getThreadCount); } private static void instantiateCPUMetrics(MetricGroup metrics) { try { final com.sun.management.OperatingSystemMXBean mxBean = (com.sun.management.OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean(); - metrics.>gauge("Load", new Gauge () { - @Override - public Double getValue() { - return mxBean.getProcessCpuLoad(); - } - }); - metrics.>gauge("Time", new Gauge () { - @Override - public Long getValue() { - return mxBean.getProcessCpuTime(); - } - }); + metrics.register("Load", mxBean::getProcessCpuLoad); + metrics.register("Time", mxBean::getProcessCpuTime); } catch (Exception e) { LOG.warn("Cannot access com.sun.management.OperatingSystemMXBean.getProcessCpuLoad()" + " - CPU load metrics will not be available.", e); } } - private static final class AttributeGauge implements Gauge { + private static final class AttributeGauge implements NumberGauge { private final MBeanServer server; private final ObjectName objectName; private final String attributeName; - private final T errorValue; + private final long errorValue; - private AttributeGauge(MBeanServer server, ObjectName objectName, String attributeName, T errorValue) { + private AttributeGauge(MBeanServer server, ObjectName objectName, String attributeName, long errorValue) { this.server = Preconditions.checkNotNull(server); this.objectName = Preconditions.checkNotNull(objectName); this.attributeName = Preconditions.checkNotNull(attributeName); @@ -284,9 +209,9 @@ private AttributeGauge(MBeanServer server, ObjectName objectName, String attribu @SuppressWarnings("unchecked") @Override - public T getValue() { + public Number getNumberValue() { try { - return (T) server.getAttribute(objectName, attributeName); + return (Number) server.getAttribute(objectName, attributeName); } catch (MBeanException | AttributeNotFoundException | InstanceNotFoundException | ReflectionException e) { LOG.warn("Could not read attribute {}.", attributeName, e); return errorValue; diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index b7956097b4332..fc38fb1dbdbc5 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -34,7 +34,7 @@ import org.apache.flink.api.common.time.Time import org.apache.flink.configuration._ import org.apache.flink.core.fs.{FileSystem, Path} import org.apache.flink.core.io.InputSplitAssigner -import org.apache.flink.metrics.{Gauge, MetricGroup} +import org.apache.flink.metrics.{MetricGroup, NumberGauge} import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour} import org.apache.flink.runtime.blob.{BlobServer, BlobStore} @@ -1855,18 +1855,20 @@ class JobManager( } private def instantiateMetrics(jobManagerMetricGroup: MetricGroup) : Unit = { - jobManagerMetricGroup.gauge[Long, Gauge[Long]]("taskSlotsAvailable", new Gauge[Long] { - override def getValue: Long = JobManager.this.instanceManager.getNumberOfAvailableSlots + jobManagerMetricGroup.register("taskSlotsAvailable", new NumberGauge { + override def getNumberValue: Number + = JobManager.this.instanceManager.getNumberOfAvailableSlots }) - jobManagerMetricGroup.gauge[Long, Gauge[Long]]("taskSlotsTotal", new Gauge[Long] { - override def getValue: Long = JobManager.this.instanceManager.getTotalNumberOfSlots + jobManagerMetricGroup.register("taskSlotsTotal", new NumberGauge { + override def getNumberValue: Number + = JobManager.this.instanceManager.getTotalNumberOfSlots }) - jobManagerMetricGroup.gauge[Long, Gauge[Long]]("numRegisteredTaskManagers", new Gauge[Long] { - override def getValue: Long + jobManagerMetricGroup.register("numRegisteredTaskManagers", new NumberGauge { + override def getNumberValue: Number = JobManager.this.instanceManager.getNumberOfRegisteredTaskManagers }) - jobManagerMetricGroup.gauge[Long, Gauge[Long]]("numRunningJobs", new Gauge[Long] { - override def getValue: Long = JobManager.this.currentJobs.size + jobManagerMetricGroup.register("numRunningJobs", new NumberGauge { + override def getNumberValue: Number = JobManager.this.currentJobs.size }) } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java index 82dcd023f9bf3..b4af04255b6a8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java @@ -33,8 +33,9 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.NumberGauge; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -286,11 +287,13 @@ public void testMetricsRegistration() throws Exception { MetricGroup metricGroup = new UnregisteredMetricsGroup() { @Override - public > G gauge(String name, G gauge) { - if (gauge != null) { - registeredGaugeNames.add(name); - } - return gauge; + public void register(String name, NumberGauge gauge) { + registeredGaugeNames.add(name); + } + + @Override + public void register(String name, StringGauge gauge) { + registeredGaugeNames.add(name); } }; @@ -326,13 +329,18 @@ public > G gauge(String name, G gauge) { @Test @SuppressWarnings("unchecked") public void testMetricsAreUpdated() throws Exception { - final Map> registeredGauges = new HashMap<>(); + final Map registeredNumberGauges = new HashMap<>(); + final Map registeredStringGauges = new HashMap<>(); MetricGroup metricGroup = new UnregisteredMetricsGroup() { @Override - public > G gauge(String name, G gauge) { - registeredGauges.put(name, gauge); - return gauge; + public void register(String name, NumberGauge gauge) { + registeredNumberGauges.put(name, gauge); + } + + @Override + public void register(String name, StringGauge gauge) { + registeredStringGauges.put(name, gauge); } }; @@ -347,28 +355,29 @@ public > G gauge(String name, G gauge) { metricGroup); // Make sure to adjust this test if metrics are added/removed - assertEquals(9, registeredGauges.size()); + assertEquals(8, registeredNumberGauges.size()); + assertEquals(1, registeredStringGauges.size()); // Check initial values - Gauge numCheckpoints = (Gauge) registeredGauges.get(CheckpointStatsTracker.NUMBER_OF_CHECKPOINTS_METRIC); - Gauge numInProgressCheckpoints = (Gauge) registeredGauges.get(CheckpointStatsTracker.NUMBER_OF_IN_PROGRESS_CHECKPOINTS_METRIC); - Gauge numCompletedCheckpoints = (Gauge) registeredGauges.get(CheckpointStatsTracker.NUMBER_OF_COMPLETED_CHECKPOINTS_METRIC); - Gauge numFailedCheckpoints = (Gauge) registeredGauges.get(CheckpointStatsTracker.NUMBER_OF_FAILED_CHECKPOINTS_METRIC); - Gauge latestRestoreTimestamp = (Gauge) registeredGauges.get(CheckpointStatsTracker.LATEST_RESTORED_CHECKPOINT_TIMESTAMP_METRIC); - Gauge latestCompletedSize = (Gauge) registeredGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_SIZE_METRIC); - Gauge latestCompletedDuration = (Gauge) registeredGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_DURATION_METRIC); - Gauge latestCompletedAlignmentBuffered = (Gauge) registeredGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_ALIGNMENT_BUFFERED_METRIC); - Gauge latestCompletedExternalPath = (Gauge) registeredGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_EXTERNAL_PATH_METRIC); - - assertEquals(Long.valueOf(0), numCheckpoints.getValue()); - assertEquals(Integer.valueOf(0), numInProgressCheckpoints.getValue()); - assertEquals(Long.valueOf(0), numCompletedCheckpoints.getValue()); - assertEquals(Long.valueOf(0), numFailedCheckpoints.getValue()); - assertEquals(Long.valueOf(-1), latestRestoreTimestamp.getValue()); - assertEquals(Long.valueOf(-1), latestCompletedSize.getValue()); - assertEquals(Long.valueOf(-1), latestCompletedDuration.getValue()); - assertEquals(Long.valueOf(-1), latestCompletedAlignmentBuffered.getValue()); - assertEquals("n/a", latestCompletedExternalPath.getValue()); + NumberGauge numCheckpoints = registeredNumberGauges.get(CheckpointStatsTracker.NUMBER_OF_CHECKPOINTS_METRIC); + NumberGauge numInProgressCheckpoints = registeredNumberGauges.get(CheckpointStatsTracker.NUMBER_OF_IN_PROGRESS_CHECKPOINTS_METRIC); + NumberGauge numCompletedCheckpoints = registeredNumberGauges.get(CheckpointStatsTracker.NUMBER_OF_COMPLETED_CHECKPOINTS_METRIC); + NumberGauge numFailedCheckpoints = registeredNumberGauges.get(CheckpointStatsTracker.NUMBER_OF_FAILED_CHECKPOINTS_METRIC); + NumberGauge latestRestoreTimestamp = registeredNumberGauges.get(CheckpointStatsTracker.LATEST_RESTORED_CHECKPOINT_TIMESTAMP_METRIC); + NumberGauge latestCompletedSize = registeredNumberGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_SIZE_METRIC); + NumberGauge latestCompletedDuration = registeredNumberGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_DURATION_METRIC); + NumberGauge latestCompletedAlignmentBuffered = registeredNumberGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_ALIGNMENT_BUFFERED_METRIC); + StringGauge latestCompletedExternalPath = registeredStringGauges.get(CheckpointStatsTracker.LATEST_COMPLETED_CHECKPOINT_EXTERNAL_PATH_METRIC); + + assertEquals(0L, numCheckpoints.getNumberValue().longValue()); + assertEquals(0, numInProgressCheckpoints.getNumberValue().longValue()); + assertEquals(0L, numCompletedCheckpoints.getNumberValue().longValue()); + assertEquals(0L, numFailedCheckpoints.getNumberValue().longValue()); + assertEquals(-1L, latestRestoreTimestamp.getNumberValue().longValue()); + assertEquals(-1L, latestCompletedSize.getNumberValue().longValue()); + assertEquals(-1L, latestCompletedDuration.getNumberValue().longValue()); + assertEquals(-1L, latestCompletedAlignmentBuffered.getNumberValue().longValue()); + assertEquals("n/a", latestCompletedExternalPath.getStringValue()); PendingCheckpointStats pending = stats.reportPendingCheckpoint( 0, @@ -376,10 +385,10 @@ public > G gauge(String name, G gauge) { CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION)); // Check counts - assertEquals(Long.valueOf(1), numCheckpoints.getValue()); - assertEquals(Integer.valueOf(1), numInProgressCheckpoints.getValue()); - assertEquals(Long.valueOf(0), numCompletedCheckpoints.getValue()); - assertEquals(Long.valueOf(0), numFailedCheckpoints.getValue()); + assertEquals(1L, numCheckpoints.getNumberValue().longValue()); + assertEquals(1L, numInProgressCheckpoints.getNumberValue().longValue()); + assertEquals(0L, numCompletedCheckpoints.getNumberValue().longValue()); + assertEquals(0L, numFailedCheckpoints.getNumberValue().longValue()); long ackTimestamp = 11231230L; long stateSize = 12381238L; @@ -401,15 +410,15 @@ public > G gauge(String name, G gauge) { pending.reportCompletedCheckpoint(externalPath); // Verify completed checkpoint updated - assertEquals(Long.valueOf(1), numCheckpoints.getValue()); - assertEquals(Integer.valueOf(0), numInProgressCheckpoints.getValue()); - assertEquals(Long.valueOf(1), numCompletedCheckpoints.getValue()); - assertEquals(Long.valueOf(0), numFailedCheckpoints.getValue()); - assertEquals(Long.valueOf(-1), latestRestoreTimestamp.getValue()); - assertEquals(Long.valueOf(stateSize), latestCompletedSize.getValue()); - assertEquals(Long.valueOf(ackTimestamp), latestCompletedDuration.getValue()); - assertEquals(Long.valueOf(alignmenetBuffered), latestCompletedAlignmentBuffered.getValue()); - assertEquals(externalPath, latestCompletedExternalPath.getValue()); + assertEquals(1L, numCheckpoints.getNumberValue()); + assertEquals(0, numInProgressCheckpoints.getNumberValue()); + assertEquals(1L, numCompletedCheckpoints.getNumberValue()); + assertEquals(0L, numFailedCheckpoints.getNumberValue()); + assertEquals((long) -1, latestRestoreTimestamp.getNumberValue()); + assertEquals(stateSize, latestCompletedSize.getNumberValue()); + assertEquals(ackTimestamp, latestCompletedDuration.getNumberValue()); + assertEquals(alignmenetBuffered, latestCompletedAlignmentBuffered.getNumberValue()); + assertEquals(externalPath, latestCompletedExternalPath.getStringValue()); // Check failed PendingCheckpointStats nextPending = stats.reportPendingCheckpoint( @@ -421,10 +430,10 @@ public > G gauge(String name, G gauge) { nextPending.reportFailedCheckpoint(failureTimestamp, null); // Verify updated - assertEquals(Long.valueOf(2), numCheckpoints.getValue()); - assertEquals(Integer.valueOf(0), numInProgressCheckpoints.getValue()); - assertEquals(Long.valueOf(1), numCompletedCheckpoints.getValue()); - assertEquals(Long.valueOf(1), numFailedCheckpoints.getValue()); // one failed now + assertEquals(2L, numCheckpoints.getNumberValue()); + assertEquals(0, numInProgressCheckpoints.getNumberValue()); + assertEquals(1L, numCompletedCheckpoints.getNumberValue()); + assertEquals(1L, numFailedCheckpoints.getNumberValue()); // one failed now // Check restore long restoreTimestamp = 183419283L; @@ -435,12 +444,12 @@ public > G gauge(String name, G gauge) { null); stats.reportRestoredCheckpoint(restored); - assertEquals(Long.valueOf(2), numCheckpoints.getValue()); - assertEquals(Integer.valueOf(0), numInProgressCheckpoints.getValue()); - assertEquals(Long.valueOf(1), numCompletedCheckpoints.getValue()); - assertEquals(Long.valueOf(1), numFailedCheckpoints.getValue()); + assertEquals(2L, numCheckpoints.getNumberValue()); + assertEquals(0, numInProgressCheckpoints.getNumberValue()); + assertEquals(1L, numCompletedCheckpoints.getNumberValue()); + assertEquals(1L, numFailedCheckpoints.getNumberValue()); - assertEquals(Long.valueOf(restoreTimestamp), latestRestoreTimestamp.getValue()); + assertEquals(restoreTimestamp, latestRestoreTimestamp.getNumberValue()); // Check Internal Checkpoint Configuration PendingCheckpointStats thirdPending = stats.reportPendingCheckpoint( @@ -452,7 +461,7 @@ public > G gauge(String name, G gauge) { thirdPending.reportCompletedCheckpoint(null); // Verify external path is "n/a", because internal checkpoint won't generate external path. - assertEquals("n/a", latestCompletedExternalPath.getValue()); + assertEquals("n/a", latestCompletedExternalPath.getStringValue()); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java index caf89e8b23401..b08c9aa4ee626 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java @@ -97,13 +97,13 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti RestartTimeGauge restartingTime = new RestartTimeGauge(executionGraph); // check that the restarting time is 0 since it's the initial start - assertEquals(0L, restartingTime.getValue().longValue()); + assertEquals(0L, restartingTime.getNumberValue().longValue()); executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); // start execution executionGraph.scheduleForExecution(); - assertEquals(0L, restartingTime.getValue().longValue()); + assertEquals(0L, restartingTime.getNumberValue().longValue()); List executionIDs = new ArrayList<>(); @@ -117,7 +117,7 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti } assertEquals(JobStatus.RUNNING, executionGraph.getState()); - assertEquals(0L, restartingTime.getValue().longValue()); + assertEquals(0L, restartingTime.getNumberValue().longValue()); // fail the job so that it goes into state restarting for (ExecutionAttemptID executionID : executionIDs) { @@ -131,11 +131,11 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti // wait some time so that the restarting time gauge shows a value different from 0 Thread.sleep(50); - long previousRestartingTime = restartingTime.getValue(); + long previousRestartingTime = restartingTime.getNumberValue(); // check that the restarting time is monotonically increasing for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); + long currentRestartingTime = restartingTime.getNumberValue(); assertTrue(currentRestartingTime >= previousRestartingTime); previousRestartingTime = currentRestartingTime; @@ -161,11 +161,11 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti assertTrue(firstRestartingTimestamp != 0); - previousRestartingTime = restartingTime.getValue(); + previousRestartingTime = restartingTime.getNumberValue(); // check that the restarting time does not increase after we've reached the running state for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); + long currentRestartingTime = restartingTime.getNumberValue(); assertTrue(currentRestartingTime == previousRestartingTime); previousRestartingTime = currentRestartingTime; @@ -184,11 +184,11 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti Thread.sleep(50); - previousRestartingTime = restartingTime.getValue(); + previousRestartingTime = restartingTime.getNumberValue(); // check that the restarting time is increasing again for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); + long currentRestartingTime = restartingTime.getNumberValue(); assertTrue(currentRestartingTime >= previousRestartingTime); previousRestartingTime = currentRestartingTime; @@ -202,10 +202,10 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti assertEquals(JobStatus.FAILED, executionGraph.getState()); - previousRestartingTime = restartingTime.getValue(); + previousRestartingTime = restartingTime.getNumberValue(); for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); + long currentRestartingTime = restartingTime.getNumberValue(); assertTrue(currentRestartingTime == previousRestartingTime); previousRestartingTime = currentRestartingTime; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerializerTest.java index 5f83e794ff986..ef41d2e1a402e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerializerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerializerTest.java @@ -20,10 +20,11 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.metrics.util.TestHistogram; import org.junit.Assert; @@ -54,20 +55,19 @@ public void testNullGaugeHandling() throws IOException { MetricDumpSerialization.MetricDumpSerializer serializer = new MetricDumpSerialization.MetricDumpSerializer(); MetricDumpSerialization.MetricDumpDeserializer deserializer = new MetricDumpSerialization.MetricDumpDeserializer(); - Map, Tuple2> gauges = new HashMap<>(); + Map> numberGauges = new HashMap<>(); + Map> stringGauges = new HashMap<>(); - gauges.put(new Gauge() { - @Override - public Object getValue() { - return null; - } - }, new Tuple2(new QueryScopeInfo.JobManagerQueryScopeInfo("A"), "g")); + numberGauges.put(() -> null, new Tuple2<>(new QueryScopeInfo.JobManagerQueryScopeInfo("A"), "ng")); + + stringGauges.put(() -> null, new Tuple2<>(new QueryScopeInfo.JobManagerQueryScopeInfo("A"), "sg")); MetricDumpSerialization.MetricSerializationResult output = serializer.serialize( - Collections.>emptyMap(), - gauges, - Collections.>emptyMap(), - Collections.>emptyMap()); + Collections.emptyMap(), + numberGauges, + stringGauges, + Collections.emptyMap(), + Collections.emptyMap()); // no metrics should be serialized Assert.assertEquals(0, output.serializedMetrics.length); @@ -84,10 +84,11 @@ public void testJavaSerialization() throws IOException { final ObjectOutputStream oos = new ObjectOutputStream(bos); oos.writeObject(serializer.serialize( - new HashMap>(), - new HashMap, Tuple2>(), - new HashMap>(), - new HashMap>())); + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + new HashMap<>())); } @Test @@ -96,7 +97,8 @@ public void testSerialization() throws IOException { MetricDumpSerialization.MetricDumpDeserializer deserializer = new MetricDumpSerialization.MetricDumpDeserializer(); Map> counters = new HashMap<>(); - Map, Tuple2> gauges = new HashMap<>(); + Map> numberGgauges = new HashMap<>(); + Map> stringGauges = new HashMap<>(); Map> histograms = new HashMap<>(); Map> meters = new HashMap<>(); @@ -106,12 +108,8 @@ public void testSerialization() throws IOException { c1.inc(1); c2.inc(2); - Gauge g1 = new Gauge() { - @Override - public Integer getValue() { - return 4; - } - }; + NumberGauge ng1 = () -> 4; + StringGauge sg1 = () -> "hello"; Histogram h1 = new TestHistogram(); @@ -135,17 +133,18 @@ public long getCount() { } }; - counters.put(c1, new Tuple2(new QueryScopeInfo.JobManagerQueryScopeInfo("A"), "c1")); - counters.put(c2, new Tuple2(new QueryScopeInfo.TaskManagerQueryScopeInfo("tmid", "B"), "c2")); - meters.put(m1, new Tuple2(new QueryScopeInfo.JobQueryScopeInfo("jid", "C"), "c3")); - gauges.put(g1, new Tuple2(new QueryScopeInfo.TaskQueryScopeInfo("jid", "vid", 2, "D"), "g1")); - histograms.put(h1, new Tuple2(new QueryScopeInfo.OperatorQueryScopeInfo("jid", "vid", 2, "opname", "E"), "h1")); + counters.put(c1, new Tuple2<>(new QueryScopeInfo.JobManagerQueryScopeInfo("A"), "c1")); + counters.put(c2, new Tuple2<>(new QueryScopeInfo.TaskManagerQueryScopeInfo("tmid", "B"), "c2")); + meters.put(m1, new Tuple2<>(new QueryScopeInfo.JobQueryScopeInfo("jid", "C"), "c3")); + numberGgauges.put(ng1, new Tuple2<>(new QueryScopeInfo.TaskQueryScopeInfo("jid", "vid", 2, "D"), "ng1")); + stringGauges.put(sg1, new Tuple2<>(new QueryScopeInfo.TaskQueryScopeInfo("jid", "vid", 2, "D"), "sg1")); + histograms.put(h1, new Tuple2<>(new QueryScopeInfo.OperatorQueryScopeInfo("jid", "vid", 2, "opname", "E"), "h1")); - MetricDumpSerialization.MetricSerializationResult serialized = serializer.serialize(counters, gauges, histograms, meters); + MetricDumpSerialization.MetricSerializationResult serialized = serializer.serialize(counters, numberGgauges, stringGauges, histograms, meters); List deserialized = deserializer.deserialize(serialized); // ===== Counters ============================================================================================== - assertEquals(5, deserialized.size()); + assertEquals(6, deserialized.size()); for (MetricDump metric : deserialized) { switch (metric.getCategory()) { @@ -171,16 +170,36 @@ public long getCount() { break; case METRIC_CATEGORY_GAUGE: MetricDump.GaugeDump gaugeDump = (MetricDump.GaugeDump) metric; - assertEquals("4", gaugeDump.value); - assertEquals("g1", gaugeDump.name); - - assertTrue(gaugeDump.scopeInfo instanceof QueryScopeInfo.TaskQueryScopeInfo); - QueryScopeInfo.TaskQueryScopeInfo taskInfo = (QueryScopeInfo.TaskQueryScopeInfo) gaugeDump.scopeInfo; - assertEquals("D", taskInfo.scope); - assertEquals("jid", taskInfo.jobID); - assertEquals("vid", taskInfo.vertexID); - assertEquals(2, taskInfo.subtaskIndex); - gauges.remove(g1); + switch (gaugeDump.name) { + case "ng1": { + assertEquals("4", gaugeDump.value); + + assertTrue(gaugeDump.scopeInfo instanceof QueryScopeInfo.TaskQueryScopeInfo); + QueryScopeInfo.TaskQueryScopeInfo taskInfo = (QueryScopeInfo.TaskQueryScopeInfo) gaugeDump.scopeInfo; + assertEquals("D", taskInfo.scope); + assertEquals("jid", taskInfo.jobID); + assertEquals("vid", taskInfo.vertexID); + assertEquals(2, taskInfo.subtaskIndex); + numberGgauges.remove(ng1); + break; + } + case "sg1": { + assertEquals("hello", gaugeDump.value); + assertEquals("sg1", gaugeDump.name); + + assertTrue(gaugeDump.scopeInfo instanceof QueryScopeInfo.TaskQueryScopeInfo); + QueryScopeInfo.TaskQueryScopeInfo taskInfo = (QueryScopeInfo.TaskQueryScopeInfo) gaugeDump.scopeInfo; + assertEquals("D", taskInfo.scope); + assertEquals("jid", taskInfo.jobID); + assertEquals("vid", taskInfo.vertexID); + assertEquals(2, taskInfo.subtaskIndex); + stringGauges.remove(sg1); + break; + } + default: + fail(); + break; + } break; case METRIC_CATEGORY_HISTOGRAM: MetricDump.HistogramDump histogramDump = (MetricDump.HistogramDump) metric; @@ -220,7 +239,8 @@ public long getCount() { } } assertTrue(counters.isEmpty()); - assertTrue(gauges.isEmpty()); + assertTrue(numberGgauges.isEmpty()); + assertTrue(stringGauges.isEmpty()); assertTrue(histograms.isEmpty()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java index 1acaf6181bb4d..b5ded4b9c31ab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java @@ -20,10 +20,10 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.metrics.util.TestHistogram; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; @@ -54,12 +54,7 @@ public void testCreateDump() throws Exception { TestActor testActor = (TestActor) testActorRef.underlyingActor(); final Counter c = new SimpleCounter(); - final Gauge g = new Gauge() { - @Override - public String getValue() { - return "Hello"; - } - }; + final StringGauge g = () -> "Hello"; final Histogram h = new TestHistogram(); final Meter m = new Meter() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java index 597508faebd0e..d691ff315e5da 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java @@ -21,7 +21,6 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.HistogramStatistics; import org.apache.flink.metrics.Metric; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java index da8182a8edea5..258967c8db2cb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java @@ -22,10 +22,11 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Histogram; import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.metrics.StringGauge; import org.apache.flink.metrics.util.TestHistogram; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.Executors; @@ -130,7 +131,8 @@ public void testUpdate() throws Exception { assertEquals("0.99", store.getJobManagerMetricStore().getMetric("abc.hist_p99")); assertEquals("0.999", store.getJobManagerMetricStore().getMetric("abc.hist_p999")); - assertEquals("x", store.getTaskManagerMetricStore(tmRID.toString()).metrics.get("abc.gauge")); + assertEquals("4", store.getTaskManagerMetricStore(tmRID.toString()).metrics.get("abc.number")); + assertEquals("x", store.getTaskManagerMetricStore(tmRID.toString()).metrics.get("abc.string")); assertEquals("5.0", store.getJobMetricStore(jobID.toString()).metrics.get("abc.jc")); assertEquals("2", store.getTaskMetricStore(jobID.toString(), "taskid").metrics.get("2.abc.tc")); assertEquals("1", store.getTaskMetricStore(jobID.toString(), "taskid").metrics.get("2.opname.abc.oc")); @@ -139,7 +141,8 @@ public void testUpdate() throws Exception { private static MetricDumpSerialization.MetricSerializationResult createRequestDumpAnswer(ResourceID tmRID, JobID jobID) { Map> counters = new HashMap<>(); - Map, Tuple2> gauges = new HashMap<>(); + Map> numberGauges = new HashMap<>(); + Map> stringGauges = new HashMap<>(); Map> histograms = new HashMap<>(); Map> meters = new HashMap<>(); @@ -170,16 +173,12 @@ public long getCount() { return 10; } }, new Tuple2<>(new QueryScopeInfo.JobQueryScopeInfo(jobID.toString(), "abc"), "jc")); - gauges.put(new Gauge() { - @Override - public String getValue() { - return "x"; - } - }, new Tuple2<>(new QueryScopeInfo.TaskManagerQueryScopeInfo(tmRID.toString(), "abc"), "gauge")); + numberGauges.put(() -> 4, new Tuple2<>(new QueryScopeInfo.TaskManagerQueryScopeInfo(tmRID.toString(), "abc"), "number")); + stringGauges.put(() -> "x", new Tuple2<>(new QueryScopeInfo.TaskManagerQueryScopeInfo(tmRID.toString(), "abc"), "string")); histograms.put(new TestHistogram(), new Tuple2<>(new QueryScopeInfo.JobManagerQueryScopeInfo("abc"), "hist")); MetricDumpSerialization.MetricDumpSerializer serializer = new MetricDumpSerialization.MetricDumpSerializer(); - MetricDumpSerialization.MetricSerializationResult dump = serializer.serialize(counters, gauges, histograms, meters); + MetricDumpSerialization.MetricSerializationResult dump = serializer.serialize(counters, numberGauges, stringGauges, histograms, meters); serializer.close(); return dump; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutput.java index 6512174c3978e..01d739ab391c4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutput.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.api.collector.selector; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.streaming.api.graph.StreamEdge; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.watermark.Watermark; @@ -157,7 +157,7 @@ public void close() { } @Override - public Gauge getWatermarkGauge() { + public NumberGauge getWatermarkGauge() { return watermarkGauge; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java index 45bbd6618997a..272fe903765a2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -19,7 +19,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.plugable.SerializationDelegate; @@ -166,7 +166,7 @@ public void clearBuffers() { } @Override - public Gauge getWatermarkGauge() { + public NumberGauge getWatermarkGauge() { return watermarkGauge; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index dc3dc5c1ef776..02a725d2209ff 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -170,7 +170,7 @@ else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) { new ForwardingValveOutputHandler(streamOperator, lock)); this.watermarkGauge = watermarkGauge; - metrics.gauge("checkpointAlignmentTime", barrierHandler::getAlignmentDurationNanos); + metrics.register("checkpointAlignmentTime", barrierHandler::getAlignmentDurationNanos); } public boolean processInput() throws Exception { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 494a82af46c5f..d92218aee3ac0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -202,7 +202,7 @@ else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) { this.input1WatermarkGauge = input1WatermarkGauge; this.input2WatermarkGauge = input2WatermarkGauge; - metrics.gauge("checkpointAlignmentTime", barrierHandler::getAlignmentDurationNanos); + metrics.register("checkpointAlignmentTime", barrierHandler::getAlignmentDurationNanos); } public boolean processInput() throws Exception { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/MinWatermarkGauge.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/MinWatermarkGauge.java index 6736dca9ac358..bbd6b504a6870 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/MinWatermarkGauge.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/MinWatermarkGauge.java @@ -18,12 +18,12 @@ package org.apache.flink.streaming.runtime.metrics; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; /** - * A {@link Gauge} for exposing the minimum watermark of a {@link WatermarkGauge} pair. + * A {@link NumberGauge} for exposing the minimum watermark of a {@link WatermarkGauge} pair. */ -public class MinWatermarkGauge implements Gauge { +public class MinWatermarkGauge implements NumberGauge { private WatermarkGauge watermarkGauge1; private WatermarkGauge watermarkGauge2; @@ -34,7 +34,7 @@ public MinWatermarkGauge(WatermarkGauge watermarkGauge1, WatermarkGauge watermar } @Override - public Long getValue() { - return Math.min(watermarkGauge1.getValue(), watermarkGauge2.getValue()); + public Long getNumberValue() { + return Math.min(watermarkGauge1.getNumberValue(), watermarkGauge2.getNumberValue()); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/WatermarkGauge.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/WatermarkGauge.java index 42c70008bdb6b..a45b5896cdff3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/WatermarkGauge.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/metrics/WatermarkGauge.java @@ -18,12 +18,12 @@ package org.apache.flink.streaming.runtime.metrics; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; /** - * A {@link Gauge} for exposing the current input/output watermark. + * A {@link NumberGauge} for exposing the current input/output watermark. */ -public class WatermarkGauge implements Gauge { +public class WatermarkGauge implements NumberGauge { private long currentWatermark = Long.MIN_VALUE; @@ -32,7 +32,7 @@ public void setCurrentWatermark(long watermark) { } @Override - public Long getValue() { + public Long getNumberValue() { return currentWatermark; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index 26088e43ad622..7dc782fe8fbbe 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -92,7 +92,7 @@ public void init() throws Exception { getEnvironment().getMetricGroup().getIOMetricGroup(), inputWatermarkGauge); } - headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, this.inputWatermarkGauge); + headOperator.getMetricGroup().register(MetricNames.IO_CURRENT_INPUT_WATERMARK, this.inputWatermarkGauge); } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index fdeea17c85e02..cef6d282ee780 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.execution.Environment; @@ -140,7 +140,7 @@ public OperatorChain(StreamTask containingTask) { WatermarkGaugeExposingOutput> output = getChainEntryPoint(); headOperator.setup(containingTask, configuration, output); - headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_OUTPUT_WATERMARK, output.getWatermarkGauge()); + headOperator.getMetricGroup().register(MetricNames.IO_CURRENT_OUTPUT_WATERMARK, output.getWatermarkGauge()); } // add head operator to end of chain @@ -374,8 +374,8 @@ private WatermarkGaugeExposingOutput> createChainedOp currentOperatorOutput = new CopyingChainingOutput<>(chainedOperator, inSerializer, outputTag, this); } - chainedOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, currentOperatorOutput.getWatermarkGauge()); - chainedOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_OUTPUT_WATERMARK, chainedOperatorOutput.getWatermarkGauge()); + chainedOperator.getMetricGroup().register(MetricNames.IO_CURRENT_INPUT_WATERMARK, currentOperatorOutput.getWatermarkGauge()); + chainedOperator.getMetricGroup().register(MetricNames.IO_CURRENT_OUTPUT_WATERMARK, chainedOperatorOutput.getWatermarkGauge()); return currentOperatorOutput; } @@ -431,7 +431,7 @@ private RecordWriterOutput createStreamOutput( * @param The type of the elements that can be emitted. */ public interface WatermarkGaugeExposingOutput extends Output { - Gauge getWatermarkGauge(); + NumberGauge getWatermarkGauge(); } private static class ChainingOutput implements WatermarkGaugeExposingOutput> { @@ -539,7 +539,7 @@ public void close() { } @Override - public Gauge getWatermarkGauge() { + public NumberGauge getWatermarkGauge() { return watermarkGauge; } } @@ -649,7 +649,7 @@ public void emitLatencyMarker(LatencyMarker latencyMarker) { } @Override - public Gauge getWatermarkGauge() { + public NumberGauge getWatermarkGauge() { return watermarkGauge; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index bd878f65950ee..9ae7ced1b6736 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -102,9 +102,9 @@ public void init() throws Exception { input1WatermarkGauge, input2WatermarkGauge); - headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, minInputWatermarkGauge); - headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_1_WATERMARK, input1WatermarkGauge); - headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_2_WATERMARK, input2WatermarkGauge); + headOperator.getMetricGroup().register(MetricNames.IO_CURRENT_INPUT_WATERMARK, minInputWatermarkGauge); + headOperator.getMetricGroup().register(MetricNames.IO_CURRENT_INPUT_1_WATERMARK, input1WatermarkGauge); + headOperator.getMetricGroup().register(MetricNames.IO_CURRENT_INPUT_2_WATERMARK, input2WatermarkGauge); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/metrics/MinWatermarkGaugeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/metrics/MinWatermarkGaugeTest.java index ae673a824e490..64cab4c083e5c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/metrics/MinWatermarkGaugeTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/metrics/MinWatermarkGaugeTest.java @@ -32,15 +32,15 @@ public void testSetCurrentLowWatermark() { WatermarkGauge metric2 = new WatermarkGauge(); MinWatermarkGauge metric = new MinWatermarkGauge(metric1, metric2); - Assert.assertEquals(Long.MIN_VALUE, metric.getValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, metric.getNumberValue().longValue()); metric1.setCurrentWatermark(1); - Assert.assertEquals(Long.MIN_VALUE, metric.getValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, metric.getNumberValue().longValue()); metric2.setCurrentWatermark(2); - Assert.assertEquals(1L, metric.getValue().longValue()); + Assert.assertEquals(1L, metric.getNumberValue().longValue()); metric1.setCurrentWatermark(3); - Assert.assertEquals(2L, metric.getValue().longValue()); + Assert.assertEquals(2L, metric.getNumberValue().longValue()); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/metrics/WatermarkGaugeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/metrics/WatermarkGaugeTest.java index c8cec887ce804..a9f5f86dfd85d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/metrics/WatermarkGaugeTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/metrics/WatermarkGaugeTest.java @@ -30,9 +30,9 @@ public class WatermarkGaugeTest { public void testSetCurrentLowWatermark() { WatermarkGauge metric = new WatermarkGauge(); - Assert.assertEquals(Long.MIN_VALUE, metric.getValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, metric.getNumberValue().longValue()); metric.setCurrentWatermark(64); - Assert.assertEquals(64, metric.getValue().longValue()); + Assert.assertEquals(64, metric.getNumberValue().longValue()); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index 3e0459d205cf4..5fc31e409b145 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -29,7 +29,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; @@ -651,29 +651,29 @@ public TaskMetricGroup getMetricGroup() { testHarness.invoke(env); testHarness.waitForTaskRunning(); - Gauge headInputWatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); - Gauge headOutputWatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); - Gauge chainedInputWatermarkGauge = (Gauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); - Gauge chainedOutputWatermarkGauge = (Gauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); + NumberGauge headInputWatermarkGauge = (NumberGauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); + NumberGauge headOutputWatermarkGauge = (NumberGauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); + NumberGauge chainedInputWatermarkGauge = (NumberGauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); + NumberGauge chainedOutputWatermarkGauge = (NumberGauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); - Assert.assertEquals(Long.MIN_VALUE, headInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, headOutputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, chainedInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, chainedOutputWatermarkGauge.getValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, headInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, headOutputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, chainedInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, chainedOutputWatermarkGauge.getNumberValue().longValue()); testHarness.processElement(new Watermark(1L)); testHarness.waitForInputProcessing(); - Assert.assertEquals(1L, headInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(2L, headOutputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(2L, chainedInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(4L, chainedOutputWatermarkGauge.getValue().longValue()); + Assert.assertEquals(1L, headInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(2L, headOutputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(2L, chainedInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(4L, chainedOutputWatermarkGauge.getNumberValue().longValue()); testHarness.processElement(new Watermark(2L)); testHarness.waitForInputProcessing(); - Assert.assertEquals(2L, headInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(4L, headOutputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(4L, chainedInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(8L, chainedOutputWatermarkGauge.getValue().longValue()); + Assert.assertEquals(2L, headInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(4L, headOutputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(4L, chainedInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(8L, chainedOutputWatermarkGauge.getNumberValue().longValue()); testHarness.endInput(); testHarness.waitForTaskCompletion(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java index 58e28b35e6519..a6972a8fa8996 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.NumberGauge; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; @@ -422,46 +422,46 @@ public TaskMetricGroup getMetricGroup() { testHarness.invoke(env); testHarness.waitForTaskRunning(); - Gauge headInput1WatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_1_WATERMARK); - Gauge headInput2WatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_2_WATERMARK); - Gauge headInputWatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); - Gauge headOutputWatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); - Gauge chainedInputWatermarkGauge = (Gauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); - Gauge chainedOutputWatermarkGauge = (Gauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); + NumberGauge headInput1WatermarkGauge = (NumberGauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_1_WATERMARK); + NumberGauge headInput2WatermarkGauge = (NumberGauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_2_WATERMARK); + NumberGauge headInputWatermarkGauge = (NumberGauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); + NumberGauge headOutputWatermarkGauge = (NumberGauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); + NumberGauge chainedInputWatermarkGauge = (NumberGauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); + NumberGauge chainedOutputWatermarkGauge = (NumberGauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); - Assert.assertEquals(Long.MIN_VALUE, headInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, headInput1WatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, headInput2WatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, headOutputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, chainedInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, chainedOutputWatermarkGauge.getValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, headInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, headInput1WatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, headInput2WatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, headOutputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, chainedInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, chainedOutputWatermarkGauge.getNumberValue().longValue()); testHarness.processElement(new Watermark(1L), 0, 0); testHarness.waitForInputProcessing(); - Assert.assertEquals(Long.MIN_VALUE, headInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(1L, headInput1WatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, headInput2WatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, headOutputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, chainedInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(Long.MIN_VALUE, chainedOutputWatermarkGauge.getValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, headInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(1L, headInput1WatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, headInput2WatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, headOutputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, chainedInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(Long.MIN_VALUE, chainedOutputWatermarkGauge.getNumberValue().longValue()); testHarness.processElement(new Watermark(2L), 1, 0); testHarness.waitForInputProcessing(); - Assert.assertEquals(1L, headInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(1L, headInput1WatermarkGauge.getValue().longValue()); - Assert.assertEquals(2L, headInput2WatermarkGauge.getValue().longValue()); - Assert.assertEquals(1L, headOutputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(1L, chainedInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(2L, chainedOutputWatermarkGauge.getValue().longValue()); + Assert.assertEquals(1L, headInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(1L, headInput1WatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(2L, headInput2WatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(1L, headOutputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(1L, chainedInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(2L, chainedOutputWatermarkGauge.getNumberValue().longValue()); testHarness.processElement(new Watermark(3L), 0, 0); testHarness.waitForInputProcessing(); - Assert.assertEquals(2L, headInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(3L, headInput1WatermarkGauge.getValue().longValue()); - Assert.assertEquals(2L, headInput2WatermarkGauge.getValue().longValue()); - Assert.assertEquals(2L, headOutputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(2L, chainedInputWatermarkGauge.getValue().longValue()); - Assert.assertEquals(4L, chainedOutputWatermarkGauge.getValue().longValue()); + Assert.assertEquals(2L, headInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(3L, headInput1WatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(2L, headInput2WatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(2L, headOutputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(2L, chainedInputWatermarkGauge.getNumberValue().longValue()); + Assert.assertEquals(4L, chainedOutputWatermarkGauge.getNumberValue().longValue()); testHarness.endInput(); testHarness.waitForTaskCompletion();