This is an automated email from the ASF dual-hosted git repository.

chesnay pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e6408403f39d2c69acab2c9f92da95aee3dbab63
Author: Chesnay Schepler <[email protected]>
AuthorDate: Wed Mar 23 15:54:56 2022 +0100

    [FLINK-26851][metrics] Migrate reporter options to proper ConfigOptions
---
 .../content.zh/docs/deployment/metric_reporters.md | 12 ++---
 docs/content/docs/deployment/metric_reporters.md   | 12 ++---
 .../shortcodes/generated/metric_configuration.html | 24 ++++++++++
 .../generated/metric_reporters_section.html        | 54 ++++++++++++++++++++++
 .../flink/annotation/docs/Documentation.java       |  2 +
 .../streaming/connectors/kafka/KafkaTestBase.java  |  3 +-
 .../flink/configuration/ConfigConstants.java       | 28 +++++------
 .../apache/flink/configuration/MetricOptions.java  | 53 +++++++++++++++++++--
 .../tests/PrometheusReporterEndToEndITCase.java    |  5 +-
 .../jobmanager/JMXJobManagerMetricTest.java        |  2 +-
 .../flink/runtime/metrics/ReporterSetup.java       | 31 +++++--------
 .../runtime/metrics/MetricRegistryImplTest.java    | 36 +++++++--------
 .../flink/runtime/metrics/ReporterSetupTest.java   | 40 ++++++++--------
 .../metrics/groups/AbstractMetricGroupTest.java    | 12 ++---
 .../runtime/metrics/groups/MetricGroupTest.java    |  3 +-
 .../flink/runtime/testutils/InMemoryReporter.java  |  3 +-
 16 files changed, 213 insertions(+), 107 deletions(-)

diff --git a/docs/content.zh/docs/deployment/metric_reporters.md 
b/docs/content.zh/docs/deployment/metric_reporters.md
index d966662..7c432e5 100644
--- a/docs/content.zh/docs/deployment/metric_reporters.md
+++ b/docs/content.zh/docs/deployment/metric_reporters.md
@@ -35,18 +35,12 @@ For more information about Flink's metric system go to the 
[metric system docume
 Metrics can be exposed to an external system by configuring one or several 
reporters in `conf/flink-conf.yaml`. These
 reporters will be instantiated on each job and task manager when they are 
started.
 
-- `metrics.reporter.<name>.<config>`: Generic setting `<config>` for the 
reporter named `<name>`.
-- `metrics.reporter.<name>.class`: The reporter class to use for the reporter 
named `<name>`.
-- `metrics.reporter.<name>.factory.class`: The reporter factory class to use 
for the reporter named `<name>`.
-- `metrics.reporter.<name>.interval`: The reporter interval to use for the 
reporter named `<name>`.
-- `metrics.reporter.<name>.scope.delimiter`: The delimiter to use for the 
identifier (default value use `metrics.scope.delimiter`) for the reporter named 
`<name>`.
-- `metrics.reporter.<name>.scope.variables.excludes`: (optional) A semi-colon 
(;) separate list of variables that should be ignored by tag-based reporters 
(e.g., Prometheus, InfluxDB). 
-- `metrics.reporters`: (optional) A comma-separated include list of reporter 
names. By default all configured reporters will be used.
-- `metrics.reporter.<name>.scope.variables.additional`: (optional) A comma 
separated map of variables and their values, which are separated by a colon 
(:). These mappings are added to the variable map by tag-based reporters (e.g. 
Prometheux, InfluxDB).
+Below is a list of parameters that are generally applicable to all reporters. 
All properties are configured by setting 
`metrics.reporter.<reporter_name>.<property>` in the configuration. Reporters 
may additionally offer implementation-specific parameters, which are documented 
in the respective reporter's section. 
+
+{{< include_reporter_config 
"layouts/shortcodes/generated/metric_reporters_section.html" >}}
 
 All reporters must at least have either the `class` or `factory.class` 
property. Which property may/should be used depends on the reporter 
implementation. See the individual reporter configuration sections for more 
information.
 Some reporters (referred to as `Scheduled`) allow specifying a reporting 
`interval`.
-Below more settings specific to each reporter will be listed.
 
 Example reporter configuration that specifies multiple reporters:
 
diff --git a/docs/content/docs/deployment/metric_reporters.md 
b/docs/content/docs/deployment/metric_reporters.md
index a78acef..c327b76 100644
--- a/docs/content/docs/deployment/metric_reporters.md
+++ b/docs/content/docs/deployment/metric_reporters.md
@@ -35,18 +35,12 @@ For more information about Flink's metric system go to the 
[metric system docume
 Metrics can be exposed to an external system by configuring one or several 
reporters in `conf/flink-conf.yaml`. These
 reporters will be instantiated on each job and task manager when they are 
started.
 
-- `metrics.reporter.<name>.<config>`: Generic setting `<config>` for the 
reporter named `<name>`.
-- `metrics.reporter.<name>.class`: The reporter class to use for the reporter 
named `<name>`.
-- `metrics.reporter.<name>.factory.class`: The reporter factory class to use 
for the reporter named `<name>`.
-- `metrics.reporter.<name>.interval`: The reporter interval to use for the 
reporter named `<name>`.
-- `metrics.reporter.<name>.scope.delimiter`: The delimiter to use for the 
identifier (default value use `metrics.scope.delimiter`) for the reporter named 
`<name>`.
-- `metrics.reporter.<name>.scope.variables.excludes`: (optional) A semi-colon 
(;) separate list of variables that should be ignored by tag-based reporters 
(e.g., Prometheus, InfluxDB). 
-- `metrics.reporter.<name>.scope.variables.additional`: (optional) A comma 
separated map of variables and their values, which are separated by a colon 
(:). These mappings are added to the variable map by tag-based reporters (e.g. 
Prometheux, InfluxDB). 
-- `metrics.reporters`: (optional) A comma-separated include list of reporter 
names. By default all configured reporters will be used.
+Below is a list of parameters that are generally applicable to all reporters. 
All properties are configured by setting 
`metrics.reporter.<reporter_name>.<property>` in the configuration. Reporters 
may additionally offer implementation-specific parameters, which are documented 
in the respective reporter's section. 
+
+{{< include_reporter_config 
"layouts/shortcodes/generated/metric_reporters_section.html" >}}
 
 All reporters must at least have either the `class` or `factory.class` 
property. Which property may/should be used depends on the reporter 
implementation. See the individual reporter configuration sections for more 
information.
 Some reporters (referred to as `Scheduled`) allow specifying a reporting 
`interval`.
-Below more settings specific to each reporter will be listed.
 
 Example reporter configuration that specifies multiple reporters:
 
diff --git a/docs/layouts/shortcodes/generated/metric_configuration.html 
b/docs/layouts/shortcodes/generated/metric_configuration.html
index c0fbc86..4fabd04 100644
--- a/docs/layouts/shortcodes/generated/metric_configuration.html
+++ b/docs/layouts/shortcodes/generated/metric_configuration.html
@@ -63,12 +63,36 @@
             <td>The reporter class to use for the reporter named 
&lt;name&gt;.</td>
         </tr>
         <tr>
+            <td><h5>metrics.reporter.&lt;name&gt;.factory.class</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>String</td>
+            <td>The reporter factory class to use for the reporter named 
&lt;name&gt;.</td>
+        </tr>
+        <tr>
             <td><h5>metrics.reporter.&lt;name&gt;.interval</h5></td>
             <td style="word-wrap: break-word;">10 s</td>
             <td>Duration</td>
             <td>The reporter interval to use for the reporter named 
&lt;name&gt;.</td>
         </tr>
         <tr>
+            <td><h5>metrics.reporter.&lt;name&gt;.scope.delimiter</h5></td>
+            <td style="word-wrap: break-word;">"."</td>
+            <td>String</td>
+            <td>The delimiter used to assemble the metric identifier for the 
reporter named &lt;name&gt;.</td>
+        </tr>
+        <tr>
+            
<td><h5>metrics.reporter.&lt;name&gt;.scope.variables.additional</h5></td>
+            <td style="word-wrap: break-word;"></td>
+            <td>Map</td>
+            <td>The map of additional variables that should be included for 
the reporter named &lt;name&gt;. Only applicable to tag-based reporters (e.g., 
PRometheus, InfluxDB).</td>
+        </tr>
+        <tr>
+            
<td><h5>metrics.reporter.&lt;name&gt;.scope.variables.excludes</h5></td>
+            <td style="word-wrap: break-word;">"."</td>
+            <td>String</td>
+            <td>The set of variables that should be excluded for the reporter 
named &lt;name&gt;. Only applicable to tag-based reporters (e.g., PRometheus, 
InfluxDB).</td>
+        </tr>
+        <tr>
             <td><h5>metrics.reporters</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
diff --git a/docs/layouts/shortcodes/generated/metric_reporters_section.html 
b/docs/layouts/shortcodes/generated/metric_reporters_section.html
new file mode 100644
index 0000000..bc4068c
--- /dev/null
+++ b/docs/layouts/shortcodes/generated/metric_reporters_section.html
@@ -0,0 +1,54 @@
+<table class="configuration table table-bordered">
+    <thead>
+        <tr>
+            <th class="text-left" style="width: 20%">Key</th>
+            <th class="text-left" style="width: 15%">Default</th>
+            <th class="text-left" style="width: 10%">Type</th>
+            <th class="text-left" style="width: 55%">Description</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td><h5>metrics.reporter.&lt;name&gt;.class</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>String</td>
+            <td>The reporter class to use for the reporter named 
&lt;name&gt;.</td>
+        </tr>
+        <tr>
+            <td><h5>metrics.reporter.&lt;name&gt;.factory.class</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>String</td>
+            <td>The reporter factory class to use for the reporter named 
&lt;name&gt;.</td>
+        </tr>
+        <tr>
+            <td><h5>metrics.reporter.&lt;name&gt;.interval</h5></td>
+            <td style="word-wrap: break-word;">10 s</td>
+            <td>Duration</td>
+            <td>The reporter interval to use for the reporter named 
&lt;name&gt;.</td>
+        </tr>
+        <tr>
+            <td><h5>metrics.reporter.&lt;name&gt;.scope.delimiter</h5></td>
+            <td style="word-wrap: break-word;">"."</td>
+            <td>String</td>
+            <td>The delimiter used to assemble the metric identifier for the 
reporter named &lt;name&gt;.</td>
+        </tr>
+        <tr>
+            
<td><h5>metrics.reporter.&lt;name&gt;.scope.variables.additional</h5></td>
+            <td style="word-wrap: break-word;"></td>
+            <td>Map</td>
+            <td>The map of additional variables that should be included for 
the reporter named &lt;name&gt;. Only applicable to tag-based reporters (e.g., 
PRometheus, InfluxDB).</td>
+        </tr>
+        <tr>
+            
<td><h5>metrics.reporter.&lt;name&gt;.scope.variables.excludes</h5></td>
+            <td style="word-wrap: break-word;">"."</td>
+            <td>String</td>
+            <td>The set of variables that should be excluded for the reporter 
named &lt;name&gt;. Only applicable to tag-based reporters (e.g., PRometheus, 
InfluxDB).</td>
+        </tr>
+        <tr>
+            <td><h5>metrics.reporter.&lt;name&gt;.&lt;parameter&gt;</h5></td>
+            <td style="word-wrap: break-word;">(none)</td>
+            <td>String</td>
+            <td>Configures the parameter &lt;parameter&gt; for the reporter 
named &lt;name&gt;.</td>
+        </tr>
+    </tbody>
+</table>
diff --git 
a/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java
 
b/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java
index e8b150f..7bdb196 100644
--- 
a/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java
+++ 
b/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java
@@ -103,6 +103,8 @@ public final class Documentation {
 
         public static final String DEPRECATED_FILE_SINKS = 
"deprecated_file_sinks";
 
+        public static final String METRIC_REPORTERS = "metric_reporters";
+
         private Sections() {}
     }
 
diff --git 
a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
 
b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
index fab7cd6..c4b0735 100644
--- 
a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
+++ 
b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
@@ -21,6 +21,7 @@ import 
org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.MetricOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.metrics.jmx.JMXReporter;
 import org.apache.flink.runtime.client.JobExecutionException;
@@ -133,7 +134,7 @@ public abstract class KafkaTestBase extends TestLogger {
         flinkConfig.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "my_reporter."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 JMXReporter.class.getName());
         return flinkConfig;
     }
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java 
b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 48b014e..29c80ca 100644
--- 
a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ 
b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -1113,28 +1113,24 @@ public final class ConfigConstants {
      */
     public static final String METRICS_REPORTER_PREFIX = "metrics.reporter.";
 
-    /** The class of the reporter to use. This is used as a suffix in an 
actual reporter config */
-    public static final String METRICS_REPORTER_CLASS_SUFFIX = "class";
+    /** @deprecated use {@link MetricOptions#REPORTER_CLASS} */
+    @Deprecated public static final String METRICS_REPORTER_CLASS_SUFFIX = 
"class";
 
-    /**
-     * The class of the reporter factory to use. This is used as a suffix in 
an actual reporter
-     * config
-     */
-    public static final String METRICS_REPORTER_FACTORY_CLASS_SUFFIX = 
"factory.class";
+    /** @deprecated use {@link MetricOptions#REPORTER_FACTORY_CLASS} */
+    @Deprecated public static final String 
METRICS_REPORTER_FACTORY_CLASS_SUFFIX = "factory.class";
 
-    /** The interval between reports. This is used as a suffix in an actual 
reporter config */
-    public static final String METRICS_REPORTER_INTERVAL_SUFFIX = "interval";
+    /** @deprecated use {@link MetricOptions#REPORTER_INTERVAL} */
+    @Deprecated public static final String METRICS_REPORTER_INTERVAL_SUFFIX = 
"interval";
 
-    /**
-     * The delimiter used to assemble the metric identifier. This is used as a 
suffix in an actual
-     * reporter config.
-     */
-    public static final String METRICS_REPORTER_SCOPE_DELIMITER = 
"scope.delimiter";
+    /** @deprecated use {@link MetricOptions#REPORTER_SCOPE_DELIMITER} */
+    @Deprecated public static final String METRICS_REPORTER_SCOPE_DELIMITER = 
"scope.delimiter";
 
-    /** The set of variables that should be excluded. */
+    /** @deprecated use {@link MetricOptions#REPORTER_EXCLUDED_VARIABLES} */
+    @Deprecated
     public static final String METRICS_REPORTER_EXCLUDED_VARIABLES = 
"scope.variables.excludes";
 
-    /** The map of additional variables that should be included. */
+    /** @deprecated use {@link MetricOptions#REPORTER_ADDITIONAL_VARIABLES} */
+    @Deprecated
     public static final String METRICS_REPORTER_ADDITIONAL_VARIABLES = 
"scope.variables.additional";
 
     /** @deprecated Use {@link MetricOptions#SCOPE_DELIMITER} instead. */
diff --git 
a/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java 
b/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java
index 70d4903..19e7bb9 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/MetricOptions.java
@@ -25,7 +25,9 @@ import 
org.apache.flink.configuration.description.InlineElement;
 import org.apache.flink.configuration.description.TextElement;
 
 import java.time.Duration;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 import static org.apache.flink.configuration.ConfigOptions.key;
 import static org.apache.flink.configuration.description.TextElement.text;
@@ -34,6 +36,9 @@ import static 
org.apache.flink.configuration.description.TextElement.text;
 @PublicEvolving
 public class MetricOptions {
 
+    private static final String NAMED_REPORTER_CONFIG_PREFIX =
+            ConfigConstants.METRICS_REPORTER_PREFIX + "<name>";
+
     /**
      * An optional list of reporter names. If configured, only reporters whose 
name matches any of
      * the names in the list will be started. Otherwise, all reporters that 
could be found in the
@@ -60,20 +65,62 @@ public class MetricOptions {
                                     + " any of the names in the list will be 
started. Otherwise, all reporters that could be found in"
                                     + " the configuration will be started.");
 
+    @Documentation.SuffixOption(NAMED_REPORTER_CONFIG_PREFIX)
+    @Documentation.Section(value = Documentation.Sections.METRIC_REPORTERS, 
position = 1)
     public static final ConfigOption<String> REPORTER_CLASS =
-            key("metrics.reporter.<name>.class")
+            key("class")
                     .stringType()
                     .noDefaultValue()
                     .withDescription("The reporter class to use for the 
reporter named <name>.");
 
+    @Documentation.SuffixOption(NAMED_REPORTER_CONFIG_PREFIX)
+    @Documentation.Section(value = Documentation.Sections.METRIC_REPORTERS, 
position = 1)
+    public static final ConfigOption<String> REPORTER_FACTORY_CLASS =
+            key("factory.class")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription(
+                            "The reporter factory class to use for the 
reporter named <name>.");
+
+    @Documentation.SuffixOption(NAMED_REPORTER_CONFIG_PREFIX)
+    @Documentation.Section(value = Documentation.Sections.METRIC_REPORTERS, 
position = 2)
     public static final ConfigOption<Duration> REPORTER_INTERVAL =
-            key("metrics.reporter.<name>.interval")
+            key("interval")
                     .durationType()
                     .defaultValue(Duration.ofSeconds(10))
                     .withDescription("The reporter interval to use for the 
reporter named <name>.");
 
+    @Documentation.SuffixOption(NAMED_REPORTER_CONFIG_PREFIX)
+    @Documentation.Section(value = Documentation.Sections.METRIC_REPORTERS, 
position = 2)
+    public static final ConfigOption<String> REPORTER_SCOPE_DELIMITER =
+            key("scope.delimiter")
+                    .stringType()
+                    .defaultValue(".")
+                    .withDescription(
+                            "The delimiter used to assemble the metric 
identifier for the reporter named <name>.");
+
+    @Documentation.SuffixOption(NAMED_REPORTER_CONFIG_PREFIX)
+    @Documentation.Section(value = Documentation.Sections.METRIC_REPORTERS, 
position = 3)
+    public static final ConfigOption<Map<String, String>> 
REPORTER_ADDITIONAL_VARIABLES =
+            key("scope.variables.additional")
+                    .mapType()
+                    .defaultValue(Collections.emptyMap())
+                    .withDescription(
+                            "The map of additional variables that should be 
included for the reporter named <name>. Only applicable to tag-based reporters 
(e.g., PRometheus, InfluxDB).");
+
+    @Documentation.SuffixOption(NAMED_REPORTER_CONFIG_PREFIX)
+    @Documentation.Section(value = Documentation.Sections.METRIC_REPORTERS, 
position = 3)
+    public static final ConfigOption<String> REPORTER_EXCLUDED_VARIABLES =
+            key("scope.variables.excludes")
+                    .stringType()
+                    .defaultValue(".")
+                    .withDescription(
+                            "The set of variables that should be excluded for 
the reporter named <name>. Only applicable to tag-based reporters (e.g., 
PRometheus, InfluxDB).");
+
+    @Documentation.SuffixOption(NAMED_REPORTER_CONFIG_PREFIX)
+    @Documentation.Section(value = Documentation.Sections.METRIC_REPORTERS, 
position = 4)
     public static final ConfigOption<String> REPORTER_CONFIG_PARAMETER =
-            key("metrics.reporter.<name>.<parameter>")
+            key("<parameter>")
                     .stringType()
                     .noDefaultValue()
                     .withDescription(
diff --git 
a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
 
b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
index 70c43cf..0b08ac9 100644
--- 
a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
+++ 
b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
@@ -20,6 +20,7 @@ package org.apache.flink.metrics.prometheus.tests;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
 import org.apache.flink.metrics.prometheus.PrometheusReporter;
 import org.apache.flink.metrics.prometheus.PrometheusReporterFactory;
 import org.apache.flink.tests.util.AutoClosableProcess;
@@ -179,14 +180,14 @@ public class PrometheusReporterEndToEndITCase extends 
TestLogger {
                 config.setString(
                         ConfigConstants.METRICS_REPORTER_PREFIX
                                 + "prom."
-                                + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                                + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                         PrometheusReporterFactory.class.getName());
                 break;
             case REFLECTION:
                 config.setString(
                         ConfigConstants.METRICS_REPORTER_PREFIX
                                 + "prom."
-                                + 
ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                                + MetricOptions.REPORTER_CLASS.key(),
                         PrometheusReporter.class.getCanonicalName());
         }
 
diff --git 
a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
 
b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
index 9da584d..025a1f0 100644
--- 
a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
+++ 
b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java
@@ -72,7 +72,7 @@ class JMXJobManagerMetricTest {
         flinkConfiguration.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 JMXReporter.class.getName());
         flinkConfiguration.setString(MetricOptions.SCOPE_NAMING_JM_JOB, 
"jobmanager.<job_name>");
 
diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
index e8b1924..c1e48df 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java
@@ -20,8 +20,6 @@ package org.apache.flink.runtime.metrics;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.DelegatingConfiguration;
 import org.apache.flink.configuration.MetricOptions;
@@ -79,16 +77,11 @@ public final class ReporterSetup {
                             // classes
                             "([\\S&&[^.]]*)\\."
                             + '('
-                            + 
Pattern.quote(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX)
+                            + Pattern.quote(MetricOptions.REPORTER_CLASS.key())
                             + '|'
-                            + 
Pattern.quote(ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX)
+                            + 
Pattern.quote(MetricOptions.REPORTER_FACTORY_CLASS.key())
                             + ')');
 
-    private static final ConfigOption<Map<String, String>> 
ADDITIONAL_VARIABLES =
-            
ConfigOptions.key(ConfigConstants.METRICS_REPORTER_ADDITIONAL_VARIABLES)
-                    .mapType()
-                    .defaultValue(Collections.emptyMap());
-
     private final String name;
     private final MetricConfig configuration;
     private final MetricReporter reporter;
@@ -107,17 +100,17 @@ public final class ReporterSetup {
 
     public Optional<String> getDelimiter() {
         return Optional.ofNullable(
-                
configuration.getString(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, 
null));
+                
configuration.getString(MetricOptions.REPORTER_SCOPE_DELIMITER.key(), null));
     }
 
     public Optional<String> getIntervalSettings() {
         return Optional.ofNullable(
-                
configuration.getString(ConfigConstants.METRICS_REPORTER_INTERVAL_SUFFIX, 
null));
+                configuration.getString(MetricOptions.REPORTER_INTERVAL.key(), 
null));
     }
 
     public Set<String> getExcludedVariables() {
         String excludedVariablesList =
-                
configuration.getString(ConfigConstants.METRICS_REPORTER_EXCLUDED_VARIABLES, 
null);
+                
configuration.getString(MetricOptions.REPORTER_EXCLUDED_VARIABLES.key(), null);
         if (excludedVariablesList == null) {
             return Collections.emptySet();
         } else {
@@ -308,7 +301,8 @@ public final class ReporterSetup {
 
                 // massage user variables keys into scope format for parity to 
variable exclusion
                 Map<String, String> additionalVariables =
-                        
reporterConfig.get(ADDITIONAL_VARIABLES).entrySet().stream()
+                        
reporterConfig.get(MetricOptions.REPORTER_ADDITIONAL_VARIABLES).entrySet()
+                                .stream()
                                 .collect(
                                         Collectors.toMap(
                                                 e -> 
ScopeFormat.asVariable(e.getKey()),
@@ -341,11 +335,8 @@ public final class ReporterSetup {
             final Map<String, MetricReporterFactory> reporterFactories)
             throws ClassNotFoundException, IllegalAccessException, 
InstantiationException {
 
-        final String reporterClassName =
-                
reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null);
-        final String factoryClassName =
-                reporterConfig.getString(
-                        ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, 
null);
+        final String reporterClassName = 
reporterConfig.get(MetricOptions.REPORTER_CLASS);
+        final String factoryClassName = 
reporterConfig.get(MetricOptions.REPORTER_FACTORY_CLASS);
 
         if (factoryClassName != null) {
             return loadViaFactory(
@@ -431,9 +422,9 @@ public final class ReporterSetup {
                             + " Please configure a factory class instead: 
'{}{}.{}: {}' to ensure that the configuration"
                             + " continues to work with future versions.",
                     reporterName,
-                    ConfigConstants.METRICS_REPORTER_PREFIX,
+                    MetricOptions.REPORTER_CLASS.key(),
                     reporterName,
-                    ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                    MetricOptions.REPORTER_FACTORY_CLASS.key(),
                     alternativeFactoryClassName);
             return loadViaFactory(
                     alternativeFactoryClassName, reporterName, reporterConfig, 
reporterFactories);
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java
index 5062c92..9a5e009 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java
@@ -142,7 +142,7 @@ public class MetricRegistryImplTest extends TestLogger {
     public void testReporterScheduling() throws Exception {
         MetricConfig config = new MetricConfig();
         config.setProperty("arg1", "hello");
-        config.setProperty(ConfigConstants.METRICS_REPORTER_INTERVAL_SUFFIX, 
"50 MILLISECONDS");
+        config.setProperty(MetricOptions.REPORTER_INTERVAL.key(), "50 
MILLISECONDS");
 
         MetricRegistryImpl registry =
                 new MetricRegistryImpl(
@@ -180,7 +180,7 @@ public class MetricRegistryImplTest extends TestLogger {
         MetricConfig config = new MetricConfig();
         // in a prior implementation the time amount was applied even if the 
time unit was invalid
         // in this case this would imply using 1 SECOND as the interval 
(seconds is the default)
-        config.setProperty(ConfigConstants.METRICS_REPORTER_INTERVAL_SUFFIX, 
"1 UNICORN");
+        config.setProperty(MetricOptions.REPORTER_INTERVAL.key(), "1 UNICORN");
 
         final ManuallyTriggeredScheduledExecutorService 
manuallyTriggeredScheduledExecutorService =
                 new ManuallyTriggeredScheduledExecutorService();
@@ -220,12 +220,12 @@ public class MetricRegistryImplTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test1."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 TestReporter6.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test2."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 TestReporter7.class.getName());
 
         MetricRegistryImpl registry =
@@ -345,13 +345,13 @@ public class MetricRegistryImplTest extends TestLogger {
     @Test
     public void testConfigurableDelimiterForReporters() throws Exception {
         MetricConfig config1 = new MetricConfig();
-        config1.setProperty(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, 
"_");
+        config1.setProperty(MetricOptions.REPORTER_SCOPE_DELIMITER.key(), "_");
 
         MetricConfig config2 = new MetricConfig();
-        config2.setProperty(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, 
"-");
+        config2.setProperty(MetricOptions.REPORTER_SCOPE_DELIMITER.key(), "-");
 
         MetricConfig config3 = new MetricConfig();
-        config3.setProperty(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, 
"AA");
+        config3.setProperty(MetricOptions.REPORTER_SCOPE_DELIMITER.key(), 
"AA");
 
         MetricRegistryImpl registry =
                 new MetricRegistryImpl(
@@ -375,13 +375,13 @@ public class MetricRegistryImplTest extends TestLogger {
     public void testConfigurableDelimiterForReportersInGroup() throws 
Exception {
         String name = "C";
         MetricConfig config1 = new MetricConfig();
-        config1.setProperty(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, 
"_");
+        config1.setProperty(MetricOptions.REPORTER_SCOPE_DELIMITER.key(), "_");
 
         MetricConfig config2 = new MetricConfig();
-        config2.setProperty(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, 
"-");
+        config2.setProperty(MetricOptions.REPORTER_SCOPE_DELIMITER.key(), "-");
 
         MetricConfig config3 = new MetricConfig();
-        config3.setProperty(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, 
"AA");
+        config3.setProperty(MetricOptions.REPORTER_SCOPE_DELIMITER.key(), 
"AA");
 
         Configuration config = new Configuration();
         config.setString(MetricOptions.SCOPE_NAMING_TM, "A.B");
@@ -389,37 +389,37 @@ public class MetricRegistryImplTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test1."
-                        + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER,
+                        + MetricOptions.REPORTER_SCOPE_DELIMITER.key(),
                 "_");
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test1."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 CollectingMetricsReporter.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test2."
-                        + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER,
+                        + MetricOptions.REPORTER_SCOPE_DELIMITER.key(),
                 "-");
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test2."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 CollectingMetricsReporter.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test3."
-                        + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER,
+                        + MetricOptions.REPORTER_SCOPE_DELIMITER.key(),
                 "AA");
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test3."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 CollectingMetricsReporter.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test4."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 CollectingMetricsReporter.class.getName());
 
         List<ReporterSetup> reporterConfigurations =
@@ -445,7 +445,7 @@ public class MetricRegistryImplTest extends TestLogger {
         for (ReporterSetup cfg : reporterConfigurations) {
             String delimiter =
                     cfg.getConfiguration()
-                            
.getProperty(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER);
+                            
.getProperty(MetricOptions.REPORTER_SCOPE_DELIMITER.key());
             if (delimiter == null || delimiter.equals("AA")) {
                 // test3 reporter: 'AA' - not correct
                 // for test4 reporter use global delimiter
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/ReporterSetupTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/ReporterSetupTest.java
index 98d6977..00eff56 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/ReporterSetupTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/ReporterSetupTest.java
@@ -125,7 +125,7 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "reporter1."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 TestReporter1.class.getName());
 
         final List<ReporterSetup> reporterSetups = 
ReporterSetup.fromConfiguration(config, null);
@@ -145,17 +145,17 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test1."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 TestReporter11.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test2."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 TestReporter12.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test3."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 TestReporter13.class.getName());
 
         List<ReporterSetup> reporterSetups = 
ReporterSetup.fromConfiguration(config, null);
@@ -201,7 +201,7 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "reporter1."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 TestReporter1.class.getName());
         config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + 
"reporter1.arg1", "value1");
         config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + 
"reporter1.arg2", "value2");
@@ -220,7 +220,7 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "reporter2."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 TestReporter2.class.getName());
         config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + 
"reporter2.arg1", "value1");
         config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + 
"reporter2.arg3", "value3");
@@ -242,12 +242,12 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                 TestReporterFactory.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + ConfigConstants.METRICS_REPORTER_EXCLUDED_VARIABLES,
+                        + MetricOptions.REPORTER_EXCLUDED_VARIABLES.key(),
                 excludedVariable1 + ";" + excludedVariable2);
 
         final List<ReporterSetup> reporterSetups = 
ReporterSetup.fromConfiguration(config, null);
@@ -270,7 +270,7 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                 TestReporterFactory.class.getName());
 
         final List<ReporterSetup> reporterSetups = 
ReporterSetup.fromConfiguration(config, null);
@@ -292,12 +292,12 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                 InstantiationTypeTrackingTestReporterFactory.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 InstantiationTypeTrackingTestReporter.class.getName());
 
         final List<ReporterSetup> reporterSetups = 
ReporterSetup.fromConfiguration(config, null);
@@ -318,12 +318,12 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                 TestReporterFactory.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "fail."
-                        + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                 FailingFactory.class.getName());
 
         final List<ReporterSetup> reporterSetups = 
ReporterSetup.fromConfiguration(config, null);
@@ -338,12 +338,12 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test1."
-                        + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                 InstantiationTypeTrackingTestReporterFactory.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test2."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 InstantiationTypeTrackingTestReporter.class.getName());
 
         final List<ReporterSetup> reporterSetups = 
ReporterSetup.fromConfiguration(config, null);
@@ -367,7 +367,7 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                 ConfigExposingReporterFactory.class.getName());
         config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.arg", 
"hello");
 
@@ -387,7 +387,7 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 InstantiationTypeTrackingTestReporter2.class.getName());
 
         final List<ReporterSetup> reporterSetups = 
ReporterSetup.fromConfiguration(config, null);
@@ -411,7 +411,7 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 InstantiationTypeTrackingTestReporter.class.getName());
 
         final List<ReporterSetup> reporterSetups =
@@ -443,12 +443,12 @@ public class ReporterSetupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                 TestReporterFactory.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + 
ConfigConstants.METRICS_REPORTER_ADDITIONAL_VARIABLES,
+                        + MetricOptions.REPORTER_ADDITIONAL_VARIABLES.key(),
                 String.join(",", tag1 + ":" + tagValue1, tag2 + ":" + 
tagValue2));
 
         final List<ReporterSetup> reporterSetups = 
ReporterSetup.fromConfiguration(config, null);
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java
index a76b192..b728d9f 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java
@@ -152,30 +152,30 @@ public class AbstractMetricGroupTest extends TestLogger {
         config.setString(MetricOptions.SCOPE_NAMING_TM, "A.B.C.D");
 
         MetricConfig metricConfig1 = new MetricConfig();
-        
metricConfig1.setProperty(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, 
"-");
+        
metricConfig1.setProperty(MetricOptions.REPORTER_SCOPE_DELIMITER.key(), "-");
 
         MetricConfig metricConfig2 = new MetricConfig();
-        
metricConfig2.setProperty(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, 
"!");
+        
metricConfig2.setProperty(MetricOptions.REPORTER_SCOPE_DELIMITER.key(), "!");
 
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test1."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 CollectingMetricsReporter.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test1."
-                        + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER,
+                        + MetricOptions.REPORTER_SCOPE_DELIMITER.key(),
                 "-");
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test2."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 CollectingMetricsReporter.class.getName());
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test2."
-                        + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER,
+                        + MetricOptions.REPORTER_SCOPE_DELIMITER.key(),
                 "!");
 
         CollectingMetricsReporter reporter1 = new 
CollectingMetricsReporter(FILTER_B);
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 15b9a27..754766d 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
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.metrics.groups;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
 import org.apache.flink.metrics.CharacterFilter;
 import org.apache.flink.metrics.Gauge;
 import org.apache.flink.metrics.Metric;
@@ -243,7 +244,7 @@ public class MetricGroupTest extends TestLogger {
         config.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "test."
-                        + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_CLASS.key(),
                 TestReporter.class.getName());
 
         MetricRegistryImpl registry =
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemoryReporter.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemoryReporter.java
index 1a992e8..a2716cd 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemoryReporter.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemoryReporter.java
@@ -21,6 +21,7 @@ import org.apache.flink.annotation.Experimental;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
 import org.apache.flink.metrics.LogicalScopeProvider;
 import org.apache.flink.metrics.Metric;
 import org.apache.flink.metrics.MetricConfig;
@@ -245,7 +246,7 @@ public class InMemoryReporter implements MetricReporter {
         configuration.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX
                         + "mini_cluster_resource_reporter."
-                        + 
ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX,
+                        + MetricOptions.REPORTER_FACTORY_CLASS.key(),
                 InMemoryReporter.Factory.class.getName());
         configuration.setString(
                 ConfigConstants.METRICS_REPORTER_PREFIX + 
"mini_cluster_resource_reporter." + ID,

Reply via email to