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 <[email protected]>
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}. */
[email protected]
[email protected](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}. */
[email protected]
[email protected](ConfigConstants.METRICS_REPORTER_PREFIX + 
"prometheus")
 public class PrometheusPushGatewayReporterOptions {
 
     @Deprecated

Reply via email to