You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2018/01/12 22:25:22 UTC

[01/14] flink git commit: [FLINK-8341] [flip6] Remove not needed options from CommandLineOptions

Repository: flink
Updated Branches:
  refs/heads/master 4ade82631 -> 3b0168685


[FLINK-8341] [flip6] Remove not needed options from CommandLineOptions

This closes #5227.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d7e9dc19
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d7e9dc19
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d7e9dc19

Branch: refs/heads/master
Commit: d7e9dc1931f1f1cedbfee12aebe34dd76e9bac10
Parents: 30011b9
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Dec 20 16:32:18 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:02 2018 +0100

----------------------------------------------------------------------
 .../flink/client/cli/CommandLineOptions.java    | 16 -----------
 .../org/apache/flink/api/scala/FlinkShell.scala | 28 ++++++++++++--------
 2 files changed, 17 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d7e9dc19/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java
index a9a29b2..1b57620 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CommandLineOptions.java
@@ -20,7 +20,6 @@ package org.apache.flink.client.cli;
 
 import org.apache.commons.cli.CommandLine;
 
-import static org.apache.flink.client.cli.CliFrontendParser.ADDRESS_OPTION;
 import static org.apache.flink.client.cli.CliFrontendParser.HELP_OPTION;
 
 /**
@@ -29,28 +28,13 @@ import static org.apache.flink.client.cli.CliFrontendParser.HELP_OPTION;
  */
 public abstract class CommandLineOptions {
 
-	private final CommandLine commandLine;
-
-	private final String jobManagerAddress;
-
 	private final boolean printHelp;
 
 	protected CommandLineOptions(CommandLine line) {
-		this.commandLine = line;
 		this.printHelp = line.hasOption(HELP_OPTION.getOpt());
-		this.jobManagerAddress = line.hasOption(ADDRESS_OPTION.getOpt()) ?
-				line.getOptionValue(ADDRESS_OPTION.getOpt()) : null;
-	}
-
-	public CommandLine getCommandLine() {
-		return commandLine;
 	}
 
 	public boolean isPrintHelp() {
 		return printHelp;
 	}
-
-	public String getJobManagerAddress() {
-		return jobManagerAddress;
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d7e9dc19/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
----------------------------------------------------------------------
diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
index 9f29ce0..349d940 100644
--- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
+++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
@@ -20,7 +20,7 @@ package org.apache.flink.api.scala
 
 import java.io._
 
-import org.apache.flink.client.cli.{CliFrontend, CliFrontendParser}
+import org.apache.flink.client.cli.{CliFrontend, CliFrontendParser, RunOptions}
 import org.apache.flink.client.program.ClusterClient
 import org.apache.flink.configuration.{Configuration, GlobalConfiguration, JobManagerOptions}
 import org.apache.flink.runtime.minicluster.StandaloneMiniCluster
@@ -253,16 +253,19 @@ object FlinkShell {
     yarnConfig.queue.foreach((queue) => args ++= Seq("-yqu", queue.toString))
     yarnConfig.slots.foreach((slots) => args ++= Seq("-ys", slots.toString))
 
-    val options = CliFrontendParser.parseRunCommand(args.toArray)
+    val commandLine = CliFrontendParser.parse(
+      CliFrontendParser.getRunCommandOptions,
+      args.toArray,
+      true)
+
     val frontend = new CliFrontend(
       configuration,
       CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
-    val config = frontend.getConfiguration
-    val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
+    val customCLI = frontend.getActiveCustomCommandLine(commandLine)
 
-    val clusterDescriptor = customCLI.createClusterDescriptor(options.getCommandLine)
+    val clusterDescriptor = customCLI.createClusterDescriptor(commandLine)
 
-    val clusterSpecification = customCLI.getClusterSpecification(options.getCommandLine)
+    val clusterSpecification = customCLI.getClusterSpecification(commandLine)
 
     val cluster = clusterDescriptor.deploySessionCluster(clusterSpecification)
 
@@ -281,16 +284,19 @@ object FlinkShell {
       "-m", "yarn-cluster"
     )
 
-    val options = CliFrontendParser.parseRunCommand(args.toArray)
+    val commandLine = CliFrontendParser.parse(
+      CliFrontendParser.getRunCommandOptions,
+      args.toArray,
+      true)
+
     val frontend = new CliFrontend(
       configuration,
       CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
-    val config = frontend.getConfiguration
-    val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
+    val customCLI = frontend.getActiveCustomCommandLine(commandLine)
 
-    val clusterDescriptor = customCLI.createClusterDescriptor(options.getCommandLine)
+    val clusterDescriptor = customCLI.createClusterDescriptor(commandLine)
 
-    val clusterId = customCLI.getClusterId(options.getCommandLine)
+    val clusterId = customCLI.getClusterId(commandLine)
 
     val cluster = clusterDescriptor.retrieve(clusterId)
 


[10/14] flink git commit: [FLINK-8119] [flip6] Wire correct Flip6 components in Flip6YarnClusterDescriptor

Posted by tr...@apache.org.
[FLINK-8119] [flip6] Wire correct Flip6 components in Flip6YarnClusterDescriptor

Let the Flip6YarnClusterDescriptor create a RestClusterClient as ClusterClient.
Moreover, this commit makes the YarnResourceManager register under the REST port
at Yarn.

This closes #5234.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/dbe0e828
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/dbe0e828
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/dbe0e828

Branch: refs/heads/master
Commit: dbe0e8286d76a5facdb49589b638b87dbde80178
Parents: 7d986ce
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Jan 3 20:38:21 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:05 2018 +0100

----------------------------------------------------------------------
 .../client/program/rest/RestClusterClient.java  | 12 ++--
 .../yarn/TestingYarnClusterDescriptor.java      |  2 +-
 .../java/org/apache/flink/yarn/YARNITCase.java  | 10 +--
 .../yarn/AbstractYarnClusterDescriptor.java     | 37 ++++++-----
 .../flink/yarn/Flip6YarnClusterDescriptor.java  | 66 ++++++++++++++++++++
 .../flink/yarn/YarnClusterDescriptor.java       | 14 +++++
 .../flink/yarn/YarnClusterDescriptorV2.java     | 49 ---------------
 .../apache/flink/yarn/YarnResourceManager.java  | 16 ++++-
 .../flink/yarn/cli/FlinkYarnSessionCli.java     | 17 +++--
 .../yarn/entrypoint/YarnEntrypointUtils.java    |  7 +++
 .../flink/yarn/AbstractYarnClusterTest.java     |  6 ++
 11 files changed, 150 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
index 3e8b136..4d8656b 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
@@ -318,28 +318,24 @@ public class RestClusterClient<T> extends ClusterClient<T> {
 		return false;
 	}
 
-	// ======================================
-	// Legacy stuff we ignore
-	// ======================================
-
 	@Override
 	public void waitForClusterToBeReady() {
-		throw new UnsupportedOperationException();
+		// no op
 	}
 
 	@Override
 	public String getWebInterfaceURL() {
-		throw new UnsupportedOperationException();
+		return "http://" + restClusterClientConfiguration.getRestServerAddress() + ':' + restClusterClientConfiguration.getRestServerPort();
 	}
 
 	@Override
 	public GetClusterStatusResponse getClusterStatus() {
-		throw new UnsupportedOperationException();
+		return null;
 	}
 
 	@Override
 	public List<String> getNewMessages() {
-		throw new UnsupportedOperationException();
+		return Collections.emptyList();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java
index e66d2e0..95a31be 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java
@@ -35,7 +35,7 @@ import java.util.List;
  * flink-yarn-tests-X-tests.jar and the flink-runtime-X-tests.jar to the set of files which
  * are shipped to the yarn cluster. This is necessary to load the testing classes.
  */
-public class TestingYarnClusterDescriptor extends AbstractYarnClusterDescriptor {
+public class TestingYarnClusterDescriptor extends YarnClusterDescriptor {
 
 	public TestingYarnClusterDescriptor(Configuration configuration, String configurationDirectory) {
 		super(

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java
index 069f68a..bea9001 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java
@@ -19,6 +19,7 @@
 package org.apache.flink.yarn;
 
 import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.AkkaOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -28,6 +29,7 @@ import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -55,13 +57,13 @@ public class YARNITCase extends YarnTestBase {
 		configuration.setString(AkkaOptions.ASK_TIMEOUT, "30 s");
 		final YarnClient yarnClient = YarnClient.createYarnClient();
 
-		try (final YarnClusterDescriptorV2 yarnClusterDescriptorV2 = new YarnClusterDescriptorV2(
+		try (final Flip6YarnClusterDescriptor flip6YarnClusterDescriptor = new Flip6YarnClusterDescriptor(
 			configuration,
 			System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR),
 			yarnClient)) {
 
-			yarnClusterDescriptorV2.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath()));
-			yarnClusterDescriptorV2.addShipFiles(Arrays.asList(flinkLibFolder.listFiles()));
+			flip6YarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath()));
+			flip6YarnClusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles()));
 
 			final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder()
 				.setMasterMemoryMB(768)
@@ -83,7 +85,7 @@ public class YARNITCase extends YarnTestBase {
 
 			jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI()));
 
-			YarnClusterClient clusterClient = yarnClusterDescriptorV2.deployJobCluster(clusterSpecification, jobGraph);
+			ClusterClient<ApplicationId> clusterClient = flip6YarnClusterDescriptor.deployJobCluster(clusterSpecification, jobGraph);
 
 			clusterClient.shutdown();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index a7e8c36..f015235 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -23,6 +23,7 @@ import org.apache.flink.client.deployment.ClusterDeploymentException;
 import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterRetrieveException;
 import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
@@ -30,6 +31,7 @@ import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.ResourceManagerOptions;
+import org.apache.flink.configuration.RestOptions;
 import org.apache.flink.configuration.SecurityOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
@@ -332,7 +334,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	// -------------------------------------------------------------
 
 	@Override
-	public YarnClusterClient retrieve(ApplicationId applicationId) throws ClusterRetrieveException {
+	public ClusterClient<ApplicationId> retrieve(ApplicationId applicationId) throws ClusterRetrieveException {
 
 		try {
 			// check if required Hadoop environment variables are set. If not, warn user
@@ -352,11 +354,17 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 				throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore.");
 			}
 
+			final String host = appReport.getHost();
+			final int rpcPort = appReport.getRpcPort();
+
 			LOG.info("Found application JobManager host name '{}' and port '{}' from supplied application id '{}'",
-				appReport.getHost(), appReport.getRpcPort(), applicationId);
+				host, rpcPort, applicationId);
+
+			flinkConfiguration.setString(JobManagerOptions.ADDRESS, host);
+			flinkConfiguration.setInteger(JobManagerOptions.PORT, rpcPort);
 
-			flinkConfiguration.setString(JobManagerOptions.ADDRESS, appReport.getHost());
-			flinkConfiguration.setInteger(JobManagerOptions.PORT, appReport.getRpcPort());
+			flinkConfiguration.setString(RestOptions.REST_ADDRESS, host);
+			flinkConfiguration.setInteger(RestOptions.REST_PORT, rpcPort);
 
 			return createYarnClusterClient(
 				this,
@@ -371,7 +379,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	}
 
 	@Override
-	public YarnClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException {
+	public ClusterClient<ApplicationId> deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException {
 		try {
 			return deployInternal(
 				clusterSpecification,
@@ -383,7 +391,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	}
 
 	@Override
-	public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) throws ClusterDeploymentException {
+	public ClusterClient<ApplicationId> deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) throws ClusterDeploymentException {
 		try {
 			return deployInternal(
 				clusterSpecification,
@@ -410,7 +418,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	 * @param clusterSpecification Initial cluster specification for the to be deployed Flink cluster
 	 * @param jobGraph A job graph which is deployed with the Flink cluster, null if none
 	 */
-	protected YarnClusterClient deployInternal(
+	protected ClusterClient<ApplicationId> deployInternal(
 			ClusterSpecification clusterSpecification,
 			String yarnClusterEntrypoint,
 			@Nullable JobGraph jobGraph) throws Exception {
@@ -488,6 +496,9 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		flinkConfiguration.setString(JobManagerOptions.ADDRESS, host);
 		flinkConfiguration.setInteger(JobManagerOptions.PORT, port);
 
+		flinkConfiguration.setString(RestOptions.REST_ADDRESS, host);
+		flinkConfiguration.setInteger(RestOptions.REST_PORT, port);
+
 		// the Flink cluster is deployed in YARN. Represent cluster
 		return createYarnClusterClient(
 			this,
@@ -1534,20 +1545,12 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	/**
 	 * Creates a YarnClusterClient; may be overriden in tests.
 	 */
-	protected YarnClusterClient createYarnClusterClient(
+	protected abstract ClusterClient<ApplicationId> createYarnClusterClient(
 			AbstractYarnClusterDescriptor descriptor,
 			int numberTaskManagers,
 			int slotsPerTaskManager,
 			ApplicationReport report,
 			org.apache.flink.configuration.Configuration flinkConfiguration,
-			boolean perJobCluster) throws Exception {
-		return new YarnClusterClient(
-			descriptor,
-			numberTaskManagers,
-			slotsPerTaskManager,
-			report,
-			flinkConfiguration,
-			perJobCluster);
-	}
+			boolean perJobCluster) throws Exception;
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java
new file mode 100644
index 0000000..e3f9646
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * 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.yarn;
+
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.rest.RestClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint;
+import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+
+/**
+ * Implementation of {@link org.apache.flink.yarn.AbstractYarnClusterDescriptor} which is used to start the
+ * new application master for a job under flip-6.
+ */
+public class Flip6YarnClusterDescriptor extends AbstractYarnClusterDescriptor {
+
+	public Flip6YarnClusterDescriptor(
+			Configuration flinkConfiguration,
+			String configurationDirectory,
+			YarnClient yarnCLient) {
+		super(flinkConfiguration, configurationDirectory, yarnCLient);
+	}
+
+	@Override
+	protected String getYarnSessionClusterEntrypoint() {
+		return YarnSessionClusterEntrypoint.class.getName();
+	}
+
+	@Override
+	protected String getYarnJobClusterEntrypoint() {
+		return YarnJobClusterEntrypoint.class.getName();
+	}
+
+	@Override
+	protected ClusterClient<ApplicationId> createYarnClusterClient(
+			AbstractYarnClusterDescriptor descriptor,
+			int numberTaskManagers,
+			int slotsPerTaskManager,
+			ApplicationReport report,
+			Configuration flinkConfiguration,
+			boolean perJobCluster) throws Exception {
+		return new RestClusterClient<>(
+			flinkConfiguration,
+			report.getApplicationId());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
index 76f9154..9467326 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
@@ -19,9 +19,12 @@
 package org.apache.flink.yarn;
 
 import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 
 /**
@@ -50,4 +53,15 @@ public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor {
 	public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
 		throw new UnsupportedOperationException("Cannot deploy a per-job yarn cluster yet.");
 	}
+
+	@Override
+	protected ClusterClient<ApplicationId> createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) throws Exception {
+		return new YarnClusterClient(
+			descriptor,
+			numberTaskManagers,
+			slotsPerTaskManager,
+			report,
+			flinkConfiguration,
+			perJobCluster);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java
deleted file mode 100644
index 6ce192c..0000000
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.yarn;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint;
-import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint;
-
-import org.apache.hadoop.yarn.client.api.YarnClient;
-
-/**
- * Implementation of {@link org.apache.flink.yarn.AbstractYarnClusterDescriptor} which is used to start the
- * new application master for a job under flip-6.
- */
-public class YarnClusterDescriptorV2 extends AbstractYarnClusterDescriptor {
-
-	public YarnClusterDescriptorV2(
-			Configuration flinkConfiguration,
-			String configurationDirectory,
-			YarnClient yarnCLient) {
-		super(flinkConfiguration, configurationDirectory, yarnCLient);
-	}
-
-	@Override
-	protected String getYarnSessionClusterEntrypoint() {
-		return YarnSessionClusterEntrypoint.class.getName();
-	}
-
-	@Override
-	protected String getYarnJobClusterEntrypoint() {
-		return YarnJobClusterEntrypoint.class.getName();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
index 5a71f41..0fa0dda 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java
@@ -177,7 +177,21 @@ public class YarnResourceManager extends ResourceManager<YarnWorkerNode> impleme
 		//TODO: change akka address to tcp host and port, the getAddress() interface should return a standard tcp address
 		Tuple2<String, Integer> hostPort = parseHostPort(getAddress());
 
-		resourceManagerClient.registerApplicationMaster(hostPort.f0, hostPort.f1, webInterfaceUrl);
+		final int restPort;
+
+		if (webInterfaceUrl != null) {
+			final int lastColon = webInterfaceUrl.lastIndexOf(':');
+
+			if (lastColon == -1) {
+				restPort = -1;
+			} else {
+				restPort = Integer.valueOf(webInterfaceUrl.substring(lastColon + 1));
+			}
+		} else {
+			restPort = -1;
+		}
+
+		resourceManagerClient.registerApplicationMaster(hostPort.f0, restPort, webInterfaceUrl);
 
 		return resourceManagerClient;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index fea8a5c..d4cf9ad 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -38,8 +38,8 @@ import org.apache.flink.util.ExecutorUtils;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.yarn.AbstractYarnClusterDescriptor;
+import org.apache.flink.yarn.Flip6YarnClusterDescriptor;
 import org.apache.flink.yarn.YarnClusterDescriptor;
-import org.apache.flink.yarn.YarnClusterDescriptorV2;
 import org.apache.flink.yarn.configuration.YarnConfigOptions;
 
 import org.apache.commons.cli.CommandLine;
@@ -522,7 +522,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId
 
 		for (Option option : commandLine.getOptions()) {
 			if (allOptions.hasOption(option.getOpt())) {
-				if (!option.getOpt().equals(detached.getOpt())) {
+				if (!option.getOpt().equals(detached.getOpt()) && !option.getOpt().equals(flip6.getOpt())) {
 					// don't resume from properties file if yarn options have been specified
 					canApplyYarnProperties = false;
 					break;
@@ -653,14 +653,19 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId
 
 						deleteYarnPropertiesFile();
 
+						ApplicationReport applicationReport;
+
 						try {
-							final ApplicationReport applicationReport = yarnClusterDescriptor
+							applicationReport = yarnClusterDescriptor
 								.getYarnClient()
 								.getApplicationReport(yarnApplicationId);
-
-							logFinalApplicationReport(applicationReport);
 						} catch (YarnException | IOException e) {
 							LOG.info("Could not log the final application report.", e);
+							applicationReport = null;
+						}
+
+						if (applicationReport != null) {
+							logFinalApplicationReport(applicationReport);
 						}
 					}
 				}
@@ -950,7 +955,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId
 	private static AbstractYarnClusterDescriptor getClusterDescriptor(Configuration configuration, String configurationDirectory, boolean flip6) {
 		final YarnClient yarnClient = YarnClient.createYarnClient();
 		if (flip6) {
-			return new YarnClusterDescriptorV2(configuration, configurationDirectory, yarnClient);
+			return new Flip6YarnClusterDescriptor(configuration, configurationDirectory, yarnClient);
 		} else {
 			return new YarnClusterDescriptor(configuration, configurationDirectory, yarnClient);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java
index 256056c..a03ed08 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java
@@ -24,6 +24,7 @@ import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.ResourceManagerOptions;
+import org.apache.flink.configuration.RestOptions;
 import org.apache.flink.configuration.SecurityOptions;
 import org.apache.flink.configuration.WebOptions;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
@@ -93,6 +94,7 @@ public class YarnEntrypointUtils {
 			ApplicationConstants.Environment.NM_HOST.key());
 
 		configuration.setString(JobManagerOptions.ADDRESS, hostname);
+		configuration.setString(RestOptions.REST_ADDRESS, hostname);
 
 		// TODO: Support port ranges for the AM
 //		final String portRange = configuration.getString(
@@ -112,6 +114,11 @@ public class YarnEntrypointUtils {
 			configuration.setInteger(WebOptions.PORT, 0);
 		}
 
+		if (configuration.getInteger(RestOptions.REST_PORT) >= 0) {
+			// set the REST port to 0 to select it randomly
+			configuration.setInteger(RestOptions.REST_PORT, 0);
+		}
+
 		// if the user has set the deprecated YARN-specific config keys, we add the
 		// corresponding generic config keys instead. that way, later code needs not
 		// deal with deprecated config keys

http://git-wip-us.apache.org/repos/asf/flink/blob/dbe0e828/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
index 4e68612..0859f03 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.yarn;
 
 import org.apache.flink.client.deployment.ClusterRetrieveException;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.TestLogger;
@@ -131,5 +132,10 @@ public class AbstractYarnClusterTest extends TestLogger {
 		protected String getYarnJobClusterEntrypoint() {
 			throw new UnsupportedOperationException("Not needed for testing");
 		}
+
+		@Override
+		protected ClusterClient<ApplicationId> createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) throws Exception {
+			throw new UnsupportedOperationException("Not needed for testing");
+		}
 	}
 }


[02/14] flink git commit: [FLINK-8340] [flip6] Remove passing of Configuration to CustomCommandLine

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index 1b0f29a..d4ab41f 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -25,7 +25,6 @@ import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.clusterframework.ApplicationStatus;
@@ -34,6 +33,8 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter;
 import org.apache.flink.runtime.security.SecurityConfiguration;
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
 import org.apache.flink.yarn.AbstractYarnClusterDescriptor;
 import org.apache.flink.yarn.YarnClusterClient;
 import org.apache.flink.yarn.YarnClusterDescriptor;
@@ -53,6 +54,8 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
@@ -70,7 +73,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
@@ -140,17 +142,36 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 
 	private final boolean acceptInteractiveInput;
 
+	private final String configurationDirectory;
+
+	private final Properties yarnPropertiesFile;
+
+	private final String yarnApplicationIdFromYarnProperties;
+
 	//------------------------------------ Internal fields -------------------------
 	private YarnClusterClient yarnCluster;
 	private boolean detachedMode = false;
 
-	public FlinkYarnSessionCli(String shortPrefix, String longPrefix) {
-		this(shortPrefix, longPrefix, true);
+	public FlinkYarnSessionCli(
+			Configuration configuration,
+			String configurationDirectory,
+			String shortPrefix,
+			String longPrefix) throws FlinkException {
+		this(configuration, configurationDirectory, shortPrefix, longPrefix, true);
 	}
 
-	public FlinkYarnSessionCli(String shortPrefix, String longPrefix, boolean acceptInteractiveInput) {
+	public FlinkYarnSessionCli(
+			Configuration configuration,
+			String configurationDirectory,
+			String shortPrefix,
+			String longPrefix,
+			boolean acceptInteractiveInput) throws FlinkException {
+		super(configuration);
+		this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory);
 		this.acceptInteractiveInput = acceptInteractiveInput;
 
+		// Create the command line options
+
 		query = new Option(shortPrefix + "q", longPrefix + "query", false, "Display available YARN resources (memory, cores)");
 		applicationId = new Option(shortPrefix + "id", longPrefix + "applicationId", true, "Attach to running YARN session");
 		queue = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue.");
@@ -183,93 +204,43 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 		allOptions.addOption(applicationId);
 		allOptions.addOption(zookeeperNamespace);
 		allOptions.addOption(flip6);
-	}
 
-	/**
-	 * Tries to load a Flink Yarn properties file and returns the Yarn application id if successful.
-	 * @param cmdLine The command-line parameters
-	 * @param flinkConfiguration The flink configuration
-	 * @return Yarn application id or null if none could be retrieved
-	 */
-	private String loadYarnPropertiesFile(CommandLine cmdLine, Configuration flinkConfiguration) {
-
-		String jobManagerOption = cmdLine.getOptionValue(addressOption.getOpt(), null);
-		if (jobManagerOption != null) {
-			// don't resume from properties file if a JobManager has been specified
-			return null;
-		}
-
-		for (Option option : cmdLine.getOptions()) {
-			if (allOptions.hasOption(option.getOpt())) {
-				if (!option.getOpt().equals(detached.getOpt())) {
-					// don't resume from properties file if yarn options have been specified
-					return null;
-				}
-			}
-		}
+		// try loading a potential yarn properties file
+		final File yarnPropertiesLocation = getYarnPropertiesLocation(configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
 
-		// load the YARN properties
-		File propertiesFile = getYarnPropertiesLocation(flinkConfiguration);
-		if (!propertiesFile.exists()) {
-			return null;
-		}
+		yarnPropertiesFile = new Properties();
 
-		logAndSysout("Found YARN properties file " + propertiesFile.getAbsolutePath());
+		if (yarnPropertiesLocation.exists()) {
+			LOG.info("Found Yarn properties file under {}.", yarnPropertiesLocation.getAbsolutePath());
 
-		Properties yarnProperties = new Properties();
-		try {
-			try (InputStream is = new FileInputStream(propertiesFile)) {
-				yarnProperties.load(is);
+			try (InputStream is = new FileInputStream(yarnPropertiesLocation)) {
+				yarnPropertiesFile.load(is);
+			} catch (IOException ioe) {
+				throw new FlinkException("Could not read the Yarn properties file " + yarnPropertiesLocation + '.');
 			}
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Cannot read the YARN properties file", e);
-		}
 
-		// get the Yarn application id from the properties file
-		String applicationID = yarnProperties.getProperty(YARN_APPLICATION_ID_KEY);
-		if (applicationID == null) {
-			throw new IllegalConfigurationException("Yarn properties file found but doesn't contain a " +
-				"Yarn application id. Please delete the file at " + propertiesFile.getAbsolutePath());
-		}
-
-		try {
-			// try converting id to ApplicationId
-			ConverterUtils.toApplicationId(applicationID);
-		}
-		catch (Exception e) {
-			throw new RuntimeException("YARN properties contains an invalid entry for " +
-				"application id: " + applicationID, e);
-		}
+			yarnApplicationIdFromYarnProperties = yarnPropertiesFile.getProperty(YARN_APPLICATION_ID_KEY);
 
-		logAndSysout("Using Yarn application id from YARN properties " + applicationID);
+			if (yarnApplicationIdFromYarnProperties == null) {
+				throw new FlinkException("Yarn properties file found but doesn't contain a " +
+					"Yarn application id. Please delete the file at " + yarnPropertiesLocation.getAbsolutePath());
+			}
 
-		// configure the default parallelism from YARN
-		String propParallelism = yarnProperties.getProperty(YARN_PROPERTIES_PARALLELISM);
-		if (propParallelism != null) { // maybe the property is not set
 			try {
-				int parallelism = Integer.parseInt(propParallelism);
-				flinkConfiguration.setInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, parallelism);
-
-				logAndSysout("YARN properties set default parallelism to " + parallelism);
+				// try converting id to ApplicationId
+				ConverterUtils.toApplicationId(yarnApplicationIdFromYarnProperties);
 			}
-			catch (NumberFormatException e) {
-				throw new RuntimeException("Error while parsing the YARN properties: " +
-					"Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer.");
+			catch (Exception e) {
+				throw new FlinkException("YARN properties contains an invalid entry for " +
+					"application id: " + yarnApplicationIdFromYarnProperties + ". Please delete the file at " +
+					yarnPropertiesLocation.getAbsolutePath(), e);
 			}
+		} else {
+			yarnApplicationIdFromYarnProperties = null;
 		}
-
-		// handle the YARN client's dynamic properties
-		String dynamicPropertiesEncoded = yarnProperties.getProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING);
-		Map<String, String> dynamicProperties = getDynamicProperties(dynamicPropertiesEncoded);
-		for (Map.Entry<String, String> dynamicProperty : dynamicProperties.entrySet()) {
-			flinkConfiguration.setString(dynamicProperty.getKey(), dynamicProperty.getValue());
-		}
-
-		return applicationID;
 	}
 
-	public AbstractYarnClusterDescriptor createDescriptor(
+	private AbstractYarnClusterDescriptor createDescriptor(
 		Configuration configuration,
 		String configurationDirectory,
 		String defaultApplicationName,
@@ -364,7 +335,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 		return yarnClusterDescriptor;
 	}
 
-	public ClusterSpecification createClusterSpecification(Configuration configuration, CommandLine cmd) {
+	private ClusterSpecification createClusterSpecification(Configuration configuration, CommandLine cmd) {
 		if (!cmd.hasOption(container.getOpt())) { // number of containers is required option!
 			LOG.error("Missing required argument {}", container.getOpt());
 			printUsage();
@@ -374,27 +345,12 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 		int numberTaskManagers = Integer.valueOf(cmd.getOptionValue(container.getOpt()));
 
 		// JobManager Memory
-		final int jobManagerMemoryMB;
-		if (cmd.hasOption(jmMemory.getOpt())) {
-			jobManagerMemoryMB = Integer.valueOf(cmd.getOptionValue(this.jmMemory.getOpt()));
-		} else {
-			jobManagerMemoryMB = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY);
-		}
+		final int jobManagerMemoryMB = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY);
 
 		// Task Managers memory
-		final int taskManagerMemoryMB;
-		if (cmd.hasOption(tmMemory.getOpt())) {
-			taskManagerMemoryMB = Integer.valueOf(cmd.getOptionValue(this.tmMemory.getOpt()));
-		} else {
-			taskManagerMemoryMB = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY);
-		}
+		final int taskManagerMemoryMB = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY);
 
-		int slotsPerTaskManager;
-		if (cmd.hasOption(slots.getOpt())) {
-			slotsPerTaskManager = Integer.valueOf(cmd.getOptionValue(this.slots.getOpt()));
-		} else {
-			slotsPerTaskManager = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
-		}
+		int slotsPerTaskManager = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
 
 		// convenience
 		int userParallelism = Integer.valueOf(cmd.getOptionValue(CliFrontendParser.PARALLELISM_OPTION.getOpt(), "-1"));
@@ -435,11 +391,11 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 	}
 
 	@Override
-	public boolean isActive(CommandLine commandLine, Configuration configuration) {
+	public boolean isActive(CommandLine commandLine) {
 		String jobManagerOption = commandLine.getOptionValue(addressOption.getOpt(), null);
 		boolean yarnJobManager = ID.equals(jobManagerOption);
 		boolean yarnAppId = commandLine.hasOption(applicationId.getOpt());
-		return yarnJobManager || yarnAppId || loadYarnPropertiesFile(commandLine, configuration) != null;
+		return yarnJobManager || yarnAppId || (isYarnPropertiesFileMode(commandLine) && yarnApplicationIdFromYarnProperties != null);
 	}
 
 	@Override
@@ -463,11 +419,8 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 	}
 
 	@Override
-	public AbstractYarnClusterDescriptor createClusterDescriptor(
-			Configuration configuration,
-			String configurationDirectory,
-			CommandLine commandLine) {
-		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(configuration, commandLine);
+	public AbstractYarnClusterDescriptor createClusterDescriptor(CommandLine commandLine) throws FlinkException {
+		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
 
 		return createDescriptor(
 			effectiveConfiguration,
@@ -477,17 +430,26 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 	}
 
 	@Override
-	public String getClusterId(Configuration configuration, CommandLine commandLine) {
-		return commandLine.hasOption(applicationId.getOpt()) ? commandLine.getOptionValue(applicationId.getOpt()) : loadYarnPropertiesFile(commandLine, configuration);
+	@Nullable
+	public String getClusterId(CommandLine commandLine) {
+		if (commandLine.hasOption(applicationId.getOpt())) {
+			return commandLine.getOptionValue(applicationId.getOpt());
+		} else if (isYarnPropertiesFileMode(commandLine)) {
+			return yarnApplicationIdFromYarnProperties;
+		} else {
+			return null;
+		}
 	}
 
 	@Override
-	public ClusterSpecification getClusterSpecification(Configuration configuration, CommandLine commandLine) {
-		return createClusterSpecification(configuration, commandLine);
+	public ClusterSpecification getClusterSpecification(CommandLine commandLine) throws FlinkException {
+		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
+
+		return createClusterSpecification(effectiveConfiguration, commandLine);
 	}
 
 	@Override
-	protected Configuration applyCommandLineOptionsToConfiguration(Configuration configuration, CommandLine commandLine) {
+	protected Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException {
 		// we ignore the addressOption because it can only contain "yarn-cluster"
 		final Configuration effectiveConfiguration = new Configuration(configuration);
 
@@ -496,7 +458,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 			effectiveConfiguration.setString(HA_CLUSTER_ID, zkNamespace);
 		}
 
-		final String applicationId = getClusterId(configuration, commandLine);
+		final String applicationId = getClusterId(commandLine);
 
 		if (applicationId != null) {
 			final String zooKeeperNamespace;
@@ -509,6 +471,66 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 			effectiveConfiguration.setString(HA_CLUSTER_ID, zooKeeperNamespace);
 		}
 
+		if (commandLine.hasOption(jmMemory.getOpt())) {
+			effectiveConfiguration.setInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, Integer.parseInt(commandLine.getOptionValue(jmMemory.getOpt())));
+		}
+
+		if (commandLine.hasOption(tmMemory.getOpt())) {
+			effectiveConfiguration.setInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, Integer.parseInt(commandLine.getOptionValue(tmMemory.getOpt())));
+		}
+
+		if (commandLine.hasOption(slots.getOpt())) {
+			effectiveConfiguration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, Integer.parseInt(commandLine.getOptionValue(slots.getOpt())));
+		}
+
+		if (isYarnPropertiesFileMode(commandLine)) {
+			return applyYarnProperties(effectiveConfiguration);
+		} else {
+			return effectiveConfiguration;
+		}
+	}
+
+	private boolean isYarnPropertiesFileMode(CommandLine commandLine) {
+		boolean canApplyYarnProperties = !commandLine.hasOption(addressOption.getOpt());
+
+		for (Option option : commandLine.getOptions()) {
+			if (allOptions.hasOption(option.getOpt())) {
+				if (!option.getOpt().equals(detached.getOpt())) {
+					// don't resume from properties file if yarn options have been specified
+					canApplyYarnProperties = false;
+					break;
+				}
+			}
+		}
+
+		return canApplyYarnProperties;
+	}
+
+	private Configuration applyYarnProperties(Configuration configuration) throws FlinkException {
+		final Configuration effectiveConfiguration = new Configuration(configuration);
+
+		// configure the default parallelism from YARN
+		String propParallelism = yarnPropertiesFile.getProperty(YARN_PROPERTIES_PARALLELISM);
+		if (propParallelism != null) { // maybe the property is not set
+			try {
+				int parallelism = Integer.parseInt(propParallelism);
+				effectiveConfiguration.setInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, parallelism);
+
+				logAndSysout("YARN properties set default parallelism to " + parallelism);
+			}
+			catch (NumberFormatException e) {
+				throw new FlinkException("Error while parsing the YARN properties: " +
+					"Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer.", e);
+			}
+		}
+
+		// handle the YARN client's dynamic properties
+		String dynamicPropertiesEncoded = yarnPropertiesFile.getProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING);
+		Map<String, String> dynamicProperties = getDynamicProperties(dynamicPropertiesEncoded);
+		for (Map.Entry<String, String> dynamicProperty : dynamicProperties.entrySet()) {
+			effectiveConfiguration.setString(dynamicProperty.getKey(), dynamicProperty.getValue());
+		}
+
 		return effectiveConfiguration;
 	}
 
@@ -596,77 +618,82 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 			}
 		} else {
 
-			AbstractYarnClusterDescriptor yarnDescriptor;
-			try {
-				yarnDescriptor = createDescriptor(configuration, configurationDirectory, null, cmd);
-			} catch (Exception e) {
-				System.err.println("Error while starting the YARN Client: " + e.getMessage());
-				e.printStackTrace(System.err);
-				return 1;
-			}
+			try (AbstractYarnClusterDescriptor yarnDescriptor = createClusterDescriptor(cmd)){
+				final ClusterSpecification clusterSpecification;
 
-			final ClusterSpecification clusterSpecification = createClusterSpecification(yarnDescriptor.getFlinkConfiguration(), cmd);
+				try {
+					clusterSpecification = getClusterSpecification(cmd);
+				} catch (FlinkException e) {
+					System.err.println("Error while creating the cluster specification: " + e.getMessage());
+					e.printStackTrace();
+					return 1;
+				}
 
-			try {
-				yarnCluster = yarnDescriptor.deploySessionCluster(clusterSpecification);
-			} catch (Exception e) {
-				System.err.println("Error while deploying YARN cluster: " + e.getMessage());
-				e.printStackTrace(System.err);
-				return 1;
-			}
-			//------------------ ClusterClient deployed, handle connection details
-			String jobManagerAddress =
-				yarnCluster.getJobManagerAddress().getAddress().getHostName() +
-					":" + yarnCluster.getJobManagerAddress().getPort();
+				try {
+					yarnCluster = yarnDescriptor.deploySessionCluster(clusterSpecification);
+				} catch (Exception e) {
+					System.err.println("Error while deploying YARN cluster: " + e.getMessage());
+					e.printStackTrace(System.err);
+					return 1;
+				}
+				//------------------ ClusterClient deployed, handle connection details
+				String jobManagerAddress =
+					yarnCluster.getJobManagerAddress().getAddress().getHostName() +
+						":" + yarnCluster.getJobManagerAddress().getPort();
 
-			System.out.println("Flink JobManager is now running on " + jobManagerAddress);
-			System.out.println("JobManager Web Interface: " + yarnCluster.getWebInterfaceURL());
+				System.out.println("Flink JobManager is now running on " + jobManagerAddress);
+				System.out.println("JobManager Web Interface: " + yarnCluster.getWebInterfaceURL());
 
-			// file that we write into the conf/ dir containing the jobManager address and the dop.
-			File yarnPropertiesFile = getYarnPropertiesLocation(yarnCluster.getFlinkConfiguration());
+				// file that we write into the conf/ dir containing the jobManager address and the dop.
+				File yarnPropertiesFile = getYarnPropertiesLocation(configuration.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
 
-			Properties yarnProps = new Properties();
-			yarnProps.setProperty(YARN_APPLICATION_ID_KEY, yarnCluster.getApplicationId().toString());
-			if (clusterSpecification.getSlotsPerTaskManager() != -1) {
-				String parallelism =
+				Properties yarnProps = new Properties();
+				yarnProps.setProperty(YARN_APPLICATION_ID_KEY, yarnCluster.getApplicationId().toString());
+				if (clusterSpecification.getSlotsPerTaskManager() != -1) {
+					String parallelism =
 						Integer.toString(clusterSpecification.getSlotsPerTaskManager() * clusterSpecification.getNumberTaskManagers());
-				yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, parallelism);
-			}
-			// add dynamic properties
-			if (yarnDescriptor.getDynamicPropertiesEncoded() != null) {
-				yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING,
+					yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, parallelism);
+				}
+				// add dynamic properties
+				if (yarnDescriptor.getDynamicPropertiesEncoded() != null) {
+					yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING,
 						yarnDescriptor.getDynamicPropertiesEncoded());
-			}
-			writeYarnProperties(yarnProps, yarnPropertiesFile);
+				}
+				writeYarnProperties(yarnProps, yarnPropertiesFile);
 
-			//------------------ ClusterClient running, let user control it ------------
+				//------------------ ClusterClient running, let user control it ------------
 
-			if (detachedMode) {
-				// print info and quit:
-				LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
+				if (detachedMode) {
+					// print info and quit:
+					LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
 						"Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
 						"yarn application -kill " + yarnCluster.getApplicationId());
-				yarnCluster.waitForClusterToBeReady();
-				yarnCluster.disconnect();
-			} else {
+					yarnCluster.waitForClusterToBeReady();
+					yarnCluster.disconnect();
+				} else {
 
-				ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+					ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
 
-				try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
+					try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
 						yarnDescriptor.getYarnClient(),
 						yarnCluster.getApplicationId(),
-						new ScheduledExecutorServiceAdapter(scheduledExecutorService))){
-					runInteractiveCli(
-						yarnCluster,
-						yarnApplicationStatusMonitor,
-						acceptInteractiveInput);
-				} finally {
-					// shut down the scheduled executor service
-					ExecutorUtils.gracefulShutdown(
-						1000L,
-						TimeUnit.MILLISECONDS,
-						scheduledExecutorService);
+						new ScheduledExecutorServiceAdapter(scheduledExecutorService))) {
+						runInteractiveCli(
+							yarnCluster,
+							yarnApplicationStatusMonitor,
+							acceptInteractiveInput);
+					} finally {
+						// shut down the scheduled executor service
+						ExecutorUtils.gracefulShutdown(
+							1000L,
+							TimeUnit.MILLISECONDS,
+							scheduledExecutorService);
+					}
 				}
+			} catch (FlinkException e) {
+				System.err.println("Error while deploying a Flink cluster: " + e.getMessage());
+				e.printStackTrace();
+				return 1;
 			}
 		}
 		return 0;
@@ -693,18 +720,20 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 	}
 
 	public static void main(final String[] args) throws Exception {
-		final FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", ""); // no prefix for the YARN session
-
 		final String configurationDirectory = CliFrontend.getConfigurationDirectoryFromEnv();
 
 		final Configuration flinkConfiguration = GlobalConfiguration.loadConfiguration();
+
+		final FlinkYarnSessionCli cli = new FlinkYarnSessionCli(
+			flinkConfiguration,
+			configurationDirectory,
+			"",
+			""); // no prefix for the YARN session
+
 		SecurityUtils.install(new SecurityConfiguration(flinkConfiguration));
-		int retCode = SecurityUtils.getInstalledContext().runSecured(new Callable<Integer>() {
-			@Override
-			public Integer call() {
-				return cli.run(args, flinkConfiguration, configurationDirectory);
-			}
-		});
+
+		final int retCode = SecurityUtils.getInstalledContext().runSecured(() -> cli.run(args, flinkConfiguration, configurationDirectory));
+
 		System.exit(retCode);
 	}
 
@@ -858,11 +887,17 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterCl
 		}
 	}
 
-	public static File getYarnPropertiesLocation(Configuration conf) {
-		String defaultPropertiesFileLocation = System.getProperty("java.io.tmpdir");
+	public static File getYarnPropertiesLocation(@Nullable String yarnPropertiesFileLocation) {
+
+		final String propertiesFileLocation;
+
+		if (yarnPropertiesFileLocation != null) {
+			propertiesFileLocation = yarnPropertiesFileLocation;
+		} else {
+			propertiesFileLocation = System.getProperty("java.io.tmpdir");
+		}
+
 		String currentUser = System.getProperty("user.name");
-		String propertiesFileLocation =
-			conf.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION, defaultPropertiesFileLocation);
 
 		return new File(propertiesFileLocation, YARN_PROPERTIES_FILE + currentUser);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
index 8f8359f..2f6c157 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
@@ -22,7 +22,7 @@ import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.TestLogger;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.flink.yarn.configuration.YarnConfigOptions;
@@ -69,6 +69,8 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 	public void testDynamicProperties() throws Exception {
 
 		FlinkYarnSessionCli cli = new FlinkYarnSessionCli(
+			new Configuration(),
+			tmp.getRoot().getAbsolutePath(),
 			"",
 			"",
 			false);
@@ -80,11 +82,7 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 		CommandLine cmd = parser.parse(options, new String[]{"run", "-j", "fake.jar", "-n", "15",
 				"-D", "akka.ask.timeout=5 min", "-D", "env.java.opts=-DappName=foobar"});
 
-		AbstractYarnClusterDescriptor flinkYarnDescriptor = cli.createDescriptor(
-			new Configuration(),
-			tmp.getRoot().getAbsolutePath(),
-			null,
-			cmd);
+		AbstractYarnClusterDescriptor flinkYarnDescriptor = cli.createClusterDescriptor(cmd);
 
 		Assert.assertNotNull(flinkYarnDescriptor);
 
@@ -100,7 +98,11 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 		String[] params =
 			new String[] {"-yn", "2", "-ys", "3", "-p", "7"};
 
-		FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli("y", "yarn");
+		FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli(
+			new Configuration(),
+			tmp.getRoot().getAbsolutePath(),
+			"y",
+			"yarn");
 
 		Options options = new Options();
 		// TODO: Nasty workaround: We should get rid of the YarnCLI and run options coupling
@@ -110,7 +112,7 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 
 		final CommandLine commandLine = CliFrontendParser.parse(options, params, true);
 
-		ClusterSpecification clusterSpecification = yarnCLI.createClusterSpecification(new Configuration(), commandLine);
+		ClusterSpecification clusterSpecification = yarnCLI.getClusterSpecification(commandLine);
 
 		// each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased.
 		assertEquals(4, clusterSpecification.getSlotsPerTaskManager());
@@ -122,21 +124,17 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 		String[] params =
 			new String[] {"-yn", "2", "-ys", "3"};
 
-		FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli("y", "yarn");
+		FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli(
+			new Configuration(),
+			tmp.getRoot().getAbsolutePath(),
+			"y",
+			"yarn");
 
 		final CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true);
 
-		final Configuration configuration = new Configuration();
+		AbstractYarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine);
 
-		AbstractYarnClusterDescriptor descriptor = yarnCLI.createDescriptor(
-			configuration,
-			tmp.getRoot().getAbsolutePath(),
-			"",
-			commandLine);
-
-		final ClusterSpecification clusterSpecification = yarnCLI.createClusterSpecification(
-			configuration,
-			commandLine);
+		final ClusterSpecification clusterSpecification = yarnCLI.getClusterSpecification(commandLine);
 
 		// each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased.
 		assertEquals(3, clusterSpecification.getSlotsPerTaskManager());
@@ -149,15 +147,15 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 
 		String[] params = new String[] {"-yn", "2", "-yz", zkNamespaceCliInput};
 
-		FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli("y", "yarn");
+		FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli(
+			new Configuration(),
+			tmp.getRoot().getAbsolutePath(),
+			"y",
+			"yarn");
 
 		CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true);
 
-		AbstractYarnClusterDescriptor descriptor = yarnCLI.createDescriptor(
-			new Configuration(),
-			tmp.getRoot().getAbsolutePath(),
-			"",
-			commandLine);
+		AbstractYarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine);
 
 		assertEquals(zkNamespaceCliInput, descriptor.getZookeeperNamespace());
 	}
@@ -170,16 +168,18 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 
 		File directoryPath = writeYarnPropertiesFile(validPropertiesFile);
 
-		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
 		final Configuration configuration = new Configuration();
 		configuration.setString(YarnConfigOptions.PROPERTIES_FILE_LOCATION, directoryPath.getAbsolutePath());
 
+		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
+			configuration,
+			tmp.getRoot().getAbsolutePath(),
+			"y",
+			"yarn");
+
 		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {}, true);
 
-		final String clusterId = flinkYarnSessionCli.getClusterId(
-			configuration,
-			commandLine);
+		final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
 
 		assertEquals(TEST_YARN_APPLICATION_ID.toString(), clusterId);
 	}
@@ -188,50 +188,49 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 	 * Tests that we fail when reading an invalid yarn properties file when retrieving
 	 * the cluster id.
 	 */
-	@Test(expected = IllegalConfigurationException.class)
+	@Test(expected = FlinkException.class)
 	public void testInvalidYarnPropertiesFile() throws Exception {
 
 		File directoryPath = writeYarnPropertiesFile(invalidPropertiesFile);
 
-		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
 		final Configuration configuration = new Configuration();
 		configuration.setString(YarnConfigOptions.PROPERTIES_FILE_LOCATION, directoryPath.getAbsolutePath());
 
-		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {}, true);
-
-		flinkYarnSessionCli.getClusterId(
+		new FlinkYarnSessionCli(
 			configuration,
-			commandLine);
+			tmp.getRoot().getAbsolutePath(),
+			"y",
+			"yarn");
 	}
 
 	@Test
 	public void testResumeFromYarnID() throws Exception {
-		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
 		final Configuration configuration = new Configuration();
+		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
+			configuration,
+			tmp.getRoot().getAbsolutePath(),
+			"y",
+			"yarn");
 
 		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true);
 
-		final String clusterId = flinkYarnSessionCli.getClusterId(
-			configuration,
-			commandLine);
+		final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
 
 		assertEquals(TEST_YARN_APPLICATION_ID.toString(), clusterId);
 	}
 
 	@Test
 	public void testResumeFromYarnIDZookeeperNamespace() throws Exception {
-		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
 		final Configuration configuration = new Configuration();
+		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
+			configuration,
+			tmp.getRoot().getAbsolutePath(),
+			"y",
+			"yarn");
 
 		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true);
 
-		final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(
-			configuration,
-			tmp.getRoot().getAbsolutePath(),
-			commandLine);
+		final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine);
 
 		final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration();
 
@@ -241,17 +240,18 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 
 	@Test
 	public void testResumeFromYarnIDZookeeperNamespaceOverride() throws Exception {
-		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
+		final Configuration configuration = new Configuration();
+		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
+			configuration,
+			tmp.getRoot().getAbsolutePath(),
+			"y",
+			"yarn");
 
 		final String overrideZkNamespace = "my_cluster";
-		final Configuration configuration = new Configuration();
 
 		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString(), "-yz", overrideZkNamespace}, true);
 
