[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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 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) {
+   Assert.fail("The distDir property was not set. You can 
set it when running maven via -DdistDir= .");
+   }
+   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);
+   final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+   Files.copy(originalConfig, backupConfig);
+   filesToDelete.add(new AutoClosablePath(backupConfig));
+   }
+
+   @Override
+   protected void after() {
+   try {
+   stopFlinkCluster();
+   } 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 {
+   Files.move(backupConfig, originalConfig, 
StandardCopyOption.REPLACE_EXISTING);
+   } catch (IOException e) {
+   LOG.error("Failed to restore flink-conf.yaml", e);
+   }
+
+  

[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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 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) {
+   Assert.fail("The distDir property was not set. You can 
set it when running maven via -DdistDir= .");
+   }
+   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);
+   final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+   Files.copy(originalConfig, backupConfig);
+   filesToDelete.add(new AutoClosablePath(backupConfig));
+   }
+
+   @Override
+   protected void after() {
+   try {
+   stopFlinkCluster();
+   } 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 {
+   Files.move(backupConfig, originalConfig, 
StandardCopyOption.REPLACE_EXISTING);
+   } catch (IOException e) {
+   LOG.error("Failed to restore flink-conf.yaml", e);
+   }
+
+  

[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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 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) {
+   Assert.fail("The distDir property was not set. You can 
set it when running maven via -DdistDir= .");
+   }
+   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);
+   final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+   Files.copy(originalConfig, backupConfig);
+   filesToDelete.add(new AutoClosablePath(backupConfig));
+   }
+
+   @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 

[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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 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) {
+   Assert.fail("The distDir property was not set. You can 
set it when running maven via -DdistDir= .");
+   }
+   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);
+   final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+   Files.copy(originalConfig, backupConfig);
+   filesToDelete.add(new AutoClosablePath(backupConfig));
+   }
+
+   @Override
+   protected void after() {
+   try {
+   stopFlinkCluster();
+   } 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 {
+   Files.move(backupConfig, originalConfig, 
StandardCopyOption.REPLACE_EXISTING);
+   } catch (IOException e) {
+   LOG.error("Failed to restore flink-conf.yaml", e);
+   }
+
+  

[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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 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) {
+   Assert.fail("The distDir property was not set. You can 
set it when running maven via -DdistDir= .");
+   }
+   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);
+   final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+   Files.copy(originalConfig, backupConfig);
+   filesToDelete.add(new AutoClosablePath(backupConfig));
+   }
+
+   @Override
+   protected void after() {
+   try {
+   stopFlinkCluster();
+   } 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 {
+   Files.move(backupConfig, originalConfig, 
StandardCopyOption.REPLACE_EXISTING);
+   } catch (IOException e) {
+   LOG.error("Failed to restore flink-conf.yaml", e);
+   }
+
+  

[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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 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) {
+   Assert.fail("The distDir property was not set. You can 
set it when running maven via -DdistDir= .");
+   }
+   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);
+   final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP);
+   Files.copy(originalConfig, backupConfig);
+   filesToDelete.add(new AutoClosablePath(backupConfig));
+   }
+
+   @Override
+   protected void after() {
+   try {
+   stopFlinkCluster();
+   } 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 {
+   Files.move(backupConfig, originalConfig, 
StandardCopyOption.REPLACE_EXISTING);
+   } catch (IOException e) {
+   LOG.error("Failed to restore flink-conf.yaml", e);
+   }
+
+  

[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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 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) {
+   Assert.fail("The distDir property was not set. You can 
set it when running maven via -DdistDir= .");
+   }
+   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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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) {
+   return new WGetBuilder(url);
+   }
+
+   /**
+* Wrapper around wget used for downloading files.
+*/
+   public static final class WGetBuilder {
+
+   private final String url;
+   private Path targetDir;
+
+   WGetBuilder(String url) {
+   this.url = 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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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());
+
+   

[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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());
+
+   

[GitHub] zentol commented on a change in pull request #7003: [FLINK-10633][prometheus] Add E2E test

2018-11-02 Thread GitBox
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());
+
+