You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2018/09/07 13:36:15 UTC

[flink] branch release-1.6 updated: [FLINK-10293][streaming] Properly forward REST port for remote environments

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

chesnay pushed a commit to branch release-1.6
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.6 by this push:
     new d8cc768  [FLINK-10293][streaming] Properly forward REST port for remote environments
d8cc768 is described below

commit d8cc7684f5ffbea782a9b3acb98759bf3c2913b0
Author: zentol <ch...@apache.org>
AuthorDate: Thu Sep 6 11:18:24 2018 +0200

    [FLINK-10293][streaming] Properly forward REST port for remote environments
---
 .../api/environment/RemoteStreamEnvironment.java   |  3 +
 .../RemoteStreamExecutionEnvironmentTest.java      | 87 ++++++++++++++++++++++
 2 files changed, 90 insertions(+)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index 480f981..9c36dab 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -29,6 +29,7 @@ import org.apache.flink.client.program.rest.RestClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.JobManagerOptions;
+import org.apache.flink.configuration.RestOptions;
 import org.apache.flink.streaming.api.graph.StreamGraph;
 
 import org.slf4j.Logger;
@@ -201,6 +202,8 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 		configuration.setString(JobManagerOptions.ADDRESS, host);
 		configuration.setInteger(JobManagerOptions.PORT, port);
 
+		configuration.setInteger(RestOptions.PORT, port);
+
 		final ClusterClient<?> client;
 		try {
 			if (CoreOptions.LEGACY_MODE.equals(configuration.getString(CoreOptions.MODE))) {
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java
new file mode 100644
index 0000000..60ee66f
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.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.streaming.api.environment;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Iterator;
+
+/**
+ * Tests for the {@link RemoteStreamEnvironment}.
+ */
+public class RemoteStreamExecutionEnvironmentTest extends TestLogger {
+
+	private static MiniCluster flink;
+
+	@BeforeClass
+	public static void setUp() throws Exception {
+		final Configuration config = new Configuration();
+		config.setInteger(RestOptions.PORT, 0);
+
+		final MiniClusterConfiguration miniClusterConfiguration = new MiniClusterConfiguration.Builder()
+			.setConfiguration(config)
+			.setNumTaskManagers(1)
+			.setNumSlotsPerTaskManager(1)
+			.build();
+
+		flink = new MiniCluster(miniClusterConfiguration);
+
+		flink.start();
+	}
+
+	@AfterClass
+	public static void tearDown() throws Exception {
+		if (flink != null) {
+			flink.close();
+		}
+	}
+
+	/**
+	 * Verifies that the port passed to the RemoteStreamEnvironment is used for connecting to the cluster.
+	 */
+	@Test
+	public void testPortForwarding() throws Exception {
+		final Configuration clientConfiguration = new Configuration();
+		clientConfiguration.setInteger(RestOptions.RETRY_MAX_ATTEMPTS, 0);
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
+			flink.getRestAddress().getHost(),
+			flink.getRestAddress().getPort(),
+			clientConfiguration);
+
+		final DataStream<Integer> resultStream = env.fromElements(1)
+			.map(x -> x * 2);
+
+		final Iterator<Integer> result = DataStreamUtils.collect(resultStream);
+		Assert.assertTrue(result.hasNext());
+		Assert.assertEquals(2, result.next().intValue());
+		Assert.assertFalse(result.hasNext());
+	}
+}