-		final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(
-			configuration,
-			tmp.getRoot().getAbsolutePath(),
-			commandLine);
+		final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine);
 
 		final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration();
 
@@ -263,17 +263,16 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 	public void testYarnIDOverridesPropertiesFile() throws Exception {
 		File directoryPath = writeYarnPropertiesFile(validPropertiesFile);
 
-		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli("y", "yarn");
-
 		final Configuration configuration = new Configuration();
 		configuration.setString(YarnConfigOptions.PROPERTIES_FILE_LOCATION, directoryPath.getAbsolutePath());
 
-		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID_2.toString() }, true);
-
-		final String clusterId = flinkYarnSessionCli.getClusterId(
+		final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli(
 			configuration,
-			commandLine);
-
+			tmp.getRoot().getAbsolutePath(),
+			"y",
+			"yarn");
+		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID_2.toString() }, true);
+		final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
 		assertEquals(TEST_YARN_APPLICATION_ID_2.toString(), clusterId);
 	}
 


[08/14] flink git commit: [hotfix] Add help command to FlinkYarnSessionCli

Posted by tr...@apache.org.
[hotfix] Add help command to FlinkYarnSessionCli


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2ce64e79
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2ce64e79
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2ce64e79

Branch: refs/heads/master
Commit: 2ce64e791b3a2e2b8a8e4ac774f2ca45da5660dc
Parents: 402499f
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Dec 29 15:12:46 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:04 2018 +0100

----------------------------------------------------------------------
 .../java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java  | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2ce64e79/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index 75a270d..24a422e 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -127,6 +127,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 	private final Option detached;
 	private final Option zookeeperNamespace;
 	private final Option flip6;
