You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2022/03/28 19:28:19 UTC

[flink] branch master updated (5d54921 -> e640840)

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

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


    from 5d54921  [FLINK-26850][metrics] Add Metric#getMetricType
     new ebca624  [FLINK-26851][docs] Document prefix for SuffixOption
     new e640840  [FLINK-26851][metrics] Migrate reporter options to proper ConfigOptions

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../content.zh/docs/deployment/metric_reporters.md |  16 +--
 docs/content/docs/deployment/metric_reporters.md   |  16 +--
 .../generated/influxdb_reporter_configuration.html |  20 ++--
 .../shortcodes/generated/metric_configuration.html |  24 +++++
 .../generated/metric_reporters_section.html        |  54 ++++++++++
 ...etheus_push_gateway_reporter_configuration.html |  12 +--
 .../{top.html => include_reporter_config.html}     |  12 ++-
 .../flink/annotation/docs/Documentation.java       |   6 +-
 .../streaming/connectors/kafka/KafkaTestBase.java  |   3 +-
 .../flink/configuration/ConfigConstants.java       |  28 +++--
 .../apache/flink/configuration/MetricOptions.java  |  53 ++++++++-
 .../configuration/ConfigOptionsDocGenerator.java   |  22 +++-
 .../ConfigOptionsDocsCompletenessITCase.java       | 118 +++++++++------------
 .../tests/PrometheusReporterEndToEndITCase.java    |   5 +-
 .../metrics/influxdb/InfluxdbReporterOptions.java  |   3 +-
 .../jobmanager/JMXJobManagerMetricTest.java        |   2 +-
 .../PrometheusPushGatewayReporterOptions.java      |   3 +-
 .../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 +-
 23 files changed, 317 insertions(+), 205 deletions(-)
 create mode 100644 docs/layouts/shortcodes/generated/metric_reporters_section.html
 copy docs/layouts/shortcodes/{top.html => include_reporter_config.html} (74%)

[flink] 02/02: [FLINK-26851][metrics] Migrate reporter options to proper ConfigOptions

Posted by ch...@apache.org.
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 <ch...@apache.org>
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,

[flink] 01/02: [FLINK-26851][docs] Document prefix for SuffixOption

Posted by ch...@apache.org.
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 ebca6249f6c5c57e9684efdc8bde9f1f3975fc2d
Author: Chesnay Schepler <ch...@apache.org>
AuthorDate: Wed Mar 23 15:52:41 2022 +0100

    [FLINK-26851][docs] Document prefix for SuffixOption
    
    Allows to define a prefix for SuffixOptions that is also written into the generated html file.
    This allows us to use the same file for documentating the fully-qualified option (prefix + suffix) or just the suffix option.
---
 .../content.zh/docs/deployment/metric_reporters.md |   4 +-
 docs/content/docs/deployment/metric_reporters.md   |   4 +-
 .../generated/influxdb_reporter_configuration.html |  20 ++--
 ...etheus_push_gateway_reporter_configuration.html |  12 +--
 .../shortcodes/include_reporter_config.html        |  26 +++++
 .../flink/annotation/docs/Documentation.java       |   4 +-
 .../configuration/ConfigOptionsDocGenerator.java   |  22 +++-
 .../ConfigOptionsDocsCompletenessITCase.java       | 118 +++++++++------------
 .../metrics/influxdb/InfluxdbReporterOptions.java  |   3 +-
 .../PrometheusPushGatewayReporterOptions.java      |   3 +-
 10 files changed, 123 insertions(+), 93 deletions(-)

diff --git a/docs/content.zh/docs/deployment/metric_reporters.md b/docs/content.zh/docs/deployment/metric_reporters.md
index 2c19f47..d966662 100644
--- a/docs/content.zh/docs/deployment/metric_reporters.md
+++ b/docs/content.zh/docs/deployment/metric_reporters.md
@@ -128,7 +128,7 @@ metrics.reporter.grph.interval: 60 SECONDS
 
 Parameters:
 
-{{< generated/influxdb_reporter_configuration >}}
+{{< include_reporter_config "layouts/shortcodes/generated/influxdb_reporter_configuration.html" >}}
 
 Example configuration:
 
@@ -180,7 +180,7 @@ All Flink metrics variables (see [List of all Variables]({{< ref "docs/ops/metri
 
 Parameters:
 
-{{< generated/prometheus_push_gateway_reporter_configuration >}}
+{{< include_reporter_config "layouts/shortcodes/generated/prometheus_push_gateway_reporter_configuration.html" >}}
 
 Example configuration:
 
diff --git a/docs/content/docs/deployment/metric_reporters.md b/docs/content/docs/deployment/metric_reporters.md
index 18b6648..a78acef 100644
--- a/docs/content/docs/deployment/metric_reporters.md
+++ b/docs/content/docs/deployment/metric_reporters.md
@@ -128,7 +128,7 @@ metrics.reporter.grph.interval: 60 SECONDS
 
 Parameters:
 
-{{< generated/influxdb_reporter_configuration >}}
+{{< include_reporter_config "layouts/shortcodes/generated/influxdb_reporter_configuration.html" >}}
 
 Example configuration:
 
@@ -180,7 +180,7 @@ All Flink metrics variables (see [List of all Variables]({{< ref "docs/ops/metri
 
 Parameters:
 
-{{< generated/prometheus_push_gateway_reporter_configuration >}}
+{{< include_reporter_config "layouts/shortcodes/generated/prometheus_push_gateway_reporter_configuration.html" >}}
 
 Example configuration:
 
diff --git a/docs/layouts/shortcodes/generated/influxdb_reporter_configuration.html b/docs/layouts/shortcodes/generated/influxdb_reporter_configuration.html
index fc1c82a..e5b231e 100644
--- a/docs/layouts/shortcodes/generated/influxdb_reporter_configuration.html
+++ b/docs/layouts/shortcodes/generated/influxdb_reporter_configuration.html
@@ -9,61 +9,61 @@
     </thead>
     <tbody>
         <tr>
-            <td><h5>connectTimeout</h5></td>
+            <td><h5>metrics.reporter.influxdb.connectTimeout</h5></td>
             <td style="word-wrap: break-word;">10000</td>
             <td>Integer</td>
             <td>(optional) the InfluxDB connect timeout for metrics</td>
         </tr>
         <tr>
-            <td><h5>consistency</h5></td>
+            <td><h5>metrics.reporter.influxdb.consistency</h5></td>
             <td style="word-wrap: break-word;">ONE</td>
             <td><p>Enum</p></td>
             <td>(optional) the InfluxDB consistency level for metrics<br /><br />Possible values:<ul><li>"ALL"</li><li>"ANY"</li><li>"ONE"</li><li>"QUORUM"</li></ul></td>
         </tr>
         <tr>
-            <td><h5>db</h5></td>
+            <td><h5>metrics.reporter.influxdb.db</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
             <td>the InfluxDB database to store metrics</td>
         </tr>
         <tr>
-            <td><h5>host</h5></td>
+            <td><h5>metrics.reporter.influxdb.host</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
             <td>the InfluxDB server host</td>
         </tr>
         <tr>
-            <td><h5>password</h5></td>
+            <td><h5>metrics.reporter.influxdb.password</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
             <td>(optional) InfluxDB username's password used for authentication</td>
         </tr>
         <tr>
-            <td><h5>port</h5></td>
+            <td><h5>metrics.reporter.influxdb.port</h5></td>
             <td style="word-wrap: break-word;">8086</td>
             <td>Integer</td>
             <td>the InfluxDB server port</td>
         </tr>
         <tr>
-            <td><h5>retentionPolicy</h5></td>
+            <td><h5>metrics.reporter.influxdb.retentionPolicy</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
             <td>(optional) the InfluxDB retention policy for metrics</td>
         </tr>
         <tr>
-            <td><h5>scheme</h5></td>
+            <td><h5>metrics.reporter.influxdb.scheme</h5></td>
             <td style="word-wrap: break-word;">http</td>
             <td><p>Enum</p></td>
             <td>the InfluxDB schema<br /><br />Possible values:<ul><li>"http"</li><li>"https"</li></ul></td>
         </tr>
         <tr>
-            <td><h5>username</h5></td>
+            <td><h5>metrics.reporter.influxdb.username</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
             <td>(optional) InfluxDB username used for authentication</td>
         </tr>
         <tr>
-            <td><h5>writeTimeout</h5></td>
+            <td><h5>metrics.reporter.influxdb.writeTimeout</h5></td>
             <td style="word-wrap: break-word;">10000</td>
             <td>Integer</td>
             <td>(optional) the InfluxDB write timeout for metrics</td>
diff --git a/docs/layouts/shortcodes/generated/prometheus_push_gateway_reporter_configuration.html b/docs/layouts/shortcodes/generated/prometheus_push_gateway_reporter_configuration.html
index 4418f46..9d03e1d 100644
--- a/docs/layouts/shortcodes/generated/prometheus_push_gateway_reporter_configuration.html
+++ b/docs/layouts/shortcodes/generated/prometheus_push_gateway_reporter_configuration.html
@@ -9,37 +9,37 @@
     </thead>
     <tbody>
         <tr>
-            <td><h5>deleteOnShutdown</h5></td>
+            <td><h5>metrics.reporter.prometheus.deleteOnShutdown</h5></td>
             <td style="word-wrap: break-word;">true</td>
             <td>Boolean</td>
             <td>Specifies whether to delete metrics from the PushGateway on shutdown. Flink will try its best to delete the metrics but this is not guaranteed. See <a href="https://issues.apache.org/jira/browse/FLINK-13787">here</a> for more details.</td>
         </tr>
         <tr>
-            <td><h5>filterLabelValueCharacters</h5></td>
+            <td><h5>metrics.reporter.prometheus.filterLabelValueCharacters</h5></td>
             <td style="word-wrap: break-word;">true</td>
             <td>Boolean</td>
             <td>Specifies whether to filter label value characters. If enabled, all characters not matching [a-zA-Z0-9:_] will be removed, otherwise no characters will be removed. Before disabling this option please ensure that your label values meet the <a href="https://prometheus.io/docs/concepts/data_model/#metric-names-and-labels">Prometheus requirements</a>.</td>
         </tr>
         <tr>
-            <td><h5>groupingKey</h5></td>
+            <td><h5>metrics.reporter.prometheus.groupingKey</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
             <td>Specifies the grouping key which is the group and global labels of all metrics. The label name and value are separated by '=', and labels are separated by ';', e.g., <code class="highlighter-rouge">k1=v1;k2=v2</code>. Please ensure that your grouping key meets the <a href="https://prometheus.io/docs/concepts/data_model/#metric-names-and-labels">Prometheus requirements</a>.</td>
         </tr>
         <tr>
-            <td><h5>hostUrl</h5></td>
+            <td><h5>metrics.reporter.prometheus.hostUrl</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
             <td>The PushGateway server host URL including scheme, host name, and port.</td>
         </tr>
         <tr>
-            <td><h5>jobName</h5></td>
+            <td><h5>metrics.reporter.prometheus.jobName</h5></td>
             <td style="word-wrap: break-word;">(none)</td>
             <td>String</td>
             <td>The job name under which metrics will be pushed</td>
         </tr>
         <tr>
-            <td><h5>randomJobNameSuffix</h5></td>
+            <td><h5>metrics.reporter.prometheus.randomJobNameSuffix</h5></td>
             <td style="word-wrap: break-word;">true</td>
             <td>Boolean</td>
             <td>Specifies whether a random suffix should be appended to the job name.</td>
diff --git a/docs/layouts/shortcodes/include_reporter_config.html b/docs/layouts/shortcodes/include_reporter_config.html
new file mode 100644
index 0000000..79330f6
--- /dev/null
+++ b/docs/layouts/shortcodes/include_reporter_config.html
@@ -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.
+*/}}
+{{/* 
+  Shortcode for embedding a reporter configuration while
+  stripping the key prefixes
+  for a concise listing of available parameters.
+*/}}
+{{ $file := .Get 0 }}
+
+{{ safeHTML (replaceRE "metrics\\.reporter\\.[&lt;&gt;>a-zA-Z]+?\\." "" (os.ReadFile $file)) }}
\ No newline at end of file
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 69bb4b0..e8b150f 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
@@ -145,7 +145,9 @@ public final class Documentation {
     @Target({ElementType.FIELD, ElementType.TYPE})
     @Retention(RetentionPolicy.RUNTIME)
     @Internal
-    public @interface SuffixOption {}
+    public @interface SuffixOption {
+        String value();
+    }
 
     /**
      * Annotation used on config option fields or REST API message headers to exclude it from
diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java
index 4602cb6..9fb068f 100644
--- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java
+++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java
@@ -465,7 +465,7 @@ public class ConfigOptionsDocGenerator {
         return ""
                 + "        <tr>\n"
                 + "            <td><h5>"
-                + escapeCharacters(option.key())
+                + escapeCharacters(getDocumentedKey(optionWithMetaInfo))
                 + "</h5>"
                 + execModeStringBuilder.toString()
                 + "</td>\n"
@@ -482,6 +482,24 @@ public class ConfigOptionsDocGenerator {
     }
 
     @VisibleForTesting
+    static String getDocumentedKey(OptionWithMetaInfo optionWithMetaInfo) {
+        Documentation.SuffixOption suffixOptionAnnotation =
+                optionWithMetaInfo.field.getAnnotation(Documentation.SuffixOption.class);
+        if (suffixOptionAnnotation == null) {
+            suffixOptionAnnotation =
+                    optionWithMetaInfo
+                            .field
+                            .getDeclaringClass()
+                            .getAnnotation(Documentation.SuffixOption.class);
+        }
+
+        final String originalKey = optionWithMetaInfo.option.key();
+        return suffixOptionAnnotation == null
+                ? originalKey
+                : suffixOptionAnnotation.value() + "." + originalKey;
+    }
+
+    @VisibleForTesting
     static String getDescription(OptionWithMetaInfo optionWithMetaInfo) {
         final String enumValuesSection =
                 Optional.ofNullable(getEnumOptionsDescription(optionWithMetaInfo))
@@ -637,7 +655,7 @@ public class ConfigOptionsDocGenerator {
     }
 
     private static void sortOptions(List<OptionWithMetaInfo> configOptions) {
-        configOptions.sort(Comparator.comparing(option -> option.option.key()));
+        configOptions.sort(Comparator.comparing(option -> getDocumentedKey(option)));
     }
 
     /**
diff --git a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java
index 584d088..41ae4d3 100644
--- a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java
+++ b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.docs.configuration;
 
-import org.apache.flink.annotation.docs.Documentation;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigOption;
 
@@ -28,7 +27,6 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -49,6 +47,7 @@ import static org.apache.flink.docs.configuration.ConfigOptionsDocGenerator.DEFA
 import static org.apache.flink.docs.configuration.ConfigOptionsDocGenerator.LOCATIONS;
 import static org.apache.flink.docs.configuration.ConfigOptionsDocGenerator.extractConfigOptions;
 import static org.apache.flink.docs.configuration.ConfigOptionsDocGenerator.getDescription;
+import static org.apache.flink.docs.configuration.ConfigOptionsDocGenerator.getDocumentedKey;
 import static org.apache.flink.docs.configuration.ConfigOptionsDocGenerator.processConfigOptions;
 import static org.apache.flink.docs.configuration.ConfigOptionsDocGenerator.stringifyDefault;
 import static org.apache.flink.docs.configuration.ConfigOptionsDocGenerator.typeToHtml;
@@ -81,59 +80,54 @@ public class ConfigOptionsDocsCompletenessITCase {
                             final List<ExistingOption> existingOptions = entry.getValue();
                             final List<ExistingOption> consolidated;
 
-                            if (existingOptions.stream()
-                                    .allMatch(option -> option.isSuffixOption)) {
-                                consolidated = existingOptions;
-                            } else {
-                                Optional<ExistingOption> deduped =
-                                        existingOptions.stream()
-                                                .reduce(
-                                                        (option1, option2) -> {
-                                                            if (option1.equals(option2)) {
-                                                                // we allow multiple instances of
-                                                                // ConfigOptions with the same key
-                                                                // if they are identical
-                                                                return option1;
+                            Optional<ExistingOption> deduped =
+                                    existingOptions.stream()
+                                            .reduce(
+                                                    (option1, option2) -> {
+                                                        if (option1.equals(option2)) {
+                                                            // we allow multiple instances of
+                                                            // ConfigOptions with the same key
+                                                            // if they are identical
+                                                            return option1;
+                                                        } else {
+                                                            // found a ConfigOption pair with
+                                                            // the same key that aren't equal
+                                                            // we fail here outright as this is
+                                                            // not a documentation-completeness
+                                                            // problem
+                                                            if (!option1.defaultValue.equals(
+                                                                    option2.defaultValue)) {
+                                                                String errorMessage =
+                                                                        String.format(
+                                                                                "Ambiguous option %s due to distinct default values (%s (in %s) vs %s (in %s)).",
+                                                                                option1.key,
+                                                                                option1.defaultValue,
+                                                                                option1
+                                                                                        .containingClass
+                                                                                        .getSimpleName(),
+                                                                                option2.defaultValue,
+                                                                                option2
+                                                                                        .containingClass
+                                                                                        .getSimpleName());
+                                                                throw new AssertionError(
+                                                                        errorMessage);
                                                             } else {
-                                                                // found a ConfigOption pair with
-                                                                // the same key that aren't equal
-                                                                // we fail here outright as this is
-                                                                // not a documentation-completeness
-                                                                // problem
-                                                                if (!option1.defaultValue.equals(
-                                                                        option2.defaultValue)) {
-                                                                    String errorMessage =
-                                                                            String.format(
-                                                                                    "Ambiguous option %s due to distinct default values (%s (in %s) vs %s (in %s)).",
-                                                                                    option1.key,
-                                                                                    option1.defaultValue,
-                                                                                    option1
-                                                                                            .containingClass
-                                                                                            .getSimpleName(),
-                                                                                    option2.defaultValue,
-                                                                                    option2
-                                                                                            .containingClass
-                                                                                            .getSimpleName());
-                                                                    throw new AssertionError(
-                                                                            errorMessage);
-                                                                } else {
-                                                                    String errorMessage =
-                                                                            String.format(
-                                                                                    "Ambiguous option %s due to distinct descriptions (%s vs %s).",
-                                                                                    option1.key,
-                                                                                    option1
-                                                                                            .containingClass
-                                                                                            .getSimpleName(),
-                                                                                    option2
-                                                                                            .containingClass
-                                                                                            .getSimpleName());
-                                                                    throw new AssertionError(
-                                                                            errorMessage);
-                                                                }
+                                                                String errorMessage =
+                                                                        String.format(
+                                                                                "Ambiguous option %s due to distinct descriptions (%s vs %s).",
+                                                                                option1.key,
+                                                                                option1
+                                                                                        .containingClass
+                                                                                        .getSimpleName(),
+                                                                                option2
+                                                                                        .containingClass
+                                                                                        .getSimpleName());
+                                                                throw new AssertionError(
+                                                                        errorMessage);
                                                             }
-                                                        });
-                                consolidated = Collections.singletonList(deduped.get());
-                            }
+                                                        }
+                                                    });
+                            consolidated = Collections.singletonList(deduped.get());
 
                             return new Tuple2<>(entry.getKey(), consolidated);
                         })
@@ -298,37 +292,25 @@ public class ConfigOptionsDocsCompletenessITCase {
     private static ExistingOption toExistingOption(
             ConfigOptionsDocGenerator.OptionWithMetaInfo optionWithMetaInfo,
             Class<?> optionsClass) {
-        String key = optionWithMetaInfo.option.key();
+        String key = getDocumentedKey(optionWithMetaInfo);
         String defaultValue = stringifyDefault(optionWithMetaInfo);
         String typeValue = typeToHtml(optionWithMetaInfo);
         String description = getDescription(optionWithMetaInfo);
-        boolean isSuffixOption = isSuffixOption(optionWithMetaInfo.field);
-        return new ExistingOption(
-                key, defaultValue, typeValue, description, optionsClass, isSuffixOption);
-    }
-
-    private static boolean isSuffixOption(Field field) {
-        final Class<?> containingOptionsClass = field.getDeclaringClass();
-
-        return field.getAnnotation(Documentation.SuffixOption.class) != null
-                || containingOptionsClass.getAnnotation(Documentation.SuffixOption.class) != null;
+        return new ExistingOption(key, defaultValue, typeValue, description, optionsClass);
     }
 
     private static final class ExistingOption extends Option {
 
         private final Class<?> containingClass;
-        private final boolean isSuffixOption;
 
         private ExistingOption(
                 String key,
                 String defaultValue,
                 String typeValue,
                 String description,
-                Class<?> containingClass,
-                boolean isSuffixOption) {
+                Class<?> containingClass) {
             super(key, defaultValue, typeValue, description);
             this.containingClass = containingClass;
-            this.isSuffixOption = isSuffixOption;
         }
     }
 
diff --git a/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporterOptions.java b/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporterOptions.java
index 870d9b9..51f4bfd 100644
--- a/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporterOptions.java
+++ b/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporterOptions.java
@@ -19,6 +19,7 @@
 package org.apache.flink.metrics.influxdb;
 
 import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.ConfigurationUtils;
@@ -28,7 +29,7 @@ import org.apache.flink.metrics.MetricConfig;
 import org.influxdb.InfluxDB;
 
 /** Config options for {@link InfluxdbReporter}. */
-@Documentation.SuffixOption
+@Documentation.SuffixOption(ConfigConstants.METRICS_REPORTER_PREFIX + "influxdb")
 public class InfluxdbReporterOptions {
 
     public static final ConfigOption<String> HOST =
diff --git a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterOptions.java b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterOptions.java
index b4e15a2..37da233 100644
--- a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterOptions.java
+++ b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterOptions.java
@@ -19,6 +19,7 @@
 package org.apache.flink.metrics.prometheus;
 
 import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.description.Description;
@@ -26,7 +27,7 @@ import org.apache.flink.configuration.description.LinkElement;
 import org.apache.flink.configuration.description.TextElement;
 
 /** Config options for the {@link PrometheusPushGatewayReporter}. */
-@Documentation.SuffixOption
+@Documentation.SuffixOption(ConfigConstants.METRICS_REPORTER_PREFIX + "prometheus")
 public class PrometheusPushGatewayReporterOptions {
 
     @Deprecated