This is an automated email from the ASF dual-hosted git repository. sxnan pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit 972b1e6138eb3ec71544f6fa14b1d3f7545b4454 Author: sxnan <[email protected]> AuthorDate: Wed Sep 11 14:10:17 2024 +0800 [FLINK-33750][config] Remove deprecated option in flink-metrics-prometheus --- .../PrometheusPushGatewayReporterFactory.java | 20 +++----------- .../PrometheusPushGatewayReporterOptions.java | 14 ---------- .../PrometheusPushGatewayReporterTest.java | 32 ---------------------- 3 files changed, 4 insertions(+), 62 deletions(-) diff --git a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterFactory.java b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterFactory.java index ab920cee8d2..35a23d9e957 100644 --- a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterFactory.java +++ b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterFactory.java @@ -35,10 +35,8 @@ import java.util.Properties; import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.DELETE_ON_SHUTDOWN; import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.GROUPING_KEY; -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.HOST; import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.HOST_URL; import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.JOB_NAME; -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.PORT; import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.RANDOM_JOB_NAME_SUFFIX; /** {@link MetricReporterFactory} for {@link PrometheusPushGatewayReporter}. */ @@ -50,8 +48,6 @@ public class PrometheusPushGatewayReporterFactory implements MetricReporterFacto @Override public PrometheusPushGatewayReporter createMetricReporter(Properties properties) { MetricConfig metricConfig = (MetricConfig) properties; - String host = metricConfig.getString(HOST.key(), HOST.defaultValue()); - int port = metricConfig.getInteger(PORT.key(), PORT.defaultValue()); String configuredJobName = metricConfig.getString(JOB_NAME.key(), JOB_NAME.defaultValue()); boolean randomSuffix = metricConfig.getBoolean( @@ -63,18 +59,10 @@ public class PrometheusPushGatewayReporterFactory implements MetricReporterFacto parseGroupingKey( metricConfig.getString(GROUPING_KEY.key(), GROUPING_KEY.defaultValue())); - String hostUrlConfig = metricConfig.getString(HOST_URL.key(), HOST_URL.defaultValue()); - - final String hostUrl; - if (!StringUtils.isNullOrWhitespaceOnly(hostUrlConfig)) { - hostUrl = hostUrlConfig; - } else { - if (StringUtils.isNullOrWhitespaceOnly(host) || port < 1) { - throw new IllegalArgumentException( - "Invalid host/port configuration. Host: " + host + " Port: " + port); - } else { - hostUrl = "http://" + host + ":" + port; - } + String hostUrl = metricConfig.getString(HOST_URL.key(), HOST_URL.defaultValue()); + + if (StringUtils.isNullOrWhitespaceOnly(hostUrl)) { + throw new IllegalArgumentException("hostUrl must not be null or empty"); } String jobName = configuredJobName; 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 b6772020aee..84404b5ef7b 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 @@ -32,20 +32,6 @@ import org.apache.flink.configuration.description.TextElement; @Documentation.SuffixOption(ConfigConstants.METRICS_REPORTER_PREFIX + "prometheus") public class PrometheusPushGatewayReporterOptions { - @Deprecated - public static final ConfigOption<String> HOST = - ConfigOptions.key("host") - .stringType() - .noDefaultValue() - .withDescription("(deprecated) The PushGateway server host."); - - @Deprecated - public static final ConfigOption<Integer> PORT = - ConfigOptions.key("port") - .intType() - .defaultValue(-1) - .withDescription("(deprecated) The PushGateway server port."); - public static final ConfigOption<String> HOST_URL = ConfigOptions.key("hostUrl") .stringType() diff --git a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterTest.java b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterTest.java index 43643776e76..6f70f45a075 100644 --- a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterTest.java +++ b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterTest.java @@ -24,9 +24,7 @@ import org.junit.jupiter.api.Test; import java.util.Map; -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.HOST; import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.HOST_URL; -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.PORT; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -54,16 +52,6 @@ class PrometheusPushGatewayReporterTest { assertThat(groupingKey).isEmpty(); } - @Test - void testConnectToPushGatewayUsingHostAndPort() { - PrometheusPushGatewayReporterFactory factory = new PrometheusPushGatewayReporterFactory(); - MetricConfig metricConfig = new MetricConfig(); - metricConfig.setProperty(HOST.key(), "localhost"); - metricConfig.setProperty(PORT.key(), "18080"); - String gatewayBaseURL = factory.createMetricReporter(metricConfig).hostUrl.toString(); - assertThat(gatewayBaseURL).isEqualTo("http://localhost:18080"); - } - @Test void testConnectToPushGatewayUsingHostUrl() { PrometheusPushGatewayReporterFactory factory = new PrometheusPushGatewayReporterFactory(); @@ -73,31 +61,11 @@ class PrometheusPushGatewayReporterTest { assertThat(gatewayBaseURL).isEqualTo("https://localhost:18080"); } - @Test - void testConnectToPushGatewayPreferHostUrl() { - PrometheusPushGatewayReporterFactory factory = new PrometheusPushGatewayReporterFactory(); - MetricConfig metricConfig = new MetricConfig(); - metricConfig.setProperty(HOST_URL.key(), "https://localhost:18080"); - metricConfig.setProperty(HOST.key(), "localhost1"); - metricConfig.setProperty(PORT.key(), "18081"); - String gatewayBaseURL = factory.createMetricReporter(metricConfig).hostUrl.toString(); - assertThat(gatewayBaseURL).isEqualTo("https://localhost:18080"); - } - @Test void testConnectToPushGatewayThrowsExceptionWithoutHostInformation() { PrometheusPushGatewayReporterFactory factory = new PrometheusPushGatewayReporterFactory(); MetricConfig metricConfig = new MetricConfig(); assertThatThrownBy(() -> factory.createMetricReporter(metricConfig)) .isInstanceOf(IllegalArgumentException.class); - - metricConfig.setProperty(HOST.key(), "localhost"); - assertThatThrownBy(() -> factory.createMetricReporter(metricConfig)) - .isInstanceOf(IllegalArgumentException.class); - - metricConfig.clear(); - metricConfig.setProperty(PORT.key(), "18080"); - assertThatThrownBy(() -> factory.createMetricReporter(metricConfig)) - .isInstanceOf(IllegalArgumentException.class); } }