+	private final Option help;
 
 	/**
 	 * @deprecated Streaming mode has been deprecated without replacement. Set the
@@ -198,6 +199,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		name = new Option(shortPrefix + "nm", longPrefix + "name", true, "Set a custom name for the application on YARN");
 		zookeeperNamespace = new Option(shortPrefix + "z", longPrefix + "zookeeperNamespace", true, "Namespace to create the Zookeeper sub-paths for high availability mode");
 		flip6 = new Option(shortPrefix + "f6", longPrefix + "flip6", false, "Specify this option to start a Flip-6 Yarn session cluster.");
+		help = new Option(shortPrefix + "h", longPrefix + "help", false, "Help for the Yarn session CLI.");
 
 		allOptions = new Options();
 		allOptions.addOption(flinkJar);
@@ -215,6 +217,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		allOptions.addOption(applicationId);
 		allOptions.addOption(zookeeperNamespace);
 		allOptions.addOption(flip6);
+		allOptions.addOption(help);
 
 		// try loading a potential yarn properties file
 		this.yarnPropertiesFileLocation = configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION);
@@ -564,6 +567,11 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		//
 		final CommandLine cmd = parseCommandLineOptions(args, true);
 
+		if (cmd.hasOption(help.getOpt())) {
+			printUsage();
+			return 0;
+		}
+
 		final AbstractYarnClusterDescriptor yarnClusterDescriptor = createClusterDescriptor(cmd);
 
 		try {


[06/14] flink git commit: [FLINK-8347] [flip6] Make cluster id used by ClusterDescriptor typesafe

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
index b17b030..938b979 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.client.program.rest;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.client.deployment.StandaloneClusterId;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.JobManagerOptions;
@@ -115,7 +116,7 @@ public class RestClusterClientTest extends TestLogger {
 
 	private RestServerEndpointConfiguration restServerEndpointConfiguration;
 
-	private RestClusterClient restClusterClient;
+	private RestClusterClient<StandaloneClusterId> restClusterClient;
 
 	@Before
 	public void setUp() throws Exception {
@@ -126,7 +127,7 @@ public class RestClusterClientTest extends TestLogger {
 		config.setString(JobManagerOptions.ADDRESS, "localhost");
 		restServerEndpointConfiguration = RestServerEndpointConfiguration.fromConfiguration(config);
 		mockGatewayRetriever = () -> CompletableFuture.completedFuture(mockRestfulGateway);
-		restClusterClient = new RestClusterClient(config, (attempt) -> 0);
+		restClusterClient = new RestClusterClient(config, StandaloneClusterId.getInstance(), (attempt) -> 0);
 	}
 
 	@After

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
index 24cb0fc..f4ea659 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
@@ -20,7 +20,6 @@ package org.apache.flink.storm.api;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.JobWithJars;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.client.program.StandaloneClusterClient;
@@ -205,7 +204,7 @@ public class FlinkClient {
 		configuration.setString(JobManagerOptions.ADDRESS, jobManagerHost);
 		configuration.setInteger(JobManagerOptions.PORT, jobManagerPort);
 
-		final ClusterClient client;
+		final StandaloneClusterClient client;
 		try {
 			client = new StandaloneClusterClient(configuration);
 		} catch (final Exception e) {
@@ -245,7 +244,7 @@ public class FlinkClient {
 		configuration.setString(JobManagerOptions.ADDRESS, this.jobManagerHost);
 		configuration.setInteger(JobManagerOptions.PORT, this.jobManagerPort);
 
-		final ClusterClient client;
+		final StandaloneClusterClient client;
 		try {
 			client = new StandaloneClusterClient(configuration);
 		} catch (final Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java
index 25ce10a..ac2f7eb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServer.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.rpc;
 import java.util.concurrent.CompletableFuture;
 
 /**
- * Interface for self gateways
+ * Interface for self gateways.
  */
 public interface RpcServer extends StartStoppable, MainThreadExecutable, RpcGateway {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
----------------------------------------------------------------------
diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
index 349d940..6fa6aa5 100644
--- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
+++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
@@ -21,6 +21,7 @@ package org.apache.flink.api.scala
 import java.io._
 
 import org.apache.flink.client.cli.{CliFrontend, CliFrontendParser, RunOptions}
+import org.apache.flink.client.deployment.{ClusterDescriptor, StandaloneClusterId}
 import org.apache.flink.client.program.ClusterClient
 import org.apache.flink.configuration.{Configuration, GlobalConfiguration, JobManagerOptions}
 import org.apache.flink.runtime.minicluster.StandaloneMiniCluster
@@ -138,7 +139,7 @@ object FlinkShell {
   def fetchConnectionInfo(
     configuration: Configuration,
     config: Config
-  ): (String, Int, Option[Either[StandaloneMiniCluster, ClusterClient]]) = {
+  ): (String, Int, Option[Either[StandaloneMiniCluster, ClusterClient[_]]]) = {
     config.executionMode match {
       case ExecutionMode.LOCAL => // Local mode
         val config = GlobalConfiguration.loadConfiguration()
@@ -294,7 +295,9 @@ object FlinkShell {
       CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
     val customCLI = frontend.getActiveCustomCommandLine(commandLine)
 
-    val clusterDescriptor = customCLI.createClusterDescriptor(commandLine)
+    val clusterDescriptor = customCLI
+      .createClusterDescriptor(commandLine)
+      .asInstanceOf[ClusterDescriptor[Any]]
 
     val clusterId = customCLI.getClusterId(commandLine)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
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 74b1c68..a0b4a40 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
@@ -21,7 +21,6 @@ import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.JobWithJars;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.client.program.StandaloneClusterClient;
@@ -199,7 +198,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 		configuration.setString(JobManagerOptions.ADDRESS, host);
 		configuration.setInteger(JobManagerOptions.PORT, port);
 
-		ClusterClient client;
+		StandaloneClusterClient client;
 		try {
 			client = new StandaloneClusterClient(configuration);
 			client.setPrintStatusDuringExecution(getConfig().isSysoutLoggingEnabled());

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
index 7f8572b..221f3fa 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
@@ -20,6 +20,7 @@
 package org.apache.flink.test.example.client;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.client.deployment.StandaloneClusterId;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.Configuration;
@@ -78,7 +79,7 @@ public class JobRetrievalITCase extends TestLogger {
 
 		final JobGraph jobGraph = new JobGraph(jobID, "testjob", imalock);
 
-		final ClusterClient client = new StandaloneClusterClient(cluster.configuration(), cluster.highAvailabilityServices());
+		final ClusterClient<StandaloneClusterId> client = new StandaloneClusterClient(cluster.configuration(), cluster.highAvailabilityServices());
 
 		// acquire the lock to make sure that the job cannot complete until the job client
 		// has been attached in resumingThread
@@ -123,7 +124,7 @@ public class JobRetrievalITCase extends TestLogger {
 	@Test
 	public void testNonExistingJobRetrieval() throws Exception {
 		final JobID jobID = new JobID();
-		ClusterClient client = new StandaloneClusterClient(cluster.configuration());
+		ClusterClient<StandaloneClusterId> client = new StandaloneClusterClient(cluster.configuration());
 
 		try {
 			client.retrieveJob(jobID);

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
index 897dc8c..55669f1 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
@@ -41,6 +41,7 @@ import akka.actor.PoisonPill;
 import akka.testkit.JavaTestKit;
 import org.apache.curator.test.TestingServer;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -125,7 +126,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 			"@@" + FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY + "=" + fsStateHandlePath + "/checkpoints" +
 			"@@" + HighAvailabilityOptions.HA_STORAGE_PATH.key() + "=" + fsStateHandlePath + "/recovery");
 
-		ClusterClient yarnCluster = null;
+		ClusterClient<ApplicationId> yarnCluster = null;
 
 		final FiniteDuration timeout = new FiniteDuration(2, TimeUnit.MINUTES);
 
@@ -141,8 +142,6 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 		try {
 			yarnCluster = flinkYarnClient.deploySessionCluster(clusterSpecification);
 
-			final ClusterClient finalYarnCluster = yarnCluster;
-
 			highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices(
 				yarnCluster.getFlinkConfiguration(),
 				Executors.directExecutor(),

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index 15fe355..ec6c105 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -246,7 +246,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 				.setSlotsPerTaskManager(1)
 				.createClusterSpecification();
 			// deploy
-			ClusterClient yarnCluster = null;
+			ClusterClient<ApplicationId> yarnCluster = null;
 			try {
 				yarnCluster = clusterDescriptor.deploySessionCluster(clusterSpecification);
 			} catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index 4affb78..a7e8c36 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -19,9 +19,10 @@
 package org.apache.flink.yarn;
 
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.client.deployment.ClusterDeploymentException;
 import org.apache.flink.client.deployment.ClusterDescriptor;
+import org.apache.flink.client.deployment.ClusterRetrieveException;
 import org.apache.flink.client.deployment.ClusterSpecification;
-import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
@@ -61,7 +62,6 @@ import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -101,7 +101,7 @@ import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties
 /**
  * The descriptor with deployment information for spawning or resuming a {@link YarnClusterClient}.
  */
-public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor {
+public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor<ApplicationId> {
 	private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnClusterDescriptor.class);
 
 	/**
@@ -332,7 +332,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	// -------------------------------------------------------------
 
 	@Override
-	public ClusterClient retrieve(String applicationID) {
+	public YarnClusterClient retrieve(ApplicationId applicationId) throws ClusterRetrieveException {
 
 		try {
 			// check if required Hadoop environment variables are set. If not, warn user
@@ -343,18 +343,17 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 					"configuration for accessing YARN.");
 			}
 
-			final ApplicationId yarnAppId = ConverterUtils.toApplicationId(applicationID);
-			final ApplicationReport appReport = yarnClient.getApplicationReport(yarnAppId);
+			final ApplicationReport appReport = yarnClient.getApplicationReport(applicationId);
 
 			if (appReport.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) {
 				// Flink cluster is not running anymore
 				LOG.error("The application {} doesn't run anymore. It has previously completed with final status: {}",
-					applicationID, appReport.getFinalApplicationStatus());
-				throw new RuntimeException("The Yarn application " + applicationID + " doesn't run anymore.");
+					applicationId, appReport.getFinalApplicationStatus());
+				throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore.");
 			}
 
 			LOG.info("Found application JobManager host name '{}' and port '{}' from supplied application id '{}'",
-				appReport.getHost(), appReport.getRpcPort(), applicationID);
+				appReport.getHost(), appReport.getRpcPort(), applicationId);
 
 			flinkConfiguration.setString(JobManagerOptions.ADDRESS, appReport.getHost());
 			flinkConfiguration.setInteger(JobManagerOptions.PORT, appReport.getRpcPort());
@@ -367,43 +366,35 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 				flinkConfiguration,
 				false);
 		} catch (Exception e) {
-			throw new RuntimeException("Couldn't retrieve Yarn cluster", e);
+			throw new ClusterRetrieveException("Couldn't retrieve Yarn cluster", e);
 		}
 	}
 
 	@Override
-	public YarnClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) {
+	public YarnClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException {
 		try {
 			return deployInternal(
 				clusterSpecification,
 				getYarnSessionClusterEntrypoint(),
 				null);
 		} catch (Exception e) {
-			throw new RuntimeException("Couldn't deploy Yarn session cluster", e);
+			throw new ClusterDeploymentException("Couldn't deploy Yarn session cluster", e);
 		}
 	}
 
 	@Override
-	public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
+	public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) throws ClusterDeploymentException {
 		try {
 			return deployInternal(
 				clusterSpecification,
 				getYarnJobClusterEntrypoint(),
 				jobGraph);
 		} catch (Exception e) {
-			throw new RuntimeException("Could not deploy Yarn job cluster.", e);
+			throw new ClusterDeploymentException("Could not deploy Yarn job cluster.", e);
 		}
 	}
 
 	@Override
-	public void terminateCluster(String clusterId) throws FlinkException {
-		try {
-			yarnClient.killApplication(ConverterUtils.toApplicationId(clusterId));
-		} catch (IOException | YarnException e) {
-			throw new FlinkException("Could not terminate cluster with id " + clusterId + '.', e);
-		}
-	}
-
 	public void terminateCluster(ApplicationId applicationId) throws FlinkException {
 		try {
 			yarnClient.killApplication(applicationId);

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
index 991b3b9..f2be264 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
@@ -41,7 +41,6 @@ import akka.util.Timeout;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,7 +57,7 @@ import scala.concurrent.duration.FiniteDuration;
 /**
  * Java representation of a running Flink cluster within YARN.
  */
-public class YarnClusterClient extends ClusterClient {
+public class YarnClusterClient extends ClusterClient<ApplicationId> {
 
 	private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClient.class);
 
@@ -170,11 +169,6 @@ public class YarnClusterClient extends ClusterClient {
 		}
 	}
 
-	@Override
-	public String getClusterIdentifier() {
-		return ConverterUtils.toString(appReport.getApplicationId());
-	}
-
 	/**
 	 * This method is only available if the cluster hasn't been started in detached mode.
 	 */
@@ -234,6 +228,11 @@ public class YarnClusterClient extends ClusterClient {
 	}
 
 	@Override
+	public ApplicationId getClusterId() {
+		return appId;
+	}
+
+	@Override
 	public boolean isDetached() {
 		return super.isDetached() || clusterDescriptor.isDetachedMode();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index 24a422e..fea8a5c 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -86,7 +86,7 @@ import static org.apache.flink.configuration.HighAvailabilityOptions.HA_CLUSTER_
 /**
  * Class handling the command line interface to the YARN session.
  */
-public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
+public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId> {
 	private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnSessionCli.class);
 
 	//------------------------------------ Constants   -------------------------
@@ -152,7 +152,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 
 	private final Properties yarnPropertiesFile;
 
-	private final String yarnApplicationIdFromYarnProperties;
+	private final ApplicationId yarnApplicationIdFromYarnProperties;
 
 	private final String yarnPropertiesFileLocation;
 
@@ -235,20 +235,20 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 					". Please delete the file at " + yarnPropertiesLocation.getAbsolutePath() + '.', ioe);
 			}
 
-			yarnApplicationIdFromYarnProperties = yarnPropertiesFile.getProperty(YARN_APPLICATION_ID_KEY);
+			final String yarnApplicationIdString = yarnPropertiesFile.getProperty(YARN_APPLICATION_ID_KEY);
 
-			if (yarnApplicationIdFromYarnProperties == null) {
+			if (yarnApplicationIdString == null) {
 				throw new FlinkException("Yarn properties file found but doesn't contain a " +
 					"Yarn application id. Please delete the file at " + yarnPropertiesLocation.getAbsolutePath());
 			}
 
 			try {
 				// try converting id to ApplicationId
-				ConverterUtils.toApplicationId(yarnApplicationIdFromYarnProperties);
+				yarnApplicationIdFromYarnProperties = ConverterUtils.toApplicationId(yarnApplicationIdString);
 			}
 			catch (Exception e) {
 				throw new FlinkException("YARN properties contains an invalid entry for " +
-					"application id: " + yarnApplicationIdFromYarnProperties + ". Please delete the file at " +
+					"application id: " + yarnApplicationIdString + ". Please delete the file at " +
 					yarnPropertiesLocation.getAbsolutePath(), e);
 			}
 		} else {
@@ -458,9 +458,9 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 
 	@Override
 	@Nullable
-	public String getClusterId(CommandLine commandLine) {
+	public ApplicationId getClusterId(CommandLine commandLine) {
 		if (commandLine.hasOption(applicationId.getOpt())) {
-			return commandLine.getOptionValue(applicationId.getOpt());
+			return ConverterUtils.toApplicationId(commandLine.getOptionValue(applicationId.getOpt()));
 		} else if (isYarnPropertiesFileMode(commandLine)) {
 			return yarnApplicationIdFromYarnProperties;
 		} else {
@@ -485,14 +485,14 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 			effectiveConfiguration.setString(HA_CLUSTER_ID, zkNamespace);
 		}
 
-		final String applicationId = getClusterId(commandLine);
+		final ApplicationId applicationId = getClusterId(commandLine);
 
 		if (applicationId != null) {
 			final String zooKeeperNamespace;
 			if (commandLine.hasOption(zookeeperNamespace.getOpt())){
 				zooKeeperNamespace = commandLine.getOptionValue(zookeeperNamespace.getOpt());
 			} else {
-				zooKeeperNamespace = effectiveConfiguration.getString(HA_CLUSTER_ID, applicationId);
+				zooKeeperNamespace = effectiveConfiguration.getString(HA_CLUSTER_ID, applicationId.toString());
 			}
 
 			effectiveConfiguration.setString(HA_CLUSTER_ID, zooKeeperNamespace);
@@ -581,20 +581,20 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 				System.out.println(description);
 				return 0;
 			} else {
-				final ClusterClient clusterClient;
+				final ClusterClient<ApplicationId> clusterClient;
 				final ApplicationId yarnApplicationId;
 
 				if (cmd.hasOption(applicationId.getOpt())) {
 					yarnApplicationId = ConverterUtils.toApplicationId(cmd.getOptionValue(applicationId.getOpt()));
 
-					clusterClient = yarnClusterDescriptor.retrieve(cmd.getOptionValue(applicationId.getOpt()));
+					clusterClient = yarnClusterDescriptor.retrieve(yarnApplicationId);
 				} else {
 					final ClusterSpecification clusterSpecification = getClusterSpecification(cmd);
 
 					clusterClient = yarnClusterDescriptor.deploySessionCluster(clusterSpecification);
 
 					//------------------ ClusterClient deployed, handle connection details
-					yarnApplicationId = ConverterUtils.toApplicationId(clusterClient.getClusterIdentifier());
+					yarnApplicationId = clusterClient.getClusterId();
 
 					String jobManagerAddress =
 						clusterClient.getJobManagerAddress().getAddress().getHostName() +
@@ -790,7 +790,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 	}
 
 	private static void runInteractiveCli(
-			ClusterClient clusterClient,
+			ClusterClient<?> clusterClient,
 			YarnApplicationStatusMonitor yarnApplicationStatusMonitor,
 			boolean readConsoleInput) {
 		try (BufferedReader in = new BufferedReader(new InputStreamReader(System.in))) {

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
index 2581e68..4e68612 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.yarn;
 
+import org.apache.flink.client.deployment.ClusterRetrieveException;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.TestLogger;
@@ -49,10 +50,9 @@ public class AbstractYarnClusterTest extends TestLogger {
 	/**
 	 * Tests that the cluster retrieval of a finished YARN application fails.
 	 */
-	@Test(expected = RuntimeException.class)
-	public void testClusterClientRetrievalOfFinishedYarnApplication() throws IOException {
+	@Test(expected = ClusterRetrieveException.class)
+	public void testClusterClientRetrievalOfFinishedYarnApplication() throws Exception {
 		final ApplicationId applicationId = ApplicationId.newInstance(System.currentTimeMillis(), 42);
-		final String clusterId = applicationId.toString();
 		final ApplicationReport applicationReport = createApplicationReport(
 			applicationId,
 			YarnApplicationState.FINISHED,
@@ -65,30 +65,7 @@ public class AbstractYarnClusterTest extends TestLogger {
 			temporaryFolder.newFolder().getAbsolutePath(),
 			yarnClient);
 
-		clusterDescriptor.retrieve(clusterId);
-	}
-
-	/**
-	 * Tests that the cluster retrieval fails if an invalid application id is provided.
-	 */
-	@Test(expected = RuntimeException.class)
-	public void testClusterClientRetrievalFromInvalidApplicationId() throws IOException {
-		final ApplicationId applicationId = ApplicationId.newInstance(System.currentTimeMillis(), 42);
-		final String clusterId = "foobar";
-
-		final ApplicationReport applicationReport = createApplicationReport(
-			applicationId,
-			YarnApplicationState.RUNNING,
-			FinalApplicationStatus.UNDEFINED);
-
-		final YarnClient yarnClient = new TestingYarnClient(Collections.singletonMap(applicationId, applicationReport));
-
-		final TestingAbstractYarnClusterDescriptor clusterDescriptor = new TestingAbstractYarnClusterDescriptor(
-			new Configuration(),
-			temporaryFolder.newFolder().getAbsolutePath(),
-			yarnClient);
-
-		clusterDescriptor.retrieve(clusterId);
+		clusterDescriptor.retrieve(applicationId);
 	}
 
 	private ApplicationReport createApplicationReport(

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
index 2f6c157..9be9cb5 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
@@ -179,9 +179,9 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 
 		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {}, true);
 
-		final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
+		final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(commandLine);
 
-		assertEquals(TEST_YARN_APPLICATION_ID.toString(), clusterId);
+		assertEquals(TEST_YARN_APPLICATION_ID, clusterId);
 	}
 
 	/**
@@ -214,9 +214,9 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 
 		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true);
 
-		final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
+		final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(commandLine);
 
-		assertEquals(TEST_YARN_APPLICATION_ID.toString(), clusterId);
+		assertEquals(TEST_YARN_APPLICATION_ID, clusterId);
 	}
 
 	@Test
@@ -272,8 +272,8 @@ public class FlinkYarnSessionCliTest extends TestLogger {
 			"y",
 			"yarn");
 		final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID_2.toString() }, true);
-		final String clusterId = flinkYarnSessionCli.getClusterId(commandLine);
-		assertEquals(TEST_YARN_APPLICATION_ID_2.toString(), clusterId);
+		final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(commandLine);
+		assertEquals(TEST_YARN_APPLICATION_ID_2, clusterId);
 	}
 
 	///////////

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
index 5be54b0..e0f69cb 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.yarn;
 
+import org.apache.flink.client.deployment.ClusterDeploymentException;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -66,7 +67,7 @@ public class YarnClusterDescriptorTest extends TestLogger {
 	}
 
 	@Test
-	public void testFailIfTaskSlotsHigherThanMaxVcores() {
+	public void testFailIfTaskSlotsHigherThanMaxVcores() throws ClusterDeploymentException {
 
 		final YarnClient yarnClient = YarnClient.createYarnClient();
 
@@ -88,7 +89,7 @@ public class YarnClusterDescriptorTest extends TestLogger {
 			clusterDescriptor.deploySessionCluster(clusterSpecification);
 
 			fail("The deploy call should have failed.");
-		} catch (RuntimeException e) {
+		} catch (ClusterDeploymentException e) {
 			// we expect the cause to be an IllegalConfigurationException
 			if (!(e.getCause() instanceof IllegalConfigurationException)) {
 				throw e;
@@ -99,7 +100,7 @@ public class YarnClusterDescriptorTest extends TestLogger {
 	}
 
 	@Test
-	public void testConfigOverwrite() {
+	public void testConfigOverwrite() throws ClusterDeploymentException {
 		Configuration configuration = new Configuration();
 		// overwrite vcores in config
 		configuration.setInteger(YarnConfigOptions.VCORES, Integer.MAX_VALUE);
@@ -125,7 +126,7 @@ public class YarnClusterDescriptorTest extends TestLogger {
 			clusterDescriptor.deploySessionCluster(clusterSpecification);
 
 			fail("The deploy call should have failed.");
-		} catch (RuntimeException e) {
+		} catch (ClusterDeploymentException e) {
 			// we expect the cause to be an IllegalConfigurationException
 			if (!(e.getCause() instanceof IllegalConfigurationException)) {
 				throw e;


[07/14] flink git commit: [FLINK-8347] [flip6] Make cluster id used by ClusterDescriptor typesafe

Posted by tr...@apache.org.
[FLINK-8347] [flip6] Make cluster id used by ClusterDescriptor typesafe

The ClusterDescriptor uses a typed cluster id for the ClusterClient retrieval.
Moreover, the ClusterClient and the CustomCommandLine are typed accordingly.

This closes #5232.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/38d37208
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/38d37208
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/38d37208

Branch: refs/heads/master
Commit: 38d3720863c6187153174d0df57fc414b0cf8e96
Parents: 2ce64e7
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Jan 11 23:46:33 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:04 2018 +0100

----------------------------------------------------------------------
 .../org/apache/flink/client/RemoteExecutor.java |   2 +-
 .../client/cli/AbstractCustomCommandLine.java   |   2 +-
 .../apache/flink/client/cli/CliFrontend.java    | 462 ++++++++++---------
 .../flink/client/cli/CliFrontendParser.java     |  14 +-
 .../flink/client/cli/CustomCommandLine.java     |   6 +-
 .../org/apache/flink/client/cli/DefaultCLI.java |  10 +-
 .../flink/client/cli/Flip6DefaultCLI.java       |  10 +-
 .../client/deployment/ClusterDescriptor.java    |  20 +-
 .../deployment/ClusterRetrieveException.java    |  41 ++
 .../Flip6StandaloneClusterDescriptor.java       |  16 +-
 .../deployment/StandaloneClusterDescriptor.java |  12 +-
 .../client/deployment/StandaloneClusterId.java  |  32 ++
 .../flink/client/program/ClusterClient.java     |  10 +-
 .../client/program/ContextEnvironment.java      |   6 +-
 .../program/ContextEnvironmentFactory.java      |   4 +-
 .../client/program/DetachedEnvironment.java     |   2 +-
 .../client/program/StandaloneClusterClient.java |   8 +-
 .../client/program/rest/RestClusterClient.java  |  27 +-
 .../flink/client/cli/CliFrontendCancelTest.java |  10 +-
 .../flink/client/cli/CliFrontendListTest.java   |   6 +-
 .../client/cli/CliFrontendSavepointTest.java    |  18 +-
 .../flink/client/cli/CliFrontendStopTest.java   |  48 +-
 .../apache/flink/client/cli/DefaultCLITest.java |  10 +-
 .../client/cli/util/DummyClusterDescriptor.java |  16 +-
 .../client/cli/util/DummyCustomCommandLine.java |  10 +-
 .../client/program/ClientConnectionTest.java    |   4 +-
 .../apache/flink/client/program/ClientTest.java |   8 +-
 .../flink/client/program/ClusterClientTest.java |  16 +-
 .../program/rest/RestClusterClientTest.java     |   5 +-
 .../org/apache/flink/storm/api/FlinkClient.java |   5 +-
 .../org/apache/flink/runtime/rpc/RpcServer.java |   2 +-
 .../org/apache/flink/api/scala/FlinkShell.scala |   7 +-
 .../environment/RemoteStreamEnvironment.java    |   3 +-
 .../test/example/client/JobRetrievalITCase.java |   5 +-
 .../flink/yarn/YARNHighAvailabilityITCase.java  |   5 +-
 .../flink/yarn/YARNSessionFIFOITCase.java       |   2 +-
 .../yarn/AbstractYarnClusterDescriptor.java     |  35 +-
 .../apache/flink/yarn/YarnClusterClient.java    |  13 +-
 .../flink/yarn/cli/FlinkYarnSessionCli.java     |  28 +-
 .../flink/yarn/AbstractYarnClusterTest.java     |  31 +-
 .../flink/yarn/FlinkYarnSessionCliTest.java     |  12 +-
 .../flink/yarn/YarnClusterDescriptorTest.java   |   9 +-
 42 files changed, 528 insertions(+), 464 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
index 1ae9b07..fcf8bab 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java
@@ -58,7 +58,7 @@ public class RemoteExecutor extends PlanExecutor {
 
 	private final Configuration clientConfiguration;
 
-	private ClusterClient client;
+	private ClusterClient<?> client;
 
 	private int defaultParallelism = 1;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
index da21556..59046bf 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
@@ -38,7 +38,7 @@ import static org.apache.flink.client.cli.CliFrontend.setJobManagerAddressInConf
  * a ZooKeeper namespace.
  *
  */
-public abstract class AbstractCustomCommandLine implements CustomCommandLine {
+public abstract class AbstractCustomCommandLine<T> implements CustomCommandLine<T> {
 
 	protected final Option zookeeperNamespaceOption = new Option("z", "zookeeperNamespace", true,
 		"Namespace to create the Zookeeper sub-paths for high availability mode");

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index 847a5f8..d661aa9 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -76,9 +76,7 @@ import java.util.Comparator;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import scala.concurrent.duration.FiniteDuration;
@@ -106,7 +104,7 @@ public class CliFrontend {
 
 	private final Configuration configuration;
 
-	private final List<CustomCommandLine> customCommandLines;
+	private final List<CustomCommandLine<?>> customCommandLines;
 
 	private final Options customCommandLineOptions;
 
@@ -116,7 +114,7 @@ public class CliFrontend {
 
 	public CliFrontend(
 			Configuration configuration,
-			List<CustomCommandLine> customCommandLines) throws Exception {
+			List<CustomCommandLine<?>> customCommandLines) throws Exception {
 		this.configuration = Preconditions.checkNotNull(configuration);
 		this.customCommandLines = Preconditions.checkNotNull(customCommandLines);
 
@@ -129,7 +127,7 @@ public class CliFrontend {
 
 		this.customCommandLineOptions = new Options();
 
-		for (CustomCommandLine customCommandLine : customCommandLines) {
+		for (CustomCommandLine<?> customCommandLine : customCommandLines) {
 			customCommandLine.addGeneralOptions(customCommandLineOptions);
 			customCommandLine.addRunOptions(customCommandLineOptions);
 		}
@@ -196,14 +194,22 @@ public class CliFrontend {
 			throw new CliArgsException("Could not build the program from JAR file.", e);
 		}
 
-		final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine);
+		final CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
+		runProgram(customCommandLine, commandLine, runOptions, program);
+	}
+
+	private <T> void runProgram(
+			CustomCommandLine<T> customCommandLine,
+			CommandLine commandLine,
+			RunOptions runOptions,
+			PackagedProgram program) throws ProgramInvocationException, FlinkException {
+		final ClusterDescriptor<T> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
 
 		try {
-			final String clusterId = customCommandLine.getClusterId(commandLine);
+			final T clusterId = customCommandLine.getClusterId(commandLine);
 
-			final ClusterClient client;
+			final ClusterClient<T> client;
 
 			if (clusterId != null) {
 				client = clusterDescriptor.retrieve(clusterId);
@@ -235,7 +241,7 @@ public class CliFrontend {
 				if (clusterId == null && !client.isDetached()) {
 					// terminate the cluster only if we have started it before and if it's not detached
 					try {
-						clusterDescriptor.terminateCluster(client.getClusterIdentifier());
+						clusterDescriptor.terminateCluster(client.getClusterId());
 					} catch (FlinkException e) {
 						LOG.info("Could not properly terminate the Flink cluster.", e);
 					}
@@ -351,100 +357,86 @@ public class CliFrontend {
 			return;
 		}
 
-		boolean running = listOptions.getRunning();
-		boolean scheduled = listOptions.getScheduled();
+		final boolean running;
+		final boolean scheduled;
 
 		// print running and scheduled jobs if not option supplied
-		if (!running && !scheduled) {
+		if (!listOptions.getRunning() && !listOptions.getScheduled()) {
 			running = true;
 			scheduled = true;
+		} else {
+			running = listOptions.getRunning();
+			scheduled = listOptions.getScheduled();
 		}
 
-		final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
-		final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final String clusterId = activeCommandLine.getClusterId(commandLine);
-
-		if (clusterId == null) {
-			throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
-				"you would like to connect.");
-		}
+		runClusterAction(
+			activeCommandLine,
+			commandLine,
+			clusterClient -> listJobs(clusterClient, running, scheduled));
 
-		final ClusterClient client = clusterDescriptor.retrieve(clusterId);
+	}
 
+	private <T> void listJobs(
+			ClusterClient<T> clusterClient,
+			boolean running,
+			boolean scheduled) throws FlinkException {
+		Collection<JobStatusMessage> jobDetails;
 		try {
-			Collection<JobStatusMessage> jobDetails;
-			try {
-				CompletableFuture<Collection<JobStatusMessage>> jobDetailsFuture = client.listJobs();
+			CompletableFuture<Collection<JobStatusMessage>> jobDetailsFuture = clusterClient.listJobs();
 
-				try {
-					logAndSysout("Waiting for response...");
-					jobDetails = jobDetailsFuture.get();
-				}
-				catch (ExecutionException ee) {
-					Throwable cause = ExceptionUtils.stripExecutionException(ee);
-					throw new Exception("Failed to retrieve job list.", cause);
-				}
-			} finally {
-				client.shutdown();
-			}
+			logAndSysout("Waiting for response...");
+			jobDetails = jobDetailsFuture.get();
 
-			LOG.info("Successfully retrieved list of jobs");
+		} catch (Exception e) {
+			Throwable cause = ExceptionUtils.stripExecutionException(e);
+			throw new FlinkException("Failed to retrieve job list.", cause);
+		}
 
-			SimpleDateFormat dateFormat = new SimpleDateFormat("dd.MM.yyyy HH:mm:ss");
-			Comparator<JobStatusMessage> startTimeComparator = (o1, o2) -> (int) (o1.getStartTime() - o2.getStartTime());
+		LOG.info("Successfully retrieved list of jobs");
 
-			final List<JobStatusMessage> runningJobs = new ArrayList<>();
-			final List<JobStatusMessage> scheduledJobs = new ArrayList<>();
-			jobDetails.forEach(details -> {
-				if (details.getJobState() == JobStatus.CREATED) {
-					scheduledJobs.add(details);
-				} else {
-					runningJobs.add(details);
-				}
-			});
+		SimpleDateFormat dateFormat = new SimpleDateFormat("dd.MM.yyyy HH:mm:ss");
+		Comparator<JobStatusMessage> startTimeComparator = (o1, o2) -> (int) (o1.getStartTime() - o2.getStartTime());
 
-			if (running) {
-				if (runningJobs.size() == 0) {
-					System.out.println("No running jobs.");
-				}
-				else {
-					runningJobs.sort(startTimeComparator);
+		final List<JobStatusMessage> runningJobs = new ArrayList<>();
+		final List<JobStatusMessage> scheduledJobs = new ArrayList<>();
+		jobDetails.forEach(details -> {
+			if (details.getJobState() == JobStatus.CREATED) {
+				scheduledJobs.add(details);
+			} else {
+				runningJobs.add(details);
+			}
+		});
 
-					System.out.println("------------------ Running/Restarting Jobs -------------------");
-					for (JobStatusMessage runningJob : runningJobs) {
-						System.out.println(dateFormat.format(new Date(runningJob.getStartTime()))
-							+ " : " + runningJob.getJobId() + " : " + runningJob.getJobName() + " (" + runningJob.getJobState() + ")");
-					}
-					System.out.println("--------------------------------------------------------------");
-				}
+		if (running) {
+			if (runningJobs.size() == 0) {
+				System.out.println("No running jobs.");
 			}
-			if (scheduled) {
-				if (scheduledJobs.size() == 0) {
-					System.out.println("No scheduled jobs.");
-				}
-				else {
-					scheduledJobs.sort(startTimeComparator);
+			else {
+				runningJobs.sort(startTimeComparator);
 
-					System.out.println("----------------------- Scheduled Jobs -----------------------");
-					for (JobStatusMessage scheduledJob : scheduledJobs) {
-						System.out.println(dateFormat.format(new Date(scheduledJob.getStartTime()))
-							+ " : " + scheduledJob.getJobId() + " : " + scheduledJob.getJobName());
-					}
-					System.out.println("--------------------------------------------------------------");
+				System.out.println("------------------ Running/Restarting Jobs -------------------");
+				for (JobStatusMessage runningJob : runningJobs) {
+					System.out.println(dateFormat.format(new Date(runningJob.getStartTime()))
+						+ " : " + runningJob.getJobId() + " : " + runningJob.getJobName() + " (" + runningJob.getJobState() + ")");
 				}
+				System.out.println("--------------------------------------------------------------");
 			}
-		} finally {
-			try {
-				client.shutdown();
-			} catch (Exception e) {
-				LOG.info("Could not properly shut down the client.", e);
+		}
+		if (scheduled) {
+			if (scheduledJobs.size() == 0) {
+				System.out.println("No scheduled jobs.");
 			}
+			else {
+				scheduledJobs.sort(startTimeComparator);
 
-			try {
-				clusterDescriptor.close();
-			} catch (Exception e) {
-				LOG.info("Could not properly close the cluster descriptor.", e);
+				System.out.println("----------------------- Scheduled Jobs -----------------------");
+				for (JobStatusMessage scheduledJob : scheduledJobs) {
+					System.out.println(dateFormat.format(new Date(scheduledJob.getStartTime()))
+						+ " : " + scheduledJob.getJobId() + " : " + scheduledJob.getJobName());
+				}
+				System.out.println("--------------------------------------------------------------");
 			}
 		}
 	}
@@ -477,41 +469,26 @@ public class CliFrontend {
 		if (stopArgs.length > 0) {
 			String jobIdString = stopArgs[0];
 			jobId = parseJobId(jobIdString);
-		}
-		else {
+		} else {
 			throw new CliArgsException("Missing JobID");
 		}
 
-		final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
-
-		final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final String clusterId = activeCommandLine.getClusterId(commandLine);
-
-		if (clusterId == null) {
-			throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
-				"you would like to connect.");
-		}
+		logAndSysout("Stopping job " + jobId + '.');
 
-		final ClusterClient client = clusterDescriptor.retrieve(clusterId);
-
-		try {
-			logAndSysout("Stopping job " + jobId + '.');
-			client.stop(jobId);
-			logAndSysout("Stopped job " + jobId + '.');
-		} finally {
-			try {
-				client.shutdown();
-			} catch (Exception e) {
-				LOG.info("Could not properly shut down the client.", e);
-			}
+		runClusterAction(
+			activeCommandLine,
+			commandLine,
+			clusterClient -> {
+				try {
+					clusterClient.stop(jobId);
+				} catch (Exception e) {
+					throw new FlinkException("Could not stop the job " + jobId + '.', e);
+				}
+			});
 
-			try {
-				clusterDescriptor.close();
-			} catch (Exception e) {
-				LOG.info("Could not properly close the cluster descriptor.", e);
-			}
-		}
+		logAndSysout("Stopped job " + jobId + '.');
 	}
 
 	/**
@@ -536,71 +513,63 @@ public class CliFrontend {
 			return;
 		}
 
-		String[] cleanedArgs = cancelOptions.getArgs();
+		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
 
-		boolean withSavepoint = cancelOptions.isWithSavepoint();
-		String targetDirectory = cancelOptions.getSavepointTargetDirectory();
+		final String[] cleanedArgs = cancelOptions.getArgs();
 
-		JobID jobId;
+		if (cancelOptions.isWithSavepoint()) {
+			final JobID jobId;
+			final String targetDirectory;
 
-		// Figure out jobID. This is a little overly complicated, because
-		// we have to figure out whether the optional target directory
-		// is set:
-		// - cancel -s <jobID> => default target dir (JobID parsed as opt arg)
-		// - cancel -s <targetDir> <jobID> => custom target dir (parsed correctly)
-		if (cleanedArgs.length > 0) {
-			String jobIdString = cleanedArgs[0];
+			if (cleanedArgs.length > 0) {
+				jobId = parseJobId(cleanedArgs[0]);
+				targetDirectory = cancelOptions.getSavepointTargetDirectory();
+			} else {
+				jobId = parseJobId(cancelOptions.getSavepointTargetDirectory());
+				targetDirectory = null;
+			}
 
-			jobId = parseJobId(jobIdString);
-		} else if (targetDirectory != null)  {
-			// Try this for case: cancel -s <jobID> (default savepoint target dir)
-			String jobIdString = targetDirectory;
-			targetDirectory = null;
+			if (targetDirectory == null) {
+				logAndSysout("Cancelling job " + jobId + " with savepoint to default savepoint directory.");
+			} else {
+				logAndSysout("Cancelling job " + jobId + " with savepoint to " + targetDirectory + '.');
+			}
 
-			jobId = parseJobId(jobIdString);
+			runClusterAction(
+				activeCommandLine,
+				commandLine,
+				clusterClient -> {
+					final String savepointPath;
+					try {
+						savepointPath = clusterClient.cancelWithSavepoint(jobId, targetDirectory);
+					} catch (Exception e) {
+						throw new FlinkException("Could not cancel job " + jobId + '.', e);
+					}
+					logAndSysout("Cancelled job " + jobId + ". Savepoint stored in " + savepointPath + '.');
+				});
 		} else {
-			throw new CliArgsException("Missing JobID in the command line arguments.");
-		}
-
-		final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
-
-		final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+			final JobID jobId;
 
-		final String clusterId = activeCommandLine.getClusterId(commandLine);
-
-		if (clusterId == null) {
-			throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
-				"you would like to connect.");
-		}
-
-		final ClusterClient client = clusterDescriptor.retrieve(clusterId);
-
-		try {
-			if (withSavepoint) {
-				if (targetDirectory == null) {
-					logAndSysout("Cancelling job " + jobId + " with savepoint to default savepoint directory.");
-				} else {
-					logAndSysout("Cancelling job " + jobId + " with savepoint to " + targetDirectory + '.');
-				}
-				String savepointPath = client.cancelWithSavepoint(jobId, targetDirectory);
-				logAndSysout("Cancelled job " + jobId + ". Savepoint stored in " + savepointPath + '.');
+			if (cleanedArgs.length > 0) {
+				jobId = parseJobId(cleanedArgs[0]);
 			} else {
-				logAndSysout("Cancelling job " + jobId + '.');
-				client.cancel(jobId);
-				logAndSysout("Cancelled job " + jobId + '.');
-			}
-		} finally {
-			try {
-				client.shutdown();
-			} catch (Exception e) {
-				LOG.info("Could not properly shut down the client.", e);
+				throw new CliArgsException("Missing JobID. Specify a JobID to cancel a job.");
 			}
 
-			try {
-				clusterDescriptor.close();
-			} catch (Exception e) {
-				LOG.info("Could not properly close the cluster descriptor.", e);
-			}
+			logAndSysout("Cancelling job " + jobId + '.');
+
+			runClusterAction(
+				activeCommandLine,
+				commandLine,
+				clusterClient -> {
+					try {
+						clusterClient.cancel(jobId);
+					} catch (Exception e) {
+						throw new FlinkException("Could not cancel job " + jobId + '.', e);
+					}
+				});
+
+			logAndSysout("Cancelled job " + jobId + '.');
 		}
 	}
 
@@ -626,69 +595,52 @@ public class CliFrontend {
 			return;
 		}
 
-		CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine);
+		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
-
-		final String clusterId = customCommandLine.getClusterId(commandLine);
+		if (savepointOptions.isDispose()) {
+			runClusterAction(
+				activeCommandLine,
+				commandLine,
+				clusterClient -> disposeSavepoint(clusterClient, savepointOptions.getSavepointPath()));
+		} else {
+			String[] cleanedArgs = savepointOptions.getArgs();
 
-		if (clusterId == null) {
-			throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
-				"you would like to connect.");
-		}
+			final JobID jobId;
 
-		final ClusterClient clusterClient = clusterDescriptor.retrieve(clusterId);
+			if (cleanedArgs.length >= 1) {
+				String jobIdString = cleanedArgs[0];
 
-		try {
-			if (savepointOptions.isDispose()) {
-				// Discard
-				disposeSavepoint(clusterClient, savepointOptions.getSavepointPath());
+				jobId = parseJobId(jobIdString);
 			} else {
-				// Trigger
-				String[] cleanedArgs = savepointOptions.getArgs();
-				JobID jobId;
-
-				if (cleanedArgs.length >= 1) {
-					String jobIdString = cleanedArgs[0];
-
-					jobId = parseJobId(jobIdString);
-				} else {
-					throw new CliArgsException("Error: The value for the Job ID is not a valid ID. " +
-						"Specify a Job ID to trigger a savepoint.");
-				}
-
-				String savepointDirectory = null;
-				if (cleanedArgs.length >= 2) {
-					savepointDirectory = cleanedArgs[1];
-				}
-
-				// Print superfluous arguments
-				if (cleanedArgs.length >= 3) {
-					logAndSysout("Provided more arguments than required. Ignoring not needed arguments.");
-				}
-
-				triggerSavepoint(clusterClient, jobId, savepointDirectory);
+				throw new CliArgsException("Missing JobID. " +
+					"Specify a Job ID to trigger a savepoint.");
 			}
-		} finally {
-			try {
-				clusterClient.shutdown();
-			} catch (Exception e) {
-				LOG.info("Could not shutdown the cluster client.", e);
+
+			final String savepointDirectory;
+			if (cleanedArgs.length >= 2) {
+				savepointDirectory = cleanedArgs[1];
+			} else {
+				savepointDirectory = null;
 			}
 
-			try {
-				clusterDescriptor.close();
-			} catch (Exception e) {
-				LOG.info("Could not properly close the cluster descriptor.", e);
+			// Print superfluous arguments
+			if (cleanedArgs.length >= 3) {
+				logAndSysout("Provided more arguments than required. Ignoring not needed arguments.");
 			}
+
+			runClusterAction(
+				activeCommandLine,
+				commandLine,
+				clusterClient -> triggerSavepoint(clusterClient, jobId, savepointDirectory));
 		}
+
 	}
 
 	/**
 	 * Sends a {@link org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint}
 	 * message to the job manager.
 	 */
-	private String triggerSavepoint(ClusterClient clusterClient, JobID jobId, String savepointDirectory) throws FlinkException {
+	private String triggerSavepoint(ClusterClient<?> clusterClient, JobID jobId, String savepointDirectory) throws FlinkException {
 		logAndSysout("Triggering savepoint for job " + jobId + '.');
 		CompletableFuture<String> savepointPathFuture = clusterClient.triggerSavepoint(jobId, savepointDirectory);
 
@@ -713,7 +665,7 @@ public class CliFrontend {
 	/**
 	 * Sends a {@link JobManagerMessages.DisposeSavepoint} message to the job manager.
 	 */
-	private void disposeSavepoint(ClusterClient clusterClient, String savepointPath) throws FlinkException {
+	private void disposeSavepoint(ClusterClient<?> clusterClient, String savepointPath) throws FlinkException {
 		Preconditions.checkNotNull(savepointPath, "Missing required argument: savepoint path. " +
 			"Usage: bin/flink savepoint -d <savepoint-path>");
 
@@ -736,7 +688,7 @@ public class CliFrontend {
 	//  Interaction with programs and JobManager
 	// --------------------------------------------------------------------------------------------
 
-	protected void executeProgram(PackagedProgram program, ClusterClient client, int parallelism) throws ProgramMissingJobException, ProgramInvocationException {
+	protected void executeProgram(PackagedProgram program, ClusterClient<?> client, int parallelism) throws ProgramMissingJobException, ProgramInvocationException {
 		logAndSysout("Starting execution of program");
 
 		final JobSubmissionResult result = client.run(program, parallelism);
@@ -860,7 +812,7 @@ public class CliFrontend {
 			System.err.println(t.getCause().getMessage());
 			StackTraceElement[] trace = t.getCause().getStackTrace();
 			for (StackTraceElement ele: trace) {
-				System.err.println("\t" + ele.toString());
+				System.err.println("\t" + ele);
 				if (ele.getMethodName().equals("main")) {
 					break;
 				}
@@ -890,6 +842,65 @@ public class CliFrontend {
 		return jobId;
 	}
 
+	/**
+	 * Retrieves the {@link ClusterClient} from the given {@link CustomCommandLine} and runs the given
+	 * {@link ClusterAction} against it.
+	 *
+	 * @param activeCommandLine to create the {@link ClusterDescriptor} from
+	 * @param commandLine containing the parsed command line options
+	 * @param clusterAction the cluster action to run against the retrieved {@link ClusterClient}.
+	 * @param <T> type of the cluster id
+	 * @throws FlinkException if something goes wrong
+	 */
+	private <T> void runClusterAction(CustomCommandLine<T> activeCommandLine, CommandLine commandLine, ClusterAction<T> clusterAction) throws FlinkException {
+		final ClusterDescriptor<T> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+
+		final T clusterId = activeCommandLine.getClusterId(commandLine);
+
+		if (clusterId == null) {
+			throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
+				"you would like to connect.");
+		} else {
+			try {
+				final ClusterClient<T> clusterClient = clusterDescriptor.retrieve(clusterId);
+
+				try {
+					clusterAction.runAction(clusterClient);
+				} finally {
+					try {
+						clusterClient.shutdown();
+					} catch (Exception e) {
+						LOG.info("Could not properly shut down the cluster client.", e);
+					}
+				}
+			} finally {
+				try {
+					clusterDescriptor.close();
+				} catch (Exception e) {
+					LOG.info("Could not properly close the cluster descriptor.", e);
+				}
+			}
+		}
+	}
+
+	/**
+	 * Internal interface to encapsulate cluster actions which are executed via
+	 * the {@link ClusterClient}.
+	 *
+	 * @param <T> tyoe pf the cluster id
+	 */
+	@FunctionalInterface
+	private interface ClusterAction<T> {
+
+		/**
+		 * Run the cluster action with the given {@link ClusterClient}.
+		 *
+		 * @param clusterClient to run the cluster action against
+		 * @throws FlinkException if something goes wrong
+		 */
+		void runAction(ClusterClient<T> clusterClient) throws FlinkException;
+	}
+
 	// --------------------------------------------------------------------------------------------
 	//  Entry point for executable
 	// --------------------------------------------------------------------------------------------
@@ -981,7 +992,7 @@ public class CliFrontend {
 		final Configuration configuration = GlobalConfiguration.loadConfiguration(configurationDirectory);
 
 		// 3. load the custom command lines
-		final List<CustomCommandLine> customCommandLines = loadCustomCommandLines(
+		final List<CustomCommandLine<?>> customCommandLines = loadCustomCommandLines(
 			configuration,
 			configurationDirectory);
 
@@ -992,12 +1003,7 @@ public class CliFrontend {
 
 			SecurityUtils.install(new SecurityConfiguration(cli.configuration));
 			int retCode = SecurityUtils.getInstalledContext()
-					.runSecured(new Callable<Integer>() {
-						@Override
-						public Integer call() {
-							return cli.parseParameters(args);
-						}
-					});
+					.runSecured(() -> cli.parseParameters(args));
 			System.exit(retCode);
 		}
 		catch (Throwable t) {
@@ -1048,8 +1054,8 @@ public class CliFrontend {
 		config.setInteger(JobManagerOptions.PORT, address.getPort());
 	}
 
-	public static List<CustomCommandLine> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
-		List<CustomCommandLine> customCommandLines = new ArrayList<>(2);
+	public static List<CustomCommandLine<?>> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
+		List<CustomCommandLine<?>> customCommandLines = new ArrayList<>(2);
 
 		//	Command line interface of the YARN session, with a special initialization here
 		//	to prefix all options with y/yarn.
@@ -1082,8 +1088,8 @@ public class CliFrontend {
 	 * @param commandLine The input to the command-line.
 	 * @return custom command-line which is active (may only be one at a time)
 	 */
-	public CustomCommandLine getActiveCustomCommandLine(CommandLine commandLine) {
-		for (CustomCommandLine cli : customCommandLines) {
+	public CustomCommandLine<?> getActiveCustomCommandLine(CommandLine commandLine) {
+		for (CustomCommandLine<?> cli : customCommandLines) {
 			if (cli.isActive(commandLine)) {
 				return cli;
 			}
@@ -1096,7 +1102,7 @@ public class CliFrontend {
 	 * @param className The fully-qualified class name to load.
 	 * @param params The constructor parameters
 	 */
-	private static CustomCommandLine loadCustomCommandLine(String className, Object... params) throws IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, NoSuchMethodException {
+	private static CustomCommandLine<?> loadCustomCommandLine(String className, Object... params) throws IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, NoSuchMethodException {
 
 		Class<? extends CustomCommandLine> customCliClass =
 			Class.forName(className).asSubclass(CustomCommandLine.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
index 10507d6..475d854 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
@@ -236,7 +236,7 @@ public class CliFrontendParser {
 	/**
 	 * Prints the help for the client.
 	 */
-	public static void printHelp(Collection<CustomCommandLine> customCommandLines) {
+	public static void printHelp(Collection<CustomCommandLine<?>> customCommandLines) {
 		System.out.println("./flink <ACTION> [OPTIONS] [ARGUMENTS]");
 		System.out.println();
 		System.out.println("The following actions are available:");
@@ -251,7 +251,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForRun(Collection<CustomCommandLine> customCommandLines) {
+	public static void printHelpForRun(Collection<CustomCommandLine<?>> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -279,7 +279,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForList(Collection<CustomCommandLine> customCommandLines) {
+	public static void printHelpForList(Collection<CustomCommandLine<?>> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -294,7 +294,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForStop(Collection<CustomCommandLine> customCommandLines) {
+	public static void printHelpForStop(Collection<CustomCommandLine<?>> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -309,7 +309,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForCancel(Collection<CustomCommandLine> customCommandLines) {
+	public static void printHelpForCancel(Collection<CustomCommandLine<?>> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -324,7 +324,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForSavepoint(Collection<CustomCommandLine> customCommandLines) {
+	public static void printHelpForSavepoint(Collection<CustomCommandLine<?>> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -345,7 +345,7 @@ public class CliFrontendParser {
 	 * @param runOptions True if the run options should be printed, False to print only general options
 	 */
 	private static void printCustomCliOptions(
-			Collection<CustomCommandLine> customCommandLines,
+			Collection<CustomCommandLine<?>> customCommandLines,
 			HelpFormatter formatter,
 			boolean runOptions) {
 		// prints options from all available command-line classes

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
index aabc224..e939974 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
@@ -30,7 +30,7 @@ import javax.annotation.Nullable;
 /**
  * Custom command-line interface to load hooks for the command-line interface.
  */
-public interface CustomCommandLine {
+public interface CustomCommandLine<T> {
 
 	/**
 	 * Signals whether the custom command-line wants to execute or not.
@@ -66,7 +66,7 @@ public interface CustomCommandLine {
 	 * @return ClusterDescriptor
 	 * @throws FlinkException if the ClusterDescriptor could not be created
 	 */
-	ClusterDescriptor createClusterDescriptor(CommandLine commandLine) throws FlinkException;
+	ClusterDescriptor<T> createClusterDescriptor(CommandLine commandLine) throws FlinkException;
 
 	/**
 	 * Returns the cluster id if a cluster id was specified on the command line, otherwise it
@@ -79,7 +79,7 @@ public interface CustomCommandLine {
 	 * @return Cluster id identifying the cluster to deploy jobs to or null
 	 */
 	@Nullable
-	String getClusterId(CommandLine commandLine);
+	T getClusterId(CommandLine commandLine);
 
 	/**
 	 * Returns the {@link ClusterSpecification} specified by the configuration and the command

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index 5660765..d2694b8 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.client.cli;
 
-import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
+import org.apache.flink.client.deployment.StandaloneClusterId;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.FlinkException;
 
@@ -31,7 +31,7 @@ import javax.annotation.Nullable;
 /**
  * The default CLI which is used for interaction with standalone clusters.
  */
-public class DefaultCLI extends AbstractCustomCommandLine {
+public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterId> {
 
 	public DefaultCLI(Configuration configuration) {
 		super(configuration);
@@ -49,7 +49,7 @@ public class DefaultCLI extends AbstractCustomCommandLine {
 	}
 
 	@Override
-	public ClusterDescriptor createClusterDescriptor(
+	public StandaloneClusterDescriptor createClusterDescriptor(
 			CommandLine commandLine) throws FlinkException {
 		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
 
@@ -58,8 +58,8 @@ public class DefaultCLI extends AbstractCustomCommandLine {
 
 	@Override
 	@Nullable
-	public String getClusterId(CommandLine commandLine) {
-		return "standalone";
+	public StandaloneClusterId getClusterId(CommandLine commandLine) {
+		return StandaloneClusterId.getInstance();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
index 1a75aac..c8fab72 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.client.cli;
 
-import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.deployment.Flip6StandaloneClusterDescriptor;
+import org.apache.flink.client.deployment.StandaloneClusterId;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.FlinkException;
 
@@ -33,7 +33,7 @@ import javax.annotation.Nullable;
 /**
  * The default CLI which is used for interaction with standalone clusters.
  */
-public class Flip6DefaultCLI extends AbstractCustomCommandLine {
+public class Flip6DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterId> {
 
 	public static final Option FLIP_6 = new Option("flip6", "Switches the client to Flip-6 mode.");
 
@@ -62,7 +62,7 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine {
 	}
 
 	@Override
-	public ClusterDescriptor createClusterDescriptor(
+	public Flip6StandaloneClusterDescriptor createClusterDescriptor(
 			CommandLine commandLine) throws FlinkException {
 		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
 
@@ -71,8 +71,8 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine {
 
 	@Override
 	@Nullable
-	public String getClusterId(CommandLine commandLine) {
-		return "flip6Standalone";
+	public StandaloneClusterId getClusterId(CommandLine commandLine) {
+		return StandaloneClusterId.getInstance();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
index b1f566c..aadb5fe 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
@@ -24,8 +24,10 @@ import org.apache.flink.util.FlinkException;
 
 /**
  * A descriptor to deploy a cluster (e.g. Yarn or Mesos) and return a Client for Cluster communication.
+ *
+ * @param <T> Type of the cluster id
  */
-public interface ClusterDescriptor extends AutoCloseable {
+public interface ClusterDescriptor<T> extends AutoCloseable {
 
 	/**
 	 * Returns a String containing details about the cluster (NodeManagers, available memory, ...).
@@ -35,19 +37,19 @@ public interface ClusterDescriptor extends AutoCloseable {
 
 	/**
 	 * Retrieves an existing Flink Cluster.
-	 * @param applicationID The unique application identifier of the running cluster
+	 * @param clusterId The unique identifier of the running cluster
 	 * @return Client for the cluster
-	 * @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
+	 * @throws ClusterRetrieveException if the cluster client could not be retrieved
 	 */
-	ClusterClient retrieve(String applicationID) throws UnsupportedOperationException;
+	ClusterClient<T> retrieve(T clusterId) throws ClusterRetrieveException;
 
 	/**
 	 * Triggers deployment of a cluster.
 	 * @param clusterSpecification Cluster specification defining the cluster to deploy
 	 * @return Client for the cluster
-	 * @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
+	 * @throws ClusterDeploymentException if the cluster could not be deployed
 	 */
-	ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException;
+	ClusterClient<T> deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException;
 
 	/**
 	 * Deploys a per-job cluster with the given job on the cluster.
@@ -57,9 +59,9 @@ public interface ClusterDescriptor extends AutoCloseable {
 	 * @return Cluster client to talk to the Flink cluster
 	 * @throws ClusterDeploymentException if the cluster could not be deployed
 	 */
-	ClusterClient deployJobCluster(
+	ClusterClient<T> deployJobCluster(
 		final ClusterSpecification clusterSpecification,
-		final JobGraph jobGraph);
+		final JobGraph jobGraph) throws ClusterDeploymentException;
 
 	/**
 	 * Terminates the cluster with the given cluster id.
@@ -67,5 +69,5 @@ public interface ClusterDescriptor extends AutoCloseable {
 	 * @param clusterId identifying the cluster to shut down
 	 * @throws FlinkException if the cluster could not be terminated
 	 */
-	void terminateCluster(String clusterId) throws FlinkException;
+	void terminateCluster(T clusterId) throws FlinkException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterRetrieveException.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterRetrieveException.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterRetrieveException.java
new file mode 100644
index 0000000..7234d1b
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterRetrieveException.java
@@ -0,0 +1,41 @@
+/*
+ * 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.client.deployment;
+
+import org.apache.flink.util.FlinkException;
+
+/**
+ * Exception which indicates that a cluster could not be retrieved.
+ */
+public class ClusterRetrieveException extends FlinkException {
+
+	private static final long serialVersionUID = 7718062507419172318L;
+
+	public ClusterRetrieveException(String message) {
+		super(message);
+	}
+
+	public ClusterRetrieveException(Throwable cause) {
+		super(cause);
+	}
+
+	public ClusterRetrieveException(String message, Throwable cause) {
+		super(message, cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
index 70fd9f7..8096da8 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
@@ -28,7 +28,7 @@ import org.apache.flink.util.Preconditions;
 /**
  * A deployment descriptor for an existing cluster.
  */
-public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor {
+public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor<StandaloneClusterId> {
 
 	private final Configuration config;
 
@@ -44,27 +44,27 @@ public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor {
 	}
 
 	@Override
-	public RestClusterClient retrieve(String applicationID) {
+	public RestClusterClient<StandaloneClusterId> retrieve(StandaloneClusterId standaloneClusterId) throws ClusterRetrieveException {
 		try {
-			return new RestClusterClient(config);
+			return new RestClusterClient<>(config, standaloneClusterId);
 		} catch (Exception e) {
-			throw new RuntimeException("Couldn't retrieve FLIP-6 standalone cluster", e);
+			throw new ClusterRetrieveException("Couldn't retrieve FLIP-6 standalone cluster", e);
 		}
 	}
 
 	@Override
-	public RestClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
+	public RestClusterClient<StandaloneClusterId> deploySessionCluster(ClusterSpecification clusterSpecification) {
 		throw new UnsupportedOperationException("Can't deploy a FLIP-6 standalone cluster.");
 	}
 
 	@Override
-	public RestClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
+	public RestClusterClient<StandaloneClusterId> deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
 		throw new UnsupportedOperationException("Can't deploy a standalone FLIP-6 per-job cluster.");
 	}
 
 	@Override
-	public void terminateCluster(String clusterId) throws FlinkException {
-		throw new UnsupportedOperationException("Cannot terminate a standalone Flip-6 cluster.");
+	public void terminateCluster(StandaloneClusterId clusterId) throws FlinkException {
+		throw new UnsupportedOperationException("Cannot terminate a Flip-6 standalone cluster.");
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
index 5638232..62908fe 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
@@ -27,7 +27,7 @@ import org.apache.flink.util.FlinkException;
 /**
  * A deployment descriptor for an existing cluster.
  */
-public class StandaloneClusterDescriptor implements ClusterDescriptor {
+public class StandaloneClusterDescriptor implements ClusterDescriptor<StandaloneClusterId> {
 
 	private final Configuration config;
 
@@ -43,16 +43,16 @@ public class StandaloneClusterDescriptor implements ClusterDescriptor {
 	}
 
 	@Override
-	public StandaloneClusterClient retrieve(String applicationID) {
+	public StandaloneClusterClient retrieve(StandaloneClusterId standaloneClusterId) throws ClusterRetrieveException {
 		try {
 			return new StandaloneClusterClient(config);
 		} catch (Exception e) {
-			throw new RuntimeException("Couldn't retrieve standalone cluster", e);
+			throw new ClusterRetrieveException("Couldn't retrieve standalone cluster", e);
 		}
 	}
 
 	@Override
-	public StandaloneClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
+	public StandaloneClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) {
 		throw new UnsupportedOperationException("Can't deploy a standalone cluster.");
 	}
 
@@ -62,8 +62,8 @@ public class StandaloneClusterDescriptor implements ClusterDescriptor {
 	}
 
 	@Override
-	public void terminateCluster(String clusterId) throws FlinkException {
-		throw new UnsupportedOperationException("Cannot terminate a standalone cluster.");
+	public void terminateCluster(StandaloneClusterId clusterId) throws FlinkException {
+		throw new UnsupportedOperationException("Cannot terminate standalone clusters.");
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterId.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterId.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterId.java
new file mode 100644
index 0000000..7201f1d
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterId.java
@@ -0,0 +1,32 @@
+/*
+ * 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.client.deployment;
+
+/**
+ * Identifier for standalone clusters.
+ */
+public class StandaloneClusterId {
+	private static final StandaloneClusterId INSTANCE = new StandaloneClusterId();
+
+	private StandaloneClusterId() {}
+
+	public static StandaloneClusterId getInstance() {
+		return INSTANCE;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
index efa23fb..b992e60 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
@@ -93,8 +93,10 @@ import scala.concurrent.duration.FiniteDuration;
 
 /**
  * Encapsulates the functionality necessary to submit a program to a remote cluster.
+ *
+ * @param <T> type of the cluster id
  */
-public abstract class ClusterClient {
+public abstract class ClusterClient<T> {
 
 	protected final Logger log = LoggerFactory.getLogger(getClass());
 
@@ -937,9 +939,11 @@ public abstract class ClusterClient {
 	public abstract List<String> getNewMessages();
 
 	/**
-	 * Returns a string representation of the cluster.
+	 * Returns the cluster id identifying the cluster to which the client is connected.
+	 *
+	 * @return cluster id of the connected cluster
 	 */
-	public abstract String getClusterIdentifier();
+	public abstract T getClusterId();
 
 	/**
 	 * Set the mode of this client (detached or blocking job execution).

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
index 7e47825..13f49d3 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
@@ -35,7 +35,7 @@ import java.util.List;
  */
 public class ContextEnvironment extends ExecutionEnvironment {
 
-	protected final ClusterClient client;
+	protected final ClusterClient<?> client;
 
 	protected final List<URL> jarFilesToAttach;
 
@@ -45,7 +45,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
 
 	protected final SavepointRestoreSettings savepointSettings;
 
-	public ContextEnvironment(ClusterClient remoteConnection, List<URL> jarFiles, List<URL> classpaths,
+	public ContextEnvironment(ClusterClient<?> remoteConnection, List<URL> jarFiles, List<URL> classpaths,
 				ClassLoader userCodeClassLoader, SavepointRestoreSettings savepointSettings) {
 		this.client = remoteConnection;
 		this.jarFilesToAttach = jarFiles;
@@ -84,7 +84,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
 				+ ") : " + getIdString();
 	}
 
-	public ClusterClient getClient() {
+	public ClusterClient<?> getClient() {
 		return this.client;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java
index 6209254..64b1863 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java
@@ -33,7 +33,7 @@ import java.util.List;
  */
 public class ContextEnvironmentFactory implements ExecutionEnvironmentFactory {
 
-	private final ClusterClient client;
+	private final ClusterClient<?> client;
 
 	private final List<URL> jarFilesToAttach;
 
@@ -49,7 +49,7 @@ public class ContextEnvironmentFactory implements ExecutionEnvironmentFactory {
 
 	private SavepointRestoreSettings savepointSettings;
 
-	public ContextEnvironmentFactory(ClusterClient client, List<URL> jarFilesToAttach,
+	public ContextEnvironmentFactory(ClusterClient<?> client, List<URL> jarFilesToAttach,
 			List<URL> classpathsToAttach, ClassLoader userCodeClassLoader, int defaultParallelism,
 			boolean isDetached, SavepointRestoreSettings savepointSettings) {
 		this.client = client;

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java
index 63aa811..9c7b639 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java
@@ -44,7 +44,7 @@ public class DetachedEnvironment extends ContextEnvironment {
 	private static final Logger LOG = LoggerFactory.getLogger(DetachedEnvironment.class);
 
 	public DetachedEnvironment(
-			ClusterClient remoteConnection,
+			ClusterClient<?> remoteConnection,
 			List<URL> jarFiles,
 			List<URL> classpaths,
 			ClassLoader userCodeClassLoader,

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
index 1782a25..df08c30 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
@@ -19,6 +19,7 @@
 package org.apache.flink.client.program;
 
 import org.apache.flink.api.common.JobSubmissionResult;
+import org.apache.flink.client.deployment.StandaloneClusterId;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.WebOptions;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus;
@@ -38,7 +39,7 @@ import scala.concurrent.Future;
  * Cluster client for communication with an standalone (on-premise) cluster or an existing cluster that has been
  * brought up independently of a specific job.
  */
-public class StandaloneClusterClient extends ClusterClient {
+public class StandaloneClusterClient extends ClusterClient<StandaloneClusterId> {
 
 	public StandaloneClusterClient(Configuration config) throws Exception {
 		super(config);
@@ -81,9 +82,8 @@ public class StandaloneClusterClient extends ClusterClient {
 	}
 
 	@Override
-	public String getClusterIdentifier() {
-		// Avoid blocking here by getting the address from the config without resolving the address
-		return "Standalone cluster with JobManager at " + this.getJobManagerAddress();
+	public StandaloneClusterId getClusterId() {
+		return StandaloneClusterId.getInstance();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
index a6bff1a..3e8b136 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
@@ -59,6 +59,7 @@ import org.apache.flink.runtime.rest.messages.queue.QueueStatus;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.util.ExecutorUtils;
 import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.SerializedThrowable;
 
 import javax.annotation.Nullable;
@@ -83,23 +84,31 @@ import static org.apache.flink.util.Preconditions.checkState;
 /**
  * A {@link ClusterClient} implementation that communicates via HTTP REST requests.
  */
-public class RestClusterClient extends ClusterClient {
+public class RestClusterClient<T> extends ClusterClient<T> {
 
 	private final RestClusterClientConfiguration restClusterClientConfiguration;
+
 	private final RestClient restClient;
+
 	private final ExecutorService executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("Flink-RestClusterClient-IO"));
 	private final WaitStrategy waitStrategy;
 
-	public RestClusterClient(Configuration config) throws Exception {
-		this(config, new ExponentialWaitStrategy(10, 2000));
+	private final T clusterId;
+
+	public RestClusterClient(Configuration config, T clusterId) throws Exception {
+		this(
+			config,
+			clusterId,
+			new ExponentialWaitStrategy(10L, 2000L));
 	}
 
 	@VisibleForTesting
-	RestClusterClient(Configuration configuration, WaitStrategy waitStrategy) throws Exception {
+	RestClusterClient(Configuration configuration, T clusterId, WaitStrategy waitStrategy) throws Exception {
 		super(configuration);
 		this.restClusterClientConfiguration = RestClusterClientConfiguration.fromConfiguration(configuration);
 		this.restClient = new RestClient(restClusterClientConfiguration.getRestClientConfiguration(), executorService);
 		this.waitStrategy = requireNonNull(waitStrategy);
+		this.clusterId = Preconditions.checkNotNull(clusterId);
 	}
 
 	@Override
@@ -295,16 +304,16 @@ public class RestClusterClient extends ClusterClient {
 			});
 	}
 
+	@Override
+	public T getClusterId() {
+		return clusterId;
+	}
+
 	// ======================================
 	// Legacy stuff we actually implement
 	// ======================================
 
 	@Override
-	public String getClusterIdentifier() {
-		return "Flip-6 Standalone cluster with dispatcher at " + restClusterClientConfiguration.getRestServerAddress() + '.';
-	}
-
-	@Override
 	public boolean hasUserJarsInClassPath(List<URL> userJarFiles) {
 		return false;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
index 60bd308..b2fa003 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
@@ -52,7 +52,7 @@ public class CliFrontendCancelTest extends TestLogger {
 		JobID jid = new JobID();
 
 		String[] parameters = { jid.toString() };
-		final ClusterClient clusterClient = createClusterClient();
+		final ClusterClient<String> clusterClient = createClusterClient();
 		MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
 
 		testFrontend.cancel(parameters);
@@ -90,7 +90,7 @@ public class CliFrontendCancelTest extends TestLogger {
 			JobID jid = new JobID();
 
 			String[] parameters = { "-s", jid.toString() };
-			final ClusterClient clusterClient = createClusterClient();
+			final ClusterClient<String> clusterClient = createClusterClient();
 			MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
 			testFrontend.cancel(parameters);
 
@@ -103,7 +103,7 @@ public class CliFrontendCancelTest extends TestLogger {
 			JobID jid = new JobID();
 
 			String[] parameters = { "-s", "targetDirectory", jid.toString() };
-			final ClusterClient clusterClient = createClusterClient();
+			final ClusterClient<String> clusterClient = createClusterClient();
 			MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
 			testFrontend.cancel(parameters);
 
@@ -134,8 +134,8 @@ public class CliFrontendCancelTest extends TestLogger {
 		testFrontend.cancel(parameters);
 	}
 
-	private static ClusterClient createClusterClient() throws Exception {
-		final ClusterClient clusterClient = mock(ClusterClient.class);
+	private static ClusterClient<String> createClusterClient() throws Exception {
+		final ClusterClient<String> clusterClient = mock(ClusterClient.class);
 
 		return clusterClient;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
index 77d8016..760b376 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
@@ -49,7 +49,7 @@ public class CliFrontendListTest extends TestLogger {
 		// test list properly
 		{
 			String[] parameters = {"-r", "-s"};
-			ClusterClient clusterClient = createClusterClient();
+			ClusterClient<String> clusterClient = createClusterClient();
 			MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
 			testFrontend.list(parameters);
 			Mockito.verify(clusterClient, times(1))
@@ -67,8 +67,8 @@ public class CliFrontendListTest extends TestLogger {
 		testFrontend.list(parameters);
 	}
 
-	private static ClusterClient createClusterClient() throws Exception {
-		final ClusterClient clusterClient = mock(ClusterClient.class);
+	private static ClusterClient<String> createClusterClient() throws Exception {
+		final ClusterClient<String> clusterClient = mock(ClusterClient.class);
 
 		when(clusterClient.listJobs()).thenReturn(CompletableFuture.completedFuture(Collections.emptyList()));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java
index 7b31a3e..d730344 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java
@@ -82,7 +82,7 @@ public class CliFrontendSavepointTest extends TestLogger {
 
 		String savepointPath = "expectedSavepointPath";
 
-		final ClusterClient clusterClient = createClusterClient(savepointPath);
+		final ClusterClient<String> clusterClient = createClusterClient(savepointPath);
 
 		try {
 			MockedCliFrontend frontend = new MockedCliFrontend(clusterClient);
@@ -110,7 +110,7 @@ public class CliFrontendSavepointTest extends TestLogger {
 		String expectedTestException = "expectedTestException";
 		Exception testException = new Exception(expectedTestException);
 
-		final ClusterClient clusterClient = createFailingClusterClient(testException);
+		final ClusterClient<String> clusterClient = createFailingClusterClient(testException);
 
 		try {
 			MockedCliFrontend frontend = new MockedCliFrontend(clusterClient);
@@ -165,7 +165,7 @@ public class CliFrontendSavepointTest extends TestLogger {
 
 		String savepointDirectory = "customTargetDirectory";
 
-		final ClusterClient clusterClient = createClusterClient(savepointDirectory);
+		final ClusterClient<String> clusterClient = createClusterClient(savepointDirectory);
 
 		try {
 			MockedCliFrontend frontend = new MockedCliFrontend(clusterClient);
@@ -224,7 +224,7 @@ public class CliFrontendSavepointTest extends TestLogger {
 
 		final CompletableFuture<String> disposeSavepointFuture = new CompletableFuture<>();
 
-		final ClusterClient clusterClient = new DisposeSavepointClusterClient(
+		final DisposeSavepointClusterClient clusterClient = new DisposeSavepointClusterClient(
 			(String savepointPath, Time timeout) -> {
 				disposeSavepointFuture.complete(savepointPath);
 				return CompletableFuture.completedFuture(Acknowledge.get());
@@ -260,7 +260,7 @@ public class CliFrontendSavepointTest extends TestLogger {
 
 		Exception testException = new Exception("expectedTestException");
 
-		ClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> FutureUtils.completedExceptionally(testException));
+		DisposeSavepointClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> FutureUtils.completedExceptionally(testException));
 
 		try {
 			CliFrontend frontend = new MockedCliFrontend(clusterClient);
@@ -313,8 +313,8 @@ public class CliFrontendSavepointTest extends TestLogger {
 		System.setErr(stdErr);
 	}
 
-	private static ClusterClient createClusterClient(String expectedResponse) throws Exception {
-		final ClusterClient clusterClient = mock(ClusterClient.class);
+	private static ClusterClient<String> createClusterClient(String expectedResponse) throws Exception {
+		final ClusterClient<String> clusterClient = mock(ClusterClient.class);
 
 		when(clusterClient.triggerSavepoint(any(JobID.class), anyString()))
 			.thenReturn(CompletableFuture.completedFuture(expectedResponse));
@@ -322,8 +322,8 @@ public class CliFrontendSavepointTest extends TestLogger {
 		return clusterClient;
 	}
 
-	private static ClusterClient createFailingClusterClient(Exception expectedException) throws Exception {
-		final ClusterClient clusterClient = mock(ClusterClient.class);
+	private static ClusterClient<String> createFailingClusterClient(Exception expectedException) throws Exception {
+		final ClusterClient<String> clusterClient = mock(ClusterClient.class);
 
 		when(clusterClient.triggerSavepoint(any(JobID.class), anyString()))
 			.thenReturn(FutureUtils.completedExceptionally(expectedException));

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
index 0120cdf..d6049e5 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.client.cli.util.MockedCliFrontend;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.TestLogger;
 
@@ -29,9 +30,12 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import javax.annotation.Nullable;
+
 import java.util.Collections;
 
 import static org.apache.flink.client.cli.CliFrontendTestUtils.pipeSystemOutToNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
@@ -51,18 +55,16 @@ public class CliFrontendStopTest extends TestLogger {
 	@Test
 	public void testStop() throws Exception {
 		// test stop properly
-		{
-			JobID jid = new JobID();
-			String jidString = jid.toString();
+		JobID jid = new JobID();
+		String jidString = jid.toString();
 
-			String[] parameters = { jidString };
-			final ClusterClient clusterClient = createClusterClient(false);
-			MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
+		String[] parameters = { jidString };
+		final ClusterClient<String> clusterClient = createClusterClient(null);
+		MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
 
-			testFrontend.stop(parameters);
+		testFrontend.stop(parameters);
 
-			Mockito.verify(clusterClient, times(1)).stop(any(JobID.class));
-		}
+		Mockito.verify(clusterClient, times(1)).stop(any(JobID.class));
 	}
 
 	@Test(expected = CliArgsException.class)
@@ -87,32 +89,30 @@ public class CliFrontendStopTest extends TestLogger {
 		testFrontend.stop(parameters);
 	}
 
-	@Test(expected = TestException.class)
+	@Test
 	public void testUnknownJobId() throws Exception {
 		// test unknown job Id
 		JobID jid = new JobID();
 
 		String[] parameters = { jid.toString() };
-		final ClusterClient clusterClient = createClusterClient(true);
+		String expectedMessage = "Test exception";
+		FlinkException testException = new FlinkException(expectedMessage);
+		final ClusterClient<String> clusterClient = createClusterClient(testException);
 		MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient);
 
-		testFrontend.stop(parameters);
-		fail("Should have failed.");
-	}
-
-	private static final class TestException extends FlinkException {
-		private static final long serialVersionUID = -2650760898729937583L;
-
-		TestException(String message) {
-			super(message);
+		try {
+			testFrontend.stop(parameters);
+			fail("Should have failed.");
+		} catch (FlinkException e) {
+			assertTrue(ExceptionUtils.findThrowableWithMessage(e, expectedMessage).isPresent());
 		}
 	}
 
-	private static ClusterClient createClusterClient(boolean reject) throws Exception {
-		final ClusterClient clusterClient = mock(ClusterClient.class);
+	private static ClusterClient<String> createClusterClient(@Nullable Exception exception) throws Exception {
+		final ClusterClient<String> clusterClient = mock(ClusterClient.class);
 
-		if (reject) {
-				doThrow(new TestException("Test Exception")).when(clusterClient).stop(any(JobID.class));
+		if (exception != null) {
+			doThrow(exception).when(clusterClient).stop(any(JobID.class));
 		}
 
 		return clusterClient;

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
index 6eb005d..aaca798 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.client.cli;
 
-import org.apache.flink.client.deployment.ClusterDescriptor;
+import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.JobManagerOptions;
@@ -62,9 +62,9 @@ public class DefaultCLITest extends TestLogger {
 
 		final InetSocketAddress expectedAddress = new InetSocketAddress(localhost, port);
 
-		final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
+		final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
 
-		final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
+		final ClusterClient<?> clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
 
 		Assert.assertEquals(expectedAddress, clusterClient.getJobManagerAddress());
 	}
@@ -89,9 +89,9 @@ public class DefaultCLITest extends TestLogger {
 
 		CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
 
-		final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
+		final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
 
-		final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
+		final ClusterClient<?> clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
 
 		final InetSocketAddress expectedAddress = new InetSocketAddress(manualHostname, manualPort);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
index 2bb3ed0..0ceb95e 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
@@ -28,11 +28,11 @@ import org.apache.flink.util.Preconditions;
 /**
  * Dummy {@link ClusterDescriptor} implementation for testing purposes.
  */
-public class DummyClusterDescriptor implements ClusterDescriptor {
+public class DummyClusterDescriptor<T> implements ClusterDescriptor<T> {
 
-	private final ClusterClient clusterClient;
+	private final ClusterClient<T> clusterClient;
 
-	public DummyClusterDescriptor(ClusterClient clusterClient) {
+	public DummyClusterDescriptor(ClusterClient<T> clusterClient) {
 		this.clusterClient = Preconditions.checkNotNull(clusterClient);
 	}
 
@@ -42,23 +42,23 @@ public class DummyClusterDescriptor implements ClusterDescriptor {
 	}
 
 	@Override
-	public ClusterClient retrieve(String applicationID) throws UnsupportedOperationException {
+	public ClusterClient<T> retrieve(T clusterId) {
 		return clusterClient;
 	}
 
 	@Override
-	public ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
+	public ClusterClient<T> deploySessionCluster(ClusterSpecification clusterSpecification) {
 		return clusterClient;
 	}
 
 	@Override
-	public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
+	public ClusterClient<T> deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
 		return clusterClient;
 	}
 
 	@Override
-	public void terminateCluster(String clusterId) throws FlinkException {
-		throw new UnsupportedOperationException("DummyClusterDescriptor does not support cluster termination.");
+	public void terminateCluster(T clusterId) throws FlinkException {
+		throw new UnsupportedOperationException("Cannot terminate a dummy cluster.");
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
index 5279d85..12bea74 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
@@ -32,10 +32,10 @@ import javax.annotation.Nullable;
 /**
  * Dummy implementation of the {@link CustomCommandLine} for testing purposes.
  */
-public class DummyCustomCommandLine implements CustomCommandLine {
-	private final ClusterClient clusterClient;
+public class DummyCustomCommandLine<T> implements CustomCommandLine {
+	private final ClusterClient<T> clusterClient;
 
-	public DummyCustomCommandLine(ClusterClient clusterClient) {
+	public DummyCustomCommandLine(ClusterClient<T> clusterClient) {
 		this.clusterClient = Preconditions.checkNotNull(clusterClient);
 	}
 
@@ -60,8 +60,8 @@ public class DummyCustomCommandLine implements CustomCommandLine {
 	}
 
 	@Override
-	public ClusterDescriptor createClusterDescriptor(CommandLine commandLine) {
-		return new DummyClusterDescriptor(clusterClient);
+	public ClusterDescriptor<T> createClusterDescriptor(CommandLine commandLine) {
+		return new DummyClusterDescriptor<>(clusterClient);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
index c2505ae..72767c7 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java
@@ -101,7 +101,7 @@ public class ClientConnectionTest extends TestLogger {
 		config.setString(JobManagerOptions.ADDRESS, unreachableEndpoint.getHostName());
 		config.setInteger(JobManagerOptions.PORT, unreachableEndpoint.getPort());
 
-		ClusterClient client = new StandaloneClusterClient(config);
+		StandaloneClusterClient client = new StandaloneClusterClient(config);
 
 		try {
 			// we have to query the cluster status to start the connection attempts
@@ -140,7 +140,7 @@ public class ClientConnectionTest extends TestLogger {
 
 			highAvailabilityServices.setJobMasterLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID, testingLeaderRetrievalService);
 
-			ClusterClient client = new StandaloneClusterClient(configuration, highAvailabilityServices);
+			StandaloneClusterClient client = new StandaloneClusterClient(configuration, highAvailabilityServices);
 
 			ActorGateway gateway = client.getJobManagerGateway();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
index 99f51ad..ca8b6fb 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
@@ -131,7 +131,7 @@ public class ClientTest extends TestLogger {
 		jobManagerSystem.actorOf(
 			Props.create(SuccessReturningActor.class),
 			JobMaster.JOB_MANAGER_NAME);
-		ClusterClient out = new StandaloneClusterClient(config);
+		StandaloneClusterClient out = new StandaloneClusterClient(config);
 		out.setDetached(true);
 
 		try {
@@ -204,7 +204,7 @@ public class ClientTest extends TestLogger {
 			Props.create(SuccessReturningActor.class),
 			JobMaster.JOB_MANAGER_NAME);
 
-		ClusterClient out = new StandaloneClusterClient(config);
+		StandaloneClusterClient out = new StandaloneClusterClient(config);
 		out.setDetached(true);
 		JobSubmissionResult result = out.run(program.getPlanWithJars(), 1);
 
@@ -222,7 +222,7 @@ public class ClientTest extends TestLogger {
 				Props.create(FailureReturningActor.class),
 				JobMaster.JOB_MANAGER_NAME);
 
-		ClusterClient out = new StandaloneClusterClient(config);
+		StandaloneClusterClient out = new StandaloneClusterClient(config);
 		out.setDetached(true);
 
 		try {
@@ -259,7 +259,7 @@ public class ClientTest extends TestLogger {
 			}).when(packagedProgramMock).invokeInteractiveModeForExecution();
 
 			try {
-				ClusterClient client = new StandaloneClusterClient(config);
+				StandaloneClusterClient client = new StandaloneClusterClient(config);
 				client.setDetached(true);
 				client.run(packagedProgramMock, 1);
 				fail("Creating the local execution environment should not be possible");

http://git-wip-us.apache.org/repos/asf/flink/blob/38d37208/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
index 7b34d4a..e2eb88d 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
@@ -71,7 +71,7 @@ public class ClusterClientTest extends TestLogger {
 		Configuration config = new Configuration();
 		HighAvailabilityServices highAvailabilityServices = mock(HighAvailabilityServices.class);
 
-		ClusterClient clusterClient = new StandaloneClusterClient(config, highAvailabilityServices);
+		StandaloneClusterClient clusterClient = new StandaloneClusterClient(config, highAvailabilityServices);
 
 		clusterClient.shutdown();
 
@@ -87,7 +87,7 @@ public class ClusterClientTest extends TestLogger {
 
 		JobID jobID = new JobID();
 		TestStopActorGateway gateway = new TestStopActorGateway(jobID);
-		ClusterClient clusterClient = new TestClusterClient(config, gateway);
+		TestClusterClient clusterClient = new TestClusterClient(config, gateway);
 		try {
 			clusterClient.stop(jobID);
 			Assert.assertTrue(gateway.messageArrived);
@@ -103,7 +103,7 @@ public class ClusterClientTest extends TestLogger {
 
 		JobID jobID = new JobID();
 		TestCancelActorGateway gateway = new TestCancelActorGateway(jobID);
-		ClusterClient clusterClient = new TestClusterClient(config, gateway);
+		TestClusterClient clusterClient = new TestClusterClient(config, gateway);
 		try {
 			clusterClient.cancel(jobID);
 			Assert.assertTrue(gateway.messageArrived);
@@ -121,7 +121,7 @@ public class ClusterClientTest extends TestLogger {
 		String savepointDirectory = "/test/directory";
 		String savepointPath = "/test/path";
 		TestCancelWithSavepointActorGateway gateway = new TestCancelWithSavepointActorGateway(jobID, savepointDirectory, savepointPath);
-		ClusterClient clusterClient = new TestClusterClient(config, gateway);
+		TestClusterClient clusterClient = new TestClusterClient(config, gateway);
 		try {
 			String path = clusterClient.cancelWithSavepoint(jobID, savepointDirectory);
 			Assert.assertTrue(gateway.messageArrived);
@@ -140,7 +140,7 @@ public class ClusterClientTest extends TestLogger {
 		String savepointDirectory = "/test/directory";
 		String savepointPath = "/test/path";
 		TestSavepointActorGateway gateway = new TestSavepointActorGateway(jobID, savepointDirectory, savepointPath);
-		ClusterClient clusterClient = new TestClusterClient(config, gateway);
+		TestClusterClient clusterClient = new TestClusterClient(config, gateway);
 		try {
 			CompletableFuture<String> pathFuture = clusterClient.triggerSavepoint(jobID, savepointDirectory);
 			Assert.assertTrue(gateway.messageArrived);
@@ -156,7 +156,7 @@ public class ClusterClientTest extends TestLogger {
 		config.setString(JobManagerOptions.ADDRESS, "localhost");
 
 		TestListActorGateway gateway = new TestListActorGateway();
-		ClusterClient clusterClient = new TestClusterClient(config, gateway);
+		TestClusterClient clusterClient = new TestClusterClient(config, gateway);
 		try {
 			CompletableFuture<Collection<JobStatusMessage>> jobDetailsFuture = clusterClient.listJobs();
 			Collection<JobStatusMessage> jobDetails = jobDetailsFuture.get();
@@ -179,7 +179,7 @@ public class ClusterClientTest extends TestLogger {
 
 		final ActorGateway jobManagerGateway = new TestDisposeWithWrongResponseActorGateway();
 
-		final ClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway);
+		final TestClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway);
 
 		CompletableFuture<Acknowledge> acknowledgeCompletableFuture = clusterClient.disposeSavepoint(savepointPath, timeout);
 
@@ -201,7 +201,7 @@ public class ClusterClientTest extends TestLogger {
 
 		final ActorGateway jobManagerGateway = new TestDisposeWithClassNotFoundExceptionActorGateway();
 
-		final ClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway);
+		final TestClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway);
 
 		CompletableFuture<Acknowledge> acknowledgeCompletableFuture = clusterClient.disposeSavepoint(savepointPath, timeout);
 


[04/14] flink git commit: [FLINK-8349] [flip6] Remove Yarn specific commands from YarnClusterDescriptor

Posted by tr...@apache.org.
[FLINK-8349] [flip6] Remove Yarn specific commands from YarnClusterDescriptor

Remove Yarn specific commands from YarnClusterDescriptor. This is a preparational
step to make the FlinkYarnSessionCli work with the Flip-6 RestClusterClient.

This closes #5229.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/402499f0
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/402499f0
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/402499f0

Branch: refs/heads/master
Commit: 402499f06a4b590ac47df64ecc01055c06b0399b
Parents: 10e900b
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Jan 11 23:25:58 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:03 2018 +0100

----------------------------------------------------------------------
 .../apache/flink/client/cli/CliFrontend.java    |   9 +
 .../client/deployment/ClusterDescriptor.java    |   9 +
 .../Flip6StandaloneClusterDescriptor.java       |   6 +
 .../deployment/StandaloneClusterDescriptor.java |   6 +
 .../flink/client/program/ClusterClient.java     |  13 +-
 .../client/program/StandaloneClusterClient.java |   4 -
 .../client/program/rest/RestClusterClient.java  |   7 +-
 .../client/cli/util/DummyClusterDescriptor.java |   6 +
 .../YARNSessionCapacitySchedulerITCase.java     |  24 +-
 .../flink/yarn/YARNSessionFIFOITCase.java       |   9 +-
 .../org/apache/flink/yarn/YarnTestBase.java     |  59 ++-
 .../yarn/AbstractYarnClusterDescriptor.java     |  21 +-
 .../apache/flink/yarn/YarnClusterClient.java    | 155 +------
 .../flink/yarn/cli/FlinkYarnSessionCli.java     | 415 ++++++++++---------
 .../yarn/cli/YarnApplicationStatusMonitor.java  |   3 +-
 15 files changed, 352 insertions(+), 394 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index 453d086..847a5f8 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -232,6 +232,15 @@ public class CliFrontend {
 
 				executeProgram(program, client, userParallelism);
 			} finally {
+				if (clusterId == null && !client.isDetached()) {
+					// terminate the cluster only if we have started it before and if it's not detached
+					try {
+						clusterDescriptor.terminateCluster(client.getClusterIdentifier());
+					} catch (FlinkException e) {
+						LOG.info("Could not properly terminate the Flink cluster.", e);
+					}
+				}
+
 				try {
 					client.shutdown();
 				} catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
index 07eeabc..b1f566c 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
@@ -20,6 +20,7 @@ package org.apache.flink.client.deployment;
 
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.util.FlinkException;
 
 /**
  * A descriptor to deploy a cluster (e.g. Yarn or Mesos) and return a Client for Cluster communication.
@@ -59,4 +60,12 @@ public interface ClusterDescriptor extends AutoCloseable {
 	ClusterClient deployJobCluster(
 		final ClusterSpecification clusterSpecification,
 		final JobGraph jobGraph);
+
+	/**
+	 * Terminates the cluster with the given cluster id.
+	 *
+	 * @param clusterId identifying the cluster to shut down
+	 * @throws FlinkException if the cluster could not be terminated
+	 */
+	void terminateCluster(String clusterId) throws FlinkException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
index a35a68b..70fd9f7 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
@@ -22,6 +22,7 @@ import org.apache.flink.client.program.rest.RestClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.Preconditions;
 
 /**
@@ -62,6 +63,11 @@ public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor {
 	}
 
 	@Override
+	public void terminateCluster(String clusterId) throws FlinkException {
+		throw new UnsupportedOperationException("Cannot terminate a standalone Flip-6 cluster.");
+	}
+
+	@Override
 	public void close() throws Exception {
 		// nothing to do
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
index ae25194..5638232 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
@@ -22,6 +22,7 @@ import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.util.FlinkException;
 
 /**
  * A deployment descriptor for an existing cluster.
@@ -61,6 +62,11 @@ public class StandaloneClusterDescriptor implements ClusterDescriptor {
 	}
 
 	@Override
+	public void terminateCluster(String clusterId) throws FlinkException {
+		throw new UnsupportedOperationException("Cannot terminate a standalone cluster.");
+	}
+
+	@Override
 	public void close() throws Exception {
 		// nothing to do
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
index 7c05e3e..efa23fb 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
@@ -262,11 +262,7 @@ public abstract class ClusterClient {
 	 */
 	public void shutdown() throws Exception {
 		synchronized (this) {
-			try {
-				finalizeCluster();
-			} finally {
-				actorSystemLoader.shutdown();
-			}
+			actorSystemLoader.shutdown();
 
 			if (highAvailabilityServices != null) {
 				highAvailabilityServices.close();
@@ -938,7 +934,7 @@ public abstract class ClusterClient {
 	 * May return new messages from the cluster.
 	 * Messages can be for example about failed containers or container launch requests.
 	 */
-	protected abstract List<String> getNewMessages();
+	public abstract List<String> getNewMessages();
 
 	/**
 	 * Returns a string representation of the cluster.
@@ -946,11 +942,6 @@ public abstract class ClusterClient {
 	public abstract String getClusterIdentifier();
 
 	/**
-	 * Request the cluster to shut down or disconnect.
-	 */
-	protected abstract void finalizeCluster();
-
-	/**
 	 * Set the mode of this client (detached or blocking job execution).
 	 * @param isDetached If true, the client will submit programs detached via the {@code run} method
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
index 10e1bdd..1782a25 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
@@ -105,8 +105,4 @@ public class StandaloneClusterClient extends ClusterClient {
 			return super.run(jobGraph, classLoader);
 		}
 	}
-
-	@Override
-	protected void finalizeCluster() {}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
index 6b21cff..a6bff1a 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
@@ -329,12 +329,7 @@ public class RestClusterClient extends ClusterClient {
 	}
 
 	@Override
-	protected List<String> getNewMessages() {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	protected void finalizeCluster() {
+	public List<String> getNewMessages() {
 		throw new UnsupportedOperationException();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
index d300055..2bb3ed0 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
@@ -22,6 +22,7 @@ import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.Preconditions;
 
 /**
@@ -56,6 +57,11 @@ public class DummyClusterDescriptor implements ClusterDescriptor {
 	}
 
 	@Override
+	public void terminateCluster(String clusterId) throws FlinkException {
+		throw new UnsupportedOperationException("DummyClusterDescriptor does not support cluster termination.");
+	}
+
+	@Override
 	public void close() {
 		// nothing to do
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index 5bed22e..c806c5e 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -27,6 +27,7 @@ import org.apache.flink.runtime.taskexecutor.TaskManagerServices;
 import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
 import org.apache.flink.test.testdata.WordCountData;
 import org.apache.flink.test.util.TestBaseUtils;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.flink.yarn.configuration.YarnConfigOptions;
 
@@ -74,6 +75,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static junit.framework.TestCase.assertTrue;
 import static org.apache.flink.yarn.UtilsTest.addTestAppender;
 import static org.apache.flink.yarn.UtilsTest.checkForLogString;
 
@@ -98,7 +100,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	 * Test regular operation, including command line parameter parsing.
 	 */
 	@Test
-	public void testClientStartup() {
+	public void testClientStartup() throws IOException {
 		LOG.info("Starting testClientStartup()");
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
 						"-n", "1",
@@ -116,7 +118,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	 * The parallelism is requested at the YARN client (-ys).
 	 */
 	@Test
-	public void perJobYarnCluster() {
+	public void perJobYarnCluster() throws IOException {
 		LOG.info("Starting perJobYarnCluster()");
 		addTestAppender(JobClient.class, Level.INFO);
 		File exampleJarLocation = new File("target/programs/BatchWordCount.jar");
@@ -145,7 +147,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	 * memory remains.
 	 */
 	@Test
-	public void perJobYarnClusterOffHeap() {
+	public void perJobYarnClusterOffHeap() throws IOException {
 		LOG.info("Starting perJobYarnCluster()");
 		addTestAppender(JobClient.class, Level.INFO);
 		File exampleJarLocation = new File("target/programs/BatchWordCount.jar");
@@ -363,15 +365,19 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	 * target queue. With an error message, we can help users identifying the issue)
 	 */
 	@Test
-	public void testNonexistingQueueWARNmessage() {
+	public void testNonexistingQueueWARNmessage() throws IOException {
 		LOG.info("Starting testNonexistingQueueWARNmessage()");
 		addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
-		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
+		try {
+			runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
 				"-t", flinkLibFolder.getAbsolutePath(),
 				"-n", "1",
 				"-jm", "768",
 				"-tm", "1024",
 				"-qu", "doesntExist"}, "to unknown queue: doesntExist", null, RunTypes.YARN_SESSION, 1);
+		} catch (Exception e) {
+			assertTrue(ExceptionUtils.findThrowableWithMessage(e, "to unknown queue: doesntExist").isPresent());
+		}
 		checkForLogString("The specified queue 'doesntExist' does not exist. Available queues");
 		LOG.info("Finished testNonexistingQueueWARNmessage()");
 	}
@@ -380,7 +386,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	 * Test per-job yarn cluster with the parallelism set at the CliFrontend instead of the YARN client.
 	 */
 	@Test
-	public void perJobYarnClusterWithParallelism() {
+	public void perJobYarnClusterWithParallelism() throws IOException {
 		LOG.info("Starting perJobYarnClusterWithParallelism()");
 		// write log messages to stdout as well, so that the runWithArgs() method
 		// is catching the log output
@@ -407,7 +413,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	 * Test a fire-and-forget job submission to a YARN cluster.
 	 */
 	@Test(timeout = 60000)
-	public void testDetachedPerJobYarnCluster() {
+	public void testDetachedPerJobYarnCluster() throws IOException {
 		LOG.info("Starting testDetachedPerJobYarnCluster()");
 
 		File exampleJarLocation = new File("target/programs/BatchWordCount.jar");
@@ -423,7 +429,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	 * Test a fire-and-forget job submission to a YARN cluster.
 	 */
 	@Test(timeout = 60000)
-	public void testDetachedPerJobYarnClusterWithStreamingJob() {
+	public void testDetachedPerJobYarnClusterWithStreamingJob() throws IOException {
 		LOG.info("Starting testDetachedPerJobYarnClusterWithStreamingJob()");
 
 		File exampleJarLocation = new File("target/programs/StreamingWordCount.jar");
@@ -435,7 +441,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 		LOG.info("Finished testDetachedPerJobYarnClusterWithStreamingJob()");
 	}
 
-	private void testDetachedPerJobYarnClusterInternal(String job) {
+	private void testDetachedPerJobYarnClusterInternal(String job) throws IOException {
 		YarnClient yc = YarnClient.createYarnClient();
 		yc.init(YARN_CONFIGURATION);
 		yc.start();

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index cc26350..15fe355 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -45,6 +45,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
@@ -80,7 +81,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	 * Test regular operation, including command line parameter parsing.
 	 */
 	@Test(timeout = 60000) // timeout after a minute.
-	public void testDetachedMode() throws InterruptedException {
+	public void testDetachedMode() throws InterruptedException, IOException {
 		LOG.info("Starting testDetachedMode()");
 		addTestAppender(FlinkYarnSessionCli.class, Level.INFO);
 		Runner runner =
@@ -158,7 +159,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	 * <p>This test validates through 666*2 cores in the "cluster".
 	 */
 	@Test
-	public void testQueryCluster() {
+	public void testQueryCluster() throws IOException {
 		LOG.info("Starting testQueryCluster()");
 		runWithArgs(new String[] {"-q"}, "Summary: totalMemory 8192 totalCores 1332", null, RunTypes.YARN_SESSION, 0); // we have 666*2 cores.
 		LOG.info("Finished testQueryCluster()");
@@ -178,7 +179,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	 */
 	@Ignore("The test is too resource consuming (8.5 GB of memory)")
 	@Test
-	public void testResourceComputation() {
+	public void testResourceComputation() throws IOException {
 		addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
 		LOG.info("Starting testResourceComputation()");
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
@@ -206,7 +207,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	 */
 	@Ignore("The test is too resource consuming (8 GB of memory)")
 	@Test
-	public void testfullAlloc() {
+	public void testfullAlloc() throws IOException {
 		addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
 		LOG.info("Starting testfullAlloc()");
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
index ed02892..eeda32d 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
@@ -61,6 +61,9 @@ import java.io.FileNotFoundException;
 import java.io.FileWriter;
 import java.io.FilenameFilter;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.util.ArrayList;
@@ -86,6 +89,7 @@ public abstract class YarnTestBase extends TestLogger {
 
 	protected static final PrintStream ORIGINAL_STDOUT = System.out;
 	protected static final PrintStream ORIGINAL_STDERR = System.err;
+	private static final InputStream ORIGINAL_STDIN = System.in;
 
 	protected static final String TEST_CLUSTER_NAME_KEY = "flink-yarn-minicluster-name";
 
@@ -510,13 +514,18 @@ public abstract class YarnTestBase extends TestLogger {
 	/**
 	 * This method returns once the "startedAfterString" has been seen.
 	 */
-	protected Runner startWithArgs(String[] args, String startedAfterString, RunTypes type) {
+	protected Runner startWithArgs(String[] args, String startedAfterString, RunTypes type) throws IOException {
 		LOG.info("Running with args {}", Arrays.toString(args));
 
 		outContent = new ByteArrayOutputStream();
 		errContent = new ByteArrayOutputStream();
+		PipedOutputStream out = new PipedOutputStream();
+		PipedInputStream in = new PipedInputStream(out);
+		PrintStream stdinPrintStream = new PrintStream(out);
+
 		System.setOut(new PrintStream(outContent));
 		System.setErr(new PrintStream(errContent));
+		System.setIn(in);
 
 		final int startTimeoutSeconds = 60;
 
@@ -525,7 +534,8 @@ public abstract class YarnTestBase extends TestLogger {
 			flinkConfiguration,
 			CliFrontend.getConfigurationDirectoryFromEnv(),
 			type,
-			0);
+			0,
+			stdinPrintStream);
 		runner.setName("Frontend (CLI/YARN Client) runner thread (startWithArgs()).");
 		runner.start();
 
@@ -539,7 +549,7 @@ public abstract class YarnTestBase extends TestLogger {
 			}
 			// check if thread died
 			if (!runner.isAlive()) {
-				sendOutput();
+				resetStreamsAndSendOutput();
 				if (runner.getRunnerError() != null) {
 					throw new RuntimeException("Runner failed with exception.", runner.getRunnerError());
 				}
@@ -547,13 +557,13 @@ public abstract class YarnTestBase extends TestLogger {
 			}
 		}
 
-		sendOutput();
+		resetStreamsAndSendOutput();
 		Assert.fail("During the timeout period of " + startTimeoutSeconds + " seconds the " +
 				"expected string did not show up");
 		return null;
 	}
 
-	protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnStrings, RunTypes type, int returnCode) {
+	protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnStrings, RunTypes type, int returnCode) throws IOException {
 		runWithArgs(args, terminateAfterString, failOnStrings, type, returnCode, false);
 	}
 
@@ -566,13 +576,17 @@ public abstract class YarnTestBase extends TestLogger {
 	 * @param expectedReturnValue Expected return code from the runner.
 	 * @param checkLogForTerminateString  If true, the runner checks also the log4j logger for the terminate string
 	 */
-	protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnPatterns, RunTypes type, int expectedReturnValue, boolean checkLogForTerminateString) {
+	protected void runWithArgs(String[] args, String terminateAfterString, String[] failOnPatterns, RunTypes type, int expectedReturnValue, boolean checkLogForTerminateString) throws IOException {
 		LOG.info("Running with args {}", Arrays.toString(args));
 
 		outContent = new ByteArrayOutputStream();
 		errContent = new ByteArrayOutputStream();
+		PipedOutputStream out = new PipedOutputStream();
+		PipedInputStream in = new PipedInputStream(out);
+		PrintStream stdinPrintStream = new PrintStream(out);
 		System.setOut(new PrintStream(outContent));
 		System.setErr(new PrintStream(errContent));
+		System.setIn(in);
 
 		// we wait for at most three minutes
 		final int startTimeoutSeconds = 180;
@@ -583,11 +597,13 @@ public abstract class YarnTestBase extends TestLogger {
 			flinkConfiguration,
 			CliFrontend.getConfigurationDirectoryFromEnv(),
 			type,
-			expectedReturnValue);
+			expectedReturnValue,
+			stdinPrintStream);
 		runner.start();
 
 		boolean expectedStringSeen = false;
 		boolean testPassedFromLog4j = false;
+		long shutdownTimeout = 30000L;
 		do {
 			sleep(1000);
 			String outContentString = outContent.toString();
@@ -597,9 +613,15 @@ public abstract class YarnTestBase extends TestLogger {
 					Pattern pattern = Pattern.compile(failOnString);
 					if (pattern.matcher(outContentString).find() || pattern.matcher(errContentString).find()) {
 						LOG.warn("Failing test. Output contained illegal string '" + failOnString + "'");
-						sendOutput();
+						resetStreamsAndSendOutput();
 						// stopping runner.
 						runner.sendStop();
+						// wait for the thread to stop
+						try {
+							runner.join(shutdownTimeout);
+						} catch (InterruptedException e) {
+							LOG.warn("Interrupted while stopping runner", e);
+						}
 						Assert.fail("Output contained illegal string '" + failOnString + "'");
 					}
 				}
@@ -622,7 +644,7 @@ public abstract class YarnTestBase extends TestLogger {
 				runner.sendStop();
 				// wait for the thread to stop
 				try {
-					runner.join(30000);
+					runner.join(shutdownTimeout);
 				}
 				catch (InterruptedException e) {
 					LOG.warn("Interrupted while stopping runner", e);
@@ -639,7 +661,7 @@ public abstract class YarnTestBase extends TestLogger {
 		}
 		while (runner.getRunnerError() == null && !expectedStringSeen && System.currentTimeMillis() < deadline);
 
-		sendOutput();
+		resetStreamsAndSendOutput();
 
 		if (runner.getRunnerError() != null) {
 			// this lets the test fail.
@@ -651,9 +673,10 @@ public abstract class YarnTestBase extends TestLogger {
 		LOG.info("Test was successful");
 	}
 
-	protected static void sendOutput() {
+	protected static void resetStreamsAndSendOutput() {
 		System.setOut(ORIGINAL_STDOUT);
 		System.setErr(ORIGINAL_STDERR);
+		System.setIn(ORIGINAL_STDIN);
 
 		LOG.info("Sending stdout content through logger: \n\n{}\n\n", outContent.toString());
 		LOG.info("Sending stderr content through logger: \n\n{}\n\n", errContent.toString());
@@ -668,6 +691,8 @@ public abstract class YarnTestBase extends TestLogger {
 		private final String configurationDirectory;
 		private final int expectedReturnValue;
 
+		private final PrintStream stdinPrintStream;
+
 		private RunTypes type;
 		private FlinkYarnSessionCli yCli;
 		private Throwable runnerError;
@@ -677,13 +702,15 @@ public abstract class YarnTestBase extends TestLogger {
 				org.apache.flink.configuration.Configuration configuration,
 				String configurationDirectory,
 				RunTypes type,
-				int expectedReturnValue) {
+				int expectedReturnValue,
+				PrintStream stdinPrintStream) {
 
 			this.args = args;
 			this.configuration = Preconditions.checkNotNull(configuration);
 			this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory);
 			this.type = type;
 			this.expectedReturnValue = expectedReturnValue;
+			this.stdinPrintStream = Preconditions.checkNotNull(stdinPrintStream);
 		}
 
 		@Override
@@ -697,8 +724,8 @@ public abstract class YarnTestBase extends TestLogger {
 							configurationDirectory,
 							"",
 							"",
-							false);
-						returnValue = yCli.run(args, configuration, configurationDirectory);
+							true);
+						returnValue = yCli.run(args);
 						break;
 					case CLI_FRONTEND:
 						try {
@@ -727,9 +754,7 @@ public abstract class YarnTestBase extends TestLogger {
 
 		/** Stops the Yarn session. */
 		public void sendStop() {
-			if (yCli != null) {
-				yCli.stop();
-			}
+			stdinPrintStream.println("stop");
 		}
 
 		public Throwable getRunnerError() {

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index 0a977df..4affb78 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -21,6 +21,7 @@ package org.apache.flink.yarn;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
@@ -33,6 +34,7 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.yarn.configuration.YarnConfigOptions;
 
@@ -330,7 +332,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	// -------------------------------------------------------------
 
 	@Override
-	public YarnClusterClient retrieve(String applicationID) {
+	public ClusterClient retrieve(String applicationID) {
 
 		try {
 			// check if required Hadoop environment variables are set. If not, warn user
@@ -393,6 +395,23 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		}
 	}
 
+	@Override
+	public void terminateCluster(String clusterId) throws FlinkException {
+		try {
+			yarnClient.killApplication(ConverterUtils.toApplicationId(clusterId));
+		} catch (IOException | YarnException e) {
+			throw new FlinkException("Could not terminate cluster with id " + clusterId + '.', e);
+		}
+	}
+
+	public void terminateCluster(ApplicationId applicationId) throws FlinkException {
+		try {
+			yarnClient.killApplication(applicationId);
+		} catch (YarnException | IOException e) {
+			throw new FlinkException("Could not kill the Yarn Flink cluster with id " + applicationId + '.', e);
+		}
+	}
+
 	/**
 	 * This method will block until the ApplicationMaster/JobManager have been
 	 * deployed on YARN.

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
index 5fb7f90..991b3b9 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
@@ -24,7 +24,6 @@ import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.clusterframework.ApplicationStatus;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
 import org.apache.flink.runtime.clusterframework.messages.InfoMessage;
@@ -33,8 +32,6 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.Preconditions;
-import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
-import org.apache.flink.yarn.configuration.YarnConfigOptions;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
@@ -43,17 +40,15 @@ import akka.pattern.Patterns;
 import akka.util.Timeout;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
 import java.io.IOException;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import scala.Option;
 import scala.concurrent.Await;
@@ -67,8 +62,6 @@ public class YarnClusterClient extends ClusterClient {
 
 	private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClient.class);
 
-	private Thread clientShutdownHook = new ClientShutdownHook();
-
 	//---------- Class internal fields -------------------
 
 	private final AbstractYarnClusterDescriptor clusterDescriptor;
@@ -80,8 +73,6 @@ public class YarnClusterClient extends ClusterClient {
 	private final ApplicationId appId;
 	private final String trackingURL;
 
-	private boolean isConnected = true;
-
 	/** Indicator whether this cluster has just been created. */
 	private final boolean newlyCreatedCluster;
 
@@ -120,32 +111,6 @@ public class YarnClusterClient extends ClusterClient {
 			flinkConfig,
 			actorSystemLoader,
 			highAvailabilityServices);
-
-		Runtime.getRuntime().addShutdownHook(clientShutdownHook);
-	}
-
-	/**
-	 * Disconnect from the Yarn cluster.
-	 */
-	public void disconnect() {
-
-		if (hasBeenShutDown.getAndSet(true)) {
-			return;
-		}
-
-		if (!isConnected) {
-			throw new IllegalStateException("Can not disconnect from an unconnected cluster.");
-		}
-
-		LOG.info("Disconnecting YarnClusterClient from ApplicationMaster");
-
-		try {
-			Runtime.getRuntime().removeShutdownHook(clientShutdownHook);
-		} catch (IllegalStateException e) {
-			// we are already in the shutdown hook
-		}
-
-		isConnected = false;
 	}
 
 	// -------------------------- Interaction with the cluster ------------------------
@@ -207,7 +172,7 @@ public class YarnClusterClient extends ClusterClient {
 
 	@Override
 	public String getClusterIdentifier() {
-		return "Yarn cluster with application id " + appReport.getApplicationId();
+		return ConverterUtils.toString(appReport.getApplicationId());
 	}
 
 	/**
@@ -215,13 +180,6 @@ public class YarnClusterClient extends ClusterClient {
 	 */
 	@Override
 	public GetClusterStatusResponse getClusterStatus() {
-		if (!isConnected) {
-			throw new IllegalStateException("The cluster is not connected to the cluster.");
-		}
-		if (hasBeenShutdown()) {
-			throw new IllegalStateException("The cluster has already been shutdown.");
-		}
-
 		try {
 			final Future<Object> clusterStatusOption =
 				getJobManagerGateway().ask(
@@ -236,15 +194,7 @@ public class YarnClusterClient extends ClusterClient {
 	@Override
 	public List<String> getNewMessages() {
 
-		if (hasBeenShutdown()) {
-			throw new RuntimeException("The YarnClusterClient has already been stopped");
-		}
-
-		if (!isConnected) {
-			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
-		}
-
-		List<String> ret = new ArrayList<String>();
+		List<String> ret = new ArrayList<>();
 		// get messages from ApplicationClient (locally)
 		while (true) {
 			Object result;
@@ -283,105 +233,6 @@ public class YarnClusterClient extends ClusterClient {
 		return ret;
 	}
 
-	// -------------------------- Shutdown handling ------------------------
-
-	private AtomicBoolean hasBeenShutDown = new AtomicBoolean(false);
-
-	/**
-	 * Shuts down or disconnects from the YARN cluster.
-	 */
-	@Override
-	public void finalizeCluster() {
-		if (isDetached() || !newlyCreatedCluster) {
-			disconnect();
-		} else {
-			shutdownCluster();
-		}
-	}
-
-	/**
-	 * Shuts down the Yarn application.
-	 */
-	public void shutdownCluster() {
-
-		if (hasBeenShutDown.getAndSet(true)) {
-			return;
-		}
-
-		if (!isConnected) {
-			throw new IllegalStateException("The cluster has been not been connected to the ApplicationMaster.");
-		}
-
-		try {
-			Runtime.getRuntime().removeShutdownHook(clientShutdownHook);
-		} catch (IllegalStateException e) {
-			// we are already in the shutdown hook
-		}
-
-		LOG.info("Sending shutdown request to the Application Master");
-		try {
-			Future<Object> response =
-				Patterns.ask(applicationClient.get(),
-					new YarnMessages.LocalStopYarnSession(ApplicationStatus.CANCELED,
-						"Flink YARN Client requested shutdown"),
-					new Timeout(akkaDuration));
-			Await.ready(response, akkaDuration);
-		} catch (Exception e) {
-			LOG.warn("Error while stopping YARN cluster.", e);
-		}
-
-		try {
-			File propertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(flinkConfig.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
-			if (propertiesFile.isFile()) {
-				if (propertiesFile.delete()) {
-					LOG.info("Deleted Yarn properties file at {}", propertiesFile.getAbsoluteFile().toString());
-				} else {
-					LOG.warn("Couldn't delete Yarn properties file at {}", propertiesFile.getAbsoluteFile().toString());
-				}
-			}
-		} catch (Exception e) {
-			LOG.warn("Exception while deleting the JobManager address file", e);
-		}
-
-		try {
-			ApplicationReport appReport = clusterDescriptor.getYarnClient().getApplicationReport(appId);
-
-			LOG.info("Application " + appId + " finished with state " + appReport
-				.getYarnApplicationState() + " and final state " + appReport
-				.getFinalApplicationStatus() + " at " + appReport.getFinishTime());
-
-			if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED || appReport.getYarnApplicationState()
-				== YarnApplicationState.KILLED) {
-				LOG.warn("Application failed. Diagnostics " + appReport.getDiagnostics());
-				LOG.warn("If log aggregation is activated in the Hadoop cluster, we recommend to retrieve "
-					+ "the full application log using this command:"
-					+ System.lineSeparator()
-					+ "\tyarn logs -applicationId " + appReport.getApplicationId()
-					+ System.lineSeparator()
-					+ "(It sometimes takes a few seconds until the logs are aggregated)");
-			}
-		} catch (Exception e) {
-			LOG.warn("Couldn't get final report", e);
-		}
-	}
-
-	public boolean hasBeenShutdown() {
-		return hasBeenShutDown.get();
-	}
-
-	private class ClientShutdownHook extends Thread {
-		@Override
-		public void run() {
-			LOG.info("Shutting down YarnClusterClient from the client shutdown hook");
-
-			try {
-				shutdown();
-			} catch (Throwable t) {
-				LOG.warn("Could not properly shut down the yarn cluster client.", t);
-			}
-		}
-	}
-
 	@Override
 	public boolean isDetached() {
 		return super.isDetached() || clusterDescriptor.isDetachedMode();

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index d797f47..75a270d 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -19,9 +19,11 @@
 package org.apache.flink.yarn.cli;
 
 import org.apache.flink.client.cli.AbstractCustomCommandLine;
+import org.apache.flink.client.cli.CliArgsException;
 import org.apache.flink.client.cli.CliFrontend;
 import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
@@ -36,20 +38,21 @@ import org.apache.flink.util.ExecutorUtils;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.yarn.AbstractYarnClusterDescriptor;
-import org.apache.flink.yarn.YarnClusterClient;
 import org.apache.flink.yarn.YarnClusterDescriptor;
 import org.apache.flink.yarn.YarnClusterDescriptorV2;
 import org.apache.flink.yarn.configuration.YarnConfigOptions;
 
 import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
-import org.apache.commons.cli.PosixParser;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -73,8 +76,10 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
 
 import static org.apache.flink.configuration.HighAvailabilityOptions.HA_CLUSTER_ID;
 
@@ -96,7 +101,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 
 	// YARN-session related constants
 	private static final String YARN_PROPERTIES_FILE = ".yarn-properties-";
-	static final String YARN_APPLICATION_ID_KEY = "applicationID";
+	private static final String YARN_APPLICATION_ID_KEY = "applicationID";
 	private static final String YARN_PROPERTIES_PARALLELISM = "parallelism";
 	private static final String YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING = "dynamicPropertiesString";
 
@@ -148,8 +153,9 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 
 	private final String yarnApplicationIdFromYarnProperties;
 
+	private final String yarnPropertiesFileLocation;
+
 	//------------------------------------ Internal fields -------------------------
-	private YarnClusterClient yarnCluster;
 	private boolean detachedMode = false;
 
 	public FlinkYarnSessionCli(
@@ -181,7 +187,12 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		tmMemory = new Option(shortPrefix + "tm", longPrefix + "taskManagerMemory", true, "Memory per TaskManager Container [in MB]");
 		container = new Option(shortPrefix + "n", longPrefix + "container", true, "Number of YARN container to allocate (=Number of Task Managers)");
 		slots = new Option(shortPrefix + "s", longPrefix + "slots", true, "Number of slots per TaskManager");
-		dynamicproperties = new Option(shortPrefix + "D", true, "Dynamic properties");
+		dynamicproperties = Option.builder(shortPrefix + "D")
+			.argName("property=value")
+			.numberOfArgs(2)
+			.valueSeparator()
+			.desc("use value for given property")
+			.build();
 		detached = new Option(shortPrefix + "d", longPrefix + "detached", false, "Start detached");
 		streaming = new Option(shortPrefix + "st", longPrefix + "streaming", false, "Start Flink in streaming mode");
 		name = new Option(shortPrefix + "nm", longPrefix + "name", true, "Set a custom name for the application on YARN");
@@ -206,7 +217,8 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		allOptions.addOption(flip6);
 
 		// try loading a potential yarn properties file
-		final File yarnPropertiesLocation = getYarnPropertiesLocation(configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
+		this.yarnPropertiesFileLocation = configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION);
+		final File yarnPropertiesLocation = getYarnPropertiesLocation(yarnPropertiesFileLocation);
 
 		yarnPropertiesFile = new Properties();
 
@@ -216,7 +228,8 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 			try (InputStream is = new FileInputStream(yarnPropertiesLocation)) {
 				yarnPropertiesFile.load(is);
 			} catch (IOException ioe) {
-				throw new FlinkException("Could not read the Yarn properties file " + yarnPropertiesLocation + '.');
+				throw new FlinkException("Could not read the Yarn properties file " + yarnPropertiesLocation +
+					". Please delete the file at " + yarnPropertiesLocation.getAbsolutePath() + '.', ioe);
 			}
 
 			yarnApplicationIdFromYarnProperties = yarnPropertiesFile.getProperty(YARN_APPLICATION_ID_KEY);
@@ -305,10 +318,21 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 			yarnClusterDescriptor.setQueue(cmd.getOptionValue(queue.getOpt()));
 		}
 
-		String[] dynamicProperties = null;
-		if (cmd.hasOption(dynamicproperties.getOpt())) {
-			dynamicProperties = cmd.getOptionValues(dynamicproperties.getOpt());
-		}
+		final Properties properties = cmd.getOptionProperties(dynamicproperties.getOpt());
+
+		String[] dynamicProperties = properties.stringPropertyNames().stream()
+			.flatMap(
+				(String key) -> {
+					final String value = properties.getProperty(key);
+
+					if (value != null) {
+						return Stream.of(key + dynamicproperties.getValueSeparator() + value);
+					} else {
+						return Stream.empty();
+					}
+				})
+			.toArray(String[]::new);
+
 		String dynamicPropertiesEncoded = StringUtils.join(dynamicProperties, YARN_DYNAMIC_PROPERTIES_SEPARATOR);
 
 		yarnClusterDescriptor.setDynamicPropertiesEncoded(dynamicPropertiesEncoded);
@@ -534,184 +558,168 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		return effectiveConfiguration;
 	}
 
-	public int run(
-			String[] args,
-			Configuration configuration,
-			String configurationDirectory) {
+	public int run(String[] args) throws CliArgsException, FlinkException {
 		//
 		//	Command Line Options
 		//
-		Options options = new Options();
-		addGeneralOptions(options);
-		addRunOptions(options);
+		final CommandLine cmd = parseCommandLineOptions(args, true);
 
-		CommandLineParser parser = new PosixParser();
-		CommandLine cmd;
-		try {
-			cmd = parser.parse(options, args);
-		} catch (Exception e) {
-			System.out.println(e.getMessage());
-			printUsage();
-			return 1;
-		}
+		final AbstractYarnClusterDescriptor yarnClusterDescriptor = createClusterDescriptor(cmd);
 
-		// Query cluster for metrics
-		if (cmd.hasOption(query.getOpt())) {
-			AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor(
-				configuration,
-				configurationDirectory,
-				cmd.hasOption(flip6.getOpt()));
-			String description;
-			try {
-				description = yarnDescriptor.getClusterDescription();
-			} catch (Exception e) {
-				System.err.println("Error while querying the YARN cluster for available resources: " + e.getMessage());
-				e.printStackTrace(System.err);
-				return 1;
-			}
-			System.out.println(description);
-			return 0;
-		} else if (cmd.hasOption(applicationId.getOpt())) {
+		try {
+			// Query cluster for metrics
+			if (cmd.hasOption(query.getOpt())) {
+				final String description = yarnClusterDescriptor.getClusterDescription();
+				System.out.println(description);
+				return 0;
+			} else {
+				final ClusterClient clusterClient;
+				final ApplicationId yarnApplicationId;
 
-			AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor(
-				configuration,
-				configurationDirectory,
-				cmd.hasOption(flip6.getOpt()));
+				if (cmd.hasOption(applicationId.getOpt())) {
+					yarnApplicationId = ConverterUtils.toApplicationId(cmd.getOptionValue(applicationId.getOpt()));
 
-			//configure ZK namespace depending on the value passed
-			String zkNamespace = cmd.hasOption(zookeeperNamespace.getOpt()) ?
-									cmd.getOptionValue(zookeeperNamespace.getOpt())
-									: yarnDescriptor.getFlinkConfiguration()
-									.getString(HA_CLUSTER_ID, cmd.getOptionValue(applicationId.getOpt()));
-			LOG.info("Going to use the ZK namespace: {}", zkNamespace);
-			yarnDescriptor.getFlinkConfiguration().setString(HA_CLUSTER_ID, zkNamespace);
+					clusterClient = yarnClusterDescriptor.retrieve(cmd.getOptionValue(applicationId.getOpt()));
+				} else {
+					final ClusterSpecification clusterSpecification = getClusterSpecification(cmd);
 
-			try {
-				yarnCluster = yarnDescriptor.retrieve(cmd.getOptionValue(applicationId.getOpt()));
-			} catch (Exception e) {
-				throw new RuntimeException("Could not retrieve existing Yarn application", e);
-			}
+					clusterClient = yarnClusterDescriptor.deploySessionCluster(clusterSpecification);
 
-			if (detachedMode) {
-				LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
-					"Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
-					"yarn application -kill " + applicationId.getOpt());
-				yarnCluster.disconnect();
-			} else {
-				ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
-
-				try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
-						yarnDescriptor.getYarnClient(),
-						yarnCluster.getApplicationId(),
-						new ScheduledExecutorServiceAdapter(scheduledExecutorService))) {
-					runInteractiveCli(
-						yarnCluster,
-						yarnApplicationStatusMonitor,
-						true);
-				} finally {
-					// shut down the scheduled executor service
-					ExecutorUtils.gracefulShutdown(
-						1000L,
-						TimeUnit.MILLISECONDS,
-						scheduledExecutorService);
-				}
-			}
-		} else {
+					//------------------ ClusterClient deployed, handle connection details
+					yarnApplicationId = ConverterUtils.toApplicationId(clusterClient.getClusterIdentifier());
 
-			try (AbstractYarnClusterDescriptor yarnDescriptor = createClusterDescriptor(cmd)){
-				final ClusterSpecification clusterSpecification;
+					String jobManagerAddress =
+						clusterClient.getJobManagerAddress().getAddress().getHostName() +
+							':' + clusterClient.getJobManagerAddress().getPort();
 
-				try {
-					clusterSpecification = getClusterSpecification(cmd);
-				} catch (FlinkException e) {
-					System.err.println("Error while creating the cluster specification: " + e.getMessage());
-					e.printStackTrace();
-					return 1;
-				}
+					System.out.println("Flink JobManager is now running on " + jobManagerAddress);
+					System.out.println("JobManager Web Interface: " + clusterClient.getWebInterfaceURL());
 
-				try {
-					yarnCluster = yarnDescriptor.deploySessionCluster(clusterSpecification);
-				} catch (Exception e) {
-					System.err.println("Error while deploying YARN cluster: " + e.getMessage());
-					e.printStackTrace(System.err);
-					return 1;
-				}
-				//------------------ ClusterClient deployed, handle connection details
-				String jobManagerAddress =
-					yarnCluster.getJobManagerAddress().getAddress().getHostName() +
-						":" + yarnCluster.getJobManagerAddress().getPort();
-
-				System.out.println("Flink JobManager is now running on " + jobManagerAddress);
-				System.out.println("JobManager Web Interface: " + yarnCluster.getWebInterfaceURL());
-
-				// file that we write into the conf/ dir containing the jobManager address and the dop.
-				File yarnPropertiesFile = getYarnPropertiesLocation(configuration.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
-
-				Properties yarnProps = new Properties();
-				yarnProps.setProperty(YARN_APPLICATION_ID_KEY, yarnCluster.getApplicationId().toString());
-				if (clusterSpecification.getSlotsPerTaskManager() != -1) {
-					String parallelism =
-						Integer.toString(clusterSpecification.getSlotsPerTaskManager() * clusterSpecification.getNumberTaskManagers());
-					yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, parallelism);
-				}
-				// add dynamic properties
-				if (yarnDescriptor.getDynamicPropertiesEncoded() != null) {
-					yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING,
-						yarnDescriptor.getDynamicPropertiesEncoded());
+					writeYarnPropertiesFile(
+						yarnApplicationId,
+						clusterSpecification.getNumberTaskManagers() * clusterSpecification.getSlotsPerTaskManager(),
+						yarnClusterDescriptor.getDynamicPropertiesEncoded());
 				}
-				writeYarnProperties(yarnProps, yarnPropertiesFile);
-
-				//------------------ ClusterClient running, let user control it ------------
 
 				if (detachedMode) {
-					// print info and quit:
 					LOG.info("The Flink YARN client has been started in detached mode. In order to stop " +
 						"Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
-						"yarn application -kill " + yarnCluster.getApplicationId());
-					yarnCluster.waitForClusterToBeReady();
-					yarnCluster.disconnect();
+						"yarn application -kill " + applicationId.getOpt());
 				} else {
+					ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
 
-					ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+					final YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
+						yarnClusterDescriptor.getYarnClient(),
+						yarnApplicationId,
+						new ScheduledExecutorServiceAdapter(scheduledExecutorService));
 
-					try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor(
-						yarnDescriptor.getYarnClient(),
-						yarnCluster.getApplicationId(),
-						new ScheduledExecutorServiceAdapter(scheduledExecutorService))) {
+					try {
 						runInteractiveCli(
-							yarnCluster,
+							clusterClient,
 							yarnApplicationStatusMonitor,
 							acceptInteractiveInput);
 					} finally {
+						try {
+							yarnApplicationStatusMonitor.close();
+						} catch (Exception e) {
+							LOG.info("Could not properly close the Yarn application status monitor.", e);
+						}
+
+						try {
+							clusterClient.shutdown();
+						} catch (Exception e) {
+							LOG.info("Could not properly shutdown cluster client.", e);
+						}
+
+						try {
+							yarnClusterDescriptor.terminateCluster(yarnApplicationId);
+						} catch (FlinkException e) {
+							LOG.info("Could not properly terminate the Flink cluster.", e);
+						}
+
 						// shut down the scheduled executor service
 						ExecutorUtils.gracefulShutdown(
 							1000L,
 							TimeUnit.MILLISECONDS,
 							scheduledExecutorService);
+
+						deleteYarnPropertiesFile();
+
+						try {
+							final ApplicationReport applicationReport = yarnClusterDescriptor
+								.getYarnClient()
+								.getApplicationReport(yarnApplicationId);
+
+							logFinalApplicationReport(applicationReport);
+						} catch (YarnException | IOException e) {
+							LOG.info("Could not log the final application report.", e);
+						}
 					}
 				}
-			} catch (FlinkException e) {
-				System.err.println("Error while deploying a Flink cluster: " + e.getMessage());
-				e.printStackTrace();
-				return 1;
+			}
+		} finally {
+			try {
+				yarnClusterDescriptor.close();
+			} catch (Exception e) {
+				LOG.info("Could not properly close the yarn cluster descriptor.", e);
 			}
 		}
+
 		return 0;
 	}
 
-	/**
-	 * Utility method for tests.
-	 */
-	public void stop() {
-		if (yarnCluster != null) {
-			LOG.info("Command line interface is shutting down the yarnCluster");
+	private void logFinalApplicationReport(ApplicationReport appReport) {
+		LOG.info("Application " + appReport.getApplicationId() + " finished with state " + appReport
+			.getYarnApplicationState() + " and final state " + appReport
+			.getFinalApplicationStatus() + " at " + appReport.getFinishTime());
+
+		if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED || appReport.getYarnApplicationState()
+			== YarnApplicationState.KILLED) {
+			LOG.warn("Application failed. Diagnostics " + appReport.getDiagnostics());
+			LOG.warn("If log aggregation is activated in the Hadoop cluster, we recommend to retrieve "
+				+ "the full application log using this command:"
+				+ System.lineSeparator()
+				+ "\tyarn logs -applicationId " + appReport.getApplicationId()
+				+ System.lineSeparator()
+				+ "(It sometimes takes a few seconds until the logs are aggregated)");
+		}
+	}
 
-			try {
-				yarnCluster.shutdown();
-			} catch (Throwable t) {
-				LOG.warn("Could not properly shutdown the yarn cluster.", t);
+	private void deleteYarnPropertiesFile() {
+		// try to clean up the old yarn properties file
+		try {
+			File propertiesFile = getYarnPropertiesLocation(yarnPropertiesFileLocation);
+			if (propertiesFile.isFile()) {
+				if (propertiesFile.delete()) {
+					LOG.info("Deleted Yarn properties file at {}", propertiesFile.getAbsoluteFile());
+				} else {
+					LOG.warn("Couldn't delete Yarn properties file at {}", propertiesFile.getAbsoluteFile());
+				}
 			}
+		} catch (Exception e) {
+			LOG.warn("Exception while deleting the JobManager address file", e);
+		}
+	}
+
+	private void writeYarnPropertiesFile(
+			ApplicationId yarnApplicationId,
+			int parallelism,
+			@Nullable String dynamicProperties) {
+		// file that we write into the conf/ dir containing the jobManager address and the dop.
+		final File yarnPropertiesFile = getYarnPropertiesLocation(yarnPropertiesFileLocation);
+
+		Properties yarnProps = new Properties();
+		yarnProps.setProperty(YARN_APPLICATION_ID_KEY, yarnApplicationId.toString());
+		if (parallelism > 0) {
+			yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, Integer.toString(parallelism));
+		}
+
+		// add dynamic properties
+		if (dynamicProperties != null) {
+			yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING, dynamicProperties);
 		}
+
+		writeYarnProperties(yarnProps, yarnPropertiesFile);
 	}
 
 	private void logAndSysout(String message) {
@@ -719,28 +727,64 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		System.out.println(message);
 	}
 
-	public static void main(final String[] args) throws Exception {
+	public static Map<String, String> getDynamicProperties(String dynamicPropertiesEncoded) {
+		if (dynamicPropertiesEncoded != null && dynamicPropertiesEncoded.length() > 0) {
+			Map<String, String> properties = new HashMap<>();
+
+			String[] propertyLines = dynamicPropertiesEncoded.split(YARN_DYNAMIC_PROPERTIES_SEPARATOR);
+			for (String propLine : propertyLines) {
+				if (propLine == null) {
+					continue;
+				}
+
+				int firstEquals = propLine.indexOf("=");
+
+				if (firstEquals >= 0) {
+					String key = propLine.substring(0, firstEquals).trim();
+					String value = propLine.substring(firstEquals + 1, propLine.length()).trim();
+
+					if (!key.isEmpty()) {
+						properties.put(key, value);
+					}
+				}
+			}
+			return properties;
+		}
+		else {
+			return Collections.emptyMap();
+		}
+	}
+
+	public static void main(final String[] args) {
 		final String configurationDirectory = CliFrontend.getConfigurationDirectoryFromEnv();
 
 		final Configuration flinkConfiguration = GlobalConfiguration.loadConfiguration();
 
-		final FlinkYarnSessionCli cli = new FlinkYarnSessionCli(
-			flinkConfiguration,
-			configurationDirectory,
-			"",
-			""); // no prefix for the YARN session
+		int retCode;
+
+		try {
+			final FlinkYarnSessionCli cli = new FlinkYarnSessionCli(
+				flinkConfiguration,
+				configurationDirectory,
+				"",
+				""); // no prefix for the YARN session
 
-		SecurityUtils.install(new SecurityConfiguration(flinkConfiguration));
+			SecurityUtils.install(new SecurityConfiguration(flinkConfiguration));
 
-		final int retCode = SecurityUtils.getInstalledContext().runSecured(() -> cli.run(args, flinkConfiguration, configurationDirectory));
+			retCode = SecurityUtils.getInstalledContext().runSecured(() -> cli.run(args));
+		} catch (CliArgsException e) {
+			retCode = handleCliArgsException(e);
+		} catch (Exception e) {
+			retCode = handleError(e);
+		}
 
 		System.exit(retCode);
 	}
 
 	private static void runInteractiveCli(
-		YarnClusterClient clusterClient,
-		YarnApplicationStatusMonitor yarnApplicationStatusMonitor,
-		boolean readConsoleInput) {
+			ClusterClient clusterClient,
+			YarnApplicationStatusMonitor yarnApplicationStatusMonitor,
+			boolean readConsoleInput) {
 		try (BufferedReader in = new BufferedReader(new InputStreamReader(System.in))) {
 			boolean continueRepl = true;
 			int numTaskmanagers = 0;
@@ -799,7 +843,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		}
 	}
 
-	private static void printClusterMessages(YarnClusterClient clusterClient) {
+	private static void printClusterMessages(ClusterClient clusterClient) {
 		final List<String> messages = clusterClient.getNewMessages();
 		if (!messages.isEmpty()) {
 			System.err.println("New messages from the YARN cluster: ");
@@ -819,8 +863,8 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 	 * @throws InterruptedException
 	 */
 	private static boolean repStep(
-		BufferedReader in,
-		boolean readConsoleInput) throws IOException, InterruptedException {
+			BufferedReader in,
+			boolean readConsoleInput) throws IOException, InterruptedException {
 
 		// wait until CLIENT_POLLING_INTERVAL is over or the user entered something.
 		long startTime = System.currentTimeMillis();
@@ -859,32 +903,25 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		propertiesFile.setReadable(true, false); // readable for all.
 	}
 
-	public static Map<String, String> getDynamicProperties(String dynamicPropertiesEncoded) {
-		if (dynamicPropertiesEncoded != null && dynamicPropertiesEncoded.length() > 0) {
-			Map<String, String> properties = new HashMap<>();
+	private static int handleCliArgsException(CliArgsException e) {
+		LOG.error("Could not parse the command line arguments.", e);
 
-			String[] propertyLines = dynamicPropertiesEncoded.split(YARN_DYNAMIC_PROPERTIES_SEPARATOR);
-			for (String propLine : propertyLines) {
-				if (propLine == null) {
-					continue;
-				}
+		System.out.println(e.getMessage());
+		System.out.println();
+		System.out.println("Use the help option (-h or --help) to get help on the command.");
+		return 1;
+	}
 
-				int firstEquals = propLine.indexOf("=");
+	private static int handleError(Exception e) {
+		LOG.error("Error while running the Flink Yarn session.", e);
 
-				if (firstEquals >= 0) {
-					String key = propLine.substring(0, firstEquals).trim();
-					String value = propLine.substring(firstEquals + 1, propLine.length()).trim();
+		System.err.println();
+		System.err.println("------------------------------------------------------------");
+		System.err.println(" The program finished with the following exception:");
+		System.err.println();
 
-					if (!key.isEmpty()) {
-						properties.put(key, value);
-					}
-				}
-			}
-			return properties;
-		}
-		else {
-			return Collections.emptyMap();
-		}
+		e.printStackTrace();
+		return 1;
 	}
 
 	public static File getYarnPropertiesLocation(@Nullable String yarnPropertiesFileLocation) {
@@ -902,7 +939,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 		return new File(propertiesFileLocation, YARN_PROPERTIES_FILE + currentUser);
 	}
 
-	protected AbstractYarnClusterDescriptor getClusterDescriptor(Configuration configuration, String configurationDirectory, boolean flip6) {
+	private static AbstractYarnClusterDescriptor getClusterDescriptor(Configuration configuration, String configurationDirectory, boolean flip6) {
 		final YarnClient yarnClient = YarnClient.createYarnClient();
 		if (flip6) {
 			return new YarnClusterDescriptorV2(configuration, configurationDirectory, yarnClient);

http://git-wip-us.apache.org/repos/asf/flink/blob/402499f0/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java
index 88d7747..f96b581 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java
@@ -59,7 +59,7 @@ public class YarnApplicationStatusMonitor implements AutoCloseable {
 
 		applicationStatusUpdateFuture = scheduledExecutor.scheduleWithFixedDelay(
 			this::updateApplicationStatus,
-			UPDATE_INTERVAL,
+			0L,
 			UPDATE_INTERVAL,
 			TimeUnit.MILLISECONDS);
 
@@ -83,6 +83,7 @@ public class YarnApplicationStatusMonitor implements AutoCloseable {
 				applicationReport = yarnClient.getApplicationReport(yarnApplicationId);
 			} catch (Exception e) {
 				LOG.info("Could not retrieve the Yarn application report for {}.", yarnApplicationId);
+				applicationStatus = ApplicationStatus.UNKNOWN;
 				return;
 			}
 


[13/14] flink git commit: [FLINK-8420] [flip6] Recognize TimeoutException in RetryingRegistration

Posted by tr...@apache.org.
[FLINK-8420] [flip6] Recognize TimeoutException in RetryingRegistration

A timeout exception will trigger an exponential backoff wrt the connection timeout.
This will guarantee that we don't overload the network with connection requests but
also to quickly connect to a newly available target.

This closes #5286.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3c99ae89
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3c99ae89
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3c99ae89

Branch: refs/heads/master
Commit: 3c99ae8959f69325bb9b7d810b41c60e42e602c5
Parents: d0bc300
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Jan 12 14:02:05 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 17:54:10 2018 +0100

----------------------------------------------------------------------
 .../registration/RetryingRegistration.java      |  3 +-
 .../registration/RetryingRegistrationTest.java  | 48 ++++++++++++++++----
 2 files changed, 42 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3c99ae89/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
index 802d361..279714b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.rpc.FencedRpcGateway;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.ExceptionUtils;
 
 import org.slf4j.Logger;
 
@@ -263,7 +264,7 @@ public abstract class RetryingRegistration<F extends Serializable, G extends Rpc
 			registrationAcceptFuture.whenCompleteAsync(
 				(Void v, Throwable failure) -> {
 					if (failure != null && !isCanceled()) {
-						if (failure instanceof TimeoutException) {
+						if (ExceptionUtils.stripCompletionException(failure) instanceof TimeoutException) {
 							// we simply have not received a response in time. maybe the timeout was
 							// very low (initial fast registration attempts), maybe the target endpoint is
 							// currently down.

http://git-wip-us.apache.org/repos/asf/flink/blob/3c99ae89/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
index 7fc6897..885a7f5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
@@ -160,7 +160,7 @@ public class RetryingRegistrationTest extends TestLogger {
 		}
 	}
 
-	@Test
+	@Test(timeout = 10000)
 	public void testRetriesOnTimeouts() throws Exception {
 		final String testId = "rien ne va plus";
 		final String testEndpointAddress = "<test-address>";
@@ -178,7 +178,15 @@ public class RetryingRegistrationTest extends TestLogger {
 		try {
 			rpc.registerGateway(testEndpointAddress, testGateway);
 
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+			final long initialTimeout = 20L;
+			TestRetryingRegistration registration = new TestRetryingRegistration(
+				rpc,
+				testEndpointAddress,
+				leaderId,
+				initialTimeout,
+				1000L,
+				15000L, // make sure that we timeout in case of an error
+				15000L);
 
 			long started = System.nanoTime();
 			registration.startRegistration();
@@ -195,7 +203,7 @@ public class RetryingRegistrationTest extends TestLogger {
 			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
 
 			// validate that some retry-delay / back-off behavior happened
-			assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT);
+			assertTrue("retries did not properly back off", elapsedMillis >= 3 * initialTimeout);
 		}
 		finally {
 			rpc.stopService();
@@ -346,11 +354,35 @@ public class RetryingRegistrationTest extends TestLogger {
 		static final long DELAY_ON_DECLINE = 200;
 
 		public TestRetryingRegistration(RpcService rpc, String targetAddress, UUID leaderId) {
-			super(LoggerFactory.getLogger(RetryingRegistrationTest.class),
-					rpc, "TestEndpoint",
-					TestRegistrationGateway.class,
-					targetAddress, leaderId,
-					INITIAL_TIMEOUT, MAX_TIMEOUT, DELAY_ON_ERROR, DELAY_ON_DECLINE);
+			this(
+				rpc,
+				targetAddress,
+				leaderId,
+				INITIAL_TIMEOUT,
+				MAX_TIMEOUT,
+				DELAY_ON_ERROR,
+				DELAY_ON_DECLINE);
+		}
+
+		public TestRetryingRegistration(
+				RpcService rpc,
+				String targetAddress,
+				UUID leaderId,
+				long initialTimeout,
+				long maxTimeout,
+				long delayOnError,
+				long delayOnDecline) {
+			super(
+				LoggerFactory.getLogger(RetryingRegistrationTest.class),
+				rpc,
+				"TestEndpoint",
+				TestRegistrationGateway.class,
+				targetAddress,
+				leaderId,
+				initialTimeout,
+				maxTimeout,
+				delayOnError,
+				delayOnDecline);
 		}
 
 		@Override


[09/14] flink git commit: [FLINK-8348] [flip6] Print help for DefaultCLI

Posted by tr...@apache.org.
[FLINK-8348] [flip6] Print help for DefaultCLI

This closes #5233.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/7d986ce0
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7d986ce0
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7d986ce0

Branch: refs/heads/master
Commit: 7d986ce0482562efd90fec416c4b2f4638a4058a
Parents: 38d3720
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Dec 29 15:29:06 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:05 2018 +0100

----------------------------------------------------------------------
 .../apache/flink/client/cli/CliFrontendParser.java  | 16 +++++++---------
 .../org/apache/flink/client/cli/DefaultCLI.java     |  2 +-
 2 files changed, 8 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7d986ce0/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
index 475d854..402b4ae 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
@@ -350,16 +350,14 @@ public class CliFrontendParser {
 			boolean runOptions) {
 		// prints options from all available command-line classes
 		for (CustomCommandLine cli: customCommandLines) {
-			if (cli.getId() != null) {
-				formatter.setSyntaxPrefix("  Options for " + cli.getId() + " mode:");
-				Options customOpts = new Options();
-				cli.addGeneralOptions(customOpts);
-				if (runOptions) {
-					cli.addRunOptions(customOpts);
-				}
-				formatter.printHelp(" ", customOpts);
-				System.out.println();
+			formatter.setSyntaxPrefix("  Options for " + cli.getId() + " mode:");
+			Options customOpts = new Options();
+			cli.addGeneralOptions(customOpts);
+			if (runOptions) {
+				cli.addRunOptions(customOpts);
 			}
+			formatter.printHelp(" ", customOpts);
+			System.out.println();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7d986ce0/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index d2694b8..2ca5a65 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -45,7 +45,7 @@ public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterId> {
 
 	@Override
 	public String getId() {
-		return null;
+		return "Default CLI";
 	}
 
 	@Override


[14/14] flink git commit: [FLINK-8288] [runtime] register job master rest endpoint url to yarn

Posted by tr...@apache.org.
[FLINK-8288] [runtime] register job master rest endpoint url to yarn

This closes #5186.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3b016868
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3b016868
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3b016868

Branch: refs/heads/master
Commit: 3b01686851e8281642924c4a620fb43b008de174
Parents: 3c99ae8
Author: shuai.xus <sh...@alibaba-inc.com>
Authored: Wed Dec 20 10:10:35 2017 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 17:55:01 2018 +0100

----------------------------------------------------------------------
 .../entrypoint/JobClusterEntrypoint.java        | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3b016868/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java
index 21a7ba4..cb1b086 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java
@@ -92,16 +92,6 @@ public abstract class JobClusterEntrypoint extends ClusterEntrypoint {
 			HeartbeatServices heartbeatServices,
 			MetricRegistry metricRegistry) throws Exception {
 
-		resourceManager = createResourceManager(
-			configuration,
-			ResourceID.generate(),
-			rpcService,
-			highAvailabilityServices,
-			heartbeatServices,
-			metricRegistry,
-			this,
-			null);
-
 		jobManagerServices = JobManagerServices.fromConfiguration(configuration, blobServer);
 
 		resourceManagerRetrievalService = highAvailabilityServices.getResourceManagerLeaderRetriever();
@@ -134,6 +124,16 @@ public abstract class JobClusterEntrypoint extends ClusterEntrypoint {
 		LOG.debug("Starting JobMaster REST endpoint.");
 		jobMasterRestEndpoint.start();
 
+		resourceManager = createResourceManager(
+			configuration,
+			ResourceID.generate(),
+			rpcService,
+			highAvailabilityServices,
+			heartbeatServices,
+			metricRegistry,
+			this,
+			jobMasterRestEndpoint.getRestAddress());
+
 		jobManagerRunner = createJobManagerRunner(
 			configuration,
 			ResourceID.generate(),


[03/14] flink git commit: [FLINK-8340] [flip6] Remove passing of Configuration to CustomCommandLine

Posted by tr...@apache.org.
[FLINK-8340] [flip6] Remove passing of Configuration to CustomCommandLine

Since the Configuration does not change over the lifetime of a CustomCommandLine,
we can safely pass it as a constructor argument instead of method argument.

This closes #5226.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/30011b9b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/30011b9b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/30011b9b

Branch: refs/heads/master
Commit: 30011b9b110aad0e1c28e7e0a025b73986781a72
Parents: 4ade826
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Jan 3 13:25:09 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:02 2018 +0100

----------------------------------------------------------------------
 .../client/cli/AbstractCustomCommandLine.java   |  16 +-
 .../apache/flink/client/cli/CliFrontend.java    |  84 ++--
 .../flink/client/cli/CustomCommandLine.java     |  23 +-
 .../org/apache/flink/client/cli/DefaultCLI.java |  17 +-
 .../flink/client/cli/Flip6DefaultCLI.java       |  17 +-
 .../flink/client/cli/CliFrontendCancelTest.java |  24 +-
 .../flink/client/cli/CliFrontendInfoTest.java   |  24 +-
 .../flink/client/cli/CliFrontendListTest.java   |   6 +-
 .../cli/CliFrontendPackageProgramTest.java      | 251 +++++-------
 .../flink/client/cli/CliFrontendRunTest.java    |  34 +-
 .../flink/client/cli/CliFrontendStopTest.java   |  12 +-
 .../apache/flink/client/cli/DefaultCLITest.java |  49 +--
 .../flink/client/cli/Flip6DefaultCLITest.java   |   4 +-
 .../client/cli/util/DummyCustomCommandLine.java |  12 +-
 .../client/cli/util/MockedCliFrontend.java      |   4 +-
 .../flink/client/program/ClusterClientTest.java |   4 +
 .../org/apache/flink/api/scala/FlinkShell.scala |  27 +-
 .../YARNSessionCapacitySchedulerITCase.java     |   4 +-
 .../flink/yarn/YARNSessionFIFOITCase.java       |   5 +-
 .../org/apache/flink/yarn/YarnTestBase.java     |   5 +-
 .../apache/flink/yarn/YarnClusterClient.java    |   3 +-
 .../flink/yarn/cli/FlinkYarnSessionCli.java     | 389 ++++++++++---------
 .../flink/yarn/FlinkYarnSessionCliTest.java     | 121 +++---
 23 files changed, 541 insertions(+), 594 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
index 70ffc5c..c7a1672 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
@@ -22,6 +22,9 @@ import org.apache.flink.client.ClientUtils;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -47,6 +50,16 @@ public abstract class AbstractCustomCommandLine<C extends ClusterClient> impleme
 		"Address of the JobManager (master) to which to connect. " +
 			"Use this flag to connect to a different JobManager than the one specified in the configuration.");
 
+	protected final Configuration configuration;
+
+	protected AbstractCustomCommandLine(Configuration configuration) {
+		this.configuration = new UnmodifiableConfiguration(Preconditions.checkNotNull(configuration));
+	}
+
+	public Configuration getConfiguration() {
+		return configuration;
+	}
+
 	@Override
 	public void addRunOptions(Options baseOptions) {
 		// nothing to add here
@@ -61,11 +74,10 @@ public abstract class AbstractCustomCommandLine<C extends ClusterClient> impleme
 	/**
 	 * Override configuration settings by specified command line options.
 	 *
-	 * @param configuration to use as the base configuration
 	 * @param commandLine containing the overriding values
 	 * @return Effective configuration with the overriden configuration settings
 	 */
-	protected Configuration applyCommandLineOptionsToConfiguration(Configuration configuration, CommandLine commandLine) {
+	protected Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException {
 		final Configuration resultingConfiguration = new Configuration(configuration);
 
 		if (commandLine.hasOption(addressOption.getOpt())) {

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index 4b5040e..630154c 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -108,8 +108,6 @@ public class CliFrontend {
 
 	private final List<CustomCommandLine<?>> customCommandLines;
 
-	private final String configurationDirectory;
-
 	private final Options customCommandLineOptions;
 
 	private final FiniteDuration clientTimeout;
@@ -118,11 +116,9 @@ public class CliFrontend {
 
 	public CliFrontend(
 			Configuration configuration,
-			List<CustomCommandLine<?>> customCommandLines,
-			String configurationDirectory) throws Exception {
+			List<CustomCommandLine<?>> customCommandLines) throws Exception {
 		this.configuration = Preconditions.checkNotNull(configuration);
 		this.customCommandLines = Preconditions.checkNotNull(customCommandLines);
-		this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory);
 
 		try {
 			FileSystem.initialize(this.configuration);
@@ -139,9 +135,9 @@ public class CliFrontend {
 		}
 
 		this.clientTimeout = AkkaUtils.getClientTimeout(this.configuration);
-		this.defaultParallelism = GlobalConfiguration.loadConfiguration().getInteger(
-														ConfigConstants.DEFAULT_PARALLELISM_KEY,
-														ConfigConstants.DEFAULT_PARALLELISM);
+		this.defaultParallelism = configuration.getInteger(
+			ConfigConstants.DEFAULT_PARALLELISM_KEY,
+			ConfigConstants.DEFAULT_PARALLELISM);
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -161,15 +157,6 @@ public class CliFrontend {
 		return copiedConfiguration;
 	}
 
-	/**
-	 * Returns the configuration directory for the CLI frontend.
-	 *
-	 * @return Configuration directory
-	 */
-	public String getConfigurationDirectory() {
-		return configurationDirectory;
-	}
-
 	// --------------------------------------------------------------------------------------------
 	//  Execute Actions
 	// --------------------------------------------------------------------------------------------
@@ -211,20 +198,17 @@ public class CliFrontend {
 
 		final CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(
-			configuration,
-			configurationDirectory,
-			commandLine);
+		final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
 
 		try {
-			final String clusterId = customCommandLine.getClusterId(configuration, commandLine);
+			final String clusterId = customCommandLine.getClusterId(commandLine);
 
 			final ClusterClient client;
 
 			if (clusterId != null) {
 				client = clusterDescriptor.retrieve(clusterId);
 			} else {
-				final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(configuration, commandLine);
+				final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(commandLine);
 				client = clusterDescriptor.deploySessionCluster(clusterSpecification);
 			}
 
@@ -368,14 +352,9 @@ public class CliFrontend {
 		}
 
 		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
-		final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(
-			configuration,
-			configurationDirectory,
-			commandLine);
+		final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
 
-		final String clusterId = activeCommandLine.getClusterId(
-			configuration,
-			commandLine);
+		final String clusterId = activeCommandLine.getClusterId(commandLine);
 
 		if (clusterId == null) {
 			throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
@@ -496,12 +475,9 @@ public class CliFrontend {
 
 		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(
-			configuration,
-			configurationDirectory,
-			commandLine);
+		final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
 
-		final String clusterId = activeCommandLine.getClusterId(configuration, commandLine);
+		final String clusterId = activeCommandLine.getClusterId(commandLine);
 
 		if (clusterId == null) {
 			throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
@@ -579,12 +555,9 @@ public class CliFrontend {
 
 		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(
-			configuration,
-			configurationDirectory,
-			commandLine);
+		final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
 
-		final String clusterId = activeCommandLine.getClusterId(configuration, commandLine);
+		final String clusterId = activeCommandLine.getClusterId(commandLine);
 
 		if (clusterId == null) {
 			throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
@@ -646,14 +619,9 @@ public class CliFrontend {
 
 		CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(
-			configuration,
-			configurationDirectory,
-			commandLine);
+		final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
 
-		final String clusterId = customCommandLine.getClusterId(
-			configuration,
-			commandLine);
+		final String clusterId = customCommandLine.getClusterId(commandLine);
 
 		if (clusterId == null) {
 			throw new FlinkException("No cluster id was specified. Please specify a cluster to which " +
@@ -1004,13 +972,14 @@ public class CliFrontend {
 		final Configuration configuration = GlobalConfiguration.loadConfiguration(configurationDirectory);
 
 		// 3. load the custom command lines
-		final List<CustomCommandLine<?>> customCommandLines = loadCustomCommandLines();
+		final List<CustomCommandLine<?>> customCommandLines = loadCustomCommandLines(
+			configuration,
+			configurationDirectory);
 
 		try {
 			final CliFrontend cli = new CliFrontend(
 				configuration,
-				customCommandLines,
-				configurationDirectory);
+				customCommandLines);
 
 			SecurityUtils.install(new SecurityConfiguration(cli.configuration));
 			int retCode = SecurityUtils.getInstalledContext()
@@ -1070,7 +1039,7 @@ public class CliFrontend {
 		config.setInteger(JobManagerOptions.PORT, address.getPort());
 	}
 
-	public static List<CustomCommandLine<?>> loadCustomCommandLines() {
+	public static List<CustomCommandLine<?>> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
 		List<CustomCommandLine<?>> customCommandLines = new ArrayList<>(2);
 
 		//	Command line interface of the YARN session, with a special initialization here
@@ -1079,13 +1048,18 @@ public class CliFrontend {
 		//	      active CustomCommandLine in order and DefaultCLI isActive always return true.
 		final String flinkYarnSessionCLI = "org.apache.flink.yarn.cli.FlinkYarnSessionCli";
 		try {
-			customCommandLines.add(loadCustomCommandLine(flinkYarnSessionCLI, "y", "yarn"));
+			customCommandLines.add(
+				loadCustomCommandLine(flinkYarnSessionCLI,
+					configuration,
+					configurationDirectory,
+					"y",
+					"yarn"));
 		} catch (Exception e) {
 			LOG.warn("Could not load CLI class {}.", flinkYarnSessionCLI, e);
 		}
 
-		customCommandLines.add(new Flip6DefaultCLI());
-		customCommandLines.add(new DefaultCLI());
+		customCommandLines.add(new Flip6DefaultCLI(configuration));
+		customCommandLines.add(new DefaultCLI(configuration));
 
 		return customCommandLines;
 	}
@@ -1101,7 +1075,7 @@ public class CliFrontend {
 	 */
 	public CustomCommandLine getActiveCustomCommandLine(CommandLine commandLine) {
 		for (CustomCommandLine cli : customCommandLines) {
-			if (cli.isActive(commandLine, configuration)) {
+			if (cli.isActive(commandLine)) {
 				return cli;
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
index cfa0827..f642484 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
@@ -21,7 +21,7 @@ package org.apache.flink.client.cli;
 import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.FlinkException;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Options;
@@ -36,10 +36,9 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
 	/**
 	 * Signals whether the custom command-line wants to execute or not.
 	 * @param commandLine The command-line options
-	 * @param configuration The Flink configuration
 	 * @return True if the command-line wants to run, False otherwise
 	 */
-	boolean isActive(CommandLine commandLine, Configuration configuration);
+	boolean isActive(CommandLine commandLine);
 
 	/**
 	 * Gets the unique identifier of this CustomCommandLine.
@@ -55,6 +54,7 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
 
 	/**
 	 * Adds custom options to the existing general options.
+	 *
 	 * @param baseOptions The existing options.
 	 */
 	void addGeneralOptions(Options baseOptions);
@@ -63,15 +63,11 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
 	 * Create a {@link ClusterDescriptor} from the given configuration, configuration directory
 	 * and the command line.
 	 *
-	 * @param configuration to create the ClusterDescriptor with
-	 * @param configurationDirectory where the configuration was loaded from
 	 * @param commandLine containing command line options relevant for the ClusterDescriptor
 	 * @return ClusterDescriptor
+	 * @throws FlinkException if the ClusterDescriptor could not be created
 	 */
-	ClusterDescriptor<ClusterType> createClusterDescriptor(
-		Configuration configuration,
-		String configurationDirectory,
-		CommandLine commandLine);
+	ClusterDescriptor<ClusterType> createClusterDescriptor(CommandLine commandLine) throws FlinkException;
 
 	/**
 	 * Returns the cluster id if a cluster id was specified on the command line, otherwise it
@@ -80,24 +76,21 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
 	 * <p>A cluster id identifies a running cluster, e.g. the Yarn application id for a Flink
 	 * cluster running on Yarn.
 	 *
-	 * @param configuration to be used for the cluster id retrieval
 	 * @param commandLine containing command line options relevant for the cluster id retrieval
 	 * @return Cluster id identifying the cluster to deploy jobs to or null
 	 */
 	@Nullable
-	String getClusterId(Configuration configuration, CommandLine commandLine);
+	String getClusterId(CommandLine commandLine);
 
 	/**
 	 * Returns the {@link ClusterSpecification} specified by the configuration and the command
 	 * line options. This specification can be used to deploy a new Flink cluster.
 	 *
-	 * @param configuration to be used for the ClusterSpecification values
 	 * @param commandLine containing command line options relevant for the ClusterSpecification
 	 * @return ClusterSpecification for a new Flink cluster
+	 * @throws FlinkException if the ClusterSpecification could not be created
 	 */
-	ClusterSpecification getClusterSpecification(
-		Configuration configuration,
-		CommandLine commandLine);
+	ClusterSpecification getClusterSpecification(CommandLine commandLine) throws FlinkException;
 
 	default CommandLine parseCommandLineOptions(String[] args, boolean stopAtNonOptions) throws CliArgsException {
 		final Options options = new Options();

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index d34e307..c29c5b7 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -23,6 +23,7 @@ import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
 import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.FlinkException;
 
 import org.apache.commons.cli.CommandLine;
 
@@ -33,8 +34,12 @@ import javax.annotation.Nullable;
  */
 public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterClient> {
 
+	public DefaultCLI(Configuration configuration) {
+		super(configuration);
+	}
+
 	@Override
-	public boolean isActive(CommandLine commandLine, Configuration configuration) {
+	public boolean isActive(CommandLine commandLine) {
 		// always active because we can try to read a JobManager address from the config
 		return true;
 	}
@@ -46,22 +51,20 @@ public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterClien
 
 	@Override
 	public ClusterDescriptor<StandaloneClusterClient> createClusterDescriptor(
-			Configuration configuration,
-			String configurationDirectory,
-			CommandLine commandLine) {
-		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(configuration, commandLine);
+			CommandLine commandLine) throws FlinkException {
+		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
 
 		return new StandaloneClusterDescriptor(effectiveConfiguration);
 	}
 
 	@Override
 	@Nullable
-	public String getClusterId(Configuration configuration, CommandLine commandLine) {
+	public String getClusterId(CommandLine commandLine) {
 		return "standalone";
 	}
 
 	@Override
-	public ClusterSpecification getClusterSpecification(Configuration configuration, CommandLine commandLine) {
+	public ClusterSpecification getClusterSpecification(CommandLine commandLine) {
 		return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
index 0e80f44..3adeca6 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
@@ -23,6 +23,7 @@ import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.deployment.Flip6StandaloneClusterDescriptor;
 import org.apache.flink.client.program.rest.RestClusterClient;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.FlinkException;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -41,8 +42,12 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine<RestClusterClient
 		FLIP_6.setRequired(false);
 	}
 
+	public Flip6DefaultCLI(Configuration configuration) {
+		super(configuration);
+	}
+
 	@Override
-	public boolean isActive(CommandLine commandLine, Configuration configuration) {
+	public boolean isActive(CommandLine commandLine) {
 		return commandLine.hasOption(FLIP_6.getOpt());
 	}
 
@@ -59,22 +64,20 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine<RestClusterClient
 
 	@Override
 	public ClusterDescriptor<RestClusterClient> createClusterDescriptor(
-			Configuration configuration,
-			String configurationDirectory,
-			CommandLine commandLine) {
-		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(configuration, commandLine);
+			CommandLine commandLine) throws FlinkException {
+		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
 
 		return new Flip6StandaloneClusterDescriptor(effectiveConfiguration);
 	}
 
 	@Override
 	@Nullable
-	public String getClusterId(Configuration configuration, CommandLine commandLine) {
+	public String getClusterId(CommandLine commandLine) {
 		return "flip6Standalone";
 	}
 
 	@Override
-	public ClusterSpecification getClusterSpecification(Configuration configuration, CommandLine commandLine) {
+	public ClusterSpecification getClusterSpecification(CommandLine commandLine) {
 		return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
index b0a308d..60bd308 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java
@@ -63,20 +63,20 @@ public class CliFrontendCancelTest extends TestLogger {
 	@Test(expected = CliArgsException.class)
 	public void testMissingJobId() throws Exception {
 		String[] parameters = {};
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.cancel(parameters);
 	}
 
 	@Test(expected = CliArgsException.class)
 	public void testUnrecognizedOption() throws Exception {
 		String[] parameters = {"-v", "-l"};
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.cancel(parameters);
 	}
 
@@ -116,10 +116,10 @@ public class CliFrontendCancelTest extends TestLogger {
 	public void testCancelWithSavepointWithoutJobId() throws Exception {
 		// Cancel with savepoint (with target directory), but no job ID
 		String[] parameters = { "-s", "targetDirectory" };
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.cancel(parameters);
 	}
 
@@ -127,10 +127,10 @@ public class CliFrontendCancelTest extends TestLogger {
 	public void testCancelWithSavepointWithoutParameters() throws Exception {
 		// Cancel with savepoint (no target directory) and no job ID
 		String[] parameters = { "-s" };
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.cancel(parameters);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java
index fa212cb..c284c61 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java
@@ -42,20 +42,20 @@ public class CliFrontendInfoTest extends TestLogger {
 	@Test(expected = CliArgsException.class)
 	public void testMissingOption() throws Exception {
 		String[] parameters = {};
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.cancel(parameters);
 	}
 
 	@Test(expected = CliArgsException.class)
 	public void testUnrecognizedOption() throws Exception {
 		String[] parameters = {"-v", "-l"};
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.cancel(parameters);
 	}
 
@@ -65,10 +65,10 @@ public class CliFrontendInfoTest extends TestLogger {
 		try {
 
 			String[] parameters = new String[]{CliFrontendTestUtils.getTestJarPath(), "-f", "true"};
+			Configuration configuration = new Configuration();
 			CliFrontend testFrontend = new CliFrontend(
-				new Configuration(),
-				Collections.singletonList(new DefaultCLI()),
-				CliFrontendTestUtils.getConfigDir());
+				configuration,
+				Collections.singletonList(new DefaultCLI(configuration)));
 			testFrontend.info(parameters);
 			assertTrue(buffer.toString().contains("\"parallelism\": \"1\""));
 		}
@@ -82,10 +82,10 @@ public class CliFrontendInfoTest extends TestLogger {
 		replaceStdOut();
 		try {
 			String[] parameters = {"-p", "17", CliFrontendTestUtils.getTestJarPath()};
+			Configuration configuration = new Configuration();
 			CliFrontend testFrontend = new CliFrontend(
-				new Configuration(),
-				Collections.singletonList(new DefaultCLI()),
-				CliFrontendTestUtils.getConfigDir());
+				configuration,
+				Collections.singletonList(new DefaultCLI(configuration)));
 			testFrontend.info(parameters);
 			assertTrue(buffer.toString().contains("\"parallelism\": \"17\""));
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
index 6639e25..77d8016 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java
@@ -60,10 +60,10 @@ public class CliFrontendListTest extends TestLogger {
 	@Test(expected = CliArgsException.class)
 	public void testUnrecognizedOption() throws Exception {
 		String[] parameters = {"-v", "-k"};
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.list(parameters);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
index a862dc1..6873e68 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
@@ -22,7 +22,6 @@ import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.PackagedProgram;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.optimizer.CompilerException;
 import org.apache.flink.optimizer.DataStatistics;
 import org.apache.flink.optimizer.Optimizer;
 import org.apache.flink.optimizer.costs.DefaultCostEstimator;
@@ -64,189 +63,145 @@ public class CliFrontendPackageProgramTest extends TestLogger {
 
 	@Before
 	public void setup() throws Exception {
+		final Configuration configuration = new Configuration();
 		frontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 	}
 
 	@Test
-	public void testNonExistingJarFile() {
-		try {
-			ProgramOptions options = mock(ProgramOptions.class);
-			when(options.getJarFilePath()).thenReturn("/some/none/existing/path");
+	public void testNonExistingJarFile() throws Exception {
+		ProgramOptions options = mock(ProgramOptions.class);
+		when(options.getJarFilePath()).thenReturn("/some/none/existing/path");
 
-			try {
-				frontend.buildProgram(options);
-				fail("should throw an exception");
-			}
-			catch (FileNotFoundException e) {
-				// that's what we want
-			}
+		try {
+			frontend.buildProgram(options);
+			fail("should throw an exception");
 		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+		catch (FileNotFoundException e) {
+			// that's what we want
 		}
 	}
 
 	@Test
-	public void testFileNotJarFile() {
-		try {
-			ProgramOptions options = mock(ProgramOptions.class);
-			when(options.getJarFilePath()).thenReturn(getNonJarFilePath());
+	public void testFileNotJarFile() throws Exception {
+		ProgramOptions options = mock(ProgramOptions.class);
+		when(options.getJarFilePath()).thenReturn(getNonJarFilePath());
 
-			try {
-				frontend.buildProgram(options);
-				fail("should throw an exception");
-			}
-			catch (ProgramInvocationException e) {
-				// that's what we want
-			}
+		try {
+			frontend.buildProgram(options);
+			fail("should throw an exception");
 		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+		catch (ProgramInvocationException e) {
+			// that's what we want
 		}
 	}
 
 	@Test
-	public void testVariantWithExplicitJarAndArgumentsOption() {
-		try {
-			String[] arguments = {
-					"--classpath", "file:///tmp/foo",
-					"--classpath", "file:///tmp/bar",
-					"-j", getTestJarPath(),
-					"-a", "--debug", "true", "arg1", "arg2" };
-			URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
-			String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"};
-
-			RunOptions options = CliFrontendParser.parseRunCommand(arguments);
-			assertEquals(getTestJarPath(), options.getJarFilePath());
-			assertArrayEquals(classpath, options.getClasspaths().toArray());
-			assertArrayEquals(reducedArguments, options.getProgramArgs());
-
-			PackagedProgram prog = frontend.buildProgram(options);
-
-			Assert.assertArrayEquals(reducedArguments, prog.getArguments());
-			Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+	public void testVariantWithExplicitJarAndArgumentsOption() throws Exception {
+		String[] arguments = {
+				"--classpath", "file:///tmp/foo",
+				"--classpath", "file:///tmp/bar",
+				"-j", getTestJarPath(),
+				"-a", "--debug", "true", "arg1", "arg2" };
+		URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
+		String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"};
+
+		RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+		assertEquals(getTestJarPath(), options.getJarFilePath());
+		assertArrayEquals(classpath, options.getClasspaths().toArray());
+		assertArrayEquals(reducedArguments, options.getProgramArgs());
+
+		PackagedProgram prog = frontend.buildProgram(options);
+
+		Assert.assertArrayEquals(reducedArguments, prog.getArguments());
+		Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
 	}
 
 	@Test
-	public void testVariantWithExplicitJarAndNoArgumentsOption() {
-		try {
-			String[] arguments = {
-					"--classpath", "file:///tmp/foo",
-					"--classpath", "file:///tmp/bar",
-					"-j", getTestJarPath(),
-					"--debug", "true", "arg1", "arg2" };
-			URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
-			String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"};
-
-			RunOptions options = CliFrontendParser.parseRunCommand(arguments);
-			assertEquals(getTestJarPath(), options.getJarFilePath());
-			assertArrayEquals(classpath, options.getClasspaths().toArray());
-			assertArrayEquals(reducedArguments, options.getProgramArgs());
-
-			PackagedProgram prog = frontend.buildProgram(options);
-
-			Assert.assertArrayEquals(reducedArguments, prog.getArguments());
-			Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+	public void testVariantWithExplicitJarAndNoArgumentsOption() throws Exception {
+		String[] arguments = {
+				"--classpath", "file:///tmp/foo",
+				"--classpath", "file:///tmp/bar",
+				"-j", getTestJarPath(),
+				"--debug", "true", "arg1", "arg2" };
+		URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
+		String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"};
+
+		RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+		assertEquals(getTestJarPath(), options.getJarFilePath());
+		assertArrayEquals(classpath, options.getClasspaths().toArray());
+		assertArrayEquals(reducedArguments, options.getProgramArgs());
+
+		PackagedProgram prog = frontend.buildProgram(options);
+
+		Assert.assertArrayEquals(reducedArguments, prog.getArguments());
+		Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
 	}
 
 	@Test
-	public void testValidVariantWithNoJarAndNoArgumentsOption() {
-		try {
-			String[] arguments = {
-					"--classpath", "file:///tmp/foo",
-					"--classpath", "file:///tmp/bar",
-					getTestJarPath(),
-					"--debug", "true", "arg1", "arg2" };
-			URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
-			String[] reducedArguments = {"--debug", "true", "arg1", "arg2"};
-
-			RunOptions options = CliFrontendParser.parseRunCommand(arguments);
-			assertEquals(getTestJarPath(), options.getJarFilePath());
-			assertArrayEquals(classpath, options.getClasspaths().toArray());
-			assertArrayEquals(reducedArguments, options.getProgramArgs());
-
-			PackagedProgram prog = frontend.buildProgram(options);
-
-			Assert.assertArrayEquals(reducedArguments, prog.getArguments());
-			Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+	public void testValidVariantWithNoJarAndNoArgumentsOption() throws Exception {
+		String[] arguments = {
+				"--classpath", "file:///tmp/foo",
+				"--classpath", "file:///tmp/bar",
+				getTestJarPath(),
+				"--debug", "true", "arg1", "arg2" };
+		URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
+		String[] reducedArguments = {"--debug", "true", "arg1", "arg2"};
+
+		RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+		assertEquals(getTestJarPath(), options.getJarFilePath());
+		assertArrayEquals(classpath, options.getClasspaths().toArray());
+		assertArrayEquals(reducedArguments, options.getProgramArgs());
+
+		PackagedProgram prog = frontend.buildProgram(options);
+
+		Assert.assertArrayEquals(reducedArguments, prog.getArguments());
+		Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName());
 	}
 
 	@Test(expected = CliArgsException.class)
 	public void testNoJarNoArgumentsAtAll() throws Exception {
 		frontend.run(new String[0]);
-
-		fail("Should have failed.");
 	}
 
 	@Test
-	public void testNonExistingFileWithArguments() {
-		try {
-			String[] arguments = {
-					"--classpath", "file:///tmp/foo",
-					"--classpath", "file:///tmp/bar",
-					"/some/none/existing/path",
-					"--debug", "true", "arg1", "arg2"  };
-			URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
-			String[] reducedArguments = {"--debug", "true", "arg1", "arg2"};
+	public void testNonExistingFileWithArguments() throws Exception {
+		String[] arguments = {
+				"--classpath", "file:///tmp/foo",
+				"--classpath", "file:///tmp/bar",
+				"/some/none/existing/path",
+				"--debug", "true", "arg1", "arg2"  };
+		URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") };
+		String[] reducedArguments = {"--debug", "true", "arg1", "arg2"};
+
+		RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+		assertEquals(arguments[4], options.getJarFilePath());
+		assertArrayEquals(classpath, options.getClasspaths().toArray());
+		assertArrayEquals(reducedArguments, options.getProgramArgs());
 
-			RunOptions options = CliFrontendParser.parseRunCommand(arguments);
-			assertEquals(arguments[4], options.getJarFilePath());
-			assertArrayEquals(classpath, options.getClasspaths().toArray());
-			assertArrayEquals(reducedArguments, options.getProgramArgs());
-
-			try {
-				frontend.buildProgram(options);
-				fail("Should fail with an exception");
-			}
-			catch (FileNotFoundException e) {
-				// that's what we want
-			}
+		try {
+			frontend.buildProgram(options);
+			fail("Should fail with an exception");
 		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+		catch (FileNotFoundException e) {
+			// that's what we want
 		}
 	}
 
 	@Test
-	public void testNonExistingFileWithoutArguments() {
-		try {
-			String[] arguments = {"/some/none/existing/path"};
+	public void testNonExistingFileWithoutArguments() throws Exception {
+		String[] arguments = {"/some/none/existing/path"};
 
-			RunOptions options = CliFrontendParser.parseRunCommand(arguments);
-			assertEquals(arguments[0], options.getJarFilePath());
-			assertArrayEquals(new String[0], options.getProgramArgs());
+		RunOptions options = CliFrontendParser.parseRunCommand(arguments);
+		assertEquals(arguments[0], options.getJarFilePath());
+		assertArrayEquals(new String[0], options.getProgramArgs());
 
-			try {
-				frontend.buildProgram(options);
-			}
-			catch (FileNotFoundException e) {
-				// that's what we want
-			}
+		try {
+			frontend.buildProgram(options);
 		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+		catch (FileNotFoundException e) {
+			// that's what we want
 		}
 	}
 
@@ -284,7 +239,7 @@ public class CliFrontendPackageProgramTest extends TestLogger {
 	 * </ul>
 	 */
 	@Test
-	public void testPlanWithExternalClass() throws CompilerException, ProgramInvocationException {
+	public void testPlanWithExternalClass() throws Exception {
 		final boolean[] callme = { false }; // create a final object reference, to be able to change its val later
 
 		try {
@@ -334,9 +289,5 @@ public class CliFrontendPackageProgramTest extends TestLogger {
 			}
 			assertTrue("Classloader was not called", callme[0]);
 		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Program failed with the wrong exception: " + e.getClass().getName());
-		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java
index fa1c76c..ebb76d8 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java
@@ -46,31 +46,32 @@ public class CliFrontendRunTest {
 
 	@Test
 	public void testRun() throws Exception {
+		final Configuration configuration = GlobalConfiguration.loadConfiguration(CliFrontendTestUtils.getConfigDir());
 		// test without parallelism
 		{
 			String[] parameters = {"-v", getTestJarPath()};
-			RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(1, true, false);
+			RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 1, true, false);
 			testFrontend.run(parameters);
 		}
 
 		// test configure parallelism
 		{
 			String[] parameters = {"-v", "-p", "42",  getTestJarPath()};
-			RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(42, true, false);
+			RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 42, true, false);
 			testFrontend.run(parameters);
 		}
 
 		// test configure sysout logging
 		{
 			String[] parameters = {"-p", "2", "-q", getTestJarPath()};
-			RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, false, false);
+			RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 2, false, false);
 			testFrontend.run(parameters);
 		}
 
 		// test detached mode
 		{
 			String[] parameters = {"-p", "2", "-d", getTestJarPath()};
-			RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, true, true);
+			RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 2, true, true);
 			testFrontend.run(parameters);
 		}
 
@@ -111,10 +112,10 @@ public class CliFrontendRunTest {
 	public void testUnrecognizedOption() throws Exception {
 		// test unrecognized option
 		String[] parameters = {"-v", "-l", "-a", "some", "program", "arguments"};
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.run(parameters);
 	}
 
@@ -122,10 +123,10 @@ public class CliFrontendRunTest {
 	public void testInvalidParallelismOption() throws Exception {
 		// test configure parallelism with non integer value
 		String[] parameters = {"-v", "-p", "text",  getTestJarPath()};
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.run(parameters);
 	}
 
@@ -133,10 +134,10 @@ public class CliFrontendRunTest {
 	public void testParallelismWithOverflow() throws Exception {
 		// test configure parallelism with overflow integer value
 		String[] parameters = {"-v", "-p", "475871387138",  getTestJarPath()};
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.run(parameters);
 	}
 
@@ -148,11 +149,10 @@ public class CliFrontendRunTest {
 		private final boolean sysoutLogging;
 		private final boolean isDetached;
 
-		public RunTestingCliFrontend(int expectedParallelism, boolean logging, boolean isDetached) throws Exception {
+		public RunTestingCliFrontend(Configuration configuration, int expectedParallelism, boolean logging, boolean isDetached) throws Exception {
 			super(
-				GlobalConfiguration.loadConfiguration(CliFrontendTestUtils.getConfigDir()),
-				Collections.singletonList(new DefaultCLI()),
-				CliFrontendTestUtils.getConfigDir());
+				configuration,
+				Collections.singletonList(new DefaultCLI(configuration)));
 			this.expectedParallelism = expectedParallelism;
 			this.sysoutLogging = logging;
 			this.isDetached = isDetached;

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
index fbbc739..0120cdf 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java
@@ -69,10 +69,10 @@ public class CliFrontendStopTest extends TestLogger {
 	public void testUnrecognizedOption() throws Exception {
 		// test unrecognized option
 		String[] parameters = { "-v", "-l" };
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.stop(parameters);
 	}
 
@@ -80,10 +80,10 @@ public class CliFrontendStopTest extends TestLogger {
 	public void testMissingJobId() throws Exception {
 		// test missing job id
 		String[] parameters = {};
+		Configuration configuration = new Configuration();
 		CliFrontend testFrontend = new CliFrontend(
-			new Configuration(),
-			Collections.singletonList(new DefaultCLI()),
-			CliFrontendTestUtils.getConfigDir());
+			configuration,
+			Collections.singletonList(new DefaultCLI(configuration)));
 		testFrontend.stop(parameters);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
index 3f05f90..e73b9c9 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
@@ -46,30 +46,25 @@ public class DefaultCLITest extends TestLogger {
 	 */
 	@Test
 	public void testConfigurationPassing() throws Exception {
-		final DefaultCLI defaultCLI = new DefaultCLI();
-
-		final String configurationDirectory = temporaryFolder.newFolder().getAbsolutePath();
-		final String[] args = {};
-
-		CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
+		final Configuration configuration = new Configuration();
 
 		final String localhost = "localhost";
 		final int port = 1234;
-		final Configuration configuration = new Configuration();
 
 		configuration.setString(JobManagerOptions.ADDRESS, localhost);
 		configuration.setInteger(JobManagerOptions.PORT, port);
 
+		final DefaultCLI defaultCLI = new DefaultCLI(configuration);
+
+		final String[] args = {};
+
+		CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
+
 		final InetSocketAddress expectedAddress = new InetSocketAddress(localhost, port);
 
-		final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(
-			configuration,
-			configurationDirectory,
-			commandLine);
+		final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
 
-		final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(
-			configuration,
-			commandLine));
+		final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
 
 		Assert.assertEquals(expectedAddress, clusterClient.getJobManagerAddress());
 	}
@@ -79,15 +74,6 @@ public class DefaultCLITest extends TestLogger {
 	 */
 	@Test
 	public void testManualConfigurationOverride() throws Exception {
-		final DefaultCLI defaultCLI = new DefaultCLI();
-
-		final String manualHostname = "123.123.123.123";
-		final int manualPort = 4321;
-		final String configurationDirectory = temporaryFolder.newFolder().getAbsolutePath();
-		final String[] args = {"-m", manualHostname + ':' + manualPort};
-
-		CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
-
 		final String localhost = "localhost";
 		final int port = 1234;
 		final Configuration configuration = new Configuration();
@@ -95,14 +81,17 @@ public class DefaultCLITest extends TestLogger {
 		configuration.setString(JobManagerOptions.ADDRESS, localhost);
 		configuration.setInteger(JobManagerOptions.PORT, port);
 
-		final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(
-			configuration,
-			configurationDirectory,
-			commandLine);
+		final DefaultCLI defaultCLI = new DefaultCLI(configuration);
+
+		final String manualHostname = "123.123.123.123";
+		final int manualPort = 4321;
+		final String[] args = {"-m", manualHostname + ':' + manualPort};
+
+		CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
+
+		final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
 
-		final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(
-			configuration,
-			commandLine));
+		final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
 
 		final InetSocketAddress expectedAddress = new InetSocketAddress(manualHostname, manualPort);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java
index 6a538d4..c3299bc 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/Flip6DefaultCLITest.java
@@ -37,7 +37,7 @@ public class Flip6DefaultCLITest extends TestLogger {
 	@Test
 	public void testFlip6Switch() throws CliArgsException {
 		final String[] args = {"-flip6"};
-		final Flip6DefaultCLI flip6DefaultCLI = new Flip6DefaultCLI();
+		final Flip6DefaultCLI flip6DefaultCLI = new Flip6DefaultCLI(new Configuration());
 
 		final Options options = new Options();
 		flip6DefaultCLI.addGeneralOptions(options);
@@ -46,6 +46,6 @@ public class Flip6DefaultCLITest extends TestLogger {
 		final CommandLine commandLine = CliFrontendParser.parse(options, args, false);
 
 		Assert.assertTrue(commandLine.hasOption(Flip6DefaultCLI.FLIP_6.getOpt()));
-		Assert.assertTrue(flip6DefaultCLI.isActive(commandLine, new Configuration()));
+		Assert.assertTrue(flip6DefaultCLI.isActive(commandLine));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
index f9c2bfa..a36e8e9 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
@@ -22,7 +22,6 @@ import org.apache.flink.client.cli.CustomCommandLine;
 import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.Preconditions;
 
 import org.apache.commons.cli.CommandLine;
@@ -43,7 +42,7 @@ public class DummyCustomCommandLine<T extends ClusterClient> implements CustomCo
 	}
 
 	@Override
-	public boolean isActive(CommandLine commandLine, Configuration configuration) {
+	public boolean isActive(CommandLine commandLine) {
 		return true;
 	}
 
@@ -63,21 +62,18 @@ public class DummyCustomCommandLine<T extends ClusterClient> implements CustomCo
 	}
 
 	@Override
-	public ClusterDescriptor<T> createClusterDescriptor(
-			Configuration configuration,
-			String configurationDirectory,
-			CommandLine commandLine) {
+	public ClusterDescriptor<T> createClusterDescriptor(CommandLine commandLine) {
 		return new DummyClusterDescriptor<>(clusterClient);
 	}
 
 	@Override
 	@Nullable
-	public String getClusterId(Configuration configuration, CommandLine commandLine) {
+	public String getClusterId(CommandLine commandLine) {
 		return "dummy";
 	}
 
 	@Override
-	public ClusterSpecification getClusterSpecification(Configuration configuration, CommandLine commandLine) {
+	public ClusterSpecification getClusterSpecification(CommandLine commandLine) {
 		return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
index 135e831..00b460f 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
@@ -19,7 +19,6 @@
 package org.apache.flink.client.cli.util;
 
 import org.apache.flink.client.cli.CliFrontend;
-import org.apache.flink.client.cli.CliFrontendTestUtils;
 import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
 
@@ -35,7 +34,6 @@ public class MockedCliFrontend extends CliFrontend {
 	public MockedCliFrontend(ClusterClient clusterClient) throws Exception {
 		super(
 			new Configuration(),
-			Collections.singletonList(new DummyCustomCommandLine<>(clusterClient)),
-			CliFrontendTestUtils.getConfigDir());
+			Collections.singletonList(new DummyCustomCommandLine<>(clusterClient)));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
index ec8091a..7b34d4a 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java
@@ -188,6 +188,8 @@ public class ClusterClientTest extends TestLogger {
 			fail("Dispose operation should have failed.");
 		} catch (ExecutionException e) {
 			assertTrue(ExceptionUtils.findThrowable(e, FlinkRuntimeException.class).isPresent());
+		} finally {
+			clusterClient.shutdown();
 		}
 	}
 
@@ -214,6 +216,8 @@ public class ClusterClientTest extends TestLogger {
 				"instance, which cannot be disposed without the user code class " +
 				"loader. Please provide the program jar with which you have created " +
 				"the savepoint via -j <JAR> for disposal.").isPresent());
+		} finally {
+			clusterClient.shutdown();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
----------------------------------------------------------------------
diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
index d4d0d05..9f29ce0 100644
--- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
+++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala
@@ -256,22 +256,15 @@ object FlinkShell {
     val options = CliFrontendParser.parseRunCommand(args.toArray)
     val frontend = new CliFrontend(
       configuration,
-      CliFrontend.loadCustomCommandLines(),
-      configurationDirectory)
+      CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
     val config = frontend.getConfiguration
     val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
 
-    val clusterDescriptor = customCLI.createClusterDescriptor(
-      config,
-      frontend.getConfigurationDirectory,
-      options.getCommandLine)
+    val clusterDescriptor = customCLI.createClusterDescriptor(options.getCommandLine)
 
-    val clusterSpecification = customCLI.getClusterSpecification(
-      config,
-      options.getCommandLine)
+    val clusterSpecification = customCLI.getClusterSpecification(options.getCommandLine)
 
-    val cluster = clusterDescriptor.deploySessionCluster(
-      clusterSpecification)
+    val cluster = clusterDescriptor.deploySessionCluster(clusterSpecification)
 
     val address = cluster.getJobManagerAddress.getAddress.getHostAddress
     val port = cluster.getJobManagerAddress.getPort
@@ -291,19 +284,13 @@ object FlinkShell {
     val options = CliFrontendParser.parseRunCommand(args.toArray)
     val frontend = new CliFrontend(
       configuration,
-      CliFrontend.loadCustomCommandLines(),
-      configurationDirectory)
+      CliFrontend.loadCustomCommandLines(configuration, configurationDirectory))
     val config = frontend.getConfiguration
     val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
 
-    val clusterDescriptor = customCLI.createClusterDescriptor(
-      configuration,
-      configurationDirectory,
-      options.getCommandLine)
+    val clusterDescriptor = customCLI.createClusterDescriptor(options.getCommandLine)
 
-    val clusterId = customCLI.getClusterId(
-      configuration,
-      options.getCommandLine)
+    val clusterId = customCLI.getClusterId(options.getCommandLine)
 
     val cluster = clusterDescriptor.retrieve(clusterId)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index 8716f8a..f347f94 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -577,10 +577,10 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 
 			// load the configuration
 			LOG.info("testDetachedPerJobYarnClusterInternal: Trying to load configuration file");
-			GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
+			Configuration configuration = GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
 
 			try {
-				File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(GlobalConfiguration.loadConfiguration());
+				File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(configuration.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
 				if (yarnPropertiesFile.exists()) {
 					LOG.info("testDetachedPerJobYarnClusterInternal: Cleaning up temporary Yarn address reference: {}", yarnPropertiesFile.getAbsolutePath());
 					yarnPropertiesFile.delete();

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index ec8ef50..d5a9883 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -25,6 +25,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -134,10 +135,10 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 
 			// load the configuration
 			LOG.info("testDetachedPerJobYarnClusterInternal: Trying to load configuration file");
-			GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
+			Configuration configuration = GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
 
 			try {
-				File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(GlobalConfiguration.loadConfiguration());
+				File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(configuration.getString(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
 				if (yarnPropertiesFile.exists()) {
 					LOG.info("testDetachedPerJobYarnClusterInternal: Cleaning up temporary Yarn address reference: {}", yarnPropertiesFile.getAbsolutePath());
 					yarnPropertiesFile.delete();

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
index 5325ae2..ed02892 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
@@ -693,6 +693,8 @@ public abstract class YarnTestBase extends TestLogger {
 				switch (type) {
 					case YARN_SESSION:
 						yCli = new FlinkYarnSessionCli(
+							configuration,
+							configurationDirectory,
 							"",
 							"",
 							false);
@@ -702,8 +704,7 @@ public abstract class YarnTestBase extends TestLogger {
 						try {
 							CliFrontend cli = new CliFrontend(
 								configuration,
-								CliFrontend.loadCustomCommandLines(),
-								configurationDirectory);
+								CliFrontend.loadCustomCommandLines(configuration, configurationDirectory));
 							returnValue = cli.parseParameters(args);
 						} catch (Exception e) {
 							throw new RuntimeException("Failed to execute the following args with CliFrontend: "

http://git-wip-us.apache.org/repos/asf/flink/blob/30011b9b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
index 63421f9..5fb7f90 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
@@ -34,6 +34,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.util.FlinkException;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
@@ -330,7 +331,7 @@ public class YarnClusterClient extends ClusterClient {
 		}
 
 		try {
-			File propertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(flinkConfig);
+			File propertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(flinkConfig.getValue(YarnConfigOptions.PROPERTIES_FILE_LOCATION));
 			if (propertiesFile.isFile()) {
 				if (propertiesFile.delete()) {
 					LOG.info("Deleted Yarn properties file at {}", propertiesFile.getAbsoluteFile().toString());


[12/14] flink git commit: [FLINK-8201] [yarn] Delete temp configuration file after uploading it to HDFS

Posted by tr...@apache.org.
[FLINK-8201] [yarn] Delete temp configuration file after uploading it to HDFS

The Utils#createTaskExecutorContext method creates a temporary local configuration
file which it then uploads to HDFS. This fille should be removed after the upload
has completed.

This closes #5123.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d0bc3000
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d0bc3000
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d0bc3000

Branch: refs/heads/master
Commit: d0bc300087b037b156425ccd509147177dd9529e
Parents: ed0716a
Author: wenlong.lwl <we...@alibaba-inc.com>
Authored: Tue Dec 5 15:54:39 2017 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:06 2018 +0100

----------------------------------------------------------------------
 .../main/java/org/apache/flink/yarn/Utils.java  | 32 +++++++++++++-------
 1 file changed, 21 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d0bc3000/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
index 652afec..9ae5b54 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
@@ -23,6 +23,7 @@ import org.apache.flink.configuration.ResourceManagerOptions;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
 import org.apache.flink.runtime.util.HadoopUtils;
+import org.apache.flink.util.FileUtils;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -408,17 +409,26 @@ public final class Utils {
 			log.debug("Writing TaskManager configuration to {}", taskManagerConfigFile.getAbsolutePath());
 			BootstrapTools.writeConfiguration(taskManagerConfig, taskManagerConfigFile);
 
-			Path homeDirPath = new Path(clientHomeDir);
-			FileSystem fs = homeDirPath.getFileSystem(yarnConfig);
-
-			flinkConf = setupLocalResource(
-				fs,
-				appId,
-				new Path(taskManagerConfigFile.toURI()),
-				homeDirPath,
-				"").f1;
-
-			log.info("Prepared local resource for modified yaml: {}", flinkConf);
+			try {
+				Path homeDirPath = new Path(clientHomeDir);
+				FileSystem fs = homeDirPath.getFileSystem(yarnConfig);
+
+				flinkConf = setupLocalResource(
+					fs,
+					appId,
+					new Path(taskManagerConfigFile.toURI()),
+					homeDirPath,
+					"").f1;
+
+				log.info("Prepared local resource for modified yaml: {}", flinkConf);
+			} finally {
+				try {
+					FileUtils.deleteFileOrDirectory(taskManagerConfigFile);
+				} catch (IOException e) {
+					log.info("Could not delete temporary configuration file " +
+						taskManagerConfigFile.getAbsolutePath() + '.', e);
+				}
+			}
 		}
 
 		Map<String, LocalResource> taskManagerLocalResources = new HashMap<>();


[05/14] flink git commit: [FLINK-8342] [flip6] Remove generic type parameter from ClusterDescriptor

Posted by tr...@apache.org.
[FLINK-8342] [flip6] Remove generic type parameter from ClusterDescriptor

This closes #5228.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/10e900b2
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/10e900b2
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/10e900b2

Branch: refs/heads/master
Commit: 10e900b25ac03876d3f9e78f260d48efe6b9d853
Parents: d7e9dc1
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Jan 2 10:19:34 2018 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:03 2018 +0100

----------------------------------------------------------------------
 .../client/cli/AbstractCustomCommandLine.java   |  4 +--
 .../apache/flink/client/cli/CliFrontend.java    | 34 ++++++++++----------
 .../flink/client/cli/CliFrontendParser.java     | 14 ++++----
 .../flink/client/cli/CustomCommandLine.java     |  5 ++-
 .../org/apache/flink/client/cli/DefaultCLI.java |  5 ++-
 .../flink/client/cli/Flip6DefaultCLI.java       |  5 ++-
 .../client/deployment/ClusterDescriptor.java    |  8 ++---
 .../Flip6StandaloneClusterDescriptor.java       |  2 +-
 .../deployment/StandaloneClusterDescriptor.java |  2 +-
 .../apache/flink/client/cli/DefaultCLITest.java |  4 +--
 .../client/cli/util/DummyClusterDescriptor.java | 14 ++++----
 .../client/cli/util/DummyCustomCommandLine.java | 12 +++----
 .../client/cli/util/MockedCliFrontend.java      |  2 +-
 .../YARNSessionCapacitySchedulerITCase.java     |  2 +-
 .../flink/yarn/YARNSessionFIFOITCase.java       |  4 +--
 .../yarn/AbstractYarnClusterDescriptor.java     |  4 +--
 .../flink/yarn/cli/FlinkYarnSessionCli.java     |  2 +-
 17 files changed, 57 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
index c7a1672..da21556 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java
@@ -19,7 +19,6 @@
 package org.apache.flink.client.cli;
 
 import org.apache.flink.client.ClientUtils;
-import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
 import org.apache.flink.configuration.UnmodifiableConfiguration;
@@ -38,9 +37,8 @@ import static org.apache.flink.client.cli.CliFrontend.setJobManagerAddressInConf
  * Base class for {@link CustomCommandLine} implementations which specify a JobManager address and
  * a ZooKeeper namespace.
  *
- * @param <C> type of the ClusterClient which is returned
  */
-public abstract class AbstractCustomCommandLine<C extends ClusterClient> implements CustomCommandLine<C> {
+public abstract class AbstractCustomCommandLine implements CustomCommandLine {
 
 	protected final Option zookeeperNamespaceOption = new Option("z", "zookeeperNamespace", true,
 		"Namespace to create the Zookeeper sub-paths for high availability mode");

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index 630154c..453d086 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -106,7 +106,7 @@ public class CliFrontend {
 
 	private final Configuration configuration;
 
-	private final List<CustomCommandLine<?>> customCommandLines;
+	private final List<CustomCommandLine> customCommandLines;
 
 	private final Options customCommandLineOptions;
 
@@ -116,7 +116,7 @@ public class CliFrontend {
 
 	public CliFrontend(
 			Configuration configuration,
-			List<CustomCommandLine<?>> customCommandLines) throws Exception {
+			List<CustomCommandLine> customCommandLines) throws Exception {
 		this.configuration = Preconditions.checkNotNull(configuration);
 		this.customCommandLines = Preconditions.checkNotNull(customCommandLines);
 
@@ -129,7 +129,7 @@ public class CliFrontend {
 
 		this.customCommandLineOptions = new Options();
 
-		for (CustomCommandLine<?> customCommandLine : customCommandLines) {
+		for (CustomCommandLine customCommandLine : customCommandLines) {
 			customCommandLine.addGeneralOptions(customCommandLineOptions);
 			customCommandLine.addRunOptions(customCommandLineOptions);
 		}
@@ -196,9 +196,9 @@ public class CliFrontend {
 			throw new CliArgsException("Could not build the program from JAR file.", e);
 		}
 
-		final CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
+		final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
+		final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
 
 		try {
 			final String clusterId = customCommandLine.getClusterId(commandLine);
@@ -351,8 +351,8 @@ public class CliFrontend {
 			scheduled = true;
 		}
 
-		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
-		final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+		final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
+		final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
 
 		final String clusterId = activeCommandLine.getClusterId(commandLine);
 
@@ -473,9 +473,9 @@ public class CliFrontend {
 			throw new CliArgsException("Missing JobID");
 		}
 
-		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
+		final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+		final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
 
 		final String clusterId = activeCommandLine.getClusterId(commandLine);
 
@@ -553,9 +553,9 @@ public class CliFrontend {
 			throw new CliArgsException("Missing JobID in the command line arguments.");
 		}
 
-		final CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(commandLine);
+		final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor<?> clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
+		final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine);
 
 		final String clusterId = activeCommandLine.getClusterId(commandLine);
 
@@ -617,9 +617,9 @@ public class CliFrontend {
 			return;
 		}
 
-		CustomCommandLine<?> customCommandLine = getActiveCustomCommandLine(commandLine);
+		CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine);
 
-		final ClusterDescriptor<?> clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
+		final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine);
 
 		final String clusterId = customCommandLine.getClusterId(commandLine);
 
@@ -972,7 +972,7 @@ public class CliFrontend {
 		final Configuration configuration = GlobalConfiguration.loadConfiguration(configurationDirectory);
 
 		// 3. load the custom command lines
-		final List<CustomCommandLine<?>> customCommandLines = loadCustomCommandLines(
+		final List<CustomCommandLine> customCommandLines = loadCustomCommandLines(
 			configuration,
 			configurationDirectory);
 
@@ -1039,8 +1039,8 @@ public class CliFrontend {
 		config.setInteger(JobManagerOptions.PORT, address.getPort());
 	}
 
-	public static List<CustomCommandLine<?>> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
-		List<CustomCommandLine<?>> customCommandLines = new ArrayList<>(2);
+	public static List<CustomCommandLine> loadCustomCommandLines(Configuration configuration, String configurationDirectory) {
+		List<CustomCommandLine> customCommandLines = new ArrayList<>(2);
 
 		//	Command line interface of the YARN session, with a special initialization here
 		//	to prefix all options with y/yarn.
@@ -1087,7 +1087,7 @@ public class CliFrontend {
 	 * @param className The fully-qualified class name to load.
 	 * @param params The constructor parameters
 	 */
-	private static CustomCommandLine<?> loadCustomCommandLine(String className, Object... params) throws IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, NoSuchMethodException {
+	private static CustomCommandLine loadCustomCommandLine(String className, Object... params) throws IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, NoSuchMethodException {
 
 		Class<? extends CustomCommandLine> customCliClass =
 			Class.forName(className).asSubclass(CustomCommandLine.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
index 475d854..10507d6 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
@@ -236,7 +236,7 @@ public class CliFrontendParser {
 	/**
 	 * Prints the help for the client.
 	 */
-	public static void printHelp(Collection<CustomCommandLine<?>> customCommandLines) {
+	public static void printHelp(Collection<CustomCommandLine> customCommandLines) {
 		System.out.println("./flink <ACTION> [OPTIONS] [ARGUMENTS]");
 		System.out.println();
 		System.out.println("The following actions are available:");
@@ -251,7 +251,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForRun(Collection<CustomCommandLine<?>> customCommandLines) {
+	public static void printHelpForRun(Collection<CustomCommandLine> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -279,7 +279,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForList(Collection<CustomCommandLine<?>> customCommandLines) {
+	public static void printHelpForList(Collection<CustomCommandLine> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -294,7 +294,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForStop(Collection<CustomCommandLine<?>> customCommandLines) {
+	public static void printHelpForStop(Collection<CustomCommandLine> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -309,7 +309,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForCancel(Collection<CustomCommandLine<?>> customCommandLines) {
+	public static void printHelpForCancel(Collection<CustomCommandLine> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -324,7 +324,7 @@ public class CliFrontendParser {
 		System.out.println();
 	}
 
-	public static void printHelpForSavepoint(Collection<CustomCommandLine<?>> customCommandLines) {
+	public static void printHelpForSavepoint(Collection<CustomCommandLine> customCommandLines) {
 		HelpFormatter formatter = new HelpFormatter();
 		formatter.setLeftPadding(5);
 		formatter.setWidth(80);
@@ -345,7 +345,7 @@ public class CliFrontendParser {
 	 * @param runOptions True if the run options should be printed, False to print only general options
 	 */
 	private static void printCustomCliOptions(
-			Collection<CustomCommandLine<?>> customCommandLines,
+			Collection<CustomCommandLine> customCommandLines,
 			HelpFormatter formatter,
 			boolean runOptions) {
 		// prints options from all available command-line classes

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
index f642484..aabc224 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
@@ -20,7 +20,6 @@ package org.apache.flink.client.cli;
 
 import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
-import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.util.FlinkException;
 
 import org.apache.commons.cli.CommandLine;
@@ -31,7 +30,7 @@ import javax.annotation.Nullable;
 /**
  * Custom command-line interface to load hooks for the command-line interface.
  */
-public interface CustomCommandLine<ClusterType extends ClusterClient> {
+public interface CustomCommandLine {
 
 	/**
 	 * Signals whether the custom command-line wants to execute or not.
@@ -67,7 +66,7 @@ public interface CustomCommandLine<ClusterType extends ClusterClient> {
 	 * @return ClusterDescriptor
 	 * @throws FlinkException if the ClusterDescriptor could not be created
 	 */
-	ClusterDescriptor<ClusterType> createClusterDescriptor(CommandLine commandLine) throws FlinkException;
+	ClusterDescriptor createClusterDescriptor(CommandLine commandLine) throws FlinkException;
 
 	/**
 	 * Returns the cluster id if a cluster id was specified on the command line, otherwise it

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index c29c5b7..5660765 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -21,7 +21,6 @@ package org.apache.flink.client.cli;
 import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
-import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.FlinkException;
 
@@ -32,7 +31,7 @@ import javax.annotation.Nullable;
 /**
  * The default CLI which is used for interaction with standalone clusters.
  */
-public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterClient> {
+public class DefaultCLI extends AbstractCustomCommandLine {
 
 	public DefaultCLI(Configuration configuration) {
 		super(configuration);
@@ -50,7 +49,7 @@ public class DefaultCLI extends AbstractCustomCommandLine<StandaloneClusterClien
 	}
 
 	@Override
-	public ClusterDescriptor<StandaloneClusterClient> createClusterDescriptor(
+	public ClusterDescriptor createClusterDescriptor(
 			CommandLine commandLine) throws FlinkException {
 		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
index 3adeca6..1a75aac 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java
@@ -21,7 +21,6 @@ package org.apache.flink.client.cli;
 import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.client.deployment.ClusterSpecification;
 import org.apache.flink.client.deployment.Flip6StandaloneClusterDescriptor;
-import org.apache.flink.client.program.rest.RestClusterClient;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.FlinkException;
 
@@ -34,7 +33,7 @@ import javax.annotation.Nullable;
 /**
  * The default CLI which is used for interaction with standalone clusters.
  */
-public class Flip6DefaultCLI extends AbstractCustomCommandLine<RestClusterClient> {
+public class Flip6DefaultCLI extends AbstractCustomCommandLine {
 
 	public static final Option FLIP_6 = new Option("flip6", "Switches the client to Flip-6 mode.");
 
@@ -63,7 +62,7 @@ public class Flip6DefaultCLI extends AbstractCustomCommandLine<RestClusterClient
 	}
 
 	@Override
-	public ClusterDescriptor<RestClusterClient> createClusterDescriptor(
+	public ClusterDescriptor createClusterDescriptor(
 			CommandLine commandLine) throws FlinkException {
 		final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
index 1603930..07eeabc 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java
@@ -24,7 +24,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 /**
  * A descriptor to deploy a cluster (e.g. Yarn or Mesos) and return a Client for Cluster communication.
  */
-public interface ClusterDescriptor<ClientType extends ClusterClient> extends AutoCloseable {
+public interface ClusterDescriptor extends AutoCloseable {
 
 	/**
 	 * Returns a String containing details about the cluster (NodeManagers, available memory, ...).
@@ -38,7 +38,7 @@ public interface ClusterDescriptor<ClientType extends ClusterClient> extends Aut
 	 * @return Client for the cluster
 	 * @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
 	 */
-	ClientType retrieve(String applicationID) throws UnsupportedOperationException;
+	ClusterClient retrieve(String applicationID) throws UnsupportedOperationException;
 
 	/**
 	 * Triggers deployment of a cluster.
@@ -46,7 +46,7 @@ public interface ClusterDescriptor<ClientType extends ClusterClient> extends Aut
 	 * @return Client for the cluster
 	 * @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
 	 */
-	ClientType deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException;
+	ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException;
 
 	/**
 	 * Deploys a per-job cluster with the given job on the cluster.
@@ -56,7 +56,7 @@ public interface ClusterDescriptor<ClientType extends ClusterClient> extends Aut
 	 * @return Cluster client to talk to the Flink cluster
 	 * @throws ClusterDeploymentException if the cluster could not be deployed
 	 */
-	ClientType deployJobCluster(
+	ClusterClient deployJobCluster(
 		final ClusterSpecification clusterSpecification,
 		final JobGraph jobGraph);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
index b8eb534..a35a68b 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java
@@ -27,7 +27,7 @@ import org.apache.flink.util.Preconditions;
 /**
  * A deployment descriptor for an existing cluster.
  */
-public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor<RestClusterClient> {
+public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor {
 
 	private final Configuration config;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
index 3808efa..ae25194 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
@@ -26,7 +26,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 /**
  * A deployment descriptor for an existing cluster.
  */
-public class StandaloneClusterDescriptor implements ClusterDescriptor<StandaloneClusterClient> {
+public class StandaloneClusterDescriptor implements ClusterDescriptor {
 
 	private final Configuration config;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
index e73b9c9..6eb005d 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java
@@ -62,7 +62,7 @@ public class DefaultCLITest extends TestLogger {
 
 		final InetSocketAddress expectedAddress = new InetSocketAddress(localhost, port);
 
-		final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
+		final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
 
 		final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
 
@@ -89,7 +89,7 @@ public class DefaultCLITest extends TestLogger {
 
 		CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false);
 
-		final ClusterDescriptor<?> clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
+		final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine);
 
 		final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
index e46957b..d300055 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java
@@ -26,14 +26,12 @@ import org.apache.flink.util.Preconditions;
 
 /**
  * Dummy {@link ClusterDescriptor} implementation for testing purposes.
- *
- * @param <C> type of the returned {@link ClusterClient}
  */
-public class DummyClusterDescriptor<C extends ClusterClient> implements ClusterDescriptor<C> {
+public class DummyClusterDescriptor implements ClusterDescriptor {
 
-	private final C clusterClient;
+	private final ClusterClient clusterClient;
 
-	public DummyClusterDescriptor(C clusterClient) {
+	public DummyClusterDescriptor(ClusterClient clusterClient) {
 		this.clusterClient = Preconditions.checkNotNull(clusterClient);
 	}
 
@@ -43,17 +41,17 @@ public class DummyClusterDescriptor<C extends ClusterClient> implements ClusterD
 	}
 
 	@Override
-	public C retrieve(String applicationID) throws UnsupportedOperationException {
+	public ClusterClient retrieve(String applicationID) throws UnsupportedOperationException {
 		return clusterClient;
 	}
 
 	@Override
-	public C deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
+	public ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws UnsupportedOperationException {
 		return clusterClient;
 	}
 
 	@Override
-	public C deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
+	public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) {
 		return clusterClient;
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
index a36e8e9..5279d85 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java
@@ -31,13 +31,11 @@ import javax.annotation.Nullable;
 
 /**
  * Dummy implementation of the {@link CustomCommandLine} for testing purposes.
- *
- * @param <T> type of the returned cluster client
  */
-public class DummyCustomCommandLine<T extends ClusterClient> implements CustomCommandLine<T> {
-	private final T clusterClient;
+public class DummyCustomCommandLine implements CustomCommandLine {
+	private final ClusterClient clusterClient;
 
-	public DummyCustomCommandLine(T clusterClient) {
+	public DummyCustomCommandLine(ClusterClient clusterClient) {
 		this.clusterClient = Preconditions.checkNotNull(clusterClient);
 	}
 
@@ -62,8 +60,8 @@ public class DummyCustomCommandLine<T extends ClusterClient> implements CustomCo
 	}
 
 	@Override
-	public ClusterDescriptor<T> createClusterDescriptor(CommandLine commandLine) {
-		return new DummyClusterDescriptor<>(clusterClient);
+	public ClusterDescriptor createClusterDescriptor(CommandLine commandLine) {
+		return new DummyClusterDescriptor(clusterClient);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
index 00b460f..477293d 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java
@@ -34,6 +34,6 @@ public class MockedCliFrontend extends CliFrontend {
 	public MockedCliFrontend(ClusterClient clusterClient) throws Exception {
 		super(
 			new Configuration(),
-			Collections.singletonList(new DummyCustomCommandLine<>(clusterClient)));
+			Collections.singletonList(new DummyCustomCommandLine(clusterClient)));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index f347f94..5bed22e 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -365,7 +365,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	@Test
 	public void testNonexistingQueueWARNmessage() {
 		LOG.info("Starting testNonexistingQueueWARNmessage()");
-		addTestAppender(YarnClusterDescriptor.class, Level.WARN);
+		addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
 				"-t", flinkLibFolder.getAbsolutePath(),
 				"-n", "1",

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index d5a9883..cc26350 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -179,7 +179,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	@Ignore("The test is too resource consuming (8.5 GB of memory)")
 	@Test
 	public void testResourceComputation() {
-		addTestAppender(YarnClusterDescriptor.class, Level.WARN);
+		addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
 		LOG.info("Starting testResourceComputation()");
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
 				"-n", "5",
@@ -207,7 +207,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	@Ignore("The test is too resource consuming (8 GB of memory)")
 	@Test
 	public void testfullAlloc() {
-		addTestAppender(YarnClusterDescriptor.class, Level.WARN);
+		addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN);
 		LOG.info("Starting testfullAlloc()");
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
 				"-n", "2",

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index 0372319..0a977df 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -99,8 +99,8 @@ import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties
 /**
  * The descriptor with deployment information for spawning or resuming a {@link YarnClusterClient}.
  */
-public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor<YarnClusterClient> {
-	private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class);
+public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor {
+	private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnClusterDescriptor.class);
 
 	/**
 	 * Minimum memory requirements, checked by the Client.

http://git-wip-us.apache.org/repos/asf/flink/blob/10e900b2/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index d4ab41f..d797f47 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -81,7 +81,7 @@ import static org.apache.flink.configuration.HighAvailabilityOptions.HA_CLUSTER_
 /**
  * Class handling the command line interface to the YARN session.
  */
-public class FlinkYarnSessionCli extends AbstractCustomCommandLine<YarnClusterClient> {
+public class FlinkYarnSessionCli extends AbstractCustomCommandLine {
 	private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnSessionCli.class);
 
 	//------------------------------------ Constants   -------------------------


[11/14] flink git commit: [hotfix] Log failure message only if Yarn application truly failed

Posted by tr...@apache.org.
[hotfix] Log failure message only if Yarn application truly failed


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ed0716ac
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ed0716ac
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ed0716ac

Branch: refs/heads/master
Commit: ed0716ac710387979735140d18a598240b43085d
Parents: dbe0e82
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Dec 29 17:08:45 2017 +0100
Committer: Till Rohrmann <tr...@apache.org>
Committed: Fri Jan 12 16:14:06 2018 +0100

----------------------------------------------------------------------
 .../main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java  | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ed0716ac/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index d4cf9ad..cc7f4c1 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -686,8 +686,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine<ApplicationId
 			.getYarnApplicationState() + " and final state " + appReport
 			.getFinalApplicationStatus() + " at " + appReport.getFinishTime());
 
-		if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED || appReport.getYarnApplicationState()
-			== YarnApplicationState.KILLED) {
+		if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED) {
 			LOG.warn("Application failed. Diagnostics " + appReport.getDiagnostics());
 			LOG.warn("If log aggregation is activated in the Hadoop cluster, we recommend to retrieve "
 				+ "the full application log using this command:"