Details
-
Sub-task
-
Status: Closed
-
Major
-
Resolution: Fixed
-
1.7.0
Description
Add an end-to-end test using the PrometheusReporter to verify that all metrics are properly reported. Additionally verify that the newly introduce RocksDB metrics are accessible (see FLINK-10423).
Attachments
Issue Links
- links to
Activity
dawidwys commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230334487
##########
File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
##########
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.metrics.prometheus.tests;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.prometheus.PrometheusReporter;
+import org.apache.flink.tests.util.AutoClosableProcess;
+import org.apache.flink.tests.util.CommandLineWrapper;
+import org.apache.flink.tests.util.FlinkDistribution;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking;
+import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking;
+
+/**
+ * End-to-end test for the PrometheusReporter.
+ */
+public class PrometheusReporterEndToEndITCase extends TestLogger {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private static final String PROMETHEUS_FILE_NAME = "prometheus-2.4.3.linux-amd64";
+
+ private static final Pattern LOG_REPORTER_PORT_PATTERN = Pattern.compile(".Started PrometheusReporter HTTP server on port ([0-9]+).");
+
+ @Rule
+ public final FlinkDistribution dist = new FlinkDistribution();
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Test
+ public void testReporter() throws Exception {
+ LOG.info("starting test");
+ dist.copyOptJarsToLib("flink-metrics-prometheus");
+
+ final Configuration config = new Configuration();
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
+
+ dist.appendConfiguration(config);
+
+ final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
+ final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
+ final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
+ final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
+ final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
+ Files.createDirectory(tmpPrometheusDir);
+
+ runBlocking(
+ "Download of Prometheus",
+ Duration.ofMinutes(5),
+ CommandLineWrapper
+ .wget("https://github.com/prometheus/prometheus/releases/download/v2.4.3/" + prometheusArchive.getFileName())
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Extraction of Prometheus archive",
+ CommandLineWrapper
+ .tar(prometheusArchive)
+ .extract()
+ .zipped()
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Set Prometheus scrape interval",
+ CommandLineWrapper
+ .sed("s/\\(scrape_interval:\\).*/
1 1s/", prometheusConfig)
Review comment:
Can't we implement behaviors like `sed` in java, otherwise we will be still tackling issues of platform dependent semantics of those tools.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230337203
##########
File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
##########
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.metrics.prometheus.tests;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.prometheus.PrometheusReporter;
+import org.apache.flink.tests.util.AutoClosableProcess;
+import org.apache.flink.tests.util.CommandLineWrapper;
+import org.apache.flink.tests.util.FlinkDistribution;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking;
+import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking;
+
+/**
+ * End-to-end test for the PrometheusReporter.
+ */
+public class PrometheusReporterEndToEndITCase extends TestLogger {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private static final String PROMETHEUS_FILE_NAME = "prometheus-2.4.3.linux-amd64";
+
+ private static final Pattern LOG_REPORTER_PORT_PATTERN = Pattern.compile(".Started PrometheusReporter HTTP server on port ([0-9]+).");
+
+ @Rule
+ public final FlinkDistribution dist = new FlinkDistribution();
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Test
+ public void testReporter() throws Exception {
+ LOG.info("starting test");
+ dist.copyOptJarsToLib("flink-metrics-prometheus");
+
+ final Configuration config = new Configuration();
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
+
+ dist.appendConfiguration(config);
+
+ final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
+ final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
+ final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
+ final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
+ final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
+ Files.createDirectory(tmpPrometheusDir);
+
+ runBlocking(
+ "Download of Prometheus",
+ Duration.ofMinutes(5),
+ CommandLineWrapper
+ .wget("https://github.com/prometheus/prometheus/releases/download/v2.4.3/" + prometheusArchive.getFileName())
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Extraction of Prometheus archive",
+ CommandLineWrapper
+ .tar(prometheusArchive)
+ .extract()
+ .zipped()
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Set Prometheus scrape interval",
+ CommandLineWrapper
+ .sed("s/\\(scrape_interval:\\).*/
1 1s/", prometheusConfig)
Review comment:
Of course we could, but I didn't want to do this now.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230337516
##########
File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
##########
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.metrics.prometheus.tests;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.prometheus.PrometheusReporter;
+import org.apache.flink.tests.util.AutoClosableProcess;
+import org.apache.flink.tests.util.CommandLineWrapper;
+import org.apache.flink.tests.util.FlinkDistribution;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking;
+import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking;
+
+/**
+ * End-to-end test for the PrometheusReporter.
+ */
+public class PrometheusReporterEndToEndITCase extends TestLogger {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private static final String PROMETHEUS_FILE_NAME = "prometheus-2.4.3.linux-amd64";
+
+ private static final Pattern LOG_REPORTER_PORT_PATTERN = Pattern.compile(".Started PrometheusReporter HTTP server on port ([0-9]+).");
+
+ @Rule
+ public final FlinkDistribution dist = new FlinkDistribution();
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Test
+ public void testReporter() throws Exception {
+ LOG.info("starting test");
+ dist.copyOptJarsToLib("flink-metrics-prometheus");
+
+ final Configuration config = new Configuration();
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
+
+ dist.appendConfiguration(config);
+
+ final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
+ final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
+ final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
+ final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
+ final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
+ Files.createDirectory(tmpPrometheusDir);
+
+ runBlocking(
+ "Download of Prometheus",
+ Duration.ofMinutes(5),
+ CommandLineWrapper
+ .wget("https://github.com/prometheus/prometheus/releases/download/v2.4.3/" + prometheusArchive.getFileName())
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Extraction of Prometheus archive",
+ CommandLineWrapper
+ .tar(prometheusArchive)
+ .extract()
+ .zipped()
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Set Prometheus scrape interval",
+ CommandLineWrapper
+ .sed("s/\\(scrape_interval:\\).*/
1 1s/", prometheusConfig)
Review comment:
I had already implemented the sed logic in bash (actually the entire test was already written in bash), so I opted for re-using some stuff.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230338568
##########
File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
##########
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.metrics.prometheus.tests;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.prometheus.PrometheusReporter;
+import org.apache.flink.tests.util.AutoClosableProcess;
+import org.apache.flink.tests.util.CommandLineWrapper;
+import org.apache.flink.tests.util.FlinkDistribution;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking;
+import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking;
+
+/**
+ * End-to-end test for the PrometheusReporter.
+ */
+public class PrometheusReporterEndToEndITCase extends TestLogger {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private static final String PROMETHEUS_FILE_NAME = "prometheus-2.4.3.linux-amd64";
+
+ private static final Pattern LOG_REPORTER_PORT_PATTERN = Pattern.compile(".Started PrometheusReporter HTTP server on port ([0-9]+).");
+
+ @Rule
+ public final FlinkDistribution dist = new FlinkDistribution();
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Test
+ public void testReporter() throws Exception {
+ LOG.info("starting test");
+ dist.copyOptJarsToLib("flink-metrics-prometheus");
+
+ final Configuration config = new Configuration();
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
+
+ dist.appendConfiguration(config);
+
+ final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
+ final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
+ final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
+ final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
+ final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
+ Files.createDirectory(tmpPrometheusDir);
+
+ runBlocking(
+ "Download of Prometheus",
+ Duration.ofMinutes(5),
+ CommandLineWrapper
+ .wget("https://github.com/prometheus/prometheus/releases/download/v2.4.3/" + prometheusArchive.getFileName())
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Extraction of Prometheus archive",
+ CommandLineWrapper
+ .tar(prometheusArchive)
+ .extract()
+ .zipped()
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Set Prometheus scrape interval",
+ CommandLineWrapper
+ .sed("s/\\(scrape_interval:\\).*/
1 1s/", prometheusConfig)
Review comment:
huh, I thought i mentioned this in the PR description but it must've been lost somewhere :/
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
dawidwys commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230339434
##########
File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
##########
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.metrics.prometheus.tests;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.prometheus.PrometheusReporter;
+import org.apache.flink.tests.util.AutoClosableProcess;
+import org.apache.flink.tests.util.CommandLineWrapper;
+import org.apache.flink.tests.util.FlinkDistribution;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking;
+import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking;
+
+/**
+ * End-to-end test for the PrometheusReporter.
+ */
+public class PrometheusReporterEndToEndITCase extends TestLogger {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private static final String PROMETHEUS_FILE_NAME = "prometheus-2.4.3.linux-amd64";
+
+ private static final Pattern LOG_REPORTER_PORT_PATTERN = Pattern.compile(".Started PrometheusReporter HTTP server on port ([0-9]+).");
+
+ @Rule
+ public final FlinkDistribution dist = new FlinkDistribution();
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Test
+ public void testReporter() throws Exception {
+ LOG.info("starting test");
+ dist.copyOptJarsToLib("flink-metrics-prometheus");
+
+ final Configuration config = new Configuration();
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
+
+ dist.appendConfiguration(config);
+
+ final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
+ final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
+ final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
+ final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
+ final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
+ Files.createDirectory(tmpPrometheusDir);
+
+ runBlocking(
+ "Download of Prometheus",
+ Duration.ofMinutes(5),
+ CommandLineWrapper
+ .wget("https://github.com/prometheus/prometheus/releases/download/v2.4.3/" + prometheusArchive.getFileName())
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Extraction of Prometheus archive",
+ CommandLineWrapper
+ .tar(prometheusArchive)
+ .extract()
+ .zipped()
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Set Prometheus scrape interval",
+ CommandLineWrapper
+ .sed("s/\\(scrape_interval:\\).*/
1 1s/", prometheusConfig)
Review comment:
got it, thanks for explanation
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230400796
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.java
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.util.FileUtils;
+
+import java.io.IOException;
+import java.nio.file.Path;
+
+/**
+ * Utility class to delete a given
when exiting a try-with-resources statement.
+ */
+public final class AutoClosablePath implements AutoCloseable {
+
+ private final Path file;
+
+ public AutoClosablePath(final Path file) {
Review comment:
Add `null` checks.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230403257
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Utility class for setting up command-line tool usages in a readable fashion.
+ */
+public enum CommandLineWrapper {
+ ;
+
+ public static WGetBuilder wget(String url)
+
+ /**
+ * Wrapper around wget used for downloading files.
+ */
+ public static final class WGetBuilder {
+
+ private final String url;
+ private Path targetDir;
+
+ WGetBuilder(String url)
+
+ public WGetBuilder targetDir(Path dir)
+
+ public String[] build() {
+ final List<String> commandsList = new ArrayList<>(5);
+ commandsList.add("wget");
+ commandsList.add("-q"); // silent
+ //commandsList.add("--show-progress"); // enable progress bar
+ if (targetDir != null)
+ commandsList.add(url);
+ return commandsList.toArray(new String[commandsList.size()]);
+ }
+ }
+
+ public static SedBuilder sed(final String command, final Path file)
+
+ /**
+ * Wrapper around sed used for processing text.
+ */
+ public static final class SedBuilder {
+
+ private final String command;
+ private final Path file;
+
+ private boolean inPlace = false;
+
+ SedBuilder(final String command, final Path file)
+
+ public SedBuilder inPlace()
+
+ public String[] build() {
+ final List<String> commandsList = new ArrayList<>(5);
+ commandsList.add("sed");
+ if (inPlace)
+ commandsList.add("-e");
+ commandsList.add(command);
+ commandsList.add(file.toAbsolutePath().toString());
+ return commandsList.toArray(new String[commandsList.size()]);
+ }
+ }
+
+ public static TarBuilder tar(final Path file)
+
+ /**
+ * Wrapper around tar used for extracting .tar archives.
+ */
+ public static final class TarBuilder {
Review comment:
Same as above.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230401171
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.java
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.util.FileUtils;
+
+import java.io.IOException;
+import java.nio.file.Path;
+
+/**
+ * Utility class to delete a given
when exiting a try-with-resources statement.
+ */
+public final class AutoClosablePath implements AutoCloseable {
+
+ private final Path file;
+
+ public AutoClosablePath(final Path file)
+
+ @Override
+ public void close() throws IOException
+
+ public Path getFile() {
Review comment:
This is not used anywhere, so why not removing it and putting it if a future test needs it?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230400941
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.java
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.util.FileUtils;
+
+import java.io.IOException;
+import java.nio.file.Path;
+
+/**
+ * Utility class to delete a given
when exiting a try-with-resources statement.
+ */
+public final class AutoClosablePath implements AutoCloseable {
+
+ private final Path file;
Review comment:
Rename to `path` or something along these lines, as this is what it is.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230402861
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Utility class for setting up command-line tool usages in a readable fashion.
+ */
+public enum CommandLineWrapper {
+ ;
+
+ public static WGetBuilder wget(String url)
+
+ /**
+ * Wrapper around wget used for downloading files.
+ */
+ public static final class WGetBuilder {
+
+ private final String url;
+ private Path targetDir;
+
+ WGetBuilder(String url)
+
+ public WGetBuilder targetDir(Path dir) {
Review comment:
We could rename it to `toTargetDir`, but most importantly, why not imposing immutability and returning a new `WGetBuilder`, instead of mutating the object?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230408639
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
Review comment:
This `30` can be a class variable with a descriptive name.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230403162
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Utility class for setting up command-line tool usages in a readable fashion.
+ */
+public enum CommandLineWrapper {
+ ;
+
+ public static WGetBuilder wget(String url)
+
+ /**
+ * Wrapper around wget used for downloading files.
+ */
+ public static final class WGetBuilder {
+
+ private final String url;
+ private Path targetDir;
+
+ WGetBuilder(String url)
+
+ public WGetBuilder targetDir(Path dir)
+
+ public String[] build() {
+ final List<String> commandsList = new ArrayList<>(5);
+ commandsList.add("wget");
+ commandsList.add("-q"); // silent
+ //commandsList.add("--show-progress"); // enable progress bar
+ if (targetDir != null)
+ commandsList.add(url);
+ return commandsList.toArray(new String[commandsList.size()]);
+ }
+ }
+
+ public static SedBuilder sed(final String command, final Path file)
+
+ /**
+ * Wrapper around sed used for processing text.
+ */
+ public static final class SedBuilder {
+
+ private final String command;
+ private final Path file;
+
+ private boolean inPlace = false;
+
+ SedBuilder(final String command, final Path file)
+
+ public SedBuilder inPlace() {
+ inPlace = true;
Review comment:
Same here as in the `WGetBuilder`. Why not imposing immutability and go for the "full" builder pattern?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230401483
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Utility class to terminate a given
when exiting a try-with-resources statement.
+ */
+public class AutoClosableProcess implements AutoCloseable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AutoClosableProcess.class);
+
+ private final Process process;
+
+ public AutoClosableProcess(Process process) {
+ this.process = process;
Review comment:
Add `null` checks.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230401719
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Utility class to terminate a given
when exiting a try-with-resources statement.
+ */
+public class AutoClosableProcess implements AutoCloseable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AutoClosableProcess.class);
+
+ private final Process process;
+
+ public AutoClosableProcess(Process process)
+
+ public static AutoClosableProcess runNonBlocking(String step, String... commands) throws IOException
+
+ public static void runBlocking(String step, String... commands) throws IOException
+
+ public static void runBlocking(String step, Duration timeout, String... commands) throws IOException {
+ LOG.info("Step started: " + step);
+ Process process = new ProcessBuilder()
+ .command(commands)
+ .inheritIO()
+ .start();
+
+ try (AutoClosableProcess autoProcess = new AutoClosableProcess(process)) {
+ final boolean success = process.waitFor(timeout.toMillis(), TimeUnit.MILLISECONDS);
+ if (!success)
+ } catch (TimeoutException | InterruptedException e)
{ + throw new RuntimeException(step + " failed due to timeout."); + }+ LOG.info("Step complete: " + step);
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (process.isAlive()) {
+ process.destroy();
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + }+ }
+ }
+
Review comment:
Again, why not removing it given that it is not used, and re-add it if a test in the future uses it?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230409384
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(request).execute()) {
+ if (response.isSuccessful()) {
+ final String json = response.body().string();
+ final JsonNode taskManagerList = OBJECT_MAPPER.readTree(json)
+ .get("taskmanagers");
+
+ if (taskManagerList != null && taskManagerList.size() > 0)
+ }
+ } catch (IOException ioe)
+
+ LOG.info("Waiting for dispatcher REST endpoint to come up...");
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + }+ }
+ throw new AssertionError("Dispatcher REST endpoint did not start in time.", reportedException);
+ }
+
+ public void stopFlinkCluster() throws IOException
+
+ public void copyOptJarsToLib(String jarNamePattern) throws FileNotFoundException, IOException {
+ final Optional<Path> reporterJarOptional = Files.walk(opt)
+ .filter(path -> path.getFileName().toString().startsWith("flink-metrics-prometheus"))
+ .findFirst();
+ if (reporterJarOptional.isPresent())
else
{ + throw new FileNotFoundException("No jar could be found matching the pattern " + jarNamePattern + "."); + }+ }
+
+ public void appendConfiguration(Configuration config) throws IOException
+
+ public Stream<String> searchAllLogs(Pattern pattern, Function<Matcher, String> matchProcessor) throws IOException {
Review comment:
Can be a future JIRA, but it could make sense to have a separate class with log parsing utilities, instead of putting it here with the `startCluster`, `stopCluster`...
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230410636
##########
File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
##########
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.metrics.prometheus.tests;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.prometheus.PrometheusReporter;
+import org.apache.flink.tests.util.AutoClosableProcess;
+import org.apache.flink.tests.util.CommandLineWrapper;
+import org.apache.flink.tests.util.FlinkDistribution;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking;
+import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking;
+
+/**
+ * End-to-end test for the PrometheusReporter.
+ */
+public class PrometheusReporterEndToEndITCase extends TestLogger {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private static final String PROMETHEUS_FILE_NAME = "prometheus-2.4.3.linux-amd64";
+
+ private static final Pattern LOG_REPORTER_PORT_PATTERN = Pattern.compile(".Started PrometheusReporter HTTP server on port ([0-9]+).");
+
+ @Rule
+ public final FlinkDistribution dist = new FlinkDistribution();
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Test
+ public void testReporter() throws Exception {
+ LOG.info("starting test");
+ dist.copyOptJarsToLib("flink-metrics-prometheus");
+
+ final Configuration config = new Configuration();
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
+
+ dist.appendConfiguration(config);
+
+ final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
+ final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
+ final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
+ final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
+ final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
+ Files.createDirectory(tmpPrometheusDir);
+
+ runBlocking(
+ "Download of Prometheus",
+ Duration.ofMinutes(5),
+ CommandLineWrapper
+ .wget("https://github.com/prometheus/prometheus/releases/download/v2.4.3/" + prometheusArchive.getFileName())
Review comment:
Given that the version is already in the path, does it make sense to move the whole path or the part that includes the version to a class variable?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230410247
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try {
Review comment:
I find it a bit counter-intuitive that we stop the cluster in the `after`, although we have not started it in the `before`. I do not know the best solution, maybe setting a flag on `startCluster` and checking it in the `after` and if `true` then stopping the cluster. This is not the best solution but feel free to ignore this comment if you do not agree.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230408989
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(request).execute()) {
+ if (response.isSuccessful()) {
+ final String json = response.body().string();
+ final JsonNode taskManagerList = OBJECT_MAPPER.readTree(json)
+ .get("taskmanagers");
+
+ if (taskManagerList != null && taskManagerList.size() > 0)
+ }
+ } catch (IOException ioe)
+
+ LOG.info("Waiting for dispatcher REST endpoint to come up...");
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + }+ }
+ throw new AssertionError("Dispatcher REST endpoint did not start in time.", reportedException);
+ }
+
+ public void stopFlinkCluster() throws IOException
+
+ public void copyOptJarsToLib(String jarNamePattern) throws FileNotFoundException, IOException {
+ final Optional<Path> reporterJarOptional = Files.walk(opt)
+ .filter(path -> path.getFileName().toString().startsWith("flink-metrics-prometheus"))
Review comment:
Here you forgot to replace the `flink-metrics-prometheus` with `jarNamePattern`.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230408436
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(request).execute()) {
Review comment:
Should we wait here for all Task Managers to be up before reporting that the cluster is up and running? This can also be a future JIRA though.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230407576
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException {
+ defaultConfig = new UnmodifiableConfiguration(GlobalConfiguration.loadConfiguration(conf.toAbsolutePath().toString()));
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
Review comment:
I would put the line 95-98 to a separate method (e.g. `backupOriginal/DefaultConfig`) and call it also in the `before()`, because this method can be also useful on its own.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230407896
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
Review comment:
As before I would create the method `restoreDefaultConfig` and also call it here.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230405669
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
Review comment:
These can move under the `LOG`, with the rest of the `static final` fields.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230404317
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
Review comment:
This is like a `E2ETestHarness` more than a `FlinkDistribution` right?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230414679
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Utility class for setting up command-line tool usages in a readable fashion.
+ */
+public enum CommandLineWrapper {
+ ;
+
+ public static WGetBuilder wget(String url)
+
+ /**
+ * Wrapper around wget used for downloading files.
+ */
+ public static final class WGetBuilder {
+
+ private final String url;
+ private Path targetDir;
+
+ WGetBuilder(String url)
+
+ public WGetBuilder targetDir(Path dir) {
Review comment:
it's harder to extend as with a variety of options you'll end up with a bloated constructor.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230415501
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
Review comment:
eh, I prefer `FlinkDistributionResource` since this class should really only provide primitives operations that map closely to `flink-dist`. Anything more complex beyond that belongs into a separate class.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230416606
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException {
+ defaultConfig = new UnmodifiableConfiguration(GlobalConfiguration.loadConfiguration(conf.toAbsolutePath().toString()));
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
Review comment:
There's no use-case for multiple backups. The backup only exists to reset flink-dist to the state before the test.
Tests can already setup multiple configs for separate clusters by calling `appendConfiguration` multiple times.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230417518
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(request).execute()) {
Review comment:
This behavior is in sync with the current bash implementation. start-cluster by default only starts 1 TM, so as it stands this is sufficient to guarantee a fully started cluster.
If more TMs are desired we can extend the class to support calls to `taskmanager.sh start` later on.
If we want to support starting a cluster with as many TMs as desired with a single call we'll first have to look at how we could determine how many TMs we should wait for.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230417575
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(request).execute()) {
+ if (response.isSuccessful()) {
+ final String json = response.body().string();
+ final JsonNode taskManagerList = OBJECT_MAPPER.readTree(json)
+ .get("taskmanagers");
+
+ if (taskManagerList != null && taskManagerList.size() > 0)
+ }
+ } catch (IOException ioe)
+
+ LOG.info("Waiting for dispatcher REST endpoint to come up...");
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + }+ }
+ throw new AssertionError("Dispatcher REST endpoint did not start in time.", reportedException);
+ }
+
+ public void stopFlinkCluster() throws IOException
+
+ public void copyOptJarsToLib(String jarNamePattern) throws FileNotFoundException, IOException {
+ final Optional<Path> reporterJarOptional = Files.walk(opt)
+ .filter(path -> path.getFileName().toString().startsWith("flink-metrics-prometheus"))
Review comment:
yes
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230418196
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(request).execute()) {
+ if (response.isSuccessful()) {
+ final String json = response.body().string();
+ final JsonNode taskManagerList = OBJECT_MAPPER.readTree(json)
+ .get("taskmanagers");
+
+ if (taskManagerList != null && taskManagerList.size() > 0)
+ }
+ } catch (IOException ioe)
+
+ LOG.info("Waiting for dispatcher REST endpoint to come up...");
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + }+ }
+ throw new AssertionError("Dispatcher REST endpoint did not start in time.", reportedException);
+ }
+
+ public void stopFlinkCluster() throws IOException
+
+ public void copyOptJarsToLib(String jarNamePattern) throws FileNotFoundException, IOException {
+ final Optional<Path> reporterJarOptional = Files.walk(opt)
+ .filter(path -> path.getFileName().toString().startsWith("flink-metrics-prometheus"))
+ .findFirst();
+ if (reporterJarOptional.isPresent())
else
{ + throw new FileNotFoundException("No jar could be found matching the pattern " + jarNamePattern + "."); + }+ }
+
+ public void appendConfiguration(Configuration config) throws IOException
+
+ public Stream<String> searchAllLogs(Pattern pattern, Function<Matcher, String> matchProcessor) throws IOException {
Review comment:
No other class will get access to the log path, and as such this is the only place to put it. All accesses to flink-dist will go through this, without exposing any resources contained within to the outside as this will only cause problems.
Effectively `FlinkDistribution` acts as a monitor to interactions with flink-dist.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230420018
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try {
Review comment:
This is purely for safety and as such I'd like to keep it without any condition.
I can see the point, and maybe we should return a `Closable` in `startCluster` to allow better management in the test.
`FlinkDistribution` does not represent a flink cluster but `flink-dist`; it thus doesn't make sense to start a cluster in `before`.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230420322
##########
File path: flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
##########
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.metrics.prometheus.tests;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.prometheus.PrometheusReporter;
+import org.apache.flink.tests.util.AutoClosableProcess;
+import org.apache.flink.tests.util.CommandLineWrapper;
+import org.apache.flink.tests.util.FlinkDistribution;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking;
+import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking;
+
+/**
+ * End-to-end test for the PrometheusReporter.
+ */
+public class PrometheusReporterEndToEndITCase extends TestLogger {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private static final String PROMETHEUS_FILE_NAME = "prometheus-2.4.3.linux-amd64";
+
+ private static final Pattern LOG_REPORTER_PORT_PATTERN = Pattern.compile(".Started PrometheusReporter HTTP server on port ([0-9]+).");
+
+ @Rule
+ public final FlinkDistribution dist = new FlinkDistribution();
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Test
+ public void testReporter() throws Exception {
+ LOG.info("starting test");
+ dist.copyOptJarsToLib("flink-metrics-prometheus");
+
+ final Configuration config = new Configuration();
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
+
+ dist.appendConfiguration(config);
+
+ final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
+ final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
+ final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
+ final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
+ final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
+ Files.createDirectory(tmpPrometheusDir);
+
+ runBlocking(
+ "Download of Prometheus",
+ Duration.ofMinutes(5),
+ CommandLineWrapper
+ .wget("https://github.com/prometheus/prometheus/releases/download/v2.4.3/" + prometheusArchive.getFileName())
Review comment:
hmm...maybe but I doubt we'll change the version anytime soon.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230425448
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(request).execute()) {
+ if (response.isSuccessful()) {
+ final String json = response.body().string();
+ final JsonNode taskManagerList = OBJECT_MAPPER.readTree(json)
+ .get("taskmanagers");
+
+ if (taskManagerList != null && taskManagerList.size() > 0)
+ }
+ } catch (IOException ioe)
+
+ LOG.info("Waiting for dispatcher REST endpoint to come up...");
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + }+ }
+ throw new AssertionError("Dispatcher REST endpoint did not start in time.", reportedException);
+ }
+
+ public void stopFlinkCluster() throws IOException
+
+ public void copyOptJarsToLib(String jarNamePattern) throws FileNotFoundException, IOException {
+ final Optional<Path> reporterJarOptional = Files.walk(opt)
+ .filter(path -> path.getFileName().toString().startsWith("flink-metrics-prometheus"))
+ .findFirst();
+ if (reporterJarOptional.isPresent())
else
{ + throw new FileNotFoundException("No jar could be found matching the pattern " + jarNamePattern + "."); + }+ }
+
+ public void appendConfiguration(Configuration config) throws IOException
+
+ public Stream<String> searchAllLogs(Pattern pattern, Function<Matcher, String> matchProcessor) throws IOException {
Review comment:
In the long-term I'd like to get rid of log-access completely.
If you cannot write a test without accessing a log it usually highlights an issue with how information is exposed to the user. IMO logs should be last place to get information, with the client/WebUI/REST API being the first.
In this case for example it shouldn't be necessary for me to iterate over all logs to find out which ports are being used by the reporter, this should be exposed as meta information for each JM/TM by the REST API.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230429154
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(request).execute()) {
+ if (response.isSuccessful()) {
+ final String json = response.body().string();
+ final JsonNode taskManagerList = OBJECT_MAPPER.readTree(json)
+ .get("taskmanagers");
+
+ if (taskManagerList != null && taskManagerList.size() > 0)
+ }
+ } catch (IOException ioe)
+
+ LOG.info("Waiting for dispatcher REST endpoint to come up...");
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + }+ }
+ throw new AssertionError("Dispatcher REST endpoint did not start in time.", reportedException);
+ }
+
+ public void stopFlinkCluster() throws IOException
+
+ public void copyOptJarsToLib(String jarNamePattern) throws FileNotFoundException, IOException {
+ final Optional<Path> reporterJarOptional = Files.walk(opt)
+ .filter(path -> path.getFileName().toString().startsWith("flink-metrics-prometheus"))
+ .findFirst();
+ if (reporterJarOptional.isPresent())
else
{ + throw new FileNotFoundException("No jar could be found matching the pattern " + jarNamePattern + "."); + }+ }
+
+ public void appendConfiguration(Configuration config) throws IOException
+
+ public Stream<String> searchAllLogs(Pattern pattern, Function<Matcher, String> matchProcessor) throws IOException {
Review comment:
I totally agree with you! But it is a pretty big effort to achieve that and also even using REST for now is not that safe, as updates happen asynchronously.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on issue #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#issuecomment-435437266
Btw, two more general notes:
1) it would be nice to have more javadocs
2) I would also suggest (slowly but steadily) to move away from `sed` or `wget` and stuff (in future JIRAs).
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u edited a comment on issue #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#issuecomment-435437266
Btw, three more notes:
1) there should be a discussion about how a testing framework should look like in the future, so that we are all on-board
2) it would be nice to have more javadocs
3) I would also suggest (slowly but steadily) to move away from `sed` or `wget` and stuff (in future JIRAs).
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on issue #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#issuecomment-435440138
And one more, a more in-depth discussion and a design doc could be very helpful to correctly design a testing framework for e2e testing.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r230443206
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(request).execute()) {
+ if (response.isSuccessful()) {
+ final String json = response.body().string();
+ final JsonNode taskManagerList = OBJECT_MAPPER.readTree(json)
+ .get("taskmanagers");
+
+ if (taskManagerList != null && taskManagerList.size() > 0)
+ }
+ } catch (IOException ioe)
+
+ LOG.info("Waiting for dispatcher REST endpoint to come up...");
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + }+ }
+ throw new AssertionError("Dispatcher REST endpoint did not start in time.", reportedException);
+ }
+
+ public void stopFlinkCluster() throws IOException
+
+ public void copyOptJarsToLib(String jarNamePattern) throws FileNotFoundException, IOException {
+ final Optional<Path> reporterJarOptional = Files.walk(opt)
+ .filter(path -> path.getFileName().toString().startsWith("flink-metrics-prometheus"))
+ .findFirst();
+ if (reporterJarOptional.isPresent())
else
{ + throw new FileNotFoundException("No jar could be found matching the pattern " + jarNamePattern + "."); + }+ }
+
+ public void appendConfiguration(Configuration config) throws IOException
+
+ public Stream<String> searchAllLogs(Pattern pattern, Function<Matcher, String> matchProcessor) throws IOException {
Review comment:
updates happening asynchronously isn't an issue in regards to safety but of test duration. There are already tests querying metrics via the REST API.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol commented on issue #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#issuecomment-435447361
For one this was an experiment; I hated writing this test in bash, so instead I ported it to java. We may use it for future tests, we may rework it entirely. But from what I've seen things get done significantly faster if at least you have some starting point. The majority of the code here is sane and easily re-usable in whatever direction we want to go with. The mistake we did in the past was to jump ahead at the first stuff we came up with (our bash stuff) and not critically analyze it when doing so.
Furthermore, I don't consider this a framework. It's too simply for that, all it adds is a programmatic way to interact with flink-dist. That's it.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
kl0u commented on a change in pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003#discussion_r231072668
##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
##########
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
Review comment:
I would still put that as a `final` field with a descriptive name.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
zentol closed pull request #7003: FLINK-10633[prometheus] Add E2E test
URL: https://github.com/apache/flink/pull/7003
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml
new file mode 100644
index 00000000000..1ef4a1bd954
— /dev/null
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml
@@ -0,0 +1,63 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <parent>
+ <artifactId>flink-end-to-end-tests</artifactId>
+ <groupId>org.apache.flink</groupId>
+ <version>1.8-SNAPSHOT</version>
+ </parent>
+
+ <modelVersion>4.0.0</modelVersion>
+
+ <artifactId>flink-end-to-end-tests-common</artifactId>
+ <version>1.8-SNAPSHOT</version>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-core</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.squareup.okhttp3</groupId>
+ <artifactId>okhttp</artifactId>
+ <version>3.7.0</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-shaded-jackson</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>${junit.version}</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <!-- To ensure that flink-dist is built beforehand -->
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-dist_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ </dependencies>
+
+</project>
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.java
new file mode 100644
index 00000000000..0098889ead6
— /dev/null
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.nio.file.Path;
+
+/**
+ * Utility class to delete a given
when exiting a try-with-resources statement.
+ */
+public final class AutoClosablePath implements AutoCloseable {
+
+ private final Path path;
+
+ public AutoClosablePath(final Path path)
+
+ @Override
+ public void close() throws IOException
+}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java
new file mode 100644
index 00000000000..02359302e13
— /dev/null
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Utility class to terminate a given
when exiting a try-with-resources statement.
+ */
+public class AutoClosableProcess implements AutoCloseable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AutoClosableProcess.class);
+
+ private final Process process;
+
+ public AutoClosableProcess(final Process process)
+
+ public static AutoClosableProcess runNonBlocking(String step, String... commands) throws IOException
+
+ public static void runBlocking(String step, String... commands) throws IOException
+
+ public static void runBlocking(String step, Duration timeout, String... commands) throws IOException {
+ LOG.info("Step started: " + step);
+ Process process = new ProcessBuilder()
+ .command(commands)
+ .inheritIO()
+ .start();
+
+ try (AutoClosableProcess autoProcess = new AutoClosableProcess(process)) {
+ final boolean success = process.waitFor(timeout.toMillis(), TimeUnit.MILLISECONDS);
+ if (!success)
+ } catch (TimeoutException | InterruptedException e)
{ + throw new RuntimeException(step + " failed due to timeout."); + }+ LOG.info("Step complete: " + step);
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (process.isAlive()) {
+ process.destroy();
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + }+ }
+ }
+}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java
new file mode 100644
index 00000000000..50fd2f81f02
— /dev/null
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Utility class for setting up command-line tool usages in a readable fashion.
+ */
+public enum CommandLineWrapper {
+ ;
+
+ public static WGetBuilder wget(String url)
+
+ /**
+ * Wrapper around wget used for downloading files.
+ */
+ public static final class WGetBuilder {
+
+ private final String url;
+ private Path targetDir;
+
+ WGetBuilder(String url)
+
+ public WGetBuilder targetDir(Path dir)
+
+ public String[] build() {
+ final List<String> commandsList = new ArrayList<>(5);
+ commandsList.add("wget");
+ commandsList.add("-q"); // silent
+ //commandsList.add("--show-progress"); // enable progress bar
+ if (targetDir != null)
+ commandsList.add(url);
+ return commandsList.toArray(new String[commandsList.size()]);
+ }
+ }
+
+ public static SedBuilder sed(final String command, final Path file)
+
+ /**
+ * Wrapper around sed used for processing text.
+ */
+ public static final class SedBuilder {
+
+ private final String command;
+ private final Path file;
+
+ private boolean inPlace = false;
+
+ SedBuilder(final String command, final Path file)
+
+ public SedBuilder inPlace()
+
+ public String[] build() {
+ final List<String> commandsList = new ArrayList<>(5);
+ commandsList.add("sed");
+ if (inPlace)
+ commandsList.add("-e");
+ commandsList.add(command);
+ commandsList.add(file.toAbsolutePath().toString());
+ return commandsList.toArray(new String[commandsList.size()]);
+ }
+ }
+
+ public static TarBuilder tar(final Path file)
+
+ /**
+ * Wrapper around tar used for extracting .tar archives.
+ */
+ public static final class TarBuilder {
+
+ private final Path file;
+ private boolean zipped = false;
+ private boolean extract = false;
+ private Path targetDir;
+
+ public TarBuilder(final Path file)
+
+ public TarBuilder zipped()
+
+ public TarBuilder extract()
+
+ public TarBuilder targetDir(final Path dir)
+
+ public String[] build() {
+ final List<String> commandsList = new ArrayList<>(4);
+ commandsList.add("tar");
+ if (zipped)
+ if (extract)
{ + commandsList.add("-x"); + }+ if (targetDir != null)
{ + commandsList.add("--directory"); + commandsList.add(targetDir.toAbsolutePath().toString()); + }+ commandsList.add("-f");
+ commandsList.add(file.toAbsolutePath().toString());
+ return commandsList.toArray(new String[commandsList.size()]);
+ }
+ }
+}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
new file mode 100644
index 00000000000..7031d88c890
— /dev/null
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.util;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assert;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A wrapper around a Flink distribution.
+ */
+public final class FlinkDistribution extends ExternalResource {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml");
+ private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak");
+
+ private final List<AutoClosablePath> filesToDelete = new ArrayList<>(4);
+
+ private final Path opt;
+ private final Path lib;
+ private final Path conf;
+ private final Path log;
+ private final Path bin;
+
+ private Configuration defaultConfig;
+
+ public FlinkDistribution() {
+ final String distDirProperty = System.getProperty("distDir");
+ if (distDirProperty == null)
+ final Path flinkDir = Paths.get(distDirProperty);
+ bin = flinkDir.resolve("bin");
+ opt = flinkDir.resolve("opt");
+ lib = flinkDir.resolve("lib");
+ conf = flinkDir.resolve("conf");
+ log = flinkDir.resolve("log");
+ }
+
+ @Override
+ protected void before() throws IOException
+
+ @Override
+ protected void after() {
+ try
catch (IOException e)
{ + LOG.error("Failure while shutting down Flink cluster.", e); + }+
+ final Path originalConfig = conf.resolve(FLINK_CONF_YAML);
+ final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+
+ try
catch (IOException e)
{ + LOG.error("Failed to restore flink-conf.yaml", e); + }+
+ for (AutoCloseable fileToDelete : filesToDelete) {
+ try
catch (Exception e)
{ + LOG.error("Failure while cleaning up file.", e); + }+ }
+ }
+
+ public void startFlinkCluster() throws IOException {
+ AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString());
+
+ final OkHttpClient client = new OkHttpClient();
+
+ final Request request = new Request.Builder()
+ .get()
+ .url("http://localhost:8081/taskmanagers")
+ .build();
+
+ Exception reportedException = null;
+ for (int retryAttempt = 0; retryAttempt < 30; retryAttempt++) {
+ try (Response response = client.newCall(request).execute()) {
+ if (response.isSuccessful()) {
+ final String json = response.body().string();
+ final JsonNode taskManagerList = OBJECT_MAPPER.readTree(json)
+ .get("taskmanagers");
+
+ if (taskManagerList != null && taskManagerList.size() > 0)
+ }
+ } catch (IOException ioe)
+
+ LOG.info("Waiting for dispatcher REST endpoint to come up...");
+ try
catch (InterruptedException e)
{ + Thread.currentThread().interrupt(); + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + }+ }
+ throw new AssertionError("Dispatcher REST endpoint did not start in time.", reportedException);
+ }
+
+ public void stopFlinkCluster() throws IOException
+
+ public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException {
+ final Optional<Path> reporterJarOptional = Files.walk(opt)
+ .filter(path -> path.getFileName().toString().startsWith(jarNamePrefix))
+ .findFirst();
+ if (reporterJarOptional.isPresent())
else
{ + throw new FileNotFoundException("No jar could be found matching the pattern " + jarNamePrefix + "."); + }+ }
+
+ public void appendConfiguration(Configuration config) throws IOException
+
+ public Stream<String> searchAllLogs(Pattern pattern, Function<Matcher, String> matchProcessor) throws IOException {
+ final List<String> matches = new ArrayList<>(2);
+
+ try (Stream<Path> logFilesStream = Files.list(log)) {
+ final Iterator<Path> logFiles = logFilesStream.iterator();
+ while (logFiles.hasNext()) {
+ final Path logFile = logFiles.next();
+ if (!logFile.getFileName().toString().endsWith(".log"))
+ try (BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(logFile.toFile()), StandardCharsets.UTF_8))) {
+ String line;
+ while ((line = br.readLine()) != null) {
+ Matcher matcher = pattern.matcher(line);
+ if (matcher.matches())
+ }
+ }
+ }
+ }
+ return matches.stream();
+ }
+}
diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml
new file mode 100644
index 00000000000..1efb40ed38e
— /dev/null
+++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml
@@ -0,0 +1,104 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <parent>
+ <artifactId>flink-end-to-end-tests</artifactId>
+ <groupId>org.apache.flink</groupId>
+ <version>1.8-SNAPSHOT</version>
+ </parent>
+
+ <modelVersion>4.0.0</modelVersion>
+
+ <artifactId>flink-metrics-reporter-prometheus-test</artifactId>
+ <version>1.8-SNAPSHOT</version>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-core</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-metrics-prometheus</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.squareup.okhttp3</groupId>
+ <artifactId>okhttp</artifactId>
+ <version>3.7.0</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-shaded-jackson</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-end-to-end-tests-common</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-test-utils-junit</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <profiles>
+ <profile>
+ <id>e2e-prometheus</id>
+ <activation>
+ <property>
+ <name>e2e-metrics</name>
+ </property>
+ </activation>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>e2e-prometheus</id>
+ <phase>integration-test</phase>
+ <goals>
+ <goal>test</goal>
+ </goals>
+ <configuration>
+ <includes>
+ <include>**/ITCase.</include>
+ </includes>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
+ </profiles>
+
+</project>
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
new file mode 100644
index 00000000000..269754eaf69
— /dev/null
+++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.metrics.prometheus.tests;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.prometheus.PrometheusReporter;
+import org.apache.flink.tests.util.AutoClosableProcess;
+import org.apache.flink.tests.util.CommandLineWrapper;
+import org.apache.flink.tests.util.FlinkDistribution;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.OperatingSystem;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking;
+import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking;
+
+/**
+ * End-to-end test for the PrometheusReporter.
+ */
+public class PrometheusReporterEndToEndITCase extends TestLogger {
+
+ private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class);
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ private static final String PROMETHEUS_VERSION = "2.4.3";
+ private static final String PROMETHEUS_FILE_NAME;
+
+ static {
+ final String base = "prometheus-" + PROMETHEUS_VERSION + '.';
+ switch (OperatingSystem.getCurrentOperatingSystem())
+ }
+
+ private static final Pattern LOG_REPORTER_PORT_PATTERN = Pattern.compile(".Started PrometheusReporter HTTP server on port ([0-9]+).");
+
+ @BeforeClass
+ public static void checkOS()
+
+ @Rule
+ public final FlinkDistribution dist = new FlinkDistribution();
+
+ @Rule
+ public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Test
+ public void testReporter() throws Exception {
+ dist.copyOptJarsToLib("flink-metrics-prometheus");
+
+ final Configuration config = new Configuration();
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName());
+ config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100");
+
+ dist.appendConfiguration(config);
+
+ final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus");
+ final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz");
+ final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME);
+ final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml");
+ final Path prometheusBinary = prometheusBinDir.resolve("prometheus");
+ Files.createDirectory(tmpPrometheusDir);
+
+ runBlocking(
+ "Download of Prometheus",
+ Duration.ofMinutes(5),
+ CommandLineWrapper
+ .wget("https://github.com/prometheus/prometheus/releases/download/v" + PROMETHEUS_VERSION + '/' + prometheusArchive.getFileName())
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Extraction of Prometheus archive",
+ CommandLineWrapper
+ .tar(prometheusArchive)
+ .extract()
+ .zipped()
+ .targetDir(tmpPrometheusDir)
+ .build());
+
+ runBlocking("Set Prometheus scrape interval",
+ CommandLineWrapper
+ .sed("s/\\(scrape_interval:\\).*/
1 1s/", prometheusConfig)
+ .inPlace()
+ .build());
+
+ dist.startFlinkCluster();
+
+ final List<Integer> ports = dist
+ .searchAllLogs(LOG_REPORTER_PORT_PATTERN, matcher -> matcher.group(1))
+ .map(Integer::valueOf)
+ .collect(Collectors.toList());
+
+ final String scrapeTargets = ports.stream()
+ .map(port -> "'localhost:" + port + "'")
+ .collect(Collectors.joining(", "));
+
+ runBlocking("Set Prometheus scrape targets to (" + scrapeTargets + ")",
+ CommandLineWrapper
+ .sed("s/\\(targets:\\).*/
1 [" + scrapeTargets + "]/", prometheusConfig)
+ .inPlace()
+ .build());
+
+ try (AutoClosableProcess prometheus = runNonBlocking(
+ "Start Prometheus server",
+ prometheusBinary.toAbsolutePath().toString(),
+ "--config.file=" + prometheusConfig.toAbsolutePath(),
+ "--storage.tsdb.path=" + prometheusBinDir.resolve("data").toAbsolutePath()))
+ }
+
+ private static void checkMetricAvailability(final OkHttpClient client, final String metric) throws InterruptedException {
+ final Request jobManagerRequest = new Request.Builder()
+ .get()
+ .url("http://localhost:9090/api/v1/query?query=" + metric)
+ .build();
+
+ Exception reportedException = null;
+ for (int x = 0; x < 30; x++) {
+ try (Response response = client.newCall(jobManagerRequest).execute()) {
+ if (response.isSuccessful()) {
+ final String json = response.body().string();
+
+ // Sample response:
+ //{
+ // "status": "success",
+ // "data": {
+ // "resultType": "vector",
+ // "result": [{
+ // "metric":
,
+ // "value": [1540548500.107, "1"]
+ // }]
+ // }
+ //}
+ OBJECT_MAPPER.readTree(json)
+ .get("data")
+ .get("result")
+ .get(0)
+ .get("value")
+ .get(1).asInt();
+ // if we reach this point some value for the given metric was reported to prometheus
+ return;
+ } else
+ } catch (Exception e)
{ + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + Thread.sleep(1000); + }+ }
+ throw new AssertionError("Could not retrieve metric " + metric + " from Prometheus.", reportedException);
+ }
+}
diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/resources/log4j-test.properties b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/resources/log4j-test.properties
new file mode 100644
index 00000000000..f7425cd14c0
— /dev/null
+++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/resources/log4j-test.properties
@@ -0,0 +1,27 @@
+################################################################################
+# 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.
+################################################################################
+
+# Set root logger level to OFF to not flood build logs
+# set manually to INFO for debugging purposes
+log4j.rootLogger=INFO, testlogger
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%m%n
diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml
index 07b65f9a1e3..4f132f923b9 100644
— a/flink-end-to-end-tests/pom.xml
+++ b/flink-end-to-end-tests/pom.xml
@@ -56,6 +56,8 @@ under the License.
<module>flink-streaming-file-sink-test</module>
<module>flink-state-evolution-test</module>
<module>flink-e2e-test-utils</module>
+ <module>flink-end-to-end-tests-common</module>
+ <module>flink-metrics-reporter-prometheus-test</module>
</modules>
<build>
@@ -67,6 +69,22 @@ under the License.
<skip>true</skip>
</configuration>
</plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <executions>
+ <!-- Do not execute any tests by default.
+ E2E tests should specify an additional execution within a profile. -->
+ <execution>
+ <id>default-test</id>
+ <phase>none</phase>
+ </execution>
+ <execution>
+ <id>integration-tests</id>
+ <phase>none</phase>
+ </execution>
+ </executions>
+ </plugin>
</plugins>
<pluginManagement>
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
master: ff744155f5e00c711ed4d6e764e99623acafdf50
1.7: 0ea67408744ca33542c8bc04ae4d1857a241fa3a
zentol opened a new pull request #7003:
FLINK-10633[prometheus] Add E2E testURL: https://github.com/apache/flink/pull/7003
This PR adds an end-to-end test for the prometheus reporter. The test starts prometheus and a cluster of 1 JM and 1 TM and verifies that metrics for both are reported to prometheus.
This test follows a new paradigm for writing end-to-end tests in that is decoupled from our bash framework and primarily based on java.
The test is written as a plain jUnit test. This means we can use the same classes/tools/designs etc. that we usually apply to our tests, along with the benefits of being able to run it in the IDE and maven, the first one making debugging significantly easier, the latter one giving us plenty of features for free, like measuring the test duration or exiting early on test failures, which we have painstakingly re-implemented in bash ourselves.
Interactions with `flink-dist` are encapsulated in a `FlinkDistribution` resource. The resource ensures that all /opt jars copied in /lib are deleted on exit, that the cluster is shutdown and that the configuration is reversed.
The test is not run by default and can be enabled by specifying the `e2e-metrics` property when executing maven. Furthermore the `distDir´ property must be set pointing to the location of flink-dist, similar to the `FLINK_DIR` variable for bash tests.
This branch illustrates how the new tests can (and supposedly will) be run on travis as part of the e2e tests: https://github.com/zentol/flink/tree/cron-master-e2e-with-prom
See the linked `cron-master-e2e-with-prom` branch above.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
users@infra.apache.org