You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by mx...@apache.org on 2016/06/17 08:45:15 UTC

[01/10] flink git commit: [FLINK-4079] YARN properties file used for per-job cluster

Repository: flink
Updated Branches:
  refs/heads/master efc344a4e -> ec6d97528


[FLINK-4079] YARN properties file used for per-job cluster


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

Branch: refs/heads/master
Commit: ec6d97528e8b21f191b7922e4810fd60804c8365
Parents: f4ac852
Author: Maximilian Michels <mx...@apache.org>
Authored: Thu Jun 16 12:03:04 2016 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Fri Jun 17 10:37:58 2016 +0200

----------------------------------------------------------------------
 .../flink/yarn/cli/FlinkYarnSessionCli.java     | 25 ++++++++++++++++----
 1 file changed, 21 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ec6d9752/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 5eca4f1..c355f0a 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
@@ -105,7 +105,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 
 	/**
 	 * Dynamic properties allow the user to specify additional configuration values with -D, such as
-	 *  -Dfs.overwrite-files=true  -Dtaskmanager.network.numberOfBuffers=16368
+	 *  -D fs.overwrite-files=true  -D taskmanager.network.numberOfBuffers=16368
 	 */
 	private final Option DYNAMIC_PROPERTIES;
 
@@ -155,10 +155,27 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 
 	/**
 	 * Resumes from a Flink Yarn properties file
+	 * @param cmdLine The command-line parameters
 	 * @param flinkConfiguration The flink configuration
 	 * @return True if the properties were loaded, false otherwise
 	 */
-	private boolean resumeFromYarnProperties(Configuration flinkConfiguration) {
+	private boolean resumeFromYarnProperties(CommandLine cmdLine, Configuration flinkConfiguration) {
+
+		String jobManagerOption = cmdLine.getOptionValue(ADDRESS_OPTION.getOpt(), null);
+		if (jobManagerOption != null) {
+			// don't resume from properties file if a JobManager has been specified
+			return false;
+		}
+
+		for (Option option : cmdLine.getOptions()) {
+			if (ALL_OPTIONS.hasOption(option.getOpt())) {
+				if (!option.getOpt().equals(DETACHED.getOpt())) {
+					// don't resume from properties file if yarn options have been specified
+					return false;
+				}
+			}
+		}
+
 		// load the YARN properties
 		File propertiesFile = getYarnPropertiesLocation(flinkConfiguration);
 		if (!propertiesFile.exists()) {
@@ -439,7 +456,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 	public boolean isActive(CommandLine commandLine, Configuration configuration) {
 		String jobManagerOption = commandLine.getOptionValue(ADDRESS_OPTION.getOpt(), null);
 		boolean yarnJobManager = ID.equals(jobManagerOption);
-		return yarnJobManager || resumeFromYarnProperties(configuration);
+		return yarnJobManager || resumeFromYarnProperties(commandLine, configuration);
 	}
 
 	@Override
@@ -471,7 +488,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 			yarnDescriptor.setFlinkConfiguration(config);
 			return yarnDescriptor.retrieve(applicationID);
 		// then try to load from yarn properties
-		} else if (resumeFromYarnProperties(config)) {
+		} else if (resumeFromYarnProperties(cmdLine, config)) {
 			AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor();
 			yarnDescriptor.setFlinkConfiguration(config);
 			return yarnDescriptor.retrieveFromConfig(config);


[04/10] flink git commit: [FLINK-3667] refactor client communication classes

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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
new file mode 100644
index 0000000..7220a29
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -0,0 +1,943 @@
+/*
+ * 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.CliFrontend;
+import org.apache.flink.client.deployment.ClusterDescriptor;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
+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.Records;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME;
+import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME;
+import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties;
+
+/**
+* All classes in this package contain code taken from
+* https://github.com/apache/hadoop-common/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java?source=cc
+* and
+* https://github.com/hortonworks/simple-yarn-app
+* and
+* https://github.com/yahoo/storm-yarn/blob/master/src/main/java/com/yahoo/storm/yarn/StormOnYarn.java
+*
+* The Flink jar is uploaded to HDFS by this client.
+* The application master and all the TaskManager containers get the jar file downloaded
+* by YARN into their local fs.
+*
+*/
+public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor<YarnClusterClient> {
+	private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class);
+
+	private static final String CONFIG_FILE_NAME = "flink-conf.yaml";
+
+	/**
+	 * Minimum memory requirements, checked by the Client.
+	 */
+	private static final int MIN_JM_MEMORY = 768; // the minimum memory should be higher than the min heap cutoff
+	private static final int MIN_TM_MEMORY = 768;
+
+	private Configuration conf = new YarnConfiguration();
+
+	/**
+	 * Files (usually in a distributed file system) used for the YARN session of Flink.
+	 * Contains configuration files and jar files.
+	 */
+	private Path sessionFilesDir;
+
+	/**
+	 * If the user has specified a different number of slots, we store them here
+	 */
+	private int slots = -1;
+
+	private int jobManagerMemoryMb = 1024;
+
+	private int taskManagerMemoryMb = 1024;
+
+	private int taskManagerCount = 1;
+
+	private String yarnQueue = null;
+
+	private String configurationDirectory;
+
+	private Path flinkConfigurationPath;
+
+	private Path flinkLoggingConfigurationPath; // optional
+
+	private Path flinkJarPath;
+
+	private String dynamicPropertiesEncoded;
+
+	private List<File> shipFiles = new ArrayList<>();
+	private org.apache.flink.configuration.Configuration flinkConfiguration;
+
+	private boolean detached;
+
+	private String customName = null;
+
+	public AbstractYarnClusterDescriptor() {
+		// for unit tests only
+		if(System.getenv("IN_TESTS") != null) {
+			try {
+				conf.addResource(new File(System.getenv("YARN_CONF_DIR") + "/yarn-site.xml").toURI().toURL());
+			} catch (Throwable t) {
+				throw new RuntimeException("Error",t);
+			}
+		}
+
+		// load the config
+		this.configurationDirectory = CliFrontend.getConfigurationDirectoryFromEnv();
+		GlobalConfiguration.loadConfiguration(configurationDirectory);
+		this.flinkConfiguration = GlobalConfiguration.getConfiguration();
+
+		File confFile = new File(configurationDirectory + File.separator + CONFIG_FILE_NAME);
+		if (!confFile.exists()) {
+			throw new RuntimeException("Unable to locate configuration file in " + confFile);
+		}
+		flinkConfigurationPath = new Path(confFile.getAbsolutePath());
+
+		//check if there is a logback or log4j file
+		if (configurationDirectory.length() > 0) {
+			File logback = new File(configurationDirectory + File.pathSeparator + CONFIG_FILE_LOGBACK_NAME);
+			if (logback.exists()) {
+				shipFiles.add(logback);
+				flinkLoggingConfigurationPath = new Path(logback.toURI());
+			}
+			File log4j = new File(configurationDirectory + File.pathSeparator + CONFIG_FILE_LOG4J_NAME);
+			if (log4j.exists()) {
+				shipFiles.add(log4j);
+				if (flinkLoggingConfigurationPath != null) {
+					// this means there is already a logback configuration file --> fail
+					LOG.warn("The configuration directory ('" + configurationDirectory + "') contains both LOG4J and " +
+						"Logback configuration files. Please delete or rename one of them.");
+				}
+				flinkLoggingConfigurationPath = new Path(log4j.toURI());
+			}
+		}
+	}
+
+	/**
+	 * The class to bootstrap the application master of the Yarn cluster (runs main method).
+	 */
+	protected abstract Class<?> getApplicationMasterClass();
+
+	public void setJobManagerMemory(int memoryMb) {
+		if(memoryMb < MIN_JM_MEMORY) {
+			throw new IllegalArgumentException("The JobManager memory (" + memoryMb + ") is below the minimum required memory amount "
+				+ "of " + MIN_JM_MEMORY+ " MB");
+		}
+		this.jobManagerMemoryMb = memoryMb;
+	}
+
+	public void setTaskManagerMemory(int memoryMb) {
+		if(memoryMb < MIN_TM_MEMORY) {
+			throw new IllegalArgumentException("The TaskManager memory (" + memoryMb + ") is below the minimum required memory amount "
+				+ "of " + MIN_TM_MEMORY+ " MB");
+		}
+		this.taskManagerMemoryMb = memoryMb;
+	}
+
+	public void setFlinkConfiguration(org.apache.flink.configuration.Configuration conf) {
+		this.flinkConfiguration = conf;
+	}
+
+	public org.apache.flink.configuration.Configuration getFlinkConfiguration() {
+		return flinkConfiguration;
+	}
+
+	public void setTaskManagerSlots(int slots) {
+		if(slots <= 0) {
+			throw new IllegalArgumentException("Number of TaskManager slots must be positive");
+		}
+		this.slots = slots;
+	}
+
+	public int getTaskManagerSlots() {
+		return this.slots;
+	}
+
+	public void setQueue(String queue) {
+		this.yarnQueue = queue;
+	}
+
+	public void setLocalJarPath(Path localJarPath) {
+		if(!localJarPath.toString().endsWith("jar")) {
+			throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension");
+		}
+		this.flinkJarPath = localJarPath;
+	}
+
+	public void setConfigurationFilePath(Path confPath) {
+		flinkConfigurationPath = confPath;
+	}
+
+	public void setConfigurationDirectory(String configurationDirectory) {
+		this.configurationDirectory = configurationDirectory;
+	}
+
+	public void setFlinkLoggingConfigurationPath(Path logConfPath) {
+		flinkLoggingConfigurationPath = logConfPath;
+	}
+
+	public Path getFlinkLoggingConfigurationPath() {
+		return flinkLoggingConfigurationPath;
+	}
+
+	public void setTaskManagerCount(int tmCount) {
+		if(tmCount < 1) {
+			throw new IllegalArgumentException("The TaskManager count has to be at least 1.");
+		}
+		this.taskManagerCount = tmCount;
+	}
+
+	public int getTaskManagerCount() {
+		return this.taskManagerCount;
+	}
+
+	public void setShipFiles(List<File> shipFiles) {
+		for(File shipFile: shipFiles) {
+			// remove uberjar from ship list (by default everything in the lib/ folder is added to
+			// the list of files to ship, but we handle the uberjar separately.
+			if(!(shipFile.getName().startsWith("flink-dist") && shipFile.getName().endsWith("jar"))) {
+				this.shipFiles.add(shipFile);
+			}
+		}
+	}
+
+	public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) {
+		this.dynamicPropertiesEncoded = dynamicPropertiesEncoded;
+	}
+
+	public String getDynamicPropertiesEncoded() {
+		return this.dynamicPropertiesEncoded;
+	}
+
+
+	private void isReadyForDeployment() throws YarnDeploymentException {
+		if(taskManagerCount <= 0) {
+			throw new YarnDeploymentException("Taskmanager count must be positive");
+		}
+		if(this.flinkJarPath == null) {
+			throw new YarnDeploymentException("The Flink jar path is null");
+		}
+		if(this.configurationDirectory == null) {
+			throw new YarnDeploymentException("Configuration directory not set");
+		}
+		if(this.flinkConfigurationPath == null) {
+			throw new YarnDeploymentException("Configuration path not set");
+		}
+		if(this.flinkConfiguration == null) {
+			throw new YarnDeploymentException("Flink configuration object has not been set");
+		}
+
+		// check if required Hadoop environment variables are set. If not, warn user
+		if(System.getenv("HADOOP_CONF_DIR") == null &&
+			System.getenv("YARN_CONF_DIR") == null) {
+			LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." +
+				"The Flink YARN Client needs one of these to be set to properly load the Hadoop " +
+				"configuration for accessing YARN.");
+		}
+	}
+
+	private static boolean allocateResource(int[] nodeManagers, int toAllocate) {
+		for(int i = 0; i < nodeManagers.length; i++) {
+			if(nodeManagers[i] >= toAllocate) {
+				nodeManagers[i] -= toAllocate;
+				return true;
+			}
+		}
+		return false;
+	}
+
+	public void setDetachedMode(boolean detachedMode) {
+		this.detached = detachedMode;
+	}
+
+	public boolean isDetachedMode() {
+		return detached;
+	}
+
+
+	/**
+	 * Gets a Hadoop Yarn client
+	 * @return Returns a YarnClient which has to be shutdown manually
+	 */
+	public static YarnClient getYarnClient(Configuration conf) {
+		YarnClient yarnClient = YarnClient.createYarnClient();
+		yarnClient.init(conf);
+		yarnClient.start();
+		return yarnClient;
+	}
+
+	@Override
+	public YarnClusterClient deploy() throws Exception {
+
+		UserGroupInformation.setConfiguration(conf);
+		UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
+		if (UserGroupInformation.isSecurityEnabled()) {
+			if (!ugi.hasKerberosCredentials()) {
+				throw new YarnDeploymentException("In secure mode. Please provide Kerberos credentials in order to authenticate. " +
+					"You may use kinit to authenticate and request a TGT from the Kerberos server.");
+			}
+			return ugi.doAs(new PrivilegedExceptionAction<YarnClusterClient>() {
+				@Override
+				public YarnClusterClient run() throws Exception {
+					return deployInternal();
+				}
+			});
+		} else {
+			return deployInternal();
+		}
+	}
+
+	/**
+	 * This method will block until the ApplicationMaster/JobManager have been
+	 * deployed on YARN.
+	 */
+	protected YarnClusterClient deployInternal() throws Exception {
+		isReadyForDeployment();
+
+		LOG.info("Using values:");
+		LOG.info("\tTaskManager count = {}", taskManagerCount);
+		LOG.info("\tJobManager memory = {}", jobManagerMemoryMb);
+		LOG.info("\tTaskManager memory = {}", taskManagerMemoryMb);
+
+		// Create application via yarnClient
+		final YarnClient yarnClient = getYarnClient(conf);
+		final YarnClientApplication yarnApplication = yarnClient.createApplication();
+		GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse();
+
+		// ------------------ Add dynamic properties to local flinkConfiguraton ------
+
+		Map<String, String> dynProperties = getDynamicProperties(dynamicPropertiesEncoded);
+		for (Map.Entry<String, String> dynProperty : dynProperties.entrySet()) {
+			flinkConfiguration.setString(dynProperty.getKey(), dynProperty.getValue());
+		}
+
+		// ------------------ Set default file system scheme -------------------------
+
+		try {
+			org.apache.flink.core.fs.FileSystem.setDefaultScheme(flinkConfiguration);
+		} catch (IOException e) {
+			throw new IOException("Error while setting the default " +
+				"filesystem scheme from configuration.", e);
+		}
+		// ------------------ Check if the specified queue exists --------------------
+
+		try {
+			List<QueueInfo> queues = yarnClient.getAllQueues();
+			if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session.
+				boolean queueFound = false;
+				for (QueueInfo queue : queues) {
+					if (queue.getQueueName().equals(this.yarnQueue)) {
+						queueFound = true;
+						break;
+					}
+				}
+				if (!queueFound) {
+					String queueNames = "";
+					for (QueueInfo queue : queues) {
+						queueNames += queue.getQueueName() + ", ";
+					}
+					LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " +
+						"Available queues: " + queueNames);
+				}
+			} else {
+				LOG.debug("The YARN cluster does not have any queues configured");
+			}
+		} catch(Throwable e) {
+			LOG.warn("Error while getting queue information from YARN: " + e.getMessage());
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("Error details", e);
+			}
+		}
+
+		// ------------------ Check if the YARN ClusterClient has the requested resources --------------
+
+		// the yarnMinAllocationMB specifies the smallest possible container allocation size.
+		// all allocations below this value are automatically set to this value.
+		final int yarnMinAllocationMB = conf.getInt("yarn.scheduler.minimum-allocation-mb", 0);
+		if(jobManagerMemoryMb < yarnMinAllocationMB || taskManagerMemoryMb < yarnMinAllocationMB) {
+			LOG.warn("The JobManager or TaskManager memory is below the smallest possible YARN Container size. "
+				+ "The value of 'yarn.scheduler.minimum-allocation-mb' is '" + yarnMinAllocationMB + "'. Please increase the memory size." +
+				"YARN will allocate the smaller containers but the scheduler will account for the minimum-allocation-mb, maybe not all instances " +
+				"you requested will start.");
+		}
+
+		// set the memory to minAllocationMB to do the next checks correctly
+		if(jobManagerMemoryMb < yarnMinAllocationMB) {
+			jobManagerMemoryMb =  yarnMinAllocationMB;
+		}
+		if(taskManagerMemoryMb < yarnMinAllocationMB) {
+			taskManagerMemoryMb =  yarnMinAllocationMB;
+		}
+
+		Resource maxRes = appResponse.getMaximumResourceCapability();
+		final String NOTE = "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n";
+		if(jobManagerMemoryMb > maxRes.getMemory() ) {
+			failSessionDuringDeployment(yarnClient, yarnApplication);
+			throw new YarnDeploymentException("The cluster does not have the requested resources for the JobManager available!\n"
+				+ "Maximum Memory: " + maxRes.getMemory() + "MB Requested: " + jobManagerMemoryMb + "MB. " + NOTE);
+		}
+
+		if(taskManagerMemoryMb > maxRes.getMemory() ) {
+			failSessionDuringDeployment(yarnClient, yarnApplication);
+			throw new YarnDeploymentException("The cluster does not have the requested resources for the TaskManagers available!\n"
+				+ "Maximum Memory: " + maxRes.getMemory() + " Requested: " + taskManagerMemoryMb + "MB. " + NOTE);
+		}
+
+		final String NOTE_RSC = "\nThe Flink YARN client will try to allocate the YARN session, but maybe not all TaskManagers are " +
+			"connecting from the beginning because the resources are currently not available in the cluster. " +
+			"The allocation might take more time than usual because the Flink YARN client needs to wait until " +
+			"the resources become available.";
+		int totalMemoryRequired = jobManagerMemoryMb + taskManagerMemoryMb * taskManagerCount;
+		ClusterResourceDescription freeClusterMem = getCurrentFreeClusterResources(yarnClient);
+		if(freeClusterMem.totalFreeMemory < totalMemoryRequired) {
+			LOG.warn("This YARN session requires " + totalMemoryRequired + "MB of memory in the cluster. "
+				+ "There are currently only " + freeClusterMem.totalFreeMemory + "MB available." + NOTE_RSC);
+
+		}
+		if(taskManagerMemoryMb > freeClusterMem.containerLimit) {
+			LOG.warn("The requested amount of memory for the TaskManagers (" + taskManagerMemoryMb + "MB) is more than "
+				+ "the largest possible YARN container: " + freeClusterMem.containerLimit + NOTE_RSC);
+		}
+		if(jobManagerMemoryMb > freeClusterMem.containerLimit) {
+			LOG.warn("The requested amount of memory for the JobManager (" + jobManagerMemoryMb + "MB) is more than "
+				+ "the largest possible YARN container: " + freeClusterMem.containerLimit + NOTE_RSC);
+		}
+
+		// ----------------- check if the requested containers fit into the cluster.
+
+		int[] nmFree = Arrays.copyOf(freeClusterMem.nodeManagersFree, freeClusterMem.nodeManagersFree.length);
+		// first, allocate the jobManager somewhere.
+		if(!allocateResource(nmFree, jobManagerMemoryMb)) {
+			LOG.warn("Unable to find a NodeManager that can fit the JobManager/Application master. " +
+				"The JobManager requires " + jobManagerMemoryMb + "MB. NodeManagers available: " +
+				Arrays.toString(freeClusterMem.nodeManagersFree) + NOTE_RSC);
+		}
+		// allocate TaskManagers
+		for(int i = 0; i < taskManagerCount; i++) {
+			if(!allocateResource(nmFree, taskManagerMemoryMb)) {
+				LOG.warn("There is not enough memory available in the YARN cluster. " +
+					"The TaskManager(s) require " + taskManagerMemoryMb + "MB each. " +
+					"NodeManagers available: " + Arrays.toString(freeClusterMem.nodeManagersFree) + "\n" +
+					"After allocating the JobManager (" + jobManagerMemoryMb + "MB) and (" + i + "/" + taskManagerCount + ") TaskManagers, " +
+					"the following NodeManagers are available: " + Arrays.toString(nmFree)  + NOTE_RSC );
+			}
+		}
+
+		// ------------------ Prepare Application Master Container  ------------------------------
+
+		// respect custom JVM options in the YAML file
+		final String javaOpts = flinkConfiguration.getString(ConfigConstants.FLINK_JVM_OPTIONS, "");
+
+		String logbackFile = configurationDirectory + File.separator + CONFIG_FILE_LOGBACK_NAME;
+		boolean hasLogback = new File(logbackFile).exists();
+		String log4jFile = configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME;
+
+		boolean hasLog4j = new File(log4jFile).exists();
+		if(hasLogback) {
+			shipFiles.add(new File(logbackFile));
+		}
+		if(hasLog4j) {
+			shipFiles.add(new File(log4jFile));
+		}
+
+		// Set up the container launch context for the application master
+		ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class);
+
+		String amCommand = "$JAVA_HOME/bin/java"
+			+ " -Xmx" + Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration)
+			+ "M " + javaOpts;
+
+		if(hasLogback || hasLog4j) {
+			amCommand += " -Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\"";
+
+			if(hasLogback) {
+				amCommand += " -Dlogback.configurationFile=file:" + CONFIG_FILE_LOGBACK_NAME;
+			}
+
+			if(hasLog4j) {
+				amCommand += " -Dlog4j.configuration=file:" + CONFIG_FILE_LOG4J_NAME;
+			}
+		}
+
+		amCommand += " " + getApplicationMasterClass().getName() + " "
+			+ " 1>"
+			+ ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out"
+			+ " 2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.err";
+		amContainer.setCommands(Collections.singletonList(amCommand));
+
+		LOG.debug("Application Master start command: " + amCommand);
+
+		// intialize HDFS
+		// Copy the application master jar to the filesystem
+		// Create a local resource to point to the destination jar path
+		final FileSystem fs = FileSystem.get(conf);
+
+		// hard coded check for the GoogleHDFS client because its not overriding the getScheme() method.
+		if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") &&
+			fs.getScheme().startsWith("file")) {
+			LOG.warn("The file system scheme is '" + fs.getScheme() + "'. This indicates that the "
+				+ "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values."
+				+ "The Flink YARN client needs to store its files in a distributed file system");
+		}
+
+		// Set-up ApplicationSubmissionContext for the application
+		ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext();
+
+		if (RecoveryMode.isHighAvailabilityModeActivated(flinkConfiguration)) {
+			// activate re-execution of failed applications
+			appContext.setMaxAppAttempts(
+				flinkConfiguration.getInteger(
+					ConfigConstants.YARN_APPLICATION_ATTEMPTS,
+					YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS));
+
+			activateHighAvailabilitySupport(appContext);
+		} else {
+			// set number of application retries to 1 in the default case
+			appContext.setMaxAppAttempts(
+				flinkConfiguration.getInteger(
+					ConfigConstants.YARN_APPLICATION_ATTEMPTS,
+					1));
+		}
+
+		final ApplicationId appId = appContext.getApplicationId();
+
+		// Setup jar for ApplicationMaster
+		LocalResource appMasterJar = Records.newRecord(LocalResource.class);
+		LocalResource flinkConf = Records.newRecord(LocalResource.class);
+		Path remotePathJar = Utils.setupLocalResource(fs, appId.toString(), flinkJarPath, appMasterJar, fs.getHomeDirectory());
+		Path remotePathConf = Utils.setupLocalResource(fs, appId.toString(), flinkConfigurationPath, flinkConf, fs.getHomeDirectory());
+		Map<String, LocalResource> localResources = new HashMap<>(2);
+		localResources.put("flink.jar", appMasterJar);
+		localResources.put("flink-conf.yaml", flinkConf);
+
+
+		// setup security tokens (code from apache storm)
+		final Path[] paths = new Path[2 + shipFiles.size()];
+		StringBuilder envShipFileList = new StringBuilder();
+		// upload ship files
+		for (int i = 0; i < shipFiles.size(); i++) {
+			File shipFile = shipFiles.get(i);
+			LocalResource shipResources = Records.newRecord(LocalResource.class);
+			Path shipLocalPath = new Path("file://" + shipFile.getAbsolutePath());
+			paths[2 + i] = Utils.setupLocalResource(fs, appId.toString(),
+				shipLocalPath, shipResources, fs.getHomeDirectory());
+			localResources.put(shipFile.getName(), shipResources);
+
+			envShipFileList.append(paths[2 + i]);
+			if(i+1 < shipFiles.size()) {
+				envShipFileList.append(',');
+			}
+		}
+
+		paths[0] = remotePathJar;
+		paths[1] = remotePathConf;
+		sessionFilesDir = new Path(fs.getHomeDirectory(), ".flink/" + appId.toString() + "/");
+
+		FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE);
+		fs.setPermission(sessionFilesDir, permission); // set permission for path.
+
+		Utils.setTokensFor(amContainer, paths, conf);
+
+		amContainer.setLocalResources(localResources);
+		fs.close();
+
+		// Setup CLASSPATH for ApplicationMaster
+		Map<String, String> appMasterEnv = new HashMap<>();
+		// set user specified app master environment variables
+		appMasterEnv.putAll(Utils.getEnvironmentVariables(ConfigConstants.YARN_APPLICATION_MASTER_ENV_PREFIX, flinkConfiguration));
+		// set classpath from YARN configuration
+		Utils.setupEnv(conf, appMasterEnv);
+		// set Flink on YARN internal configuration values
+		appMasterEnv.put(YarnConfigKeys.ENV_TM_COUNT, String.valueOf(taskManagerCount));
+		appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(taskManagerMemoryMb));
+		appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, remotePathJar.toString() );
+		appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString());
+		appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, fs.getHomeDirectory().toString());
+		appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString());
+		appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_USERNAME, UserGroupInformation.getCurrentUser().getShortUserName());
+		appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(slots));
+		appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(detached));
+
+		if(dynamicPropertiesEncoded != null) {
+			appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded);
+		}
+
+		amContainer.setEnvironment(appMasterEnv);
+
+		// Set up resource type requirements for ApplicationMaster
+		Resource capability = Records.newRecord(Resource.class);
+		capability.setMemory(jobManagerMemoryMb);
+		capability.setVirtualCores(1);
+
+		String name;
+		if(customName == null) {
+			name = "Flink session with " + taskManagerCount + " TaskManagers";
+			if(detached) {
+				name += " (detached)";
+			}
+		} else {
+			name = customName;
+		}
+
+		appContext.setApplicationName(name); // application name
+		appContext.setApplicationType("Apache Flink");
+		appContext.setAMContainerSpec(amContainer);
+		appContext.setResource(capability);
+		if(yarnQueue != null) {
+			appContext.setQueue(yarnQueue);
+		}
+
+		// add a hook to clean up in case deployment fails
+		Thread deploymentFailureHook = new DeploymentFailureHook(yarnClient, yarnApplication);
+		Runtime.getRuntime().addShutdownHook(deploymentFailureHook);
+		LOG.info("Submitting application master " + appId);
+		yarnClient.submitApplication(appContext);
+
+		LOG.info("Waiting for the cluster to be allocated");
+		int waittime = 0;
+		ApplicationReport report;
+		loop: while( true ) {
+			try {
+				report = yarnClient.getApplicationReport(appId);
+			} catch (IOException e) {
+				throw new YarnDeploymentException("Failed to deploy the cluster: " + e.getMessage());
+			}
+			YarnApplicationState appState = report.getYarnApplicationState();
+			switch(appState) {
+				case FAILED:
+				case FINISHED:
+				case KILLED:
+					throw new YarnDeploymentException("The YARN application unexpectedly switched to state "
+						+ appState + " during deployment. \n" +
+						"Diagnostics from YARN: " + report.getDiagnostics() + "\n" +
+						"If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" +
+						"yarn logs -applicationId " + appId);
+					//break ..
+				case RUNNING:
+					LOG.info("YARN application has been deployed successfully.");
+					break loop;
+				default:
+					LOG.info("Deploying cluster, current state " + appState);
+					if(waittime > 60000) {
+						LOG.info("Deployment took more than 60 seconds. Please check if the requested resources are available in the YARN cluster");
+					}
+
+			}
+			waittime += 1000;
+			Thread.sleep(1000);
+		}
+		// print the application id for user to cancel themselves.
+		if (isDetachedMode()) {
+			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:\nyarn application -kill " + appId + "\nPlease also note that the " +
+					"temporary files of the YARN session in the home directoy will not be removed.");
+		}
+		// since deployment was successful, remove the hook
+		try {
+			Runtime.getRuntime().removeShutdownHook(deploymentFailureHook);
+		} catch (IllegalStateException e) {
+			// we're already in the shut down hook.
+		}
+
+		String host = report.getHost();
+		int port = report.getRpcPort();
+		String trackingURL = report.getTrackingUrl();
+
+		// Correctly initialize the Flink config
+		flinkConfiguration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, host);
+		flinkConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port);
+
+		// the Flink cluster is deployed in YARN. Represent cluster
+		return new YarnClusterClient(this, yarnClient, report, flinkConfiguration, sessionFilesDir);
+	}
+
+	/**
+	 * Kills YARN application and stops YARN client.
+	 *
+	 * Use this method to kill the App before it has been properly deployed
+	 */
+	private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) {
+		LOG.info("Killing YARN application");
+
+		try {
+			yarnClient.killApplication(yarnApplication.getNewApplicationResponse().getApplicationId());
+		} catch (Exception e) {
+			// we only log a debug message here because the "killApplication" call is a best-effort
+			// call (we don't know if the application has been deployed when the error occured).
+			LOG.debug("Error while killing YARN application", e);
+		}
+		yarnClient.stop();
+	}
+
+
+	private static class ClusterResourceDescription {
+		final public int totalFreeMemory;
+		final public int containerLimit;
+		final public int[] nodeManagersFree;
+
+		public ClusterResourceDescription(int totalFreeMemory, int containerLimit, int[] nodeManagersFree) {
+			this.totalFreeMemory = totalFreeMemory;
+			this.containerLimit = containerLimit;
+			this.nodeManagersFree = nodeManagersFree;
+		}
+	}
+
+	private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) throws YarnException, IOException {
+		List<NodeReport> nodes = yarnClient.getNodeReports(NodeState.RUNNING);
+
+		int totalFreeMemory = 0;
+		int containerLimit = 0;
+		int[] nodeManagersFree = new int[nodes.size()];
+
+		for(int i = 0; i < nodes.size(); i++) {
+			NodeReport rep = nodes.get(i);
+			int free = rep.getCapability().getMemory() - (rep.getUsed() != null ? rep.getUsed().getMemory() : 0 );
+			nodeManagersFree[i] = free;
+			totalFreeMemory += free;
+			if(free > containerLimit) {
+				containerLimit = free;
+			}
+		}
+		return new ClusterResourceDescription(totalFreeMemory, containerLimit, nodeManagersFree);
+	}
+
+	@Override
+	public String getClusterDescription() throws Exception {
+
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		PrintStream ps = new PrintStream(baos);
+
+		YarnClient yarnClient = getYarnClient(conf);
+		YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics();
+
+		ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers());
+		List<NodeReport> nodes = yarnClient.getNodeReports(NodeState.RUNNING);
+		final String format = "|%-16s |%-16s %n";
+		ps.printf("|Property         |Value          %n");
+		ps.println("+---------------------------------------+");
+		int totalMemory = 0;
+		int totalCores = 0;
+		for(NodeReport rep : nodes) {
+			final Resource res = rep.getCapability();
+			totalMemory += res.getMemory();
+			totalCores += res.getVirtualCores();
+			ps.format(format, "NodeID", rep.getNodeId());
+			ps.format(format, "Memory", res.getMemory() + " MB");
+			ps.format(format, "vCores", res.getVirtualCores());
+			ps.format(format, "HealthReport", rep.getHealthReport());
+			ps.format(format, "Containers", rep.getNumContainers());
+			ps.println("+---------------------------------------+");
+		}
+		ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores);
+		List<QueueInfo> qInfo = yarnClient.getAllQueues();
+		for(QueueInfo q : qInfo) {
+			ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " +
+				q.getMaximumCapacity() + " Applications: " + q.getApplications().size());
+		}
+		yarnClient.stop();
+		return baos.toString();
+	}
+
+	public String getSessionFilesDir() {
+		return sessionFilesDir.toString();
+	}
+
+	public void setName(String name) {
+		if(name == null) {
+			throw new IllegalArgumentException("The passed name is null");
+		}
+		customName = name;
+	}
+
+	private void activateHighAvailabilitySupport(ApplicationSubmissionContext appContext) throws InvocationTargetException, IllegalAccessException {
+		ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance();
+
+		reflector.setKeepContainersAcrossApplicationAttempts(appContext, true);
+		reflector.setAttemptFailuresValidityInterval(appContext, AkkaUtils.getTimeout(flinkConfiguration).toMillis());
+	}
+
+	/**
+	 * Singleton object which uses reflection to determine whether the {@link ApplicationSubmissionContext}
+	 * supports the setKeepContainersAcrossApplicationAttempts and the setAttemptFailuresValidityInterval
+	 * methods. Depending on the Hadoop version these methods are supported or not. If the methods
+	 * are not supported, then nothing happens when setKeepContainersAcrossApplicationAttempts or
+	 * setAttemptFailuresValidityInterval are called.
+	 */
+	private static class ApplicationSubmissionContextReflector {
+		private static final Logger LOG = LoggerFactory.getLogger(ApplicationSubmissionContextReflector.class);
+
+		private static final ApplicationSubmissionContextReflector instance = new ApplicationSubmissionContextReflector(ApplicationSubmissionContext.class);
+
+		public static ApplicationSubmissionContextReflector getInstance() {
+			return instance;
+		}
+
+		private static final String keepContainersMethodName = "setKeepContainersAcrossApplicationAttempts";
+		private static final String attemptsFailuresValidityIntervalMethodName = "setAttemptFailuresValidityInterval";
+
+		private final Method keepContainersMethod;
+		private final Method attemptFailuresValidityIntervalMethod;
+
+		private ApplicationSubmissionContextReflector(Class<ApplicationSubmissionContext> clazz) {
+			Method keepContainersMethod;
+			Method attemptFailuresValidityIntervalMethod;
+
+			try {
+				// this method is only supported by Hadoop 2.4.0 onwards
+				keepContainersMethod = clazz.getMethod(keepContainersMethodName, boolean.class);
+				LOG.debug("{} supports method {}.", clazz.getCanonicalName(), keepContainersMethodName);
+			} catch (NoSuchMethodException e) {
+				LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), keepContainersMethodName);
+				// assign null because the Hadoop version apparently does not support this call.
+				keepContainersMethod = null;
+			}
+
+			this.keepContainersMethod = keepContainersMethod;
+
+			try {
+				// this method is only supported by Hadoop 2.6.0 onwards
+				attemptFailuresValidityIntervalMethod = clazz.getMethod(attemptsFailuresValidityIntervalMethodName, long.class);
+				LOG.debug("{} supports method {}.", clazz.getCanonicalName(), attemptsFailuresValidityIntervalMethodName);
+			} catch (NoSuchMethodException e) {
+				LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), attemptsFailuresValidityIntervalMethodName);
+				// assign null because the Hadoop version apparently does not support this call.
+				attemptFailuresValidityIntervalMethod = null;
+			}
+
+			this.attemptFailuresValidityIntervalMethod = attemptFailuresValidityIntervalMethod;
+		}
+
+		public void setKeepContainersAcrossApplicationAttempts(
+				ApplicationSubmissionContext appContext,
+				boolean keepContainers) throws InvocationTargetException, IllegalAccessException {
+
+			if (keepContainersMethod != null) {
+				LOG.debug("Calling method {} of {}.", keepContainersMethod.getName(),
+					appContext.getClass().getCanonicalName());
+				keepContainersMethod.invoke(appContext, keepContainers);
+			} else {
+				LOG.debug("{} does not support method {}. Doing nothing.",
+					appContext.getClass().getCanonicalName(), keepContainersMethodName);
+			}
+		}
+
+		public void setAttemptFailuresValidityInterval(
+				ApplicationSubmissionContext appContext,
+				long validityInterval) throws InvocationTargetException, IllegalAccessException {
+			if (attemptFailuresValidityIntervalMethod != null) {
+				LOG.debug("Calling method {} of {}.",
+					attemptFailuresValidityIntervalMethod.getName(),
+					appContext.getClass().getCanonicalName());
+				attemptFailuresValidityIntervalMethod.invoke(appContext, validityInterval);
+			} else {
+				LOG.debug("{} does not support method {}. Doing nothing.",
+					appContext.getClass().getCanonicalName(),
+					attemptsFailuresValidityIntervalMethodName);
+			}
+		}
+	}
+
+	private static class YarnDeploymentException extends RuntimeException {
+		private static final long serialVersionUID = -812040641215388943L;
+
+		public YarnDeploymentException() {
+		}
+
+		public YarnDeploymentException(String message) {
+			super(message);
+		}
+
+		public YarnDeploymentException(String message, Throwable cause) {
+			super(message, cause);
+		}
+	}
+
+	private class DeploymentFailureHook extends Thread {
+
+		DeploymentFailureHook(YarnClient yarnClient, YarnClientApplication yarnApplication) {
+			this.yarnClient = yarnClient;
+			this.yarnApplication = yarnApplication;
+		}
+
+		private YarnClient yarnClient;
+		private YarnClientApplication yarnApplication;
+
+		@Override
+		public void run() {
+			LOG.info("Cancelling deployment from Deployment Failure Hook");
+			failSessionDuringDeployment(yarnClient, yarnApplication);
+			LOG.info("Deleting files in " + sessionFilesDir);
+			try {
+				FileSystem fs = FileSystem.get(conf);
+				fs.delete(sessionFilesDir, true);
+				fs.close();
+			} catch (IOException e) {
+				LOG.error("Failed to delete Flink Jar and conf files in HDFS", e);
+			}
+		}
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java
deleted file mode 100644
index 467e06d..0000000
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java
+++ /dev/null
@@ -1,28 +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;
-
-/**
- * Default implementation of {@link FlinkYarnClientBase} which starts an {@link YarnApplicationMasterRunner}.
- */
-public class FlinkYarnClient extends FlinkYarnClientBase {
-	@Override
-	protected Class<?> getApplicationMasterClass() {
-		return YarnApplicationMasterRunner.class;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClientBase.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClientBase.java b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClientBase.java
deleted file mode 100644
index 6f81d09..0000000
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClientBase.java
+++ /dev/null
@@ -1,907 +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.client.CliFrontend;
-import org.apache.flink.client.FlinkYarnSessionCli;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.jobmanager.RecoveryMode;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.NodeState;
-import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
-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.Records;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
-* All classes in this package contain code taken from
-* https://github.com/apache/hadoop-common/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java?source=cc
-* and
-* https://github.com/hortonworks/simple-yarn-app
-* and
-* https://github.com/yahoo/storm-yarn/blob/master/src/main/java/com/yahoo/storm/yarn/StormOnYarn.java
-*
-* The Flink jar is uploaded to HDFS by this client.
-* The application master and all the TaskManager containers get the jar file downloaded
-* by YARN into their local fs.
-*
-*/
-public abstract class FlinkYarnClientBase extends AbstractFlinkYarnClient {
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnClient.class);
-
-	/**
-	 * Minimum memory requirements, checked by the Client.
-	 */
-	private static final int MIN_JM_MEMORY = 768; // the minimum memory should be higher than the min heap cutoff
-	private static final int MIN_TM_MEMORY = 768;
-
-	private Configuration conf;
-	private YarnClient yarnClient;
-	private YarnClientApplication yarnApplication;
-	private Thread deploymentFailureHook = new DeploymentFailureHook();
-
-	/**
-	 * Files (usually in a distributed file system) used for the YARN session of Flink.
-	 * Contains configuration files and jar files.
-	 */
-	private Path sessionFilesDir;
-
-	/**
-	 * If the user has specified a different number of slots, we store them here
-	 */
-	private int slots = -1;
-
-	private int jobManagerMemoryMb = 1024;
-
-	private int taskManagerMemoryMb = 1024;
-
-	private int taskManagerCount = 1;
-
-	private String yarnQueue = null;
-
-	private String configurationDirectory;
-
-	private Path flinkConfigurationPath;
-
-	private Path flinkLoggingConfigurationPath; // optional
-
-	private Path flinkJarPath;
-
-	private String dynamicPropertiesEncoded;
-
-	private List<File> shipFiles = new ArrayList<>();
-	private org.apache.flink.configuration.Configuration flinkConfiguration;
-
-	private boolean detached;
-
-	private String customName = null;
-
-	public FlinkYarnClientBase() {
-		conf = new YarnConfiguration();
-		if(this.yarnClient == null) {
-			// Create yarnClient
-			yarnClient = YarnClient.createYarnClient();
-			yarnClient.init(conf);
-			yarnClient.start();
-		}
-
-		// for unit tests only
-		if(System.getenv("IN_TESTS") != null) {
-			try {
-				conf.addResource(new File(System.getenv("YARN_CONF_DIR") + "/yarn-site.xml").toURI().toURL());
-			} catch (Throwable t) {
-				throw new RuntimeException("Error",t);
-			}
-		}
-	}
-
-	/**
-	 * The class to bootstrap the application master of the Yarn cluster (runs main method).
-	 */
-	protected abstract Class<?> getApplicationMasterClass();
-
-	@Override
-	public void setJobManagerMemory(int memoryMb) {
-		if(memoryMb < MIN_JM_MEMORY) {
-			throw new IllegalArgumentException("The JobManager memory (" + memoryMb + ") is below the minimum required memory amount "
-				+ "of " + MIN_JM_MEMORY+ " MB");
-		}
-		this.jobManagerMemoryMb = memoryMb;
-	}
-
-	@Override
-	public void setTaskManagerMemory(int memoryMb) {
-		if(memoryMb < MIN_TM_MEMORY) {
-			throw new IllegalArgumentException("The TaskManager memory (" + memoryMb + ") is below the minimum required memory amount "
-				+ "of " + MIN_TM_MEMORY+ " MB");
-		}
-		this.taskManagerMemoryMb = memoryMb;
-	}
-
-	@Override
-	public void setFlinkConfiguration(org.apache.flink.configuration.Configuration conf) {
-		this.flinkConfiguration = conf;
-	}
-
-	@Override
-	public org.apache.flink.configuration.Configuration getFlinkConfiguration() {
-		return flinkConfiguration;
-	}
-
-	@Override
-	public void setTaskManagerSlots(int slots) {
-		if(slots <= 0) {
-			throw new IllegalArgumentException("Number of TaskManager slots must be positive");
-		}
-		this.slots = slots;
-	}
-
-	@Override
-	public int getTaskManagerSlots() {
-		return this.slots;
-	}
-
-	@Override
-	public void setQueue(String queue) {
-		this.yarnQueue = queue;
-	}
-
-	@Override
-	public void setLocalJarPath(Path localJarPath) {
-		if(!localJarPath.toString().endsWith("jar")) {
-			throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension");
-		}
-		this.flinkJarPath = localJarPath;
-	}
-
-	@Override
-	public void setConfigurationFilePath(Path confPath) {
-		flinkConfigurationPath = confPath;
-	}
-
-	@Override
-	public void setConfigurationDirectory(String configurationDirectory) {
-		this.configurationDirectory = configurationDirectory;
-	}
-
-	@Override
-	public void setFlinkLoggingConfigurationPath(Path logConfPath) {
-		flinkLoggingConfigurationPath = logConfPath;
-	}
-
-	@Override
-	public Path getFlinkLoggingConfigurationPath() {
-		return flinkLoggingConfigurationPath;
-	}
-
-	@Override
-	public void setTaskManagerCount(int tmCount) {
-		if(tmCount < 1) {
-			throw new IllegalArgumentException("The TaskManager count has to be at least 1.");
-		}
-		this.taskManagerCount = tmCount;
-	}
-
-	@Override
-	public int getTaskManagerCount() {
-		return this.taskManagerCount;
-	}
-
-	@Override
-	public void setShipFiles(List<File> shipFiles) {
-		for(File shipFile: shipFiles) {
-			// remove uberjar from ship list (by default everything in the lib/ folder is added to
-			// the list of files to ship, but we handle the uberjar separately.
-			if(!(shipFile.getName().startsWith("flink-dist") && shipFile.getName().endsWith("jar"))) {
-				this.shipFiles.add(shipFile);
-			}
-		}
-	}
-
-	@Override
-	public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) {
-		this.dynamicPropertiesEncoded = dynamicPropertiesEncoded;
-	}
-
-	@Override
-	public String getDynamicPropertiesEncoded() {
-		return this.dynamicPropertiesEncoded;
-	}
-
-
-	public void isReadyForDeployment() throws YarnDeploymentException {
-		if(taskManagerCount <= 0) {
-			throw new YarnDeploymentException("Taskmanager count must be positive");
-		}
-		if(this.flinkJarPath == null) {
-			throw new YarnDeploymentException("The Flink jar path is null");
-		}
-		if(this.configurationDirectory == null) {
-			throw new YarnDeploymentException("Configuration directory not set");
-		}
-		if(this.flinkConfigurationPath == null) {
-			throw new YarnDeploymentException("Configuration path not set");
-		}
-		if(this.flinkConfiguration == null) {
-			throw new YarnDeploymentException("Flink configuration object has not been set");
-		}
-
-		// check if required Hadoop environment variables are set. If not, warn user
-		if(System.getenv("HADOOP_CONF_DIR") == null &&
-			System.getenv("YARN_CONF_DIR") == null) {
-			LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." +
-				"The Flink YARN Client needs one of these to be set to properly load the Hadoop " +
-				"configuration for accessing YARN.");
-		}
-	}
-
-	public static boolean allocateResource(int[] nodeManagers, int toAllocate) {
-		for(int i = 0; i < nodeManagers.length; i++) {
-			if(nodeManagers[i] >= toAllocate) {
-				nodeManagers[i] -= toAllocate;
-				return true;
-			}
-		}
-		return false;
-	}
-
-	@Override
-	public void setDetachedMode(boolean detachedMode) {
-		this.detached = detachedMode;
-	}
-
-	@Override
-	public boolean isDetached() {
-		return detached;
-	}
-
-	@Override
-	public AbstractFlinkYarnCluster deploy() throws Exception {
-
-		UserGroupInformation.setConfiguration(conf);
-		UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-
-		if (UserGroupInformation.isSecurityEnabled()) {
-			if (!ugi.hasKerberosCredentials()) {
-				throw new YarnDeploymentException("In secure mode. Please provide Kerberos credentials in order to authenticate. " +
-					"You may use kinit to authenticate and request a TGT from the Kerberos server.");
-			}
-			return ugi.doAs(new PrivilegedExceptionAction<AbstractFlinkYarnCluster>() {
-				@Override
-				public AbstractFlinkYarnCluster run() throws Exception {
-					return deployInternal();
-				}
-			});
-		} else {
-			return deployInternal();
-		}
-	}
-
-
-
-	/**
-	 * This method will block until the ApplicationMaster/JobManager have been
-	 * deployed on YARN.
-	 */
-	protected AbstractFlinkYarnCluster deployInternal() throws Exception {
-		isReadyForDeployment();
-
-		LOG.info("Using values:");
-		LOG.info("\tTaskManager count = {}", taskManagerCount);
-		LOG.info("\tJobManager memory = {}", jobManagerMemoryMb);
-		LOG.info("\tTaskManager memory = {}", taskManagerMemoryMb);
-
-		// Create application via yarnClient
-		yarnApplication = yarnClient.createApplication();
-		GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse();
-
-		// ------------------ Add dynamic properties to local flinkConfiguraton ------
-
-		Map<String, String> dynProperties = CliFrontend.getDynamicProperties(dynamicPropertiesEncoded);
-		for (Map.Entry<String, String> dynProperty : dynProperties.entrySet()) {
-			flinkConfiguration.setString(dynProperty.getKey(), dynProperty.getValue());
-		}
-
-		try {
-			org.apache.flink.core.fs.FileSystem.setDefaultScheme(flinkConfiguration);
-		} catch (IOException e) {
-			throw new IOException("Error while setting the default " +
-				"filesystem scheme from configuration.", e);
-		}
-		// ------------------ Check if the specified queue exists --------------
-
-		try {
-			List<QueueInfo> queues = yarnClient.getAllQueues();
-			if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session.
-				boolean queueFound = false;
-				for (QueueInfo queue : queues) {
-					if (queue.getQueueName().equals(this.yarnQueue)) {
-						queueFound = true;
-						break;
-					}
-				}
-				if (!queueFound) {
-					String queueNames = "";
-					for (QueueInfo queue : queues) {
-						queueNames += queue.getQueueName() + ", ";
-					}
-					LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " +
-						"Available queues: " + queueNames);
-				}
-			} else {
-				LOG.debug("The YARN cluster does not have any queues configured");
-			}
-		} catch(Throwable e) {
-			LOG.warn("Error while getting queue information from YARN: " + e.getMessage());
-			if(LOG.isDebugEnabled()) {
-				LOG.debug("Error details", e);
-			}
-		}
-
-		// ------------------ Check if the YARN Cluster has the requested resources --------------
-
-		// the yarnMinAllocationMB specifies the smallest possible container allocation size.
-		// all allocations below this value are automatically set to this value.
-		final int yarnMinAllocationMB = conf.getInt("yarn.scheduler.minimum-allocation-mb", 0);
-		if(jobManagerMemoryMb < yarnMinAllocationMB || taskManagerMemoryMb < yarnMinAllocationMB) {
-			LOG.warn("The JobManager or TaskManager memory is below the smallest possible YARN Container size. "
-				+ "The value of 'yarn.scheduler.minimum-allocation-mb' is '" + yarnMinAllocationMB + "'. Please increase the memory size." +
-				"YARN will allocate the smaller containers but the scheduler will account for the minimum-allocation-mb, maybe not all instances " +
-				"you requested will start.");
-		}
-
-		// set the memory to minAllocationMB to do the next checks correctly
-		if(jobManagerMemoryMb < yarnMinAllocationMB) {
-			jobManagerMemoryMb =  yarnMinAllocationMB;
-		}
-		if(taskManagerMemoryMb < yarnMinAllocationMB) {
-			taskManagerMemoryMb =  yarnMinAllocationMB;
-		}
-
-		Resource maxRes = appResponse.getMaximumResourceCapability();
-		final String NOTE = "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n";
-		if(jobManagerMemoryMb > maxRes.getMemory() ) {
-			failSessionDuringDeployment();
-			throw new YarnDeploymentException("The cluster does not have the requested resources for the JobManager available!\n"
-				+ "Maximum Memory: " + maxRes.getMemory() + "MB Requested: " + jobManagerMemoryMb + "MB. " + NOTE);
-		}
-
-		if(taskManagerMemoryMb > maxRes.getMemory() ) {
-			failSessionDuringDeployment();
-			throw new YarnDeploymentException("The cluster does not have the requested resources for the TaskManagers available!\n"
-				+ "Maximum Memory: " + maxRes.getMemory() + " Requested: " + taskManagerMemoryMb + "MB. " + NOTE);
-		}
-
-		final String NOTE_RSC = "\nThe Flink YARN client will try to allocate the YARN session, but maybe not all TaskManagers are " +
-			"connecting from the beginning because the resources are currently not available in the cluster. " +
-			"The allocation might take more time than usual because the Flink YARN client needs to wait until " +
-			"the resources become available.";
-		int totalMemoryRequired = jobManagerMemoryMb + taskManagerMemoryMb * taskManagerCount;
-		ClusterResourceDescription freeClusterMem = getCurrentFreeClusterResources(yarnClient);
-		if(freeClusterMem.totalFreeMemory < totalMemoryRequired) {
-			LOG.warn("This YARN session requires " + totalMemoryRequired + "MB of memory in the cluster. "
-				+ "There are currently only " + freeClusterMem.totalFreeMemory + "MB available." + NOTE_RSC);
-
-		}
-		if(taskManagerMemoryMb > freeClusterMem.containerLimit) {
-			LOG.warn("The requested amount of memory for the TaskManagers (" + taskManagerMemoryMb + "MB) is more than "
-				+ "the largest possible YARN container: " + freeClusterMem.containerLimit + NOTE_RSC);
-		}
-		if(jobManagerMemoryMb > freeClusterMem.containerLimit) {
-			LOG.warn("The requested amount of memory for the JobManager (" + jobManagerMemoryMb + "MB) is more than "
-				+ "the largest possible YARN container: " + freeClusterMem.containerLimit + NOTE_RSC);
-		}
-
-		// ----------------- check if the requested containers fit into the cluster.
-
-		int[] nmFree = Arrays.copyOf(freeClusterMem.nodeManagersFree, freeClusterMem.nodeManagersFree.length);
-		// first, allocate the jobManager somewhere.
-		if(!allocateResource(nmFree, jobManagerMemoryMb)) {
-			LOG.warn("Unable to find a NodeManager that can fit the JobManager/Application master. " +
-				"The JobManager requires " + jobManagerMemoryMb + "MB. NodeManagers available: " +
-				Arrays.toString(freeClusterMem.nodeManagersFree) + NOTE_RSC);
-		}
-		// allocate TaskManagers
-		for(int i = 0; i < taskManagerCount; i++) {
-			if(!allocateResource(nmFree, taskManagerMemoryMb)) {
-				LOG.warn("There is not enough memory available in the YARN cluster. " +
-					"The TaskManager(s) require " + taskManagerMemoryMb + "MB each. " +
-					"NodeManagers available: " + Arrays.toString(freeClusterMem.nodeManagersFree) + "\n" +
-					"After allocating the JobManager (" + jobManagerMemoryMb + "MB) and (" + i + "/" + taskManagerCount + ") TaskManagers, " +
-					"the following NodeManagers are available: " + Arrays.toString(nmFree)  + NOTE_RSC );
-			}
-		}
-
-		// ------------------ Prepare Application Master Container  ------------------------------
-
-		// respect custom JVM options in the YAML file
-		final String javaOpts = flinkConfiguration.getString(ConfigConstants.FLINK_JVM_OPTIONS, "");
-
-		String logbackFile = configurationDirectory + File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME;
-		boolean hasLogback = new File(logbackFile).exists();
-		String log4jFile = configurationDirectory + File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME;
-
-		boolean hasLog4j = new File(log4jFile).exists();
-		if(hasLogback) {
-			shipFiles.add(new File(logbackFile));
-		}
-		if(hasLog4j) {
-			shipFiles.add(new File(log4jFile));
-		}
-
-		// Set up the container launch context for the application master
-		ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class);
-
-		String amCommand = "$JAVA_HOME/bin/java"
-			+ " -Xmx" + Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration)
-			+ "M " + javaOpts;
-
-		if(hasLogback || hasLog4j) {
-			amCommand += " -Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\"";
-
-			if(hasLogback) {
-				amCommand += " -Dlogback.configurationFile=file:" + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME;
-			}
-
-			if(hasLog4j) {
-				amCommand += " -Dlog4j.configuration=file:" + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME;
-			}
-		}
-
-		amCommand += " " + getApplicationMasterClass().getName() + " "
-			+ " 1>"
-			+ ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out"
-			+ " 2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.err";
-		amContainer.setCommands(Collections.singletonList(amCommand));
-
-		LOG.debug("Application Master start command: " + amCommand);
-
-		// intialize HDFS
-		// Copy the application master jar to the filesystem
-		// Create a local resource to point to the destination jar path
-		final FileSystem fs = FileSystem.get(conf);
-
-		// hard coded check for the GoogleHDFS client because its not overriding the getScheme() method.
-		if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") &&
-			fs.getScheme().startsWith("file")) {
-			LOG.warn("The file system scheme is '" + fs.getScheme() + "'. This indicates that the "
-				+ "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values."
-				+ "The Flink YARN client needs to store its files in a distributed file system");
-		}
-
-		// Set-up ApplicationSubmissionContext for the application
-		ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext();
-
-		if (RecoveryMode.isHighAvailabilityModeActivated(flinkConfiguration)) {
-			// activate re-execution of failed applications
-			appContext.setMaxAppAttempts(
-				flinkConfiguration.getInteger(
-					ConfigConstants.YARN_APPLICATION_ATTEMPTS,
-					YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS));
-
-			activateHighAvailabilitySupport(appContext);
-		} else {
-			// set number of application retries to 1 in the default case
-			appContext.setMaxAppAttempts(
-				flinkConfiguration.getInteger(
-					ConfigConstants.YARN_APPLICATION_ATTEMPTS,
-					1));
-		}
-
-		final ApplicationId appId = appContext.getApplicationId();
-
-		// Setup jar for ApplicationMaster
-		LocalResource appMasterJar = Records.newRecord(LocalResource.class);
-		LocalResource flinkConf = Records.newRecord(LocalResource.class);
-		Path remotePathJar = Utils.setupLocalResource(fs, appId.toString(), flinkJarPath, appMasterJar, fs.getHomeDirectory());
-		Path remotePathConf = Utils.setupLocalResource(fs, appId.toString(), flinkConfigurationPath, flinkConf, fs.getHomeDirectory());
-		Map<String, LocalResource> localResources = new HashMap<>(2);
-		localResources.put("flink.jar", appMasterJar);
-		localResources.put("flink-conf.yaml", flinkConf);
-
-
-		// setup security tokens (code from apache storm)
-		final Path[] paths = new Path[2 + shipFiles.size()];
-		StringBuilder envShipFileList = new StringBuilder();
-		// upload ship files
-		for (int i = 0; i < shipFiles.size(); i++) {
-			File shipFile = shipFiles.get(i);
-			LocalResource shipResources = Records.newRecord(LocalResource.class);
-			Path shipLocalPath = new Path("file://" + shipFile.getAbsolutePath());
-			paths[2 + i] = Utils.setupLocalResource(fs, appId.toString(),
-				shipLocalPath, shipResources, fs.getHomeDirectory());
-			localResources.put(shipFile.getName(), shipResources);
-
-			envShipFileList.append(paths[2 + i]);
-			if(i+1 < shipFiles.size()) {
-				envShipFileList.append(',');
-			}
-		}
-
-		paths[0] = remotePathJar;
-		paths[1] = remotePathConf;
-		sessionFilesDir = new Path(fs.getHomeDirectory(), ".flink/" + appId.toString() + "/");
-
-		FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE);
-		fs.setPermission(sessionFilesDir, permission); // set permission for path.
-
-		Utils.setTokensFor(amContainer, paths, conf);
-
-		amContainer.setLocalResources(localResources);
-		fs.close();
-
-		// Setup CLASSPATH for ApplicationMaster
-		Map<String, String> appMasterEnv = new HashMap<>();
-		// set user specified app master environment variables
-		appMasterEnv.putAll(Utils.getEnvironmentVariables(ConfigConstants.YARN_APPLICATION_MASTER_ENV_PREFIX, flinkConfiguration));
-		// set classpath from YARN configuration
-		Utils.setupEnv(conf, appMasterEnv);
-		// set Flink on YARN internal configuration values
-		appMasterEnv.put(YarnConfigKeys.ENV_TM_COUNT, String.valueOf(taskManagerCount));
-		appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(taskManagerMemoryMb));
-		appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, remotePathJar.toString() );
-		appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString());
-		appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, fs.getHomeDirectory().toString());
-		appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString());
-		appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_USERNAME, UserGroupInformation.getCurrentUser().getShortUserName());
-		appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(slots));
-		appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(detached));
-
-		if(dynamicPropertiesEncoded != null) {
-			appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded);
-		}
-
-		amContainer.setEnvironment(appMasterEnv);
-
-		// Set up resource type requirements for ApplicationMaster
-		Resource capability = Records.newRecord(Resource.class);
-		capability.setMemory(jobManagerMemoryMb);
-		capability.setVirtualCores(1);
-
-		String name;
-		if(customName == null) {
-			name = "Flink session with " + taskManagerCount + " TaskManagers";
-			if(detached) {
-				name += " (detached)";
-			}
-		} else {
-			name = customName;
-		}
-
-		appContext.setApplicationName(name); // application name
-		appContext.setApplicationType("Apache Flink");
-		appContext.setAMContainerSpec(amContainer);
-		appContext.setResource(capability);
-		if(yarnQueue != null) {
-			appContext.setQueue(yarnQueue);
-		}
-
-		// add a hook to clean up in case deployment fails
-		Runtime.getRuntime().addShutdownHook(deploymentFailureHook);
-		LOG.info("Submitting application master " + appId);
-		yarnClient.submitApplication(appContext);
-
-		LOG.info("Waiting for the cluster to be allocated");
-		int waittime = 0;
-		loop: while( true ) {
-			ApplicationReport report;
-			try {
-				report = yarnClient.getApplicationReport(appId);
-			} catch (IOException e) {
-				throw new YarnDeploymentException("Failed to deploy the cluster: " + e.getMessage());
-			}
-			YarnApplicationState appState = report.getYarnApplicationState();
-			switch(appState) {
-				case FAILED:
-				case FINISHED:
-				case KILLED:
-					throw new YarnDeploymentException("The YARN application unexpectedly switched to state "
-						+ appState + " during deployment. \n" +
-						"Diagnostics from YARN: " + report.getDiagnostics() + "\n" +
-						"If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" +
-						"yarn logs -applicationId " + appId);
-					//break ..
-				case RUNNING:
-					LOG.info("YARN application has been deployed successfully.");
-					break loop;
-				default:
-					LOG.info("Deploying cluster, current state " + appState);
-					if(waittime > 60000) {
-						LOG.info("Deployment took more than 60 seconds. Please check if the requested resources are available in the YARN cluster");
-					}
-
-			}
-			waittime += 1000;
-			Thread.sleep(1000);
-		}
-		// print the application id for user to cancel themselves.
-		if (isDetached()) {
-			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:\nyarn application -kill " + appId + "\nPlease also note that the " +
-					"temporary files of the YARN session in the home directoy will not be removed.");
-		}
-		// since deployment was successful, remove the hook
-		try {
-			Runtime.getRuntime().removeShutdownHook(deploymentFailureHook);
-		} catch (IllegalStateException e) {
-			// we're already in the shut down hook.
-		}
-		// the Flink cluster is deployed in YARN. Represent cluster
-		return new FlinkYarnCluster(yarnClient, appId, conf, flinkConfiguration, sessionFilesDir, detached);
-	}
-
-	/**
-	 * Kills YARN application and stops YARN client.
-	 *
-	 * Use this method to kill the App before it has been properly deployed
-	 */
-	private void failSessionDuringDeployment() {
-		LOG.info("Killing YARN application");
-
-		try {
-			yarnClient.killApplication(yarnApplication.getNewApplicationResponse().getApplicationId());
-		} catch (Exception e) {
-			// we only log a debug message here because the "killApplication" call is a best-effort
-			// call (we don't know if the application has been deployed when the error occured).
-			LOG.debug("Error while killing YARN application", e);
-		}
-		yarnClient.stop();
-	}
-
-
-	private static class ClusterResourceDescription {
-		final public int totalFreeMemory;
-		final public int containerLimit;
-		final public int[] nodeManagersFree;
-
-		public ClusterResourceDescription(int totalFreeMemory, int containerLimit, int[] nodeManagersFree) {
-			this.totalFreeMemory = totalFreeMemory;
-			this.containerLimit = containerLimit;
-			this.nodeManagersFree = nodeManagersFree;
-		}
-	}
-
-	private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) throws YarnException, IOException {
-		List<NodeReport> nodes = yarnClient.getNodeReports(NodeState.RUNNING);
-
-		int totalFreeMemory = 0;
-		int containerLimit = 0;
-		int[] nodeManagersFree = new int[nodes.size()];
-
-		for(int i = 0; i < nodes.size(); i++) {
-			NodeReport rep = nodes.get(i);
-			int free = rep.getCapability().getMemory() - (rep.getUsed() != null ? rep.getUsed().getMemory() : 0 );
-			nodeManagersFree[i] = free;
-			totalFreeMemory += free;
-			if(free > containerLimit) {
-				containerLimit = free;
-			}
-		}
-		return new ClusterResourceDescription(totalFreeMemory, containerLimit, nodeManagersFree);
-	}
-
-	@Override
-	public String getClusterDescription() throws Exception {
-
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		PrintStream ps = new PrintStream(baos);
-
-		YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics();
-
-		ps.append("NodeManagers in the Cluster " + metrics.getNumNodeManagers());
-		List<NodeReport> nodes = yarnClient.getNodeReports(NodeState.RUNNING);
-		final String format = "|%-16s |%-16s %n";
-		ps.printf("|Property         |Value          %n");
-		ps.println("+---------------------------------------+");
-		int totalMemory = 0;
-		int totalCores = 0;
-		for(NodeReport rep : nodes) {
-			final Resource res = rep.getCapability();
-			totalMemory += res.getMemory();
-			totalCores += res.getVirtualCores();
-			ps.format(format, "NodeID", rep.getNodeId());
-			ps.format(format, "Memory", res.getMemory() + " MB");
-			ps.format(format, "vCores", res.getVirtualCores());
-			ps.format(format, "HealthReport", rep.getHealthReport());
-			ps.format(format, "Containers", rep.getNumContainers());
-			ps.println("+---------------------------------------+");
-		}
-		ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores);
-		List<QueueInfo> qInfo = yarnClient.getAllQueues();
-		for(QueueInfo q : qInfo) {
-			ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " +
-				q.getMaximumCapacity() + " Applications: " + q.getApplications().size());
-		}
-		yarnClient.stop();
-		return baos.toString();
-	}
-
-	@Override
-	public String getSessionFilesDir() {
-		return sessionFilesDir.toString();
-	}
-
-	@Override
-	public void setName(String name) {
-		if(name == null) {
-			throw new IllegalArgumentException("The passed name is null");
-		}
-		customName = name;
-	}
-
-	private void activateHighAvailabilitySupport(ApplicationSubmissionContext appContext) throws InvocationTargetException, IllegalAccessException {
-		ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance();
-
-		reflector.setKeepContainersAcrossApplicationAttempts(appContext, true);
-		reflector.setAttemptFailuresValidityInterval(appContext, AkkaUtils.getTimeout(flinkConfiguration).toMillis());
-	}
-
-	/**
-	 * Singleton object which uses reflection to determine whether the {@link ApplicationSubmissionContext}
-	 * supports the setKeepContainersAcrossApplicationAttempts and the setAttemptFailuresValidityInterval
-	 * methods. Depending on the Hadoop version these methods are supported or not. If the methods
-	 * are not supported, then nothing happens when setKeepContainersAcrossApplicationAttempts or
-	 * setAttemptFailuresValidityInterval are called.
-	 */
-	private static class ApplicationSubmissionContextReflector {
-		private static final Logger LOG = LoggerFactory.getLogger(ApplicationSubmissionContextReflector.class);
-
-		private static final ApplicationSubmissionContextReflector instance = new ApplicationSubmissionContextReflector(ApplicationSubmissionContext.class);
-
-		public static ApplicationSubmissionContextReflector getInstance() {
-			return instance;
-		}
-
-		private static final String keepContainersMethodName = "setKeepContainersAcrossApplicationAttempts";
-		private static final String attemptsFailuresValidityIntervalMethodName = "setAttemptFailuresValidityInterval";
-
-		private final Method keepContainersMethod;
-		private final Method attemptFailuresValidityIntervalMethod;
-
-		private ApplicationSubmissionContextReflector(Class<ApplicationSubmissionContext> clazz) {
-			Method keepContainersMethod;
-			Method attemptFailuresValidityIntervalMethod;
-
-			try {
-				// this method is only supported by Hadoop 2.4.0 onwards
-				keepContainersMethod = clazz.getMethod(keepContainersMethodName, boolean.class);
-				LOG.debug("{} supports method {}.", clazz.getCanonicalName(), keepContainersMethodName);
-			} catch (NoSuchMethodException e) {
-				LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), keepContainersMethodName);
-				// assign null because the Hadoop version apparently does not support this call.
-				keepContainersMethod = null;
-			}
-
-			this.keepContainersMethod = keepContainersMethod;
-
-			try {
-				// this method is only supported by Hadoop 2.6.0 onwards
-				attemptFailuresValidityIntervalMethod = clazz.getMethod(attemptsFailuresValidityIntervalMethodName, long.class);
-				LOG.debug("{} supports method {}.", clazz.getCanonicalName(), attemptsFailuresValidityIntervalMethodName);
-			} catch (NoSuchMethodException e) {
-				LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), attemptsFailuresValidityIntervalMethodName);
-				// assign null because the Hadoop version apparently does not support this call.
-				attemptFailuresValidityIntervalMethod = null;
-			}
-
-			this.attemptFailuresValidityIntervalMethod = attemptFailuresValidityIntervalMethod;
-		}
-
-		public void setKeepContainersAcrossApplicationAttempts(
-				ApplicationSubmissionContext appContext,
-				boolean keepContainers) throws InvocationTargetException, IllegalAccessException {
-
-			if (keepContainersMethod != null) {
-				LOG.debug("Calling method {} of {}.", keepContainersMethod.getName(),
-					appContext.getClass().getCanonicalName());
-				keepContainersMethod.invoke(appContext, keepContainers);
-			} else {
-				LOG.debug("{} does not support method {}. Doing nothing.",
-					appContext.getClass().getCanonicalName(), keepContainersMethodName);
-			}
-		}
-
-		public void setAttemptFailuresValidityInterval(
-				ApplicationSubmissionContext appContext,
-				long validityInterval) throws InvocationTargetException, IllegalAccessException {
-			if (attemptFailuresValidityIntervalMethod != null) {
-				LOG.debug("Calling method {} of {}.",
-					attemptFailuresValidityIntervalMethod.getName(),
-					appContext.getClass().getCanonicalName());
-				attemptFailuresValidityIntervalMethod.invoke(appContext, validityInterval);
-			} else {
-				LOG.debug("{} does not support method {}. Doing nothing.",
-					appContext.getClass().getCanonicalName(),
-					attemptsFailuresValidityIntervalMethodName);
-			}
-		}
-	}
-
-	public static class YarnDeploymentException extends RuntimeException {
-		private static final long serialVersionUID = -812040641215388943L;
-
-		public YarnDeploymentException() {
-		}
-
-		public YarnDeploymentException(String message) {
-			super(message);
-		}
-
-		public YarnDeploymentException(String message, Throwable cause) {
-			super(message, cause);
-		}
-	}
-
-	private class DeploymentFailureHook extends Thread {
-		@Override
-		public void run() {
-			LOG.info("Cancelling deployment from Deployment Failure Hook");
-			failSessionDuringDeployment();
-			LOG.info("Deleting files in " + sessionFilesDir);
-			try {
-				FileSystem fs = FileSystem.get(conf);
-				fs.delete(sessionFilesDir, true);
-				fs.close();
-			} catch (IOException e) {
-				LOG.error("Failed to delete Flink Jar and conf files in HDFS", e);
-			}
-		}
-	}
-}
-


[07/10] flink git commit: [FLINK-3667] refactor client communication classes

Posted by mx...@apache.org.
[FLINK-3667] refactor client communication classes

- ClusterDescriptor: base interface for cluster deployment descriptors
- ClusterDescriptor: YarnClusterDescriptor

- ClusterClient: base class for ClusterClients, handles lifecycle of cluster
- ClusterClient: shares configuration with the implementations
- ClusterClient: StandaloneClusterClient, YarnClusterClient
- ClusterClient: remove run methods and enable detached mode via flag

- CliFrontend: remove all Yarn specific logic
- CliFrontend: remove all cluster setup logic

- CustomCommandLine: interface for other cluster implementations
- Customcommandline: enables creation of new cluster or resuming from existing

- Yarn: move Yarn classes and functionality to the yarn module (yarn
  properties, yarn interfaces)
- Yarn: improve reliability of cluster startup
- Yarn Tests: only disable parallel execution of ITCases

This closes #1978


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

Branch: refs/heads/master
Commit: f9b52a3114a2114e6846091acf3abb294a49615b
Parents: efc344a
Author: Maximilian Michels <mx...@apache.org>
Authored: Fri Apr 22 19:52:54 2016 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Fri Jun 17 10:37:58 2016 +0200

----------------------------------------------------------------------
 .../api/avro/AvroExternalJarProgramITCase.java  |  15 +-
 .../org/apache/flink/client/CliFrontend.java    | 359 ++-----
 .../flink/client/FlinkYarnSessionCli.java       | 505 ----------
 .../org/apache/flink/client/RemoteExecutor.java |   9 +-
 .../flink/client/cli/CliFrontendParser.java     | 114 ++-
 .../flink/client/cli/CustomCommandLine.java     |  57 ++
 .../client/deployment/ClusterDescriptor.java    |  41 +
 .../org/apache/flink/client/program/Client.java | 624 ------------
 .../flink/client/program/ClusterClient.java     | 695 ++++++++++++++
 .../client/program/ContextEnvironment.java      |  12 +-
 .../program/ContextEnvironmentFactory.java      |  18 +-
 .../client/program/DetachedEnvironment.java     |   6 +-
 .../client/program/StandaloneClusterClient.java |  98 ++
 .../CliFrontendAddressConfigurationTest.java    | 125 +--
 .../client/CliFrontendPackageProgramTest.java   |   5 +-
 .../apache/flink/client/CliFrontendRunTest.java |  26 +-
 .../flink/client/CliFrontendTestUtils.java      |  32 +-
 .../TestingClusterClientWithoutActorSystem.java |  55 ++
 .../client/program/ClientConnectionTest.java    |   2 +-
 .../apache/flink/client/program/ClientTest.java |  33 +-
 .../program/ExecutionPlanCreationTest.java      |   2 +-
 .../org/apache/flink/storm/api/FlinkClient.java |  11 +-
 .../flink/api/common/JobExecutionResult.java    |   3 +
 .../flink/api/common/JobSubmissionResult.java   |  24 +-
 .../main/flink-bin/conf/log4j-cli.properties    |   2 +-
 .../src/main/flink-bin/yarn-bin/yarn-session.sh |   2 +-
 .../operations/DegreesWithExceptionITCase.java  |   2 +-
 .../ReduceOnEdgesWithExceptionITCase.java       |   2 +-
 .../ReduceOnNeighborsWithExceptionITCase.java   |   2 +-
 .../webmonitor/handlers/JarActionHandler.java   |   4 +-
 .../apache/flink/runtime/client/JobClient.java  |  17 +-
 .../clusterframework/ApplicationStatus.java     |   1 +
 .../clusterframework/FlinkResourceManager.java  |   2 +-
 .../messages/GetClusterStatusResponse.java      |   2 +-
 .../runtime/yarn/AbstractFlinkYarnClient.java   | 143 ---
 .../runtime/yarn/AbstractFlinkYarnCluster.java  | 123 ---
 .../org/apache/flink/api/scala/FlinkShell.scala |  82 +-
 .../flink/api/scala/ExecutionEnvironment.scala  |   2 +-
 .../elasticsearch2/ElasticsearchSinkITCase.java |   2 +-
 .../environment/RemoteStreamEnvironment.java    |   9 +-
 .../environment/StreamContextEnvironment.java   |   5 +-
 .../RemoteEnvironmentITCase.java                |   2 +-
 .../flink/test/misc/AutoParallelismITCase.java  |   2 +-
 .../test/recovery/SimpleRecoveryITCase.java     |   2 +-
 flink-yarn-tests/pom.xml                        |  15 +-
 ...CliFrontendYarnAddressConfigurationTest.java | 220 +++++
 .../flink/yarn/FlinkYarnSessionCliTest.java     |  14 +-
 .../flink/yarn/TestingFlinkYarnClient.java      |  71 --
 .../yarn/TestingYarnClusterDescriptor.java      |  71 ++
 .../flink/yarn/YARNHighAvailabilityITCase.java  |   9 +-
 .../YARNSessionCapacitySchedulerITCase.java     |   6 +-
 .../flink/yarn/YARNSessionFIFOITCase.java       |  20 +-
 .../org/apache/flink/yarn/YarnTestBase.java     |   4 +-
 .../yarn/AbstractYarnClusterDescriptor.java     | 943 +++++++++++++++++++
 .../org/apache/flink/yarn/FlinkYarnClient.java  |  28 -
 .../apache/flink/yarn/FlinkYarnClientBase.java  | 907 ------------------
 .../org/apache/flink/yarn/FlinkYarnCluster.java | 559 -----------
 .../flink/yarn/YarnApplicationMasterRunner.java |   7 +-
 .../apache/flink/yarn/YarnClusterClient.java    | 577 ++++++++++++
 .../flink/yarn/YarnClusterDescriptor.java       |  28 +
 .../flink/yarn/cli/FlinkYarnSessionCli.java     | 606 ++++++++++++
 .../apache/flink/yarn/ApplicationClient.scala   |   8 +-
 .../org/apache/flink/yarn/YarnMessages.scala    |   7 +-
 63 files changed, 3799 insertions(+), 3580 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
index ac10074..29a7e58 100644
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
+++ b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
@@ -19,19 +19,12 @@
 package org.apache.flink.api.avro;
 
 import java.io.File;
-import java.net.InetAddress;
 
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.client.CliFrontend;
-import org.apache.flink.client.RemoteExecutor;
-import org.apache.flink.client.program.Client;
-import org.apache.flink.client.program.JobWithJars;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.PackagedProgram;
+import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.plan.FlinkPlan;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 
 import org.junit.Assert;
@@ -64,10 +57,10 @@ public class AvroExternalJarProgramITCase {
 			config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
 			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, testMiniCluster.getLeaderRPCPort());
 
-			Client client = new Client(config);
+			ClusterClient client = new StandaloneClusterClient(config);
 
 			client.setPrintStatusDuringExecution(false);
-			client.runBlocking(program, 4);
+			client.run(program, 4);
 
 		}
 		catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index 6d972bc..cf7a8c2 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -20,8 +20,6 @@ package org.apache.flink.client;
 
 import akka.actor.ActorSystem;
 
-import org.apache.commons.cli.CommandLine;
-
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
@@ -31,18 +29,21 @@ import org.apache.flink.client.cli.CancelOptions;
 import org.apache.flink.client.cli.CliArgsException;
 import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.cli.CommandLineOptions;
+import org.apache.flink.client.cli.CustomCommandLine;
 import org.apache.flink.client.cli.InfoOptions;
 import org.apache.flink.client.cli.ListOptions;
 import org.apache.flink.client.cli.ProgramOptions;
 import org.apache.flink.client.cli.RunOptions;
 import org.apache.flink.client.cli.SavepointOptions;
 import org.apache.flink.client.cli.StopOptions;
-import org.apache.flink.client.program.Client;
+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.client.program.StandaloneClusterClient;
 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.core.fs.FileSystem;
 import org.apache.flink.optimizer.DataStatistics;
 import org.apache.flink.optimizer.Optimizer;
@@ -53,7 +54,6 @@ import org.apache.flink.optimizer.plan.StreamingPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.client.JobStatusMessage;
-import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
@@ -68,8 +68,6 @@ import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSucc
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
 import org.apache.flink.util.StringUtils;
 
 import org.slf4j.Logger;
@@ -81,10 +79,8 @@ import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.net.URL;
 import java.text.SimpleDateFormat;
@@ -93,10 +89,8 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint;
@@ -121,20 +115,6 @@ public class CliFrontend {
 	private static final String CONFIG_DIRECTORY_FALLBACK_1 = "../conf";
 	private static final String CONFIG_DIRECTORY_FALLBACK_2 = "conf";
 
-	// YARN-session related constants
-	public static final String YARN_PROPERTIES_FILE = ".yarn-properties-";
-	public static final String YARN_PROPERTIES_JOBMANAGER_KEY = "jobManager";
-	public static final String YARN_PROPERTIES_PARALLELISM = "parallelism";
-	public static final String YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING = "dynamicPropertiesString";
-
-	public static final String YARN_DYNAMIC_PROPERTIES_SEPARATOR = "@@"; // this has to be a regex for String.split()
-
-	/**
-	 * A special host name used to run a job by deploying Flink into a YARN cluster,
-	 * if this string is specified as the JobManager address
-	 */
-	public static final String YARN_DEPLOY_JOBMANAGER = "yarn-cluster";
-
 	// --------------------------------------------------------------------------------------------
 	// --------------------------------------------------------------------------------------------
 
@@ -149,12 +129,9 @@ public class CliFrontend {
 
 	private ActorSystem actorSystem;
 
-	private AbstractFlinkYarnCluster yarnCluster;
-
 	/**
 	 *
-	 * @throws Exception Thrown if the configuration directory was not found, the configuration could not
-	 *                   be loaded, or the YARN properties could not be parsed.
+	 * @throws Exception Thrown if the configuration directory was not found, the configuration could not be loaded
 	 */
 	public CliFrontend() throws Exception {
 		this(getConfigurationDirectoryFromEnv());
@@ -171,61 +148,6 @@ public class CliFrontend {
 		GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
 		this.config = GlobalConfiguration.getConfiguration();
 
-		// load the YARN properties
-		File propertiesFile = new File(getYarnPropertiesLocation(config));
-		if (propertiesFile.exists()) {
-
-			logAndSysout("Found YARN properties file " + propertiesFile.getAbsolutePath());
-
-			Properties yarnProperties = new Properties();
-			try {
-				try (InputStream is = new FileInputStream(propertiesFile)) {
-					yarnProperties.load(is);
-				}
-			}
-			catch (IOException e) {
-				throw new Exception("Cannot read the YARN properties file", e);
-			}
-
-			// 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);
-					this.config.setInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, parallelism);
-
-					logAndSysout("YARN properties set default parallelism to " + parallelism);
-				}
-				catch (NumberFormatException e) {
-					throw new Exception("Error while parsing the YARN properties: " +
-							"Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer.");
-				}
-			}
-
-			// get the JobManager address from the YARN properties
-			String address = yarnProperties.getProperty(YARN_PROPERTIES_JOBMANAGER_KEY);
-			InetSocketAddress jobManagerAddress;
-			if (address != null) {
-				try {
-					jobManagerAddress = ClientUtils.parseHostPortAddress(address);
-					// store address in config from where it is retrieved by the retrieval service
-					writeJobManagerAddressToConfig(jobManagerAddress);
-				}
-				catch (Exception e) {
-					throw new Exception("YARN properties contain an invalid entry for JobManager address.", e);
-				}
-
-				logAndSysout("Using JobManager address from YARN properties " + jobManagerAddress);
-			}
-
-			// 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()) {
-				this.config.setString(dynamicProperty.getKey(), dynamicProperty.getValue());
-			}
-		}
-
 		try {
 			FileSystem.setDefaultScheme(config);
 		} catch (IOException e) {
@@ -301,61 +223,33 @@ public class CliFrontend {
 			return handleError(t);
 		}
 
-		int exitCode = 1;
+		ClusterClient client = null;
 		try {
-			int userParallelism = options.getParallelism();
-			LOG.debug("User parallelism is set to {}", userParallelism);
 
-			Client client = getClient(options, program.getMainClassName(), userParallelism, options.getDetachedMode());
+			client = getClient(options, program.getMainClassName());
 			client.setPrintStatusDuringExecution(options.getStdoutLogging());
+			client.setDetached(options.getDetachedMode());
 			LOG.debug("Client slots is set to {}", client.getMaxSlots());
 
 			LOG.debug("Savepoint path is set to {}", options.getSavepointPath());
 
-			try {
-				if (client.getMaxSlots() != -1 && userParallelism == -1) {
-					logAndSysout("Using the parallelism provided by the remote cluster ("+client.getMaxSlots()+"). " +
-							"To use another parallelism, set it at the ./bin/flink client.");
-					userParallelism = client.getMaxSlots();
-				}
-
-				// detached mode
-				if (options.getDetachedMode() || (yarnCluster != null && yarnCluster.isDetached())) {
-					exitCode = executeProgramDetached(program, client, userParallelism);
-				}
-				else {
-					exitCode = executeProgramBlocking(program, client, userParallelism);
-				}
-
-				// show YARN cluster status if its not a detached YARN cluster.
-				if (yarnCluster != null && !yarnCluster.isDetached()) {
-					List<String> msgs = yarnCluster.getNewMessages();
-					if (msgs != null && msgs.size() > 1) {
-
-						logAndSysout("The following messages were created by the YARN cluster while running the Job:");
-						for (String msg : msgs) {
-							logAndSysout(msg);
-						}
-					}
-					if (yarnCluster.hasFailed()) {
-						logAndSysout("YARN cluster is in failed state!");
-						logAndSysout("YARN Diagnostics: " + yarnCluster.getDiagnostics());
-					}
-				}
-
-				return exitCode;
-			}
-			finally {
-				client.shutdown();
+			int userParallelism = options.getParallelism();
+			LOG.debug("User parallelism is set to {}", userParallelism);
+			if (client.getMaxSlots() != -1 && userParallelism == -1) {
+				logAndSysout("Using the parallelism provided by the remote cluster ("
+					+ client.getMaxSlots()+"). "
+					+ "To use another parallelism, set it at the ./bin/flink client.");
+				userParallelism = client.getMaxSlots();
 			}
+
+			return executeProgram(program, client, userParallelism);
 		}
 		catch (Throwable t) {
 			return handleError(t);
 		}
 		finally {
-			if (yarnCluster != null && !yarnCluster.isDetached()) {
-				logAndSysout("Shutting down YARN cluster");
-				yarnCluster.shutdown(exitCode != 0);
+			if (client != null) {
+				client.shutdown();
 			}
 			if (program != null) {
 				program.deleteExtractedLibraries();
@@ -410,7 +304,7 @@ public class CliFrontend {
 			LOG.info("Creating program plan dump");
 
 			Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config);
-			FlinkPlan flinkPlan = Client.getOptimizedPlan(compiler, program, parallelism);
+			FlinkPlan flinkPlan = ClusterClient.getOptimizedPlan(compiler, program, parallelism);
 			
 			String jsonPlan = null;
 			if (flinkPlan instanceof OptimizedPlan) {
@@ -830,53 +724,30 @@ public class CliFrontend {
 	//  Interaction with programs and JobManager
 	// --------------------------------------------------------------------------------------------
 
-	protected int executeProgramDetached(PackagedProgram program, Client client, int parallelism) {
-		LOG.info("Starting execution of program");
+	protected int executeProgram(PackagedProgram program, ClusterClient client, int parallelism) {
+		logAndSysout("Starting execution of program");
 
 		JobSubmissionResult result;
 		try {
-			result = client.runDetached(program, parallelism);
+			result = client.run(program, parallelism);
 		} catch (ProgramInvocationException e) {
 			return handleError(e);
 		} finally {
 			program.deleteExtractedLibraries();
 		}
 
-		if (yarnCluster != null) {
-			yarnCluster.stopAfterJob(result.getJobID());
-			yarnCluster.disconnect();
-		}
-		
-		System.out.println("Job has been submitted with JobID " + result.getJobID());
-
-		return 0;
-	}
-
-	protected int executeProgramBlocking(PackagedProgram program, Client client, int parallelism) {
-		LOG.info("Starting execution of program");
-
-		JobSubmissionResult result;
-		try {
-			result = client.runBlocking(program, parallelism);
-		}
-		catch (ProgramInvocationException e) {
-			return handleError(e);
-		}
-		finally {
-			program.deleteExtractedLibraries();
-		}
-
-		LOG.info("Program execution finished");
-
-		if (result instanceof JobExecutionResult) {
-			JobExecutionResult execResult = (JobExecutionResult) result;
+		if(result.isJobExecutionResults()) {
+			logAndSysout("Program execution finished");
+			JobExecutionResult execResult = result.getJobExecutionResult();
 			System.out.println("Job with JobID " + execResult.getJobID() + " has finished.");
 			System.out.println("Job Runtime: " + execResult.getNetRuntime() + " ms");
 			Map<String, Object> accumulatorsResult = execResult.getAllAccumulatorResults();
 			if (accumulatorsResult.size() > 0) {
-					System.out.println("Accumulator Results: ");
-					System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult));
+				System.out.println("Accumulator Results: ");
+				System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult));
 			}
+		} else {
+			logAndSysout("Job has been submitted with JobID " + result.getJobID());
 		}
 
 		return 0;
@@ -923,16 +794,6 @@ public class CliFrontend {
 	}
 
 	/**
-	 * Writes the given job manager address to the associated configuration object
-	 *
-	 * @param address Address to write to the configuration
-	 */
-	protected void writeJobManagerAddressToConfig(InetSocketAddress address) {
-		config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.getHostName());
-		config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.getPort());
-	}
-
-	/**
 	 * Updates the associated configuration with the given command line options
 	 *
 	 * @param options Command line options
@@ -940,7 +801,7 @@ public class CliFrontend {
 	protected void updateConfig(CommandLineOptions options) {
 		if(options.getJobManagerAddress() != null){
 			InetSocketAddress jobManagerAddress = ClientUtils.parseHostPortAddress(options.getJobManagerAddress());
-			writeJobManagerAddressToConfig(jobManagerAddress);
+			writeJobManagerAddressToConfig(config, jobManagerAddress);
 		}
 	}
 
@@ -980,110 +841,65 @@ public class CliFrontend {
 	}
 
 	/**
-	 * Retrieves a {@link Client} object from the given command line options and other parameters.
+	 * Retrieves a {@link ClusterClient} object from the given command line options and other parameters.
 	 *
 	 * @param options Command line options which contain JobManager address
 	 * @param programName Program name
-	 * @param userParallelism Given user parallelism
 	 * @throws Exception
 	 */
-	protected Client getClient(
+	protected ClusterClient getClient(
 			CommandLineOptions options,
-			String programName,
-			int userParallelism,
-			boolean detachedMode)
+			String programName)
 		throws Exception {
 		InetSocketAddress jobManagerAddress;
-		int maxSlots = -1;
 
-		if (YARN_DEPLOY_JOBMANAGER.equals(options.getJobManagerAddress())) {
-			logAndSysout("YARN cluster mode detected. Switching Log4j output to console");
+		// try to get the JobManager address via command-line args
+		if (options.getJobManagerAddress() != null) {
 
-			// Default yarn application name to use, if nothing is specified on the command line
-			String applicationName = "Flink Application: " + programName;
+			// Get the custom command-lines (e.g. Yarn/Mesos)
+			CustomCommandLine<?> activeCommandLine =
+				CliFrontendParser.getActiveCustomCommandLine(options.getJobManagerAddress());
 
-			// user wants to run Flink in YARN cluster.
-			CommandLine commandLine = options.getCommandLine();
-			AbstractFlinkYarnClient flinkYarnClient = CliFrontendParser
-														.getFlinkYarnSessionCli()
-														.withDefaultApplicationName(applicationName)
-														.createFlinkYarnClient(commandLine);
+			if (activeCommandLine != null) {
+				logAndSysout(activeCommandLine.getIdentifier() + " mode detected. Switching Log4j output to console");
 
-			if (flinkYarnClient == null) {
-				throw new RuntimeException("Unable to create Flink YARN Client. Check previous log messages");
-			}
+				// Default yarn application name to use, if nothing is specified on the command line
+				String applicationName = "Flink Application: " + programName;
 
-			// in case the main detached mode wasn't set, we don't wanna overwrite the one loaded
-			// from yarn options.
-			if (detachedMode) {
-				flinkYarnClient.setDetachedMode(true);
-			}
+				ClusterClient client = activeCommandLine.createClient(applicationName, options.getCommandLine());
 
-			// the number of slots available from YARN:
-			int yarnTmSlots = flinkYarnClient.getTaskManagerSlots();
-			if (yarnTmSlots == -1) {
-				yarnTmSlots = 1;
-			}
-			maxSlots = yarnTmSlots * flinkYarnClient.getTaskManagerCount();
-			if (userParallelism != -1) {
-				int slotsPerTM = userParallelism / flinkYarnClient.getTaskManagerCount();
-				logAndSysout("The YARN cluster has " + maxSlots + " slots available, " +
-						"but the user requested a parallelism of " + userParallelism + " on YARN. " +
-						"Each of the " + flinkYarnClient.getTaskManagerCount() + " TaskManagers " +
-						"will get "+slotsPerTM+" slots.");
-				flinkYarnClient.setTaskManagerSlots(slotsPerTM);
-			}
+				logAndSysout("Cluster started");
+				logAndSysout("JobManager web interface address " + client.getWebInterfaceURL());
 
-			try {
-				yarnCluster = flinkYarnClient.deploy();
-				yarnCluster.connectToCluster();
-			}
-			catch (Exception e) {
-				throw new RuntimeException("Error deploying the YARN cluster", e);
+				return client;
+			} else {
+				// job manager address supplied on the command-line
+				LOG.info("Using address {} to connect to JobManager.", options.getJobManagerAddress());
+				jobManagerAddress = ClientUtils.parseHostPortAddress(options.getJobManagerAddress());
+				writeJobManagerAddressToConfig(config, jobManagerAddress);
+				return new StandaloneClusterClient(config);
 			}
 
-			jobManagerAddress = yarnCluster.getJobManagerAddress();
-			writeJobManagerAddressToConfig(jobManagerAddress);
-			
-			// overwrite the yarn client config (because the client parses the dynamic properties)
-			this.config.addAll(flinkYarnClient.getFlinkConfiguration());
-
-			logAndSysout("YARN cluster started");
-			logAndSysout("JobManager web interface address " + yarnCluster.getWebInterfaceURL());
-			logAndSysout("Waiting until all TaskManagers have connected");
-
-			while(true) {
-				GetClusterStatusResponse status = yarnCluster.getClusterStatus();
-				if (status != null) {
-					if (status.numRegisteredTaskManagers() < flinkYarnClient.getTaskManagerCount()) {
-						logAndSysout("TaskManager status (" + status.numRegisteredTaskManagers() + "/"
-							+ flinkYarnClient.getTaskManagerCount() + ")");
-					} else {
-						logAndSysout("All TaskManagers are connected");
-						break;
-					}
-				} else {
-					logAndSysout("No status updates from the YARN cluster received so far. Waiting ...");
-				}
-
-				try {
-					Thread.sleep(500);
-				}
-				catch (InterruptedException e) {
-					LOG.error("Interrupted while waiting for TaskManagers");
-					System.err.println("Thread is interrupted");
-					Thread.currentThread().interrupt();
+		// try to get the JobManager address via resuming of a cluster
+		} else {
+			for (CustomCommandLine cli : CliFrontendParser.getAllCustomCommandLine().values()) {
+				ClusterClient client = cli.retrieveCluster(config);
+				if (client != null) {
+					LOG.info("Using address {} to connect to JobManager.", client.getJobManagerAddressFromConfig());
+					return client;
 				}
 			}
 		}
-		else {
-			if(options.getJobManagerAddress() != null) {
-				jobManagerAddress = ClientUtils.parseHostPortAddress(options.getJobManagerAddress());
-				writeJobManagerAddressToConfig(jobManagerAddress);
-			}
-		}
 
-		return new Client(config, maxSlots);
+		// read JobManager address from the config
+		if (config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null) != null) {
+			return new StandaloneClusterClient(config);
+		// We tried hard but couldn't find a JobManager address
+		} else {
+			throw new IllegalConfigurationException(
+				"The JobManager address is neither provided at the command-line, " +
+					"nor configured in flink-conf.yaml.");
+		}
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -1275,33 +1091,16 @@ public class CliFrontend {
 		return location;
 	}
 
-	public static Map<String, String> getDynamicProperties(String dynamicPropertiesEncoded) {
-		if (dynamicPropertiesEncoded != null && dynamicPropertiesEncoded.length() > 0) {
-			Map<String, String> properties = new HashMap<>();
-			
-			String[] propertyLines = dynamicPropertiesEncoded.split(CliFrontend.YARN_DYNAMIC_PROPERTIES_SEPARATOR);
-			for (String propLine : propertyLines) {
-				if (propLine == null) {
-					continue;
-				}
-				
-				String[] kv = propLine.split("=");
-				if (kv.length >= 2 && kv[0] != null && kv[1] != null && kv[0].length() > 0) {
-					properties.put(kv[0], kv[1]);
-				}
-			}
-			return properties;
-		}
-		else {
-			return Collections.emptyMap();
-		}
-	}
-
-	public static String getYarnPropertiesLocation(Configuration conf) {
-		String defaultPropertiesFileLocation = System.getProperty("java.io.tmpdir");
-		String currentUser = System.getProperty("user.name");
-		String propertiesFileLocation = conf.getString(ConfigConstants.YARN_PROPERTIES_FILE_LOCATION, defaultPropertiesFileLocation);
 
-		return propertiesFileLocation + File.separator + CliFrontend.YARN_PROPERTIES_FILE + currentUser;
+	/**
+	 * Writes the given job manager address to the associated configuration object
+	 *
+	 * @param address Address to write to the configuration
+	 * @param config The config to write to
+	 */
+	public static void writeJobManagerAddressToConfig(Configuration config, InetSocketAddress address) {
+		config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.getHostName());
+		config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.getPort());
 	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java
deleted file mode 100644
index bb61ffb..0000000
--- a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java
+++ /dev/null
@@ -1,505 +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.client;
-
-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.flink.configuration.Configuration;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-
-/**
- * Class handling the command line interface to the YARN session.
- */
-public class FlinkYarnSessionCli {
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnSessionCli.class);
-
-	//------------------------------------ Constants   -------------------------
-
-	private static final String CONFIG_FILE_NAME = "flink-conf.yaml";
-	public static final String CONFIG_FILE_LOGBACK_NAME = "logback.xml";
-	public static final String CONFIG_FILE_LOG4J_NAME = "log4j.properties";
-
-	private static final int CLIENT_POLLING_INTERVALL = 3;
-
-
-	//------------------------------------ Command Line argument options -------------------------
-	// the prefix transformation is used by the CliFrontend static constructor.
-	private final Option QUERY;
-	// --- or ---
-	private final Option QUEUE;
-	private final Option SHIP_PATH;
-	private final Option FLINK_JAR;
-	private final Option JM_MEMORY;
-	private final Option TM_MEMORY;
-	private final Option CONTAINER;
-	private final Option SLOTS;
-	private final Option DETACHED;
-	private final Option STREAMING;
-	private final Option NAME;
-
-	/**
-	 * Dynamic properties allow the user to specify additional configuration values with -D, such as
-	 *  -Dfs.overwrite-files=true  -Dtaskmanager.network.numberOfBuffers=16368
-	 */
-	private final Option DYNAMIC_PROPERTIES;
-
-	private final boolean acceptInteractiveInput;
-	
-	//------------------------------------ Internal fields -------------------------
-	private AbstractFlinkYarnCluster yarnCluster = null;
-	private boolean detachedMode = false;
-
-	/** Default yarn application name. */
-	private String defaultApplicationName = null;
-
-	public FlinkYarnSessionCli(String shortPrefix, String longPrefix, boolean acceptInteractiveInput) {
-		this.acceptInteractiveInput = acceptInteractiveInput;
-		
-		QUERY = new Option(shortPrefix + "q", longPrefix + "query", false, "Display available YARN resources (memory, cores)");
-		QUEUE = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue.");
-		SHIP_PATH = new Option(shortPrefix + "t", longPrefix + "ship", true, "Ship files in the specified directory (t for transfer)");
-		FLINK_JAR = new Option(shortPrefix + "j", longPrefix + "jar", true, "Path to Flink jar file");
-		JM_MEMORY = new Option(shortPrefix + "jm", longPrefix + "jobManagerMemory", true, "Memory for JobManager Container [in MB]");
-		TM_MEMORY = 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");
-		DYNAMIC_PROPERTIES = new Option(shortPrefix + "D", true, "Dynamic properties");
-		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");
-	}
-
-	/**
-	 * Creates a new Yarn Client.
-	 * @param cmd the command line to parse options from
-	 * @return an instance of the client or null if there was an error
-	 */
-	public AbstractFlinkYarnClient createFlinkYarnClient(CommandLine cmd) {
-
-		AbstractFlinkYarnClient flinkYarnClient = getFlinkYarnClient();
-		if (flinkYarnClient == null) {
-			return null;
-		}
-
-		if (!cmd.hasOption(CONTAINER.getOpt())) { // number of containers is required option!
-			LOG.error("Missing required argument " + CONTAINER.getOpt());
-			printUsage();
-			return null;
-		}
-		flinkYarnClient.setTaskManagerCount(Integer.valueOf(cmd.getOptionValue(CONTAINER.getOpt())));
-
-		// Jar Path
-		Path localJarPath;
-		if (cmd.hasOption(FLINK_JAR.getOpt())) {
-			String userPath = cmd.getOptionValue(FLINK_JAR.getOpt());
-			if(!userPath.startsWith("file://")) {
-				userPath = "file://" + userPath;
-			}
-			localJarPath = new Path(userPath);
-		} else {
-			LOG.info("No path for the flink jar passed. Using the location of "+flinkYarnClient.getClass()+" to locate the jar");
-			localJarPath = new Path("file://"+flinkYarnClient.getClass().getProtectionDomain().getCodeSource().getLocation().getPath());
-		}
-
-		flinkYarnClient.setLocalJarPath(localJarPath);
-
-		// Conf Path
-		String confDirPath = CliFrontend.getConfigurationDirectoryFromEnv();
-		GlobalConfiguration.loadConfiguration(confDirPath);
-		Configuration flinkConfiguration = GlobalConfiguration.getConfiguration();
-		flinkYarnClient.setFlinkConfiguration(flinkConfiguration);
-		flinkYarnClient.setConfigurationDirectory(confDirPath);
-		File confFile = new File(confDirPath + File.separator + CONFIG_FILE_NAME);
-		if (!confFile.exists()) {
-			LOG.error("Unable to locate configuration file in "+confFile);
-			return null;
-		}
-		Path confPath = new Path(confFile.getAbsolutePath());
-
-		flinkYarnClient.setConfigurationFilePath(confPath);
-
-		List<File> shipFiles = new ArrayList<>();
-		// path to directory to ship
-		if (cmd.hasOption(SHIP_PATH.getOpt())) {
-			String shipPath = cmd.getOptionValue(SHIP_PATH.getOpt());
-			File shipDir = new File(shipPath);
-			if (shipDir.isDirectory()) {
-				shipFiles = new ArrayList<>(Arrays.asList(shipDir.listFiles(new FilenameFilter() {
-					@Override
-					public boolean accept(File dir, String name) {
-						return !(name.equals(".") || name.equals(".."));
-					}
-				})));
-			} else {
-				LOG.warn("Ship directory is not a directory. Ignoring it.");
-			}
-		}
-
-		//check if there is a logback or log4j file
-		if (confDirPath.length() > 0) {
-			File logback = new File(confDirPath + File.pathSeparator + CONFIG_FILE_LOGBACK_NAME);
-			if (logback.exists()) {
-				shipFiles.add(logback);
-				flinkYarnClient.setFlinkLoggingConfigurationPath(new Path(logback.toURI()));
-			}
-			File log4j = new File(confDirPath + File.pathSeparator + CONFIG_FILE_LOG4J_NAME);
-			if (log4j.exists()) {
-				shipFiles.add(log4j);
-				if (flinkYarnClient.getFlinkLoggingConfigurationPath() != null) {
-					// this means there is already a logback configuration file --> fail
-					LOG.warn("The configuration directory ('" + confDirPath + "') contains both LOG4J and " +
-							"Logback configuration files. Please delete or rename one of them.");
-				} // else
-				flinkYarnClient.setFlinkLoggingConfigurationPath(new Path(log4j.toURI()));
-			}
-		}
-
-		flinkYarnClient.setShipFiles(shipFiles);
-
-		// queue
-		if (cmd.hasOption(QUEUE.getOpt())) {
-			flinkYarnClient.setQueue(cmd.getOptionValue(QUEUE.getOpt()));
-		}
-
-		// JobManager Memory
-		if (cmd.hasOption(JM_MEMORY.getOpt())) {
-			int jmMemory = Integer.valueOf(cmd.getOptionValue(JM_MEMORY.getOpt()));
-			flinkYarnClient.setJobManagerMemory(jmMemory);
-		}
-
-		// Task Managers memory
-		if (cmd.hasOption(TM_MEMORY.getOpt())) {
-			int tmMemory = Integer.valueOf(cmd.getOptionValue(TM_MEMORY.getOpt()));
-			flinkYarnClient.setTaskManagerMemory(tmMemory);
-		}
-
-		if (cmd.hasOption(SLOTS.getOpt())) {
-			int slots = Integer.valueOf(cmd.getOptionValue(SLOTS.getOpt()));
-			flinkYarnClient.setTaskManagerSlots(slots);
-		}
-
-		String[] dynamicProperties = null;
-		if (cmd.hasOption(DYNAMIC_PROPERTIES.getOpt())) {
-			dynamicProperties = cmd.getOptionValues(DYNAMIC_PROPERTIES.getOpt());
-		}
-		String dynamicPropertiesEncoded = StringUtils.join(dynamicProperties,
-				CliFrontend.YARN_DYNAMIC_PROPERTIES_SEPARATOR);
-
-		flinkYarnClient.setDynamicPropertiesEncoded(dynamicPropertiesEncoded);
-
-		if (cmd.hasOption(DETACHED.getOpt())) {
-			this.detachedMode = true;
-			flinkYarnClient.setDetachedMode(detachedMode);
-		}
-
-		if(cmd.hasOption(NAME.getOpt())) {
-			flinkYarnClient.setName(cmd.getOptionValue(NAME.getOpt()));
-		} else {
-			// set the default application name, if none is specified
-			if(defaultApplicationName != null) {
-				flinkYarnClient.setName(defaultApplicationName);
-			}
-		}
-
-		return flinkYarnClient;
-	}
-
-
-	private void printUsage() {
-		System.out.println("Usage:");
-		HelpFormatter formatter = new HelpFormatter();
-		formatter.setWidth(200);
-		formatter.setLeftPadding(5);
-		formatter.setSyntaxPrefix("   Required");
-		Options req = new Options();
-		req.addOption(CONTAINER);
-		formatter.printHelp(" ", req);
-
-		formatter.setSyntaxPrefix("   Optional");
-		Options opt = new Options();
-		opt.addOption(JM_MEMORY);
-		opt.addOption(TM_MEMORY);
-		opt.addOption(QUERY);
-		opt.addOption(QUEUE);
-		opt.addOption(SLOTS);
-		opt.addOption(DYNAMIC_PROPERTIES);
-		opt.addOption(DETACHED);
-		opt.addOption(STREAMING);
-		opt.addOption(NAME);
-		formatter.printHelp(" ", opt);
-	}
-
-	public static AbstractFlinkYarnClient getFlinkYarnClient() {
-		AbstractFlinkYarnClient yarnClient;
-		try {
-			Class<? extends AbstractFlinkYarnClient> yarnClientClass =
-					Class.forName("org.apache.flink.yarn.FlinkYarnClient").asSubclass(AbstractFlinkYarnClient.class);
-			yarnClient = InstantiationUtil.instantiate(yarnClientClass, AbstractFlinkYarnClient.class);
-		}
-		catch (ClassNotFoundException e) {
-			System.err.println("Unable to locate the Flink YARN Client. " +
-					"Please ensure that you are using a Flink build with Hadoop2/YARN support. Message: " +
-					e.getMessage());
-			e.printStackTrace(System.err);
-			return null; // make it obvious
-		}
-		return yarnClient;
-	}
-
-	private static void writeYarnProperties(Properties properties, File propertiesFile) {
-		try {
-			OutputStream out = new FileOutputStream(propertiesFile);
-			properties.store(out, "Generated YARN properties file");
-			out.close();
-		} catch (IOException e) {
-			throw new RuntimeException("Error writing the properties file", e);
-		}
-		propertiesFile.setReadable(true, false); // readable for all.
-	}
-
-	public static void runInteractiveCli(AbstractFlinkYarnCluster yarnCluster, boolean readConsoleInput) {
-		final String HELP = "Available commands:\n" +
-				"help - show these commands\n" +
-				"stop - stop the YARN session";
-		int numTaskmanagers = 0;
-		try {
-			BufferedReader in = new BufferedReader(new InputStreamReader(System.in));
-			label:
-			while (true) {
-				// ------------------ check if there are updates by the cluster -----------
-
-				GetClusterStatusResponse status = yarnCluster.getClusterStatus();
-				LOG.debug("Received status message: {}", status);
-
-				if (status != null && numTaskmanagers != status.numRegisteredTaskManagers()) {
-					System.err.println("Number of connected TaskManagers changed to " +
-							status.numRegisteredTaskManagers() + ". " +
-						"Slots available: " + status.totalNumberOfSlots());
-					numTaskmanagers = status.numRegisteredTaskManagers();
-				}
-
-				List<String> messages = yarnCluster.getNewMessages();
-				if (messages != null && messages.size() > 0) {
-					System.err.println("New messages from the YARN cluster: ");
-					for (String msg : messages) {
-						System.err.println(msg);
-					}
-				}
-
-				if (yarnCluster.hasFailed()) {
-					System.err.println("The YARN cluster has failed");
-					yarnCluster.shutdown(true);
-				}
-
-				// wait until CLIENT_POLLING_INTERVAL is over or the user entered something.
-				long startTime = System.currentTimeMillis();
-				while ((System.currentTimeMillis() - startTime) < CLIENT_POLLING_INTERVALL * 1000
-						&& (!readConsoleInput || !in.ready()))
-				{
-					Thread.sleep(200);
-				}
-				//------------- handle interactive command by user. ----------------------
-				
-				if (readConsoleInput && in.ready()) {
-					String command = in.readLine();
-					switch (command) {
-						case "quit":
-						case "stop":
-							break label;
-
-						case "help":
-							System.err.println(HELP);
-							break;
-						default:
-							System.err.println("Unknown command '" + command + "'. Showing help: \n" + HELP);
-							break;
-					}
-				}
-				
-				if (yarnCluster.hasBeenStopped()) {
-					LOG.info("Stopping interactive command line interface, YARN cluster has been stopped.");
-					break;
-				}
-			}
-		} catch(Exception e) {
-			LOG.warn("Exception while running the interactive command line interface", e);
-		}
-	}
-
-	public static void main(String[] args) {
-		FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", "", true); // no prefix for the YARN session
-		System.exit(cli.run(args));
-	}
-
-	public void getYARNSessionCLIOptions(Options options) {
-		options.addOption(FLINK_JAR);
-		options.addOption(JM_MEMORY);
-		options.addOption(TM_MEMORY);
-		options.addOption(CONTAINER);
-		options.addOption(QUEUE);
-		options.addOption(QUERY);
-		options.addOption(SHIP_PATH);
-		options.addOption(SLOTS);
-		options.addOption(DYNAMIC_PROPERTIES);
-		options.addOption(DETACHED);
-		options.addOption(STREAMING);
-		options.addOption(NAME);
-	}
-
-	public int run(String[] args) {
-		//
-		//	Command Line Options
-		//
-		Options options = new Options();
-		getYARNSessionCLIOptions(options);
-
-		CommandLineParser parser = new PosixParser();
-		CommandLine cmd;
-		try {
-			cmd = parser.parse(options, args);
-		} catch(Exception e) {
-			System.out.println(e.getMessage());
-			printUsage();
-			return 1;
-		}
-		
-		// Query cluster for metrics
-		if (cmd.hasOption(QUERY.getOpt())) {
-			AbstractFlinkYarnClient flinkYarnClient = getFlinkYarnClient();
-			String description;
-			try {
-				description = flinkYarnClient.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 {
-			AbstractFlinkYarnClient flinkYarnClient = createFlinkYarnClient(cmd);
-
-			if (flinkYarnClient == null) {
-				System.err.println("Error while starting the YARN Client. Please check log output!");
-				return 1;
-			}
-
-			try {
-				yarnCluster = flinkYarnClient.deploy();
-				// only connect to cluster if its not a detached session.
-				if(!flinkYarnClient.isDetached()) {
-					yarnCluster.connectToCluster();
-				}
-			} catch (Exception e) {
-				System.err.println("Error while deploying YARN cluster: "+e.getMessage());
-				e.printStackTrace(System.err);
-				return 1;
-			}
-			//------------------ Cluster deployed, handle connection details
-			String jobManagerAddress = yarnCluster.getJobManagerAddress().getAddress().getHostAddress() + ":" + 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 = new File(CliFrontend.getYarnPropertiesLocation(yarnCluster.getFlinkConfiguration()));
-
-			Properties yarnProps = new Properties();
-			yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_JOBMANAGER_KEY, jobManagerAddress);
-			if (flinkYarnClient.getTaskManagerSlots() != -1) {
-				String parallelism =
-						Integer.toString(flinkYarnClient.getTaskManagerSlots() * flinkYarnClient.getTaskManagerCount());
-				yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_PARALLELISM, parallelism);
-			}
-			// add dynamic properties
-			if (flinkYarnClient.getDynamicPropertiesEncoded() != null) {
-				yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING,
-						flinkYarnClient.getDynamicPropertiesEncoded());
-			}
-			writeYarnProperties(yarnProps, yarnPropertiesFile);
-
-			//------------------ Cluster 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()+"\n" +
-						"Please also note that the temporary files of the YARN session in {} will not be removed.",
-						flinkYarnClient.getSessionFilesDir());
-			} else {
-				runInteractiveCli(yarnCluster, acceptInteractiveInput);
-
-				if (!yarnCluster.hasBeenStopped()) {
-					LOG.info("Command Line Interface requested session shutdown");
-					yarnCluster.shutdown(false);
-				}
-
-				try {
-					yarnPropertiesFile.delete();
-				} catch (Exception e) {
-					LOG.warn("Exception while deleting the JobManager address file", e);
-				}
-			}
-		}
-		return 0;
-	}
-
-	/**
-	 * Sets the default Yarn Application Name.
-	 * @param defaultApplicationName the name of the yarn application to use
-	 * @return FlinkYarnSessionCli instance, for chaining
-     */
-	public FlinkYarnSessionCli withDefaultApplicationName(String defaultApplicationName) {
-		this.defaultApplicationName = defaultApplicationName;
-		return this;
-	}
-
-	/**
-	 * Utility method for tests.
-	 */
-	public void stop() {
-		if (yarnCluster != null) {
-			LOG.info("Command line interface is shutting down the yarnCluster");
-			yarnCluster.shutdown(false);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 ab70453..86b36b3 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
@@ -27,8 +27,9 @@ import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.PlanExecutor;
-import org.apache.flink.client.program.Client;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.JobWithJars;
+import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.optimizer.DataStatistics;
 import org.apache.flink.optimizer.Optimizer;
 import org.apache.flink.configuration.ConfigConstants;
@@ -57,7 +58,7 @@ public class RemoteExecutor extends PlanExecutor {
 
 	private final Configuration clientConfiguration;
 
-	private Client client;
+	private ClusterClient client;
 
 	private int defaultParallelism = 1;
 
@@ -149,7 +150,7 @@ public class RemoteExecutor extends PlanExecutor {
 	public void start() throws Exception {
 		synchronized (lock) {
 			if (client == null) {
-				client = new Client(clientConfiguration);
+				client = new StandaloneClusterClient(clientConfiguration);
 				client.setPrintStatusDuringExecution(isPrintingStatusDuringExecution());
 			}
 			else {
@@ -207,7 +208,7 @@ public class RemoteExecutor extends PlanExecutor {
 			}
 
 			try {
-				return client.runBlocking(program, defaultParallelism);
+				return client.run(program, defaultParallelism).getJobExecutionResult();
 			}
 			finally {
 				if (shutDownAtEnd) {

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 b75952e..f28d1b6 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
@@ -24,8 +24,16 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.PosixParser;
 
-import org.apache.flink.client.CliFrontend;
-import org.apache.flink.client.FlinkYarnSessionCli;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
 
 /**
  * A simple command line parser (based on Apache Commons CLI) that extracts command
@@ -33,9 +41,17 @@ import org.apache.flink.client.FlinkYarnSessionCli;
  */
 public class CliFrontendParser {
 
+	private static final Logger LOG = LoggerFactory.getLogger(CliFrontendParser.class);
+
+
 	/** command line interface of the YARN session, with a special initialization here
 	 *  to prefix all options with y/yarn. */
-	private static final FlinkYarnSessionCli yarnSessionCLi = new FlinkYarnSessionCli("y", "yarn", true);
+	private static final Map<String, CustomCommandLine> customCommandLine = new HashMap<>(1);
+
+	static {
+		// we could easily add more here in the future
+		loadCustomCommandLine("org.apache.flink.yarn.cli.FlinkYarnSessionCli", "y", "yarn");
+	}
 
 
 	static final Option HELP_OPTION = new Option("h", "help", false,
@@ -43,7 +59,7 @@ public class CliFrontendParser {
 
 	static final Option JAR_OPTION = new Option("j", "jarfile", true, "Flink program JAR file.");
 
-	public static final Option CLASS_OPTION = new Option("c", "class", true,
+	static final Option CLASS_OPTION = new Option("c", "class", true,
 			"Class with the program entry point (\"main\" method or \"getPlan()\" method. Only needed if the " +
 			"JAR file does not specify the class in its manifest.");
 
@@ -53,23 +69,23 @@ public class CliFrontendParser {
 					"times for specifying more than one URL. The protocol must be supported by the " +
 					"{@link java.net.URLClassLoader}.");
 
-	static final Option PARALLELISM_OPTION = new Option("p", "parallelism", true,
+	public static final Option PARALLELISM_OPTION = new Option("p", "parallelism", true,
 			"The parallelism with which to run the program. Optional flag to override the default value " +
 			"specified in the configuration.");
 
 	static final Option LOGGING_OPTION = new Option("q", "sysoutLogging", false, "If present, " +
 			"supress logging output to standard out.");
 
-	static final Option DETACHED_OPTION = new Option("d", "detached", false, "If present, runs " +
+	public static final Option DETACHED_OPTION = new Option("d", "detached", false, "If present, runs " +
 			"the job in detached mode");
 
 	static final Option ARGS_OPTION = new Option("a", "arguments", true,
 			"Program arguments. Arguments can also be added without -a, simply as trailing parameters.");
 
 	static final Option ADDRESS_OPTION = new Option("m", "jobmanager", true,
-			"Address of the JobManager (master) to which to connect. Specify '" + CliFrontend.YARN_DEPLOY_JOBMANAGER +
-			"' as the JobManager to deploy a YARN cluster for the job. Use this flag to connect to a " +
-			"different JobManager than the one specified in the configuration.");
+			"Address of the JobManager (master) to which to connect. " +
+			"Specify " + getCliIdentifierString() +" as the JobManager to deploy a cluster for the job. " +
+			"Use this flag to connect to a different JobManager than the one specified in the configuration.");
 
 	static final Option SAVEPOINT_PATH_OPTION = new Option("s", "fromSavepoint", true,
 			"Path to a savepoint to reset the job back to (for example file:///flink/savepoint-1537).");
@@ -143,8 +159,10 @@ public class CliFrontendParser {
 		options.addOption(DETACHED_OPTION);
 		options.addOption(SAVEPOINT_PATH_OPTION);
 
-		// also add the YARN options so that the parser can parse them
-		yarnSessionCLi.getYARNSessionCLIOptions(options);
+		for (CustomCommandLine customCLI : customCommandLine.values()) {
+			customCLI.addOptions(options);
+		}
+
 		return options;
 	}
 
@@ -240,10 +258,16 @@ public class CliFrontendParser {
 		System.out.println("\n  Syntax: run [OPTIONS] <jar-file> <arguments>");
 		formatter.setSyntaxPrefix("  \"run\" action options:");
 		formatter.printHelp(" ", getRunOptionsWithoutDeprecatedOptions(new Options()));
-		formatter.setSyntaxPrefix("  Additional arguments if -m " + CliFrontend.YARN_DEPLOY_JOBMANAGER + " is set:");
-		Options yarnOpts = new Options();
-		yarnSessionCLi.getYARNSessionCLIOptions(yarnOpts);
-		formatter.printHelp(" ", yarnOpts);
+
+		// prints options from all available command-line classes
+		for (Map.Entry<String, CustomCommandLine> entry: customCommandLine.entrySet()) {
+			formatter.setSyntaxPrefix("  Additional arguments if -m " + entry.getKey() + " is set:");
+			Options customOpts = new Options();
+			entry.getValue().addOptions(customOpts);
+			formatter.printHelp(" ", customOpts);
+			System.out.println();
+		}
+
 		System.out.println();
 	}
 
@@ -376,7 +400,63 @@ public class CliFrontendParser {
 		}
 	}
 
-	public static FlinkYarnSessionCli getFlinkYarnSessionCli() {
-		return yarnSessionCLi;
+	public static Map<String, CustomCommandLine> getAllCustomCommandLine() {
+		if (customCommandLine.isEmpty()) {
+			LOG.warn("No custom command-line classes were loaded.");
+		}
+		return Collections.unmodifiableMap(customCommandLine);
+	}
+
+	private static String getCliIdentifierString() {
+		StringBuilder builder = new StringBuilder();
+		boolean first = true;
+		for (String identifier : customCommandLine.keySet()) {
+			if (!first) {
+				builder.append(", ");
+			}
+			first = false;
+			builder.append("'").append(identifier).append("'");
+		}
+		return builder.toString();
+	}
+
+	/**
+	 * Gets the custom command-line for this identifier.
+	 * @param identifier The unique identifier for this command-line implementation.
+	 * @return CustomCommandLine or null if none was found
+	 */
+	public static CustomCommandLine getActiveCustomCommandLine(String identifier) {
+		return CliFrontendParser.getAllCustomCommandLine().get(identifier);
 	}
+
+	private static void loadCustomCommandLine(String className, Object... params) {
+
+		try {
+			Class<? extends CustomCommandLine> customCliClass =
+				Class.forName(className).asSubclass(CustomCommandLine.class);
+
+			// construct class types from the parameters
+			Class<?>[] types = new Class<?>[params.length];
+			for (int i = 0; i < params.length; i++) {
+				Preconditions.checkNotNull(params[i], "Parameters for custom command-lines may not be null.");
+				types[i] = params[i].getClass();
+			}
+
+			Constructor<? extends CustomCommandLine> constructor = customCliClass.getConstructor(types);
+			final CustomCommandLine cli = constructor.newInstance(params);
+
+			String cliIdentifier = Preconditions.checkNotNull(cli.getIdentifier());
+			CustomCommandLine existing = customCommandLine.put(cliIdentifier, cli);
+
+			if (existing != null) {
+				throw new IllegalStateException("Attempted to register " + cliIdentifier +
+					" but there is already a command-line with this identifier.");
+			}
+		} catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException | InstantiationException
+			| InvocationTargetException e) {
+			LOG.warn("Unable to locate custom CLI class {}. " +
+				"Flink is not compiled with support for this class.", className, e);
+		}
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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
new file mode 100644
index 0000000..cd5e0e6
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java
@@ -0,0 +1,57 @@
+/*
+ * 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.cli;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.configuration.Configuration;
+
+
+/**
+ * Custom command-line interface to load hooks for the command-line interface.
+ */
+public interface CustomCommandLine<ClusterType extends ClusterClient> {
+
+	/**
+	 * Returns a unique identifier for this custom command-line.
+	 * @return An unique identifier string
+	 */
+	String getIdentifier();
+
+	/**
+	 * Adds custom options to the existing options.
+	 * @param baseOptions The existing options.
+	 */
+	void addOptions(Options baseOptions);
+
+	/**
+	 * Retrieves a client for a running cluster
+	 * @param config The Flink config
+	 * @return Client if a cluster could be retrieve, null otherwise
+	 */
+	ClusterClient retrieveCluster(Configuration config) throws Exception;
+
+	/**
+	 * Creates the client for the cluster
+	 * @param applicationName The application name to use
+	 * @param commandLine The command-line options parsed by the CliFrontend
+	 * @return The client to communicate with the cluster which the CustomCommandLine brought up.
+	 */
+	ClusterType createClient(String applicationName, CommandLine commandLine) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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
new file mode 100644
index 0000000..cf0595b
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.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.client.program.ClusterClient;
+
+/**
+ * A descriptor to deploy a cluster (e.g. Yarn or Mesos) and return a Client for Cluster communication.
+ */
+public interface ClusterDescriptor<ClientType extends ClusterClient> {
+
+	/**
+	 * Returns a String containing details about the cluster (NodeManagers, available memory, ...)
+	 *
+	 */
+	String getClusterDescription() throws Exception;
+
+	/**
+	 * Triggers deployment of a cluster
+	 * @return Client for the cluster
+	 * @throws Exception
+	 */
+	ClientType deploy() throws Exception;
+}


[06/10] flink git commit: [FLINK-3667] refactor client communication classes

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
deleted file mode 100644
index dcf542a..0000000
--- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
+++ /dev/null
@@ -1,624 +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.client.program;
-
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.JobSubmissionResult;
-import org.apache.flink.api.common.accumulators.AccumulatorHelper;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.common.Plan;
-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;
-import org.apache.flink.optimizer.plan.FlinkPlan;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plan.StreamingPlan;
-import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
-import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.client.JobClient;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsErroneous;
-import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsFound;
-import org.apache.flink.runtime.messages.accumulators.RequestAccumulatorResults;
-import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.SerializedValue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-import akka.actor.ActorSystem;
-
-/**
- * Encapsulates the functionality necessary to submit a program to a remote cluster.
- */
-public class Client {
-
-	private static final Logger LOG = LoggerFactory.getLogger(Client.class);
-
-	/** The optimizer used in the optimization of batch programs */
-	final Optimizer compiler;
-
-	/** The actor system used to communicate with the JobManager */
-	private final ActorSystem actorSystem;
-
-	/** Configuration of the client */
-	private final Configuration config;
-
-	/** Timeout for futures */
-	private final FiniteDuration timeout;
-
-	/** Lookup timeout for the job manager retrieval service */
-	private final FiniteDuration lookupTimeout;
-
-	/**
-	 * If != -1, this field specifies the total number of available slots on the cluster
-	 * connected to the client.
-	 */
-	private final int maxSlots;
-
-	/** Flag indicating whether to sysout print execution updates */
-	private boolean printStatusDuringExecution = true;
-
-	/**
-	 * For interactive invocations, the Job ID is only available after the ContextEnvironment has
-	 * been run inside the user JAR. We pass the Client to every instance of the ContextEnvironment
-	 * which lets us access the last JobID here.
-	 */
-	private JobID lastJobID;
-
-	// ------------------------------------------------------------------------
-	//                            Construction
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a instance that submits the programs to the JobManager defined in the
-	 * configuration. This method will try to resolve the JobManager hostname and throw an exception
-	 * if that is not possible.
-	 *
-	 * @param config The config used to obtain the job-manager's address, and used to configure the optimizer.
-	 *
-	 * @throws java.io.IOException Thrown, if the client's actor system could not be started.
-	 * @throws java.net.UnknownHostException Thrown, if the JobManager's hostname could not be resolved.
-	 */
-	public Client(Configuration config) throws IOException {
-		this(config, -1);
-	}
-
-	/**
-	 * Creates a new instance of the class that submits the jobs to a job-manager.
-	 * at the given address using the default port.
-	 *
-	 * @param config The configuration for the client-side processes, like the optimizer.
-	 * @param maxSlots maxSlots The number of maxSlots on the cluster if != -1.
-	 *
-	 * @throws java.io.IOException Thrown, if the client's actor system could not be started.
-	 * @throws java.net.UnknownHostException Thrown, if the JobManager's hostname could not be resolved.
-	 */
-	public Client(Configuration config, int maxSlots) throws IOException {
-		this.config = Preconditions.checkNotNull(config);
-		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config);
-		this.maxSlots = maxSlots;
-
-		LOG.info("Starting client actor system");
-
-		try {
-			this.actorSystem = JobClient.startJobClientActorSystem(config);
-		} catch (Exception e) {
-			throw new IOException("Could start client actor system.", e);
-		}
-
-		timeout = AkkaUtils.getClientTimeout(config);
-		lookupTimeout = AkkaUtils.getLookupTimeout(config);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Startup & Shutdown
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Shuts down the client. This stops the internal actor system and actors.
-	 */
-	public void shutdown() {
-		if (!this.actorSystem.isTerminated()) {
-			this.actorSystem.shutdown();
-			this.actorSystem.awaitTermination();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Configuration
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Configures whether the client should print progress updates during the execution to {@code System.out}.
-	 * All updates are logged via the SLF4J loggers regardless of this setting.
-	 *
-	 * @param print True to print updates to standard out during execution, false to not print them.
-	 */
-	public void setPrintStatusDuringExecution(boolean print) {
-		this.printStatusDuringExecution = print;
-	}
-
-	/**
-	 * @return whether the client will print progress updates during the execution to {@code System.out}
-	 */
-	public boolean getPrintStatusDuringExecution() {
-		return this.printStatusDuringExecution;
-	}
-
-	/**
-	 * @return -1 if unknown. The maximum number of available processing slots at the Flink cluster
-	 * connected to this client.
-	 */
-	public int getMaxSlots() {
-		return this.maxSlots;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Access to the Program's Plan
-	// ------------------------------------------------------------------------
-
-	public static String getOptimizedPlanAsJson(Optimizer compiler, PackagedProgram prog, int parallelism)
-			throws CompilerException, ProgramInvocationException
-	{
-		PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator();
-		return jsonGen.getOptimizerPlanAsJSON((OptimizedPlan) getOptimizedPlan(compiler, prog, parallelism));
-	}
-
-	public static FlinkPlan getOptimizedPlan(Optimizer compiler, PackagedProgram prog, int parallelism)
-			throws CompilerException, ProgramInvocationException
-	{
-		Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader());
-		if (prog.isUsingProgramEntryPoint()) {
-			return getOptimizedPlan(compiler, prog.getPlanWithJars(), parallelism);
-		} else if (prog.isUsingInteractiveMode()) {
-			// temporary hack to support the optimizer plan preview
-			OptimizerPlanEnvironment env = new OptimizerPlanEnvironment(compiler);
-			if (parallelism > 0) {
-				env.setParallelism(parallelism);
-			}
-
-			return env.getOptimizedPlan(prog);
-		} else {
-			throw new RuntimeException("Couldn't determine program mode.");
-		}
-	}
-
-	public static OptimizedPlan getOptimizedPlan(Optimizer compiler, Plan p, int parallelism) throws CompilerException {
-		if (parallelism > 0 && p.getDefaultParallelism() <= 0) {
-			LOG.debug("Changing plan default parallelism from {} to {}", p.getDefaultParallelism(), parallelism);
-			p.setDefaultParallelism(parallelism);
-		}
-		LOG.debug("Set parallelism {}, plan default parallelism {}", parallelism, p.getDefaultParallelism());
-
-		return compiler.compile(p);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Program submission / execution
-	// ------------------------------------------------------------------------
-
-	public JobSubmissionResult runBlocking(PackagedProgram prog, int parallelism) throws ProgramInvocationException {
-		Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader());
-		if (prog.isUsingProgramEntryPoint()) {
-			return runBlocking(prog.getPlanWithJars(), parallelism, prog.getSavepointPath());
-		}
-		else if (prog.isUsingInteractiveMode()) {
-			LOG.info("Starting program in interactive mode");
-			ContextEnvironment.setAsContext(new ContextEnvironmentFactory(this, prog.getAllLibraries(),
-					prog.getClasspaths(), prog.getUserCodeClassLoader(), parallelism, true,
-					prog.getSavepointPath()));
-
-			// invoke here
-			try {
-				prog.invokeInteractiveModeForExecution();
-			}
-			finally {
-				ContextEnvironment.unsetContext();
-			}
-
-			return new JobSubmissionResult(lastJobID);
-		}
-		else {
-			throw new RuntimeException();
-		}
-	}
-
-	public JobSubmissionResult runDetached(PackagedProgram prog, int parallelism)
-			throws ProgramInvocationException
-	{
-		Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader());
-		if (prog.isUsingProgramEntryPoint()) {
-			return runDetached(prog.getPlanWithJars(), parallelism, prog.getSavepointPath());
-		}
-		else if (prog.isUsingInteractiveMode()) {
-			LOG.info("Starting program in interactive mode");
-			ContextEnvironmentFactory factory = new ContextEnvironmentFactory(this, prog.getAllLibraries(),
-					prog.getClasspaths(), prog.getUserCodeClassLoader(), parallelism, false,
-					prog.getSavepointPath());
-			ContextEnvironment.setAsContext(factory);
-
-			// invoke here
-			try {
-				prog.invokeInteractiveModeForExecution();
-				return ((DetachedEnvironment) factory.getLastEnvCreated()).finalizeExecute();
-			}
-			finally {
-				ContextEnvironment.unsetContext();
-			}
-		}
-		else {
-			throw new RuntimeException("PackagedProgram does not have a valid invocation mode.");
-		}
-	}
-
-	public JobExecutionResult runBlocking(JobWithJars program, int parallelism) throws ProgramInvocationException {
-		return runBlocking(program, parallelism, null);
-	}
-
-	/**
-	 * Runs a program on the Flink cluster to which this client is connected. The call blocks until the
-	 * execution is complete, and returns afterwards.
-	 *
-	 * @param program The program to be executed.
-	 * @param parallelism The default parallelism to use when running the program. The default parallelism is used
-	 *                    when the program does not set a parallelism by itself.
-	 *
-	 * @throws CompilerException Thrown, if the compiler encounters an illegal situation.
-	 * @throws ProgramInvocationException Thrown, if the program could not be instantiated from its jar file,
-	 *                                    or if the submission failed. That might be either due to an I/O problem,
-	 *                                    i.e. the job-manager is unreachable, or due to the fact that the
-	 *                                    parallel execution failed.
-	 */
-	public JobExecutionResult runBlocking(JobWithJars program, int parallelism, String savepointPath)
-			throws CompilerException, ProgramInvocationException {
-		ClassLoader classLoader = program.getUserCodeClassLoader();
-		if (classLoader == null) {
-			throw new IllegalArgumentException("The given JobWithJars does not provide a usercode class loader.");
-		}
-
-		OptimizedPlan optPlan = getOptimizedPlan(compiler, program, parallelism);
-		return runBlocking(optPlan, program.getJarFiles(), program.getClasspaths(), classLoader, savepointPath);
-	}
-
-	public JobSubmissionResult runDetached(JobWithJars program, int parallelism) throws ProgramInvocationException {
-		return runDetached(program, parallelism, null);
-	}
-
-	/**
-	 * Submits a program to the Flink cluster to which this client is connected. The call returns after the
-	 * program was submitted and does not wait for the program to complete.
-	 *
-	 * @param program The program to be executed.
-	 * @param parallelism The default parallelism to use when running the program. The default parallelism is used
-	 *                    when the program does not set a parallelism by itself.
-	 *
-	 * @throws CompilerException Thrown, if the compiler encounters an illegal situation.
-	 * @throws ProgramInvocationException Thrown, if the program could not be instantiated from its jar file,
-	 *                                    or if the submission failed. That might be either due to an I/O problem,
-	 *                                    i.e. the job-manager is unreachable.
-	 */
-	public JobSubmissionResult runDetached(JobWithJars program, int parallelism, String savepointPath)
-			throws CompilerException, ProgramInvocationException {
-		ClassLoader classLoader = program.getUserCodeClassLoader();
-		if (classLoader == null) {
-			throw new IllegalArgumentException("The given JobWithJars does not provide a usercode class loader.");
-		}
-
-		OptimizedPlan optimizedPlan = getOptimizedPlan(compiler, program, parallelism);
-		return runDetached(optimizedPlan, program.getJarFiles(), program.getClasspaths(), classLoader, savepointPath);
-	}
-
-	public JobExecutionResult runBlocking(
-			FlinkPlan compiledPlan, List<URL> libraries, List<URL> classpaths, ClassLoader classLoader) throws ProgramInvocationException {
-		return runBlocking(compiledPlan, libraries, classpaths, classLoader, null);
-	}
-
-	public JobExecutionResult runBlocking(FlinkPlan compiledPlan, List<URL> libraries, List<URL> classpaths,
-			ClassLoader classLoader, String savepointPath) throws ProgramInvocationException
-	{
-		JobGraph job = getJobGraph(compiledPlan, libraries, classpaths, savepointPath);
-		return runBlocking(job, classLoader);
-	}
-
-	public JobSubmissionResult runDetached(FlinkPlan compiledPlan, List<URL> libraries, List<URL> classpaths, ClassLoader classLoader) throws ProgramInvocationException {
-		return runDetached(compiledPlan, libraries, classpaths, classLoader, null);
-	}
-
-	public JobSubmissionResult runDetached(FlinkPlan compiledPlan, List<URL> libraries, List<URL> classpaths,
-			ClassLoader classLoader, String savepointPath) throws ProgramInvocationException
-	{
-		JobGraph job = getJobGraph(compiledPlan, libraries, classpaths, savepointPath);
-		return runDetached(job, classLoader);
-	}
-
-	public JobExecutionResult runBlocking(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
-		LeaderRetrievalService leaderRetrievalService;
-		try {
-			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
-		} catch (Exception e) {
-			throw new ProgramInvocationException("Could not create the leader retrieval service.", e);
-		}
-
-		try {
-			this.lastJobID = jobGraph.getJobID();
-			return JobClient.submitJobAndWait(actorSystem, leaderRetrievalService, jobGraph, timeout, printStatusDuringExecution, classLoader);
-		} catch (JobExecutionException e) {
-			throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e);
-		}
-	}
-
-	public JobSubmissionResult runDetached(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
-		ActorGateway jobManagerGateway;
-
-		try {
-			jobManagerGateway = getJobManagerGateway();
-		} catch (Exception e) {
-			throw new ProgramInvocationException("Failed to retrieve the JobManager gateway.", e);
-		}
-
-		LOG.info("Checking and uploading JAR files");
-		try {
-			JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout);
-		}
-		catch (IOException e) {
-			throw new ProgramInvocationException("Could not upload the program's JAR files to the JobManager.", e);
-		}
-		try {
-			this.lastJobID = jobGraph.getJobID();
-			JobClient.submitJobDetached(jobManagerGateway, jobGraph, timeout, classLoader);
-			return new JobSubmissionResult(jobGraph.getJobID());
-		} catch (JobExecutionException e) {
-				throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e);
-		}
-	}
-
-	/**
-	 * Cancels a job identified by the job id.
-	 * @param jobId the job id
-	 * @throws Exception In case an error occurred.
-	 */
-	public void cancel(JobID jobId) throws Exception {
-		final ActorGateway jobManagerGateway = getJobManagerGateway();
-
-		final Future<Object> response;
-		try {
-			response = jobManagerGateway.ask(new JobManagerMessages.CancelJob(jobId), timeout);
-		} catch (final Exception e) {
-			throw new ProgramInvocationException("Failed to query the job manager gateway.", e);
-		}
-
-		final Object result = Await.result(response, timeout);
-
-		if (result instanceof JobManagerMessages.CancellationSuccess) {
-			LOG.info("Job cancellation with ID " + jobId + " succeeded.");
-		} else if (result instanceof JobManagerMessages.CancellationFailure) {
-			final Throwable t = ((JobManagerMessages.CancellationFailure) result).cause();
-			LOG.info("Job cancellation with ID " + jobId + " failed.", t);
-			throw new Exception("Failed to cancel the job because of \n" + t.getMessage());
-		} else {
-			throw new Exception("Unknown message received while cancelling: " + result.getClass().getName());
-		}
-	}
-
-	/**
-	 * Stops a program on Flink cluster whose job-manager is configured in this client's configuration.
-	 * Stopping works only for streaming programs. Be aware, that the program might continue to run for
-	 * a while after sending the stop command, because after sources stopped to emit data all operators
-	 * need to finish processing.
-	 * 
-	 * @param jobId
-	 *            the job ID of the streaming program to stop
-	 * @throws Exception
-	 *             If the job ID is invalid (ie, is unknown or refers to a batch job) or if sending the stop signal
-	 *             failed. That might be due to an I/O problem, ie, the job-manager is unreachable.
-	 */
-	public void stop(final JobID jobId) throws Exception {
-		final ActorGateway jobManagerGateway = getJobManagerGateway();
-
-		final Future<Object> response;
-		try {
-			response = jobManagerGateway.ask(new JobManagerMessages.StopJob(jobId), timeout);
-		} catch (final Exception e) {
-			throw new ProgramInvocationException("Failed to query the job manager gateway.", e);
-		}
-
-		final Object result = Await.result(response, timeout);
-
-		if (result instanceof JobManagerMessages.StoppingSuccess) {
-			LOG.info("Job stopping with ID " + jobId + " succeeded.");
-		} else if (result instanceof JobManagerMessages.StoppingFailure) {
-			final Throwable t = ((JobManagerMessages.StoppingFailure) result).cause();
-			LOG.info("Job stopping with ID " + jobId + " failed.", t);
-			throw new Exception("Failed to stop the job because of \n" + t.getMessage());
-		} else {
-			throw new Exception("Unknown message received while stopping: " + result.getClass().getName());
-		}
-	}
-
-	/**
-	 * Requests and returns the accumulators for the given job identifier. Accumulators can be
-	 * requested while a is running or after it has finished. The default class loader is used
-	 * to deserialize the incoming accumulator results.
-	 * @param jobID The job identifier of a job.
-	 * @return A Map containing the accumulator's name and its value.
-	 */
-	public Map<String, Object> getAccumulators(JobID jobID) throws Exception {
-		return getAccumulators(jobID, ClassLoader.getSystemClassLoader());
-	}
-
-	/**
-	 * Requests and returns the accumulators for the given job identifier. Accumulators can be
-	 * requested while a is running or after it has finished.
-	 * @param jobID The job identifier of a job.
-	 * @param loader The class loader for deserializing the accumulator results.
-	 * @return A Map containing the accumulator's name and its value.
-	 */
-	public Map<String, Object> getAccumulators(JobID jobID, ClassLoader loader) throws Exception {
-		ActorGateway jobManagerGateway = getJobManagerGateway();
-
-		Future<Object> response;
-		try {
-			response = jobManagerGateway.ask(new RequestAccumulatorResults(jobID), timeout);
-		} catch (Exception e) {
-			throw new Exception("Failed to query the job manager gateway for accumulators.", e);
-		}
-
-		Object result = Await.result(response, timeout);
-
-		if (result instanceof AccumulatorResultsFound) {
-			Map<String, SerializedValue<Object>> serializedAccumulators =
-					((AccumulatorResultsFound) result).result();
-
-			return AccumulatorHelper.deserializeAccumulators(serializedAccumulators, loader);
-
-		} else if (result instanceof AccumulatorResultsErroneous) {
-			throw ((AccumulatorResultsErroneous) result).cause();
-		} else {
-			throw new Exception("Failed to fetch accumulators for the job " + jobID + ".");
-		}
-	}
-
-
-	// ------------------------------------------------------------------------
-	//  Sessions
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Tells the JobManager to finish the session (job) defined by the given ID.
-	 * 
-	 * @param jobId The ID that identifies the session.
-	 */
-	public void endSession(JobID jobId) throws Exception {
-		if (jobId == null) {
-			throw new IllegalArgumentException("The JobID must not be null.");
-		}
-		endSessions(Collections.singletonList(jobId));
-	}
-
-	/**
-	 * Tells the JobManager to finish the sessions (jobs) defined by the given IDs.
-	 *
-	 * @param jobIds The IDs that identify the sessions.
-	 */
-	public void endSessions(List<JobID> jobIds) throws Exception {
-		if (jobIds == null) {
-			throw new IllegalArgumentException("The JobIDs must not be null");
-		}
-
-		ActorGateway jobManagerGateway = getJobManagerGateway();
-		
-		for (JobID jid : jobIds) {
-			if (jid != null) {
-				LOG.info("Telling job manager to end the session {}.", jid);
-				jobManagerGateway.tell(new JobManagerMessages.RemoveCachedJob(jid));
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Internal translation methods
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates the optimized plan for a given program, using this client's compiler.
-	 *
-	 * @param prog The program to be compiled.
-	 * @return The compiled and optimized plan, as returned by the compiler.
-	 * @throws CompilerException Thrown, if the compiler encounters an illegal situation.
-	 * @throws ProgramInvocationException Thrown, if the program could not be instantiated from its jar file.
-	 */
-	private static OptimizedPlan getOptimizedPlan(Optimizer compiler, JobWithJars prog, int parallelism)
-			throws CompilerException, ProgramInvocationException {
-		return getOptimizedPlan(compiler, prog.getPlan(), parallelism);
-	}
-
-	public JobGraph getJobGraph(PackagedProgram prog, FlinkPlan optPlan) throws ProgramInvocationException {
-		return getJobGraph(optPlan, prog.getAllLibraries(), prog.getClasspaths(), null);
-	}
-
-	public JobGraph getJobGraph(PackagedProgram prog, FlinkPlan optPlan, String savepointPath) throws ProgramInvocationException {
-		return getJobGraph(optPlan, prog.getAllLibraries(), prog.getClasspaths(), savepointPath);
-	}
-
-	private JobGraph getJobGraph(FlinkPlan optPlan, List<URL> jarFiles, List<URL> classpaths, String savepointPath) {
-		JobGraph job;
-		if (optPlan instanceof StreamingPlan) {
-			job = ((StreamingPlan) optPlan).getJobGraph();
-			job.setSavepointPath(savepointPath);
-		} else {
-			JobGraphGenerator gen = new JobGraphGenerator(this.config);
-			job = gen.compileJobGraph((OptimizedPlan) optPlan);
-		}
-
-		for (URL jar : jarFiles) {
-			try {
-				job.addJar(new Path(jar.toURI()));
-			} catch (URISyntaxException e) {
-				throw new RuntimeException("URL is invalid. This should not happen.", e);
-			}
-		}
- 
-		job.setClasspaths(classpaths);
-
-		return job;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Helper methods
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Returns the {@link ActorGateway} of the current job manager leader using
-	 * the {@link LeaderRetrievalService}.
-	 *
-	 * @return ActorGateway of the current job manager leader
-	 * @throws Exception
-	 */
-	private ActorGateway getJobManagerGateway() throws Exception {
-		LOG.info("Looking up JobManager");
-		LeaderRetrievalService leaderRetrievalService;
-
-		leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config);
-
-		return LeaderRetrievalUtils.retrieveLeaderGateway(
-			leaderRetrievalService,
-			actorSystem,
-			lookupTimeout);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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
new file mode 100644
index 0000000..b56428d
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
@@ -0,0 +1,695 @@
+/*
+ * 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.program;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import akka.actor.ActorRef;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobSubmissionResult;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.accumulators.AccumulatorHelper;
+import org.apache.flink.configuration.ConfigConstants;
+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;
+import org.apache.flink.optimizer.plan.FlinkPlan;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.StreamingPlan;
+import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
+import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.client.JobClient;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsErroneous;
+import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsFound;
+import org.apache.flink.runtime.messages.accumulators.RequestAccumulatorResults;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.net.ConnectionUtils;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+import akka.actor.ActorSystem;
+
+
+/**
+ * Encapsulates the functionality necessary to submit a program to a remote cluster.
+ */
+public abstract class ClusterClient {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ClusterClient.class);
+
+	/** The optimizer used in the optimization of batch programs */
+	final Optimizer compiler;
+
+	/** The actor system used to communicate with the JobManager */
+	protected final ActorSystem actorSystem;
+
+	/** Configuration of the client */
+	protected final Configuration flinkConfig;
+
+	/** Timeout for futures */
+	protected final FiniteDuration timeout;
+
+	/** Lookup timeout for the job manager retrieval service */
+	private final FiniteDuration lookupTimeout;
+
+	/** Flag indicating whether to sysout print execution updates */
+	private boolean printStatusDuringExecution = true;
+
+	/**
+	 * For interactive invocations, the Job ID is only available after the ContextEnvironment has
+	 * been run inside the user JAR. We pass the Client to every instance of the ContextEnvironment
+	 * which lets us access the last JobID here.
+	 */
+	private JobID lastJobID;
+
+	/** Switch for blocking/detached job submission of the client */
+	private boolean detachedJobSubmission = false;
+
+	// ------------------------------------------------------------------------
+	//                            Construction
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a instance that submits the programs to the JobManager defined in the
+	 * configuration. This method will try to resolve the JobManager hostname and throw an exception
+	 * if that is not possible.
+	 *
+	 * @param flinkConfig The config used to obtain the job-manager's address, and used to configure the optimizer.
+	 *
+	 * @throws java.io.IOException Thrown, if the client's actor system could not be started.
+	 */
+	public ClusterClient(Configuration flinkConfig) throws IOException {
+
+		this.flinkConfig = Preconditions.checkNotNull(flinkConfig);
+		this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig);
+
+		this.timeout = AkkaUtils.getClientTimeout(flinkConfig);
+		this.lookupTimeout = AkkaUtils.getLookupTimeout(flinkConfig);
+
+		this.actorSystem = createActorSystem();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Startup & Shutdown
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Method to create the ActorSystem of the Client. May be overriden in subclasses.
+	 * @return ActorSystem
+	 * @throws IOException
+	 */
+	protected ActorSystem createActorSystem() throws IOException {
+
+		if (actorSystem != null) {
+			throw new RuntimeException("This method may only be called once.");
+		}
+
+		// start actor system
+		LOG.info("Starting client actor system.");
+
+		String hostName = flinkConfig.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
+		int port = flinkConfig.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
+		if (hostName == null || port == -1) {
+			throw new IOException("The initial JobManager address has not been set correctly.");
+		}
+		InetSocketAddress initialJobManagerAddress = new InetSocketAddress(hostName, port);
+
+		// find name of own public interface, able to connect to the JM
+		// try to find address for 2 seconds. log after 400 ms.
+		InetAddress ownHostname = ConnectionUtils.findConnectingAddress(initialJobManagerAddress, 2000, 400);
+		return AkkaUtils.createActorSystem(flinkConfig,
+			new Some<>(new Tuple2<String, Object>(ownHostname.getCanonicalHostName(), 0)));
+	}
+
+	/**
+	 * Shuts down the client. This stops the internal actor system and actors.
+	 */
+	public void shutdown() {
+		try {
+			finalizeCluster();
+		} finally {
+			if (!this.actorSystem.isTerminated()) {
+				this.actorSystem.shutdown();
+				this.actorSystem.awaitTermination();
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Configuration
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Configures whether the client should print progress updates during the execution to {@code System.out}.
+	 * All updates are logged via the SLF4J loggers regardless of this setting.
+	 *
+	 * @param print True to print updates to standard out during execution, false to not print them.
+	 */
+	public void setPrintStatusDuringExecution(boolean print) {
+		this.printStatusDuringExecution = print;
+	}
+
+	/**
+	 * @return whether the client will print progress updates during the execution to {@code System.out}
+	 */
+	public boolean getPrintStatusDuringExecution() {
+		return this.printStatusDuringExecution;
+	}
+
+	/**
+	 * Gets the current JobManager address from the Flink configuration (may change in case of a HA setup).
+	 * @return The address (host and port) of the leading JobManager
+	 */
+	public InetSocketAddress getJobManagerAddressFromConfig() {
+		try {
+		String hostName = flinkConfig.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
+		int port = flinkConfig.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
+		return new InetSocketAddress(hostName, port);
+		} catch (Exception e) {
+			throw new RuntimeException("Failed to retrieve JobManager address", e);
+		}
+	}
+
+	/**
+	 * Gets the current JobManager address (may change in case of a HA setup).
+	 * @return The address (host and port) of the leading JobManager
+	 */
+	public InetSocketAddress getJobManagerAddress() {
+		try {
+			final ActorRef jmActor = getJobManagerGateway().actor();
+			return AkkaUtils.getInetSockeAddressFromAkkaURL(jmActor.path().toSerializationFormat());
+		} catch (Exception e) {
+			throw new RuntimeException("Failed to retrieve JobManager address", e);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Access to the Program's Plan
+	// ------------------------------------------------------------------------
+
+	public static String getOptimizedPlanAsJson(Optimizer compiler, PackagedProgram prog, int parallelism)
+			throws CompilerException, ProgramInvocationException
+	{
+		PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator();
+		return jsonGen.getOptimizerPlanAsJSON((OptimizedPlan) getOptimizedPlan(compiler, prog, parallelism));
+	}
+
+	public static FlinkPlan getOptimizedPlan(Optimizer compiler, PackagedProgram prog, int parallelism)
+			throws CompilerException, ProgramInvocationException
+	{
+		Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader());
+		if (prog.isUsingProgramEntryPoint()) {
+			return getOptimizedPlan(compiler, prog.getPlanWithJars(), parallelism);
+		} else if (prog.isUsingInteractiveMode()) {
+			// temporary hack to support the optimizer plan preview
+			OptimizerPlanEnvironment env = new OptimizerPlanEnvironment(compiler);
+			if (parallelism > 0) {
+				env.setParallelism(parallelism);
+			}
+
+			return env.getOptimizedPlan(prog);
+		} else {
+			throw new RuntimeException("Couldn't determine program mode.");
+		}
+	}
+
+	public static OptimizedPlan getOptimizedPlan(Optimizer compiler, Plan p, int parallelism) throws CompilerException {
+		if (parallelism > 0 && p.getDefaultParallelism() <= 0) {
+			LOG.debug("Changing plan default parallelism from {} to {}", p.getDefaultParallelism(), parallelism);
+			p.setDefaultParallelism(parallelism);
+		}
+		LOG.debug("Set parallelism {}, plan default parallelism {}", parallelism, p.getDefaultParallelism());
+
+		return compiler.compile(p);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Program submission / execution
+	// ------------------------------------------------------------------------
+
+	/**
+	 * General purpose method to run a user jar from the CliFrontend in either blocking or detached mode, depending
+	 * on whether {@code setDetached(true)} or {@code setDetached(false)}.
+	 * @param prog the packaged program
+	 * @param parallelism the parallelism to execute the contained Flink job
+	 * @return The result of the execution
+	 * @throws ProgramInvocationException
+	 */
+	public JobSubmissionResult run(PackagedProgram prog, int parallelism)
+			throws ProgramInvocationException
+	{
+		Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader());
+		if (prog.isUsingProgramEntryPoint()) {
+			return run(prog.getPlanWithJars(), parallelism, prog.getSavepointPath());
+		}
+		else if (prog.isUsingInteractiveMode()) {
+			LOG.info("Starting program in interactive mode");
+			ContextEnvironmentFactory factory = new ContextEnvironmentFactory(this, prog.getAllLibraries(),
+					prog.getClasspaths(), prog.getUserCodeClassLoader(), parallelism, isDetached(),
+					prog.getSavepointPath());
+			ContextEnvironment.setAsContext(factory);
+
+			try {
+				// invoke main method
+				prog.invokeInteractiveModeForExecution();
+				if (isDetached()) {
+					// in detached mode, we execute the whole user code to extract the Flink job, afterwards we run it here
+					return ((DetachedEnvironment) factory.getLastEnvCreated()).finalizeExecute();
+				}
+				else {
+					// in blocking mode, we execute all Flink jobs contained in the user code and then return here
+					return new JobSubmissionResult(lastJobID);
+				}
+			}
+			finally {
+				ContextEnvironment.unsetContext();
+			}
+		}
+		else {
+			throw new RuntimeException("PackagedProgram does not have a valid invocation mode.");
+		}
+	}
+
+	public JobSubmissionResult run(JobWithJars program, int parallelism) throws ProgramInvocationException {
+		return run(program, parallelism, null);
+	}
+
+	/**
+	 * Runs a program on the Flink cluster to which this client is connected. The call blocks until the
+	 * execution is complete, and returns afterwards.
+	 *
+	 * @param program The program to be executed.
+	 * @param parallelism The default parallelism to use when running the program. The default parallelism is used
+	 *                    when the program does not set a parallelism by itself.
+	 *
+	 * @throws CompilerException Thrown, if the compiler encounters an illegal situation.
+	 * @throws ProgramInvocationException Thrown, if the program could not be instantiated from its jar file,
+	 *                                    or if the submission failed. That might be either due to an I/O problem,
+	 *                                    i.e. the job-manager is unreachable, or due to the fact that the
+	 *                                    parallel execution failed.
+	 */
+	public JobSubmissionResult run(JobWithJars program, int parallelism, String savepointPath)
+			throws CompilerException, ProgramInvocationException {
+		ClassLoader classLoader = program.getUserCodeClassLoader();
+		if (classLoader == null) {
+			throw new IllegalArgumentException("The given JobWithJars does not provide a usercode class loader.");
+		}
+
+		OptimizedPlan optPlan = getOptimizedPlan(compiler, program, parallelism);
+		return run(optPlan, program.getJarFiles(), program.getClasspaths(), classLoader, savepointPath);
+	}
+
+	public JobSubmissionResult run(
+			FlinkPlan compiledPlan, List<URL> libraries, List<URL> classpaths, ClassLoader classLoader) throws ProgramInvocationException {
+		return run(compiledPlan, libraries, classpaths, classLoader, null);
+	}
+
+	public JobSubmissionResult run(FlinkPlan compiledPlan,
+			List<URL> libraries, List<URL> classpaths, ClassLoader classLoader, String savepointPath)
+		throws ProgramInvocationException
+	{
+		JobGraph job = getJobGraph(compiledPlan, libraries, classpaths, savepointPath);
+		return submitJob(job, classLoader);
+	}
+
+	/**
+	 * Submits a JobGraph blocking.
+	 * @param jobGraph The JobGraph
+	 * @param classLoader User code class loader to deserialize the results and errors (may contain custom classes).
+	 * @return JobExecutionResult
+	 * @throws ProgramInvocationException
+	 */
+	public JobExecutionResult run(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
+		LeaderRetrievalService leaderRetrievalService;
+		try {
+			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
+		} catch (Exception e) {
+			throw new ProgramInvocationException("Could not create the leader retrieval service", e);
+		}
+
+		try {
+			this.lastJobID = jobGraph.getJobID();
+			return JobClient.submitJobAndWait(actorSystem,
+				leaderRetrievalService, jobGraph, timeout, printStatusDuringExecution, classLoader);
+		} catch (JobExecutionException e) {
+			throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e);
+		}
+	}
+
+	/**
+	 * Submits a JobGraph detached.
+	 * @param jobGraph The JobGraph
+	 * @param classLoader User code class loader to deserialize the results and errors (may contain custom classes).
+	 * @return JobSubmissionResult
+	 * @throws ProgramInvocationException
+	 */
+	public JobSubmissionResult runDetached(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
+		ActorGateway jobManagerGateway;
+
+		try {
+			jobManagerGateway = getJobManagerGateway();
+		} catch (Exception e) {
+			throw new ProgramInvocationException("Failed to retrieve the JobManager gateway.", e);
+		}
+
+		try {
+			JobClient.submitJobDetached(jobManagerGateway, jobGraph, timeout, classLoader);
+			return new JobSubmissionResult(jobGraph.getJobID());
+		} catch (JobExecutionException e) {
+			throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e);
+		}
+	}
+
+	/**
+	 * Cancels a job identified by the job id.
+	 * @param jobId the job id
+	 * @throws Exception In case an error occurred.
+	 */
+	public void cancel(JobID jobId) throws Exception {
+		final ActorGateway jobManagerGateway = getJobManagerGateway();
+
+		final Future<Object> response;
+		try {
+			response = jobManagerGateway.ask(new JobManagerMessages.CancelJob(jobId), timeout);
+		} catch (final Exception e) {
+			throw new ProgramInvocationException("Failed to query the job manager gateway.", e);
+		}
+
+		final Object result = Await.result(response, timeout);
+
+		if (result instanceof JobManagerMessages.CancellationSuccess) {
+			LOG.info("Job cancellation with ID " + jobId + " succeeded.");
+		} else if (result instanceof JobManagerMessages.CancellationFailure) {
+			final Throwable t = ((JobManagerMessages.CancellationFailure) result).cause();
+			LOG.info("Job cancellation with ID " + jobId + " failed.", t);
+			throw new Exception("Failed to cancel the job because of \n" + t.getMessage());
+		} else {
+			throw new Exception("Unknown message received while cancelling: " + result.getClass().getName());
+		}
+	}
+
+	/**
+	 * Stops a program on Flink cluster whose job-manager is configured in this client's configuration.
+	 * Stopping works only for streaming programs. Be aware, that the program might continue to run for
+	 * a while after sending the stop command, because after sources stopped to emit data all operators
+	 * need to finish processing.
+	 * 
+	 * @param jobId
+	 *            the job ID of the streaming program to stop
+	 * @throws Exception
+	 *             If the job ID is invalid (ie, is unknown or refers to a batch job) or if sending the stop signal
+	 *             failed. That might be due to an I/O problem, ie, the job-manager is unreachable.
+	 */
+	public void stop(final JobID jobId) throws Exception {
+		final ActorGateway jobManagerGateway = getJobManagerGateway();
+
+		final Future<Object> response;
+		try {
+			response = jobManagerGateway.ask(new JobManagerMessages.StopJob(jobId), timeout);
+		} catch (final Exception e) {
+			throw new ProgramInvocationException("Failed to query the job manager gateway.", e);
+		}
+
+		final Object result = Await.result(response, timeout);
+
+		if (result instanceof JobManagerMessages.StoppingSuccess) {
+			LOG.info("Job stopping with ID " + jobId + " succeeded.");
+		} else if (result instanceof JobManagerMessages.StoppingFailure) {
+			final Throwable t = ((JobManagerMessages.StoppingFailure) result).cause();
+			LOG.info("Job stopping with ID " + jobId + " failed.", t);
+			throw new Exception("Failed to stop the job because of \n" + t.getMessage());
+		} else {
+			throw new Exception("Unknown message received while stopping: " + result.getClass().getName());
+		}
+	}
+
+	/**
+	 * Requests and returns the accumulators for the given job identifier. Accumulators can be
+	 * requested while a is running or after it has finished. The default class loader is used
+	 * to deserialize the incoming accumulator results.
+	 * @param jobID The job identifier of a job.
+	 * @return A Map containing the accumulator's name and its value.
+	 */
+	public Map<String, Object> getAccumulators(JobID jobID) throws Exception {
+		return getAccumulators(jobID, ClassLoader.getSystemClassLoader());
+	}
+
+	/**
+	 * Requests and returns the accumulators for the given job identifier. Accumulators can be
+	 * requested while a is running or after it has finished.
+	 * @param jobID The job identifier of a job.
+	 * @param loader The class loader for deserializing the accumulator results.
+	 * @return A Map containing the accumulator's name and its value.
+	 */
+	public Map<String, Object> getAccumulators(JobID jobID, ClassLoader loader) throws Exception {
+		ActorGateway jobManagerGateway = getJobManagerGateway();
+
+		Future<Object> response;
+		try {
+			response = jobManagerGateway.ask(new RequestAccumulatorResults(jobID), timeout);
+		} catch (Exception e) {
+			throw new Exception("Failed to query the job manager gateway for accumulators.", e);
+		}
+
+		Object result = Await.result(response, timeout);
+
+		if (result instanceof AccumulatorResultsFound) {
+			Map<String, SerializedValue<Object>> serializedAccumulators =
+					((AccumulatorResultsFound) result).result();
+
+			return AccumulatorHelper.deserializeAccumulators(serializedAccumulators, loader);
+
+		} else if (result instanceof AccumulatorResultsErroneous) {
+			throw ((AccumulatorResultsErroneous) result).cause();
+		} else {
+			throw new Exception("Failed to fetch accumulators for the job " + jobID + ".");
+		}
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  Sessions
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Tells the JobManager to finish the session (job) defined by the given ID.
+	 * 
+	 * @param jobId The ID that identifies the session.
+	 */
+	public void endSession(JobID jobId) throws Exception {
+		if (jobId == null) {
+			throw new IllegalArgumentException("The JobID must not be null.");
+		}
+		endSessions(Collections.singletonList(jobId));
+	}
+
+	/**
+	 * Tells the JobManager to finish the sessions (jobs) defined by the given IDs.
+	 *
+	 * @param jobIds The IDs that identify the sessions.
+	 */
+	public void endSessions(List<JobID> jobIds) throws Exception {
+		if (jobIds == null) {
+			throw new IllegalArgumentException("The JobIDs must not be null");
+		}
+
+		ActorGateway jobManagerGateway = getJobManagerGateway();
+		
+		for (JobID jid : jobIds) {
+			if (jid != null) {
+				LOG.info("Telling job manager to end the session {}.", jid);
+				jobManagerGateway.tell(new JobManagerMessages.RemoveCachedJob(jid));
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Internal translation methods
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates the optimized plan for a given program, using this client's compiler.
+	 *
+	 * @param prog The program to be compiled.
+	 * @return The compiled and optimized plan, as returned by the compiler.
+	 * @throws CompilerException Thrown, if the compiler encounters an illegal situation.
+	 * @throws ProgramInvocationException Thrown, if the program could not be instantiated from its jar file.
+	 */
+	private static OptimizedPlan getOptimizedPlan(Optimizer compiler, JobWithJars prog, int parallelism)
+			throws CompilerException, ProgramInvocationException {
+		return getOptimizedPlan(compiler, prog.getPlan(), parallelism);
+	}
+
+	public JobGraph getJobGraph(PackagedProgram prog, FlinkPlan optPlan) throws ProgramInvocationException {
+		return getJobGraph(optPlan, prog.getAllLibraries(), prog.getClasspaths(), null);
+	}
+
+	public JobGraph getJobGraph(PackagedProgram prog, FlinkPlan optPlan, String savepointPath) throws ProgramInvocationException {
+		return getJobGraph(optPlan, prog.getAllLibraries(), prog.getClasspaths(), savepointPath);
+	}
+
+	private JobGraph getJobGraph(FlinkPlan optPlan, List<URL> jarFiles, List<URL> classpaths, String savepointPath) {
+		JobGraph job;
+		if (optPlan instanceof StreamingPlan) {
+			job = ((StreamingPlan) optPlan).getJobGraph();
+			job.setSavepointPath(savepointPath);
+		} else {
+			JobGraphGenerator gen = new JobGraphGenerator(this.flinkConfig);
+			job = gen.compileJobGraph((OptimizedPlan) optPlan);
+		}
+
+		for (URL jar : jarFiles) {
+			try {
+				job.addJar(new Path(jar.toURI()));
+			} catch (URISyntaxException e) {
+				throw new RuntimeException("URL is invalid. This should not happen.", e);
+			}
+		}
+ 
+		job.setClasspaths(classpaths);
+
+		return job;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Helper methods
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Returns the {@link ActorGateway} of the current job manager leader using
+	 * the {@link LeaderRetrievalService}.
+	 *
+	 * @return ActorGateway of the current job manager leader
+	 * @throws Exception
+	 */
+	protected ActorGateway getJobManagerGateway() throws Exception {
+		LOG.info("Looking up JobManager");
+
+		return LeaderRetrievalUtils.retrieveLeaderGateway(
+			LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig),
+			actorSystem,
+			lookupTimeout);
+	}
+
+	/**
+	 * Logs and prints to sysout if printing to stdout is enabled.
+	 * @param message The message to log/print
+	 */
+	protected void logAndSysout(String message) {
+		LOG.info(message);
+		if (printStatusDuringExecution) {
+			System.out.println(message);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Abstract methods to be implemented by the cluster specific Client
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Returns an URL (as a string) to the JobManager web interface
+	 */
+	public abstract String getWebInterfaceURL();
+
+	/**
+	 * Returns the latest cluster status, with number of Taskmanagers and slots
+	 */
+	public abstract GetClusterStatusResponse getClusterStatus();
+
+	/**
+	 * May return new messages from the cluster.
+	 * Messages can be for example about failed containers or container launch requests.
+	 */
+	protected abstract List<String> getNewMessages();
+
+	/**
+	 * Returns a string representation of the cluster.
+	 */
+	protected 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
+	 */
+	public void setDetached(boolean isDetached) {
+		this.detachedJobSubmission = isDetached;
+	}
+
+	/**
+	 * A flag to indicate whether this clients submits jobs detached.
+	 * @return True if the Client submits detached, false otherwise
+	 */
+	public boolean isDetached() {
+		return detachedJobSubmission;
+	}
+
+	/**
+	 * Return the Flink configuration object
+	 * @return The Flink configuration object
+	 */
+	public Configuration getFlinkConfiguration() {
+		return flinkConfig.clone();
+	}
+
+	/**
+	 * The client may define an upper limit on the number of slots to use
+	 * @return -1 if unknown
+	 */
+	public abstract int getMaxSlots();
+
+	/**
+	 * Calls the subclasses' submitJob method. It may decide to simply call one of the run methods or it may perform
+	 * some custom job submission logic.
+	 * @param jobGraph The JobGraph to be submitted
+	 * @return JobSubmissionResult
+	 */
+	protected abstract JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader)
+		throws ProgramInvocationException;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 dfb5f2e..fe2d7e0 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
@@ -34,7 +34,7 @@ import java.util.List;
  */
 public class ContextEnvironment extends ExecutionEnvironment {
 
-	protected final Client client;
+	protected final ClusterClient client;
 
 	protected final List<URL> jarFilesToAttach;
 
@@ -44,8 +44,8 @@ public class ContextEnvironment extends ExecutionEnvironment {
 
 	protected final String savepointPath;
 	
-	public ContextEnvironment(Client remoteConnection, List<URL> jarFiles, List<URL> classpaths,
-			ClassLoader userCodeClassLoader, String savepointPath) {
+	public ContextEnvironment(ClusterClient remoteConnection, List<URL> jarFiles, List<URL> classpaths,
+				ClassLoader userCodeClassLoader, String savepointPath) {
 		this.client = remoteConnection;
 		this.jarFilesToAttach = jarFiles;
 		this.classpathsToAttach = classpaths;
@@ -58,7 +58,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
 		Plan p = createProgramPlan(jobName);
 		JobWithJars toRun = new JobWithJars(p, this.jarFilesToAttach, this.classpathsToAttach,
 				this.userCodeClassLoader);
-		this.lastJobExecutionResult = client.runBlocking(toRun, getParallelism(), savepointPath);
+		this.lastJobExecutionResult = client.run(toRun, getParallelism(), savepointPath).getJobExecutionResult();
 		return this.lastJobExecutionResult;
 	}
 
@@ -66,7 +66,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
 	public String getExecutionPlan() throws Exception {
 		Plan plan = createProgramPlan("unnamed job");
 
-		OptimizedPlan op = Client.getOptimizedPlan(client.compiler, plan, getParallelism());
+		OptimizedPlan op = ClusterClient.getOptimizedPlan(client.compiler, plan, getParallelism());
 		PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator();
 		return gen.getOptimizerPlanAsJSON(op);
 	}
@@ -83,7 +83,7 @@ public class ContextEnvironment extends ExecutionEnvironment {
 				+ ") : " + getIdString();
 	}
 	
-	public Client getClient() {
+	public ClusterClient getClient() {
 		return this.client;
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 e820bad..f9b1fc2 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
@@ -32,7 +32,7 @@ import java.util.List;
  */
 public class ContextEnvironmentFactory implements ExecutionEnvironmentFactory {
 
-	private final Client client;
+	private final ClusterClient client;
 
 	private final List<URL> jarFilesToAttach;
 
@@ -42,34 +42,34 @@ public class ContextEnvironmentFactory implements ExecutionEnvironmentFactory {
 
 	private final int defaultParallelism;
 
-	private final boolean wait;
+	private final boolean isDetached;
 
 	private ExecutionEnvironment lastEnvCreated;
 
 	private String savepointPath;
 
-	public ContextEnvironmentFactory(Client client, List<URL> jarFilesToAttach,
+	public ContextEnvironmentFactory(ClusterClient client, List<URL> jarFilesToAttach,
 			List<URL> classpathsToAttach, ClassLoader userCodeClassLoader, int defaultParallelism,
-			boolean wait, String savepointPath)
+			boolean isDetached, String savepointPath)
 	{
 		this.client = client;
 		this.jarFilesToAttach = jarFilesToAttach;
 		this.classpathsToAttach = classpathsToAttach;
 		this.userCodeClassLoader = userCodeClassLoader;
 		this.defaultParallelism = defaultParallelism;
-		this.wait = wait;
+		this.isDetached = isDetached;
 		this.savepointPath = savepointPath;
 	}
 
 	@Override
 	public ExecutionEnvironment createExecutionEnvironment() {
-		if (!wait && lastEnvCreated != null) {
+		if (isDetached && lastEnvCreated != null) {
 			throw new InvalidProgramException("Multiple enviornments cannot be created in detached mode");
 		}
 
-		lastEnvCreated = wait ?
-				new ContextEnvironment(client, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath) :
-				new DetachedEnvironment(client, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath);
+		lastEnvCreated = isDetached ?
+				new DetachedEnvironment(client, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath):
+				new ContextEnvironment(client, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath);
 		if (defaultParallelism > 0) {
 			lastEnvCreated.setParallelism(defaultParallelism);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 037c36b..8298933 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
@@ -42,7 +42,7 @@ public class DetachedEnvironment extends ContextEnvironment {
 	private static final Logger LOG = LoggerFactory.getLogger(DetachedEnvironment.class);
 
 	public DetachedEnvironment(
-			Client remoteConnection,
+			ClusterClient remoteConnection,
 			List<URL> jarFiles,
 			List<URL> classpaths,
 			ClassLoader userCodeClassLoader,
@@ -53,7 +53,7 @@ public class DetachedEnvironment extends ContextEnvironment {
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
 		Plan p = createProgramPlan(jobName);
-		setDetachedPlan(Client.getOptimizedPlan(client.compiler, p, getParallelism()));
+		setDetachedPlan(ClusterClient.getOptimizedPlan(client.compiler, p, getParallelism()));
 		LOG.warn("Job was executed in detached mode, the results will be available on completion.");
 		this.lastJobExecutionResult = DetachedJobExecutionResult.INSTANCE;
 		return this.lastJobExecutionResult;
@@ -72,7 +72,7 @@ public class DetachedEnvironment extends ContextEnvironment {
 	 * Finishes this Context Environment's execution by explicitly running the plan constructed.
 	 */
 	JobSubmissionResult finalizeExecute() throws ProgramInvocationException {
-		return client.runDetached(detachedPlan, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath);
+		return client.run(detachedPlan, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath);
 	}
 
 	public static final class DetachedJobExecutionResult extends JobExecutionResult {

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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
new file mode 100644
index 0000000..82f350a
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java
@@ -0,0 +1,98 @@
+/*
+ * 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.program;
+
+import org.apache.flink.api.common.JobSubmissionResult;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus;
+import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * 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 StandaloneClusterClient(Configuration config) throws IOException {
+		super(config);
+	}
+
+
+	@Override
+	public String getWebInterfaceURL() {
+		String host = this.getJobManagerAddress().getHostName();
+		int port = getFlinkConfiguration().getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY,
+			ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT);
+		return "http://" +  host + ":" + port;
+	}
+
+	@Override
+	public GetClusterStatusResponse getClusterStatus() {
+		ActorGateway jmGateway;
+		try {
+			jmGateway = getJobManagerGateway();
+			Future<Object> future = jmGateway.ask(GetClusterStatus.getInstance(), timeout);
+			Object result = Await.result(future, timeout);
+			if (result instanceof GetClusterStatusResponse) {
+				return (GetClusterStatusResponse) result;
+			} else {
+				throw new RuntimeException("Received the wrong reply " + result + " from cluster.");
+			}
+		} catch (Exception e) {
+			throw new RuntimeException("Couldn't retrieve the Cluster status.", e);
+		}
+	}
+
+	@Override
+	public List<String> getNewMessages() {
+		return Collections.emptyList();
+	}
+
+	@Override
+	public String getClusterIdentifier() {
+		return "Standalone cluster with JobManager running at " + this.getJobManagerAddress();
+	}
+
+	@Override
+	public int getMaxSlots() {
+		return -1;
+	}
+
+	@Override
+	protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader)
+			throws ProgramInvocationException {
+		if (isDetached()) {
+			return super.runDetached(jobGraph, classLoader);
+		} else {
+			return super.run(jobGraph, classLoader);
+		}
+	}
+
+	@Override
+	protected void finalizeCluster() {}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
index 9d0b691..de85ca8 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
@@ -18,17 +18,13 @@
 
 package org.apache.flink.client;
 
-import static org.junit.Assert.assertEquals;
+import static org.apache.flink.client.CliFrontendTestUtils.checkJobManagerAddress;
 import static org.junit.Assert.fail;
 
 import static org.mockito.Mockito.*;
 
-import java.io.File;
-import java.nio.file.Files;
-import java.nio.file.StandardOpenOption;
 import org.apache.flink.client.cli.CommandLineOptions;
 
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -46,57 +42,18 @@ public class CliFrontendAddressConfigurationTest {
 
 	@Rule
 	public TemporaryFolder folder = new TemporaryFolder();
-	
+
 	@BeforeClass
 	public static void init() {
 		CliFrontendTestUtils.pipeSystemOutToNull();
 	}
-	
+
 	@Before
 	public void clearConfig() {
 		CliFrontendTestUtils.clearGlobalConfiguration();
 	}
 
 	@Test
-	public void testInvalidConfigAndNoOption() {
-		try {
-			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir());
-			CommandLineOptions options = mock(CommandLineOptions.class);
-
-			frontend.updateConfig(options);
-			Configuration config = frontend.getConfiguration();
-
-			checkJobManagerAddress(config, null, -1);
-
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testInvalidConfigAndOption() {
-		try {
-			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir());
-
-			CommandLineOptions options = mock(CommandLineOptions.class);
-			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
-
-			frontend.updateConfig(options);
-			Configuration config = frontend.getConfiguration();
-
-			InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
-
-			checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
 	public void testValidConfig() {
 		try {
 			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir());
@@ -112,83 +69,38 @@ public class CliFrontendAddressConfigurationTest {
 					CliFrontendTestUtils.TEST_JOB_MANAGER_PORT);
 		}
 		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+				e.printStackTrace();
+				fail(e.getMessage());
+			}
 	}
 
-	/**
-	 * Test that the CliFrontent is able to pick up the .yarn-properties file from a specified location.
-	 */
 	@Test
-	public void testYarnConfig() {
+	public void testInvalidConfigAndNoOption() {
 		try {
-			File tmpFolder = folder.newFolder();
-			String currentUser = System.getProperty("user.name");
-
-			// copy reference flink-conf.yaml to temporary test directory and append custom configuration path.
-			File confFile = new File(CliFrontendRunTest.class.getResource("/testconfigwithyarn/flink-conf.yaml").getFile());
-			File testConfFile = new File(tmpFolder, "flink-conf.yaml");
-			org.apache.commons.io.FileUtils.copyFile(confFile, testConfFile);
-			String toAppend = "\nyarn.properties-file.location: " + tmpFolder;
-			// append to flink-conf.yaml
-			Files.write(testConfFile.toPath(), toAppend.getBytes(), StandardOpenOption.APPEND);
-			// copy .yarn-properties-<username>
-			File propertiesFile = new File(CliFrontendRunTest.class.getResource("/testconfigwithyarn/.yarn-properties").getFile());
-			File testPropertiesFile = new File(tmpFolder, ".yarn-properties-"+currentUser);
-			org.apache.commons.io.FileUtils.copyFile(propertiesFile, testPropertiesFile);
-
-			// start CLI Frontend
-			CliFrontend frontend = new CliFrontend(tmpFolder.getAbsolutePath());
-
+			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir());
 			CommandLineOptions options = mock(CommandLineOptions.class);
 
 			frontend.updateConfig(options);
 			Configuration config = frontend.getConfiguration();
 
-			checkJobManagerAddress(
-					config,
-					CliFrontendTestUtils.TEST_YARN_JOB_MANAGER_ADDRESS,
-					CliFrontendTestUtils.TEST_YARN_JOB_MANAGER_PORT);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testInvalidYarnConfig() {
-		try {
-			CliFrontend cli = new CliFrontend(CliFrontendTestUtils.getConfigDirWithInvalidYarnFile());
-
-			CommandLineOptions options = mock(CommandLineOptions.class);
-
-			cli.updateConfig(options);
-
-			Configuration config = cli.getConfiguration();
+			checkJobManagerAddress(config, null, -1);
 
-			checkJobManagerAddress(
-				config,
-				CliFrontendTestUtils.TEST_JOB_MANAGER_ADDRESS,
-				CliFrontendTestUtils.TEST_JOB_MANAGER_PORT);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
-	public void testManualOptionsOverridesConfig() {
+	public void testInvalidConfigAndOption() {
 		try {
-			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir());
+			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir());
 
 			CommandLineOptions options = mock(CommandLineOptions.class);
 			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
 
 			frontend.updateConfig(options);
-
 			Configuration config = frontend.getConfiguration();
 
 			InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
@@ -200,11 +112,11 @@ public class CliFrontendAddressConfigurationTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
-	public void testManualOptionsOverridesYarn() {
+	public void testManualOptionsOverridesConfig() {
 		try {
-			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDirWithYarnFile());
+			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir());
 
 			CommandLineOptions options = mock(CommandLineOptions.class);
 			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
@@ -223,11 +135,4 @@ public class CliFrontendAddressConfigurationTest {
 		}
 	}
 
-	public void checkJobManagerAddress(Configuration config, String expectedAddress, int expectedPort) {
-		String jobManagerAddress = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
-		int jobManagerPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
-
-		assertEquals(expectedAddress, jobManagerAddress);
-		assertEquals(expectedPort, jobManagerPort);
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java
index 5439742..f47ca69 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java
@@ -30,11 +30,10 @@ import static org.mockito.Mockito.*;
 import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.cli.ProgramOptions;
 import org.apache.flink.client.cli.RunOptions;
-import org.apache.flink.client.program.Client;
+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.optimizer.CompilerException;
-import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 
 import org.apache.flink.optimizer.DataStatistics;
@@ -328,7 +327,7 @@ public class CliFrontendPackageProgramTest {
 			Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), c);
 
 			// we expect this to fail with a "ClassNotFoundException"
-			Client.getOptimizedPlanAsJson(compiler, prog, 666);
+			ClusterClient.getOptimizedPlanAsJson(compiler, prog, 666);
 			fail("Should have failed with a ClassNotFoundException");
 		}
 		catch (ProgramInvocationException e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
index 56173bd..ceba6cb 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java
@@ -25,11 +25,10 @@ import static org.junit.Assert.*;
 import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.cli.CommandLineOptions;
 import org.apache.flink.client.cli.RunOptions;
-import org.apache.flink.client.program.Client;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.PackagedProgram;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.mockito.Mockito;
 
 
 public class CliFrontendRunTest {
@@ -75,7 +74,7 @@ public class CliFrontendRunTest {
 			// test detached mode
 			{
 				String[] parameters = {"-p", "2", "-d", getTestJarPath()};
-				RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, false, true);
+				RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, true, true);
 				assertEquals(0, testFrontend.run(parameters));
 			}
 
@@ -96,9 +95,6 @@ public class CliFrontendRunTest {
 			// test configure savepoint path
 			{
 				String[] parameters = {"-s", "expectedSavepointPath", getTestJarPath()};
-				RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(1, false, false);
-				assertEquals(0, testFrontend.run(parameters));
-
 				RunOptions options = CliFrontendParser.parseRunCommand(parameters);
 				assertEquals("expectedSavepointPath", options.getSavepointPath());
 			}
@@ -125,22 +121,16 @@ public class CliFrontendRunTest {
 		}
 
 		@Override
-		protected int executeProgramDetached(PackagedProgram program, Client client, int parallelism) {
-			assertTrue(isDetached);
-			assertEquals(this.expectedParallelism, parallelism);
-			assertEquals(this.sysoutLogging, client.getPrintStatusDuringExecution());
-			return 0;
-		}
-
-		@Override
-		protected int executeProgramBlocking(PackagedProgram program, Client client, int parallelism) {
-			assertTrue(!isDetached);
+		protected int executeProgram(PackagedProgram program, ClusterClient client, int parallelism) {
+			assertEquals(isDetached, client.isDetached());
+			assertEquals(sysoutLogging, client.getPrintStatusDuringExecution());
+			assertEquals(expectedParallelism, parallelism);
 			return 0;
 		}
 
 		@Override
-		protected Client getClient(CommandLineOptions options, String programName, int userParallelism, boolean detached) throws Exception {
-			return Mockito.mock(Client.class);
+		protected ClusterClient getClient(CommandLineOptions options, String programName) throws Exception {
+			return TestingClusterClientWithoutActorSystem.create();
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java
index 7d01ab6..1872133 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java
@@ -25,9 +25,12 @@ import java.io.PrintStream;
 import java.lang.reflect.Field;
 import java.net.MalformedURLException;
 import java.util.Map;
+
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 public class CliFrontendTestUtils {
@@ -35,16 +38,11 @@ public class CliFrontendTestUtils {
 	public static final String TEST_JAR_MAIN_CLASS = "org.apache.flink.client.testjar.WordCount";
 	
 	public static final String TEST_JAR_CLASSLOADERTEST_CLASS = "org.apache.flink.client.testjar.JobWithExternalDependency";
-	
-	
+
 	public static final String TEST_JOB_MANAGER_ADDRESS = "192.168.1.33";
-	
+
 	public static final int TEST_JOB_MANAGER_PORT = 55443;
 	
-	public static final String TEST_YARN_JOB_MANAGER_ADDRESS = "22.33.44.55";
-	
-	public static final int TEST_YARN_JOB_MANAGER_PORT = 6655;
-	
 	
 	public static String getTestJarPath() throws FileNotFoundException, MalformedURLException {
 		File f = new File("target/maven-test-jar.jar");
@@ -68,17 +66,7 @@ public class CliFrontendTestUtils {
 		String confFile = CliFrontendRunTest.class.getResource("/invalidtestconfig/flink-conf.yaml").getFile();
 		return new File(confFile).getAbsoluteFile().getParent();
 	}
-	
-	public static String getConfigDirWithYarnFile() {
-		String confFile = CliFrontendRunTest.class.getResource("/testconfigwithyarn/flink-conf.yaml").getFile();
-		return new File(confFile).getAbsoluteFile().getParent();
-	}
-	
-	public static String getConfigDirWithInvalidYarnFile() {
-		String confFile = CliFrontendRunTest.class.getResource("/testconfigwithinvalidyarn/flink-conf.yaml").getFile();
-		return new File(confFile).getAbsoluteFile().getParent();
-	}
-	
+
 	public static void pipeSystemOutToNull() {
 		System.setOut(new PrintStream(new BlackholeOutputSteam()));
 		System.setErr(new PrintStream(new BlackholeOutputSteam()));
@@ -114,6 +102,14 @@ public class CliFrontendTestUtils {
 		@Override
 		public void write(int b){}
 	}
+
+	public static void checkJobManagerAddress(Configuration config, String expectedAddress, int expectedPort) {
+		String jobManagerAddress = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
+		int jobManagerPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
+
+		assertEquals(expectedAddress, jobManagerAddress);
+		assertEquals(expectedPort, jobManagerPort);
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-clients/src/test/java/org/apache/flink/client/TestingClusterClientWithoutActorSystem.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/TestingClusterClientWithoutActorSystem.java b/flink-clients/src/test/java/org/apache/flink/client/TestingClusterClientWithoutActorSystem.java
new file mode 100644
index 0000000..ab608cb
--- /dev/null
+++ b/flink-clients/src/test/java/org/apache/flink/client/TestingClusterClientWithoutActorSystem.java
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+import akka.actor.ActorSystem;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.StandaloneClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+
+/**
+ * A client to use in tests which does not instantiate an ActorSystem.
+ */
+public class TestingClusterClientWithoutActorSystem extends StandaloneClusterClient {
+
+	private TestingClusterClientWithoutActorSystem() throws IOException {
+		super(new Configuration());
+	}
+
+	/**
+	 * Do not instantiate the Actor System to save resources.
+	 * @return Mocked ActorSystem
+	 * @throws IOException
+	 */
+	@Override
+	protected ActorSystem createActorSystem() throws IOException {
+		return Mockito.mock(ActorSystem.class);
+	}
+
+	public static ClusterClient create() {
+		try {
+			return new TestingClusterClientWithoutActorSystem();
+		} catch (IOException e) {
+			throw new RuntimeException("Could not create TestingClientWithoutActorSystem.", e);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 14a1fff..4eb5269 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
@@ -98,7 +98,7 @@ public class ClientConnectionTest {
 				@Override
 				public void run() {
 					try {
-						new Client(config);
+						new StandaloneClusterClient(config);
 						fail("This should fail with an exception since the JobManager is unreachable.");
 					}
 					catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 4f9b367..96785f4 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
@@ -66,7 +66,7 @@ import static org.mockito.Mockito.when;
 
 
 /**
- * Simple and maybe stupid test to check the {@link Client} class.
+ * Simple and maybe stupid test to check the {@link ClusterClient} class.
  */
 public class ClientTest {
 
@@ -127,11 +127,12 @@ public class ClientTest {
 	@Test
 	public void testDetachedMode() throws Exception{
 		jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME());
-		Client out = new Client(config);
+		ClusterClient out = new StandaloneClusterClient(config);
+		out.setDetached(true);
 
 		try {
 			PackagedProgram prg = new PackagedProgram(TestExecuteTwice.class);
-			out.runDetached(prg, 1);
+			out.run(prg, 1);
 			fail(FAIL_MESSAGE);
 		} catch (ProgramInvocationException e) {
 			assertEquals(
@@ -141,7 +142,7 @@ public class ClientTest {
 
 		try {
 			PackagedProgram prg = new PackagedProgram(TestEager.class);
-			out.runDetached(prg, 1);
+			out.run(prg, 1);
 			fail(FAIL_MESSAGE);
 		} catch (ProgramInvocationException e) {
 			assertEquals(
@@ -151,7 +152,7 @@ public class ClientTest {
 
 		try {
 			PackagedProgram prg = new PackagedProgram(TestGetRuntime.class);
-			out.runDetached(prg, 1);
+			out.run(prg, 1);
 			fail(FAIL_MESSAGE);
 		} catch (ProgramInvocationException e) {
 			assertEquals(
@@ -161,7 +162,7 @@ public class ClientTest {
 
 		try {
 			PackagedProgram prg = new PackagedProgram(TestGetJobID.class);
-			out.runDetached(prg, 1);
+			out.run(prg, 1);
 			fail(FAIL_MESSAGE);
 		} catch (ProgramInvocationException e) {
 			assertEquals(
@@ -171,7 +172,7 @@ public class ClientTest {
 
 		try {
 			PackagedProgram prg = new PackagedProgram(TestGetAccumulator.class);
-			out.runDetached(prg, 1);
+			out.run(prg, 1);
 			fail(FAIL_MESSAGE);
 		} catch (ProgramInvocationException e) {
 			assertEquals(
@@ -181,7 +182,7 @@ public class ClientTest {
 
 		try {
 			PackagedProgram prg = new PackagedProgram(TestGetAllAccumulator.class);
-			out.runDetached(prg, 1);
+			out.run(prg, 1);
 			fail(FAIL_MESSAGE);
 		} catch (ProgramInvocationException e) {
 			assertEquals(
@@ -198,8 +199,9 @@ public class ClientTest {
 		try {
 			jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME());
 
-			Client out = new Client(config);
-			JobSubmissionResult result = out.runDetached(program.getPlanWithJars(), 1);
+			ClusterClient out = new StandaloneClusterClient(config);
+			out.setDetached(true);
+			JobSubmissionResult result = out.run(program.getPlanWithJars(), 1);
 
 			assertNotNull(result);
 
@@ -219,10 +221,11 @@ public class ClientTest {
 		try {
 			jobManagerSystem.actorOf(Props.create(FailureReturningActor.class), JobManager.JOB_MANAGER_NAME());
 
-			Client out = new Client(config);
+			ClusterClient out = new StandaloneClusterClient(config);
+			out.setDetached(true);
 
 			try {
-				out.runDetached(program.getPlanWithJars(), 1);
+				out.run(program.getPlanWithJars(), 1);
 				fail("This should fail with an exception");
 			}
 			catch (ProgramInvocationException e) {
@@ -258,7 +261,9 @@ public class ClientTest {
 			}).when(packagedProgramMock).invokeInteractiveModeForExecution();
 
 			try {
-				new Client(config).runBlocking(packagedProgramMock, 1);
+				ClusterClient client = new StandaloneClusterClient(config);
+				client.setDetached(true);
+				client.run(packagedProgramMock, 1);
 				fail("Creating the local execution environment should not be possible");
 			}
 			catch (InvalidProgramException e) {
@@ -280,7 +285,7 @@ public class ClientTest {
 			assertNotNull(prg.getPreviewPlan());
 			
 			Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config);
-			OptimizedPlan op = (OptimizedPlan) Client.getOptimizedPlan(optimizer, prg, 1);
+			OptimizedPlan op = (OptimizedPlan) ClusterClient.getOptimizedPlan(optimizer, prg, 1);
 			assertNotNull(op);
 
 			PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator();


[08/10] flink git commit: [FLINK-3937] programmatic resuming of clusters

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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 a5b8af7..9130fdd 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
@@ -37,6 +37,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.util.Preconditions;
+import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -55,6 +56,7 @@ import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -77,9 +79,6 @@ public class YarnClusterClient extends ClusterClient {
 	// (HDFS) location of the files required to run on YARN. Needed here to delete them on shutdown.
 	private final Path sessionFilesDir;
 
-	/** The leader retrieval service for connecting to the cluster and finding the active leader. */
-	private final LeaderRetrievalService leaderRetrievalService;
-
 	//---------- Class internal fields -------------------
 
 	private final AbstractYarnClusterDescriptor clusterDescriptor;
@@ -92,6 +91,7 @@ public class YarnClusterClient extends ClusterClient {
 
 	private boolean isConnected = false;
 
+	private final boolean perJobCluster;
 
 	/**
 	 * Create a new Flink on YARN cluster.
@@ -101,6 +101,7 @@ public class YarnClusterClient extends ClusterClient {
 	 * @param appReport the YARN application ID
 	 * @param flinkConfig Flink configuration
 	 * @param sessionFilesDir Location of files required for YARN session
+	 * @param perJobCluster Indicator whether this cluster is only created for a single job and then shutdown
 	 * @throws IOException
 	 * @throws YarnException
 	 */
@@ -109,7 +110,8 @@ public class YarnClusterClient extends ClusterClient {
 		final YarnClient yarnClient,
 		final ApplicationReport appReport,
 		org.apache.flink.configuration.Configuration flinkConfig,
-		Path sessionFilesDir) throws IOException, YarnException {
+		Path sessionFilesDir,
+		boolean perJobCluster) throws IOException, YarnException {
 
 		super(flinkConfig);
 
@@ -122,18 +124,16 @@ public class YarnClusterClient extends ClusterClient {
 		this.applicationId = appReport;
 		this.appId = appReport.getApplicationId();
 		this.trackingURL = appReport.getTrackingUrl();
+		this.perJobCluster = perJobCluster;
 
+		/* The leader retrieval service for connecting to the cluster and finding the active leader. */
+		LeaderRetrievalService leaderRetrievalService;
 		try {
 			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
 		} catch (Exception e) {
 			throw new IOException("Could not create the leader retrieval service.", e);
 		}
 
-
-		if (isConnected) {
-			throw new IllegalStateException("Already connected to the cluster.");
-		}
-
 		// start application client
 		LOG.info("Start application client.");
 
@@ -182,28 +182,31 @@ public class YarnClusterClient extends ClusterClient {
 
 		isConnected = true;
 
-		logAndSysout("Waiting until all TaskManagers have connected");
+		if (perJobCluster) {
 
-		while(true) {
-			GetClusterStatusResponse status = getClusterStatus();
-			if (status != null) {
-				if (status.numRegisteredTaskManagers() < clusterDescriptor.getTaskManagerCount()) {
-					logAndSysout("TaskManager status (" + status.numRegisteredTaskManagers() + "/"
-						+ clusterDescriptor.getTaskManagerCount() + ")");
+			logAndSysout("Waiting until all TaskManagers have connected");
+
+			while (true) {
+				GetClusterStatusResponse status = getClusterStatus();
+				if (status != null) {
+					if (status.numRegisteredTaskManagers() < clusterDescriptor.getTaskManagerCount()) {
+						logAndSysout("TaskManager status (" + status.numRegisteredTaskManagers() + "/"
+							+ clusterDescriptor.getTaskManagerCount() + ")");
+					} else {
+						logAndSysout("All TaskManagers are connected");
+						break;
+					}
 				} else {
-					logAndSysout("All TaskManagers are connected");
-					break;
+					logAndSysout("No status updates from the YARN cluster received so far. Waiting ...");
 				}
-			} else {
-				logAndSysout("No status updates from the YARN cluster received so far. Waiting ...");
-			}
 
-			try {
-				Thread.sleep(500);
-			} catch (InterruptedException e) {
-				LOG.error("Interrupted while waiting for TaskManagers");
-				System.err.println("Thread is interrupted");
-				throw new IOException("Interrupted while waiting for TaskManagers", e);
+				try {
+					Thread.sleep(500);
+				} catch (InterruptedException e) {
+					LOG.error("Interrupted while waiting for TaskManagers");
+					System.err.println("Thread is interrupted");
+					throw new IOException("Interrupted while waiting for TaskManagers", e);
+				}
 			}
 		}
 	}
@@ -214,9 +217,12 @@ public class YarnClusterClient extends ClusterClient {
 		}
 		LOG.info("Disconnecting YarnClusterClient from ApplicationMaster");
 
-		if(!Runtime.getRuntime().removeShutdownHook(clientShutdownHook)) {
-			LOG.warn("Error while removing the shutdown hook. The YARN session might be killed unintentionally");
+		try {
+			Runtime.getRuntime().removeShutdownHook(clientShutdownHook);
+		} catch (IllegalStateException e) {
+			// we are already in the shutdown hook
 		}
+
 		// tell the actor to shut down.
 		applicationClient.tell(PoisonPill.getInstance(), applicationClient);
 
@@ -265,12 +271,30 @@ public class YarnClusterClient extends ClusterClient {
 
 	@Override
 	protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
-		if (isDetached()) {
-			JobSubmissionResult result = super.runDetached(jobGraph, classLoader);
+		if (perJobCluster) {
 			stopAfterJob(jobGraph.getJobID());
-			return result;
+		}
+
+		if (isDetached()) {
+			return super.runDetached(jobGraph, classLoader);
 		} else {
-			return super.run(jobGraph, classLoader);
+			try {
+				return super.run(jobGraph, classLoader);
+			} finally {
+				// show cluster status
+				List<String> msgs = getNewMessages();
+				if (msgs != null && msgs.size() > 1) {
+
+					logAndSysout("The following messages were created by the YARN cluster while running the Job:");
+					for (String msg : msgs) {
+						logAndSysout(msg);
+					}
+				}
+				if (getApplicationStatus() != ApplicationStatus.SUCCEEDED) {
+					logAndSysout("YARN cluster is in non-successful state " + getApplicationStatus());
+					logAndSysout("YARN Diagnostics: " + getDiagnostics());
+				}
+			}
 		}
 	}
 
@@ -298,8 +322,9 @@ public class YarnClusterClient extends ClusterClient {
 			throw new IllegalStateException("The cluster is not connected to the ApplicationMaster.");
 		}
 		if(hasBeenShutdown()) {
-			throw new RuntimeException("The YarnClusterClient has already been stopped");
+			return null;
 		}
+
 		Future<Object> clusterStatusOption = ask(applicationClient, YarnMessages.getLocalGetyarnClusterStatus(), akkaTimeout);
 		Object clusterStatus;
 		try {
@@ -417,32 +442,20 @@ public class YarnClusterClient extends ClusterClient {
 	@Override
 	public void finalizeCluster() {
 
-		if (!isConnected) {
-			throw new IllegalStateException("The cluster has been not been connected to the ApplicationMaster.");
-		}
-
-		if (isDetached()) {
-			// only disconnect if we are running detached
+		if (isDetached() || !perJobCluster) {
+			// only disconnect if we are not running a per job cluster
 			disconnect();
-			return;
+		} else {
+			shutdownCluster();
 		}
+	}
 
-		// show cluster status
-
-		List<String> msgs = getNewMessages();
-		if (msgs != null && msgs.size() > 1) {
+	public void shutdownCluster() {
 
-			logAndSysout("The following messages were created by the YARN cluster while running the Job:");
-			for (String msg : msgs) {
-				logAndSysout(msg);
-			}
-		}
-		if (getApplicationStatus() != ApplicationStatus.SUCCEEDED) {
-			logAndSysout("YARN cluster is in non-successful state " + getApplicationStatus());
-			logAndSysout("YARN Diagnostics: " + getDiagnostics());
+		if (!isConnected) {
+			throw new IllegalStateException("The cluster has been not been connected to the ApplicationMaster.");
 		}
 
-
 		if(hasBeenShutDown.getAndSet(true)) {
 			return;
 		}
@@ -471,13 +484,30 @@ public class YarnClusterClient extends ClusterClient {
 			actorSystem.awaitTermination();
 		}
 
-		LOG.info("Deleting files in " + sessionFilesDir);
 		try {
-			FileSystem shutFS = FileSystem.get(hadoopConfig);
-			shutFS.delete(sessionFilesDir, true); // delete conf and jar file.
-			shutFS.close();
-		}catch(IOException e){
-			LOG.error("Could not delete the Flink jar and configuration files in HDFS..", e);
+			File propertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(flinkConfig);
+			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);
+		}
+
+		if (sessionFilesDir != null) {
+			LOG.info("Deleting files in " + sessionFilesDir);
+			try {
+				FileSystem shutFS = FileSystem.get(hadoopConfig);
+				shutFS.delete(sessionFilesDir, true); // delete conf and jar file.
+				shutFS.close();
+			} catch (IOException e) {
+				LOG.error("Could not delete the Flink jar and configuration files in HDFS..", e);
+			}
+		} else {
+			LOG.warn("Session file directory not set. Not deleting session files");
 		}
 
 		try {
@@ -571,7 +601,6 @@ public class YarnClusterClient extends ClusterClient {
 
 	@Override
 	public boolean isDetached() {
-		// either we have set detached mode using the general '-d' flag or using the Yarn CLI flag 'yd'
 		return super.isDetached() || clusterDescriptor.isDetachedMode();
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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 43e7c7b..5f745b2 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
@@ -17,10 +17,12 @@
  */
 package org.apache.flink.yarn;
 
+
 /**
  * Default implementation of {@link AbstractYarnClusterDescriptor} which starts an {@link YarnApplicationMasterRunner}.
  */
 public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor {
+
 	@Override
 	protected Class<?> getApplicationMasterClass() {
 		return YarnApplicationMasterRunner.class;

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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 fdcc858..5eca4f1 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
@@ -28,11 +28,9 @@ import org.apache.flink.client.CliFrontend;
 import org.apache.flink.client.ClientUtils;
 import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.cli.CustomCommandLine;
-import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.yarn.AbstractYarnClusterDescriptor;
 import org.apache.flink.yarn.YarnClusterDescriptor;
 import org.apache.flink.yarn.YarnClusterClient;
 import org.apache.flink.runtime.clusterframework.ApplicationStatus;
@@ -59,6 +57,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import static org.apache.flink.client.cli.CliFrontendParser.ADDRESS_OPTION;
+
 /**
  * Class handling the command line interface to the YARN session.
  */
@@ -97,8 +97,11 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 	private final Option CONTAINER;
 	private final Option SLOTS;
 	private final Option DETACHED;
+	@Deprecated
 	private final Option STREAMING;
 	private final Option NAME;
+	
+	private final Options ALL_OPTIONS;
 
 	/**
 	 * Dynamic properties allow the user to specify additional configuration values with -D, such as
@@ -118,7 +121,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 
 	public FlinkYarnSessionCli(String shortPrefix, String longPrefix, boolean acceptInteractiveInput) {
 		this.acceptInteractiveInput = acceptInteractiveInput;
-		
+
 		QUERY = new Option(shortPrefix + "q", longPrefix + "query", false, "Display available YARN resources (memory, cores)");
 		APPLICATION_ID = new Option(shortPrefix + "id", longPrefix + "applicationId", true, "Attach to running YARN session");
 		QUEUE = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue.");
@@ -132,37 +135,24 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 		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");
+		
+		ALL_OPTIONS = new Options();
+		ALL_OPTIONS.addOption(FLINK_JAR);
+		ALL_OPTIONS.addOption(JM_MEMORY);
+		ALL_OPTIONS.addOption(TM_MEMORY);
+		ALL_OPTIONS.addOption(CONTAINER);
+		ALL_OPTIONS.addOption(QUEUE);
+		ALL_OPTIONS.addOption(QUERY);
+		ALL_OPTIONS.addOption(SHIP_PATH);
+		ALL_OPTIONS.addOption(SLOTS);
+		ALL_OPTIONS.addOption(DYNAMIC_PROPERTIES);
+		ALL_OPTIONS.addOption(DETACHED);
+		ALL_OPTIONS.addOption(STREAMING);
+		ALL_OPTIONS.addOption(NAME);
+		ALL_OPTIONS.addOption(APPLICATION_ID);
 	}
 
-	/**
-	 * Attaches a new Yarn Client to running YARN application.
-	 *
-	 */
-	public AbstractFlinkYarnCluster attachFlinkYarnClient(CommandLine cmd) {
-		AbstractFlinkYarnClient flinkYarnClient = getFlinkYarnClient();
-		if (flinkYarnClient == null) {
-			return null;
-		}
 
-		if (!cmd.hasOption(APPLICATION_ID.getOpt())) {
-			LOG.error("Missing required argument " + APPLICATION_ID.getOpt());
-			printUsage();
-			return null;
-		}
-
-		String confDirPath = CliFrontend.getConfigurationDirectoryFromEnv();
-		GlobalConfiguration.loadConfiguration(confDirPath);
-		Configuration flinkConfiguration = GlobalConfiguration.getConfiguration();
-		flinkYarnClient.setFlinkConfiguration(flinkConfiguration);
-		flinkYarnClient.setConfigurationDirectory(confDirPath);
-
-		try {
-			return flinkYarnClient.attach(cmd.getOptionValue(APPLICATION_ID.getOpt()));
-		} catch (Exception e) {
-			LOG.error("Could not attach to YARN session", e);
-			return null;
-		}
-	}
 	/**
 	 * Resumes from a Flink Yarn properties file
 	 * @param flinkConfiguration The flink configuration
@@ -170,7 +160,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 	 */
 	private boolean resumeFromYarnProperties(Configuration flinkConfiguration) {
 		// load the YARN properties
-		File propertiesFile = new File(getYarnPropertiesLocation(flinkConfiguration));
+		File propertiesFile = getYarnPropertiesLocation(flinkConfiguration);
 		if (!propertiesFile.exists()) {
 			return false;
 		}
@@ -209,7 +199,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 			try {
 				jobManagerAddress = ClientUtils.parseHostPortAddress(address);
 				// store address in config from where it is retrieved by the retrieval service
-				CliFrontend.writeJobManagerAddressToConfig(flinkConfiguration, jobManagerAddress);
+				CliFrontend.setJobManagerAddressInConfig(flinkConfiguration, jobManagerAddress);
 			}
 			catch (Exception e) {
 				throw new RuntimeException("YARN properties contain an invalid entry for JobManager address.", e);
@@ -228,10 +218,9 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 		return true;
 	}
 
-	public YarnClusterDescriptor createDescriptor(String defaultApplicationName, CommandLine cmd) {
-
+	public AbstractYarnClusterDescriptor createDescriptor(String defaultApplicationName, CommandLine cmd) {
 
-		YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor();
+		AbstractYarnClusterDescriptor yarnClusterDescriptor = getClusterDescriptor();
 
 		if (!cmd.hasOption(CONTAINER.getOpt())) { // number of containers is required option!
 			LOG.error("Missing required argument {}", CONTAINER.getOpt());
@@ -343,19 +332,6 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 		return yarnClusterDescriptor;
 	}
 
-	@Override
-	public YarnClusterClient createClient(String applicationName, CommandLine cmdLine) throws Exception {
-
-		YarnClusterDescriptor yarnClusterDescriptor = createDescriptor(applicationName, cmdLine);
-
-		try {
-			return yarnClusterDescriptor.deploy();
-		} catch (Exception e) {
-			throw new RuntimeException("Error deploying the YARN cluster", e);
-		}
-
-	}
-
 	private void printUsage() {
 		System.out.println("Usage:");
 		HelpFormatter formatter = new HelpFormatter();
@@ -367,17 +343,10 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 		formatter.printHelp(" ", req);
 
 		formatter.setSyntaxPrefix("   Optional");
-		Options opt = new Options();
-		opt.addOption(JM_MEMORY);
-		opt.addOption(TM_MEMORY);
-		opt.addOption(QUERY);
-		opt.addOption(QUEUE);
-		opt.addOption(SLOTS);
-		opt.addOption(DYNAMIC_PROPERTIES);
-		opt.addOption(DETACHED);
-		opt.addOption(STREAMING);
-		opt.addOption(NAME);
-		formatter.printHelp(" ", opt);
+		Options options = new Options();
+		addGeneralOptions(options);
+		addRunOptions(options);
+		formatter.printHelp(" ", options);
 	}
 
 	private static void writeYarnProperties(Properties properties, File propertiesFile) {
@@ -439,6 +408,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 					switch (command) {
 						case "quit":
 						case "stop":
+							yarnCluster.shutdownCluster();
 							break label;
 
 						case "help":
@@ -466,38 +436,62 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 	}
 
 	@Override
-	public String getIdentifier() {
+	public boolean isActive(CommandLine commandLine, Configuration configuration) {
+		String jobManagerOption = commandLine.getOptionValue(ADDRESS_OPTION.getOpt(), null);
+		boolean yarnJobManager = ID.equals(jobManagerOption);
+		return yarnJobManager || resumeFromYarnProperties(configuration);
+	}
+
+	@Override
+	public String getId() {
 		return ID;
 	}
 
-	public void addOptions(Options options) {
-		options.addOption(FLINK_JAR);
-		options.addOption(JM_MEMORY);
-		options.addOption(TM_MEMORY);
-		options.addOption(CONTAINER);
-		options.addOption(QUEUE);
-		options.addOption(QUERY);
-		options.addOption(SHIP_PATH);
-		options.addOption(SLOTS);
-		options.addOption(DYNAMIC_PROPERTIES);
-		options.addOption(DETACHED);
-		options.addOption(STREAMING);
-		options.addOption(NAME);
+	@Override
+	public void addRunOptions(Options baseOptions) {
+		for (Object option : ALL_OPTIONS.getOptions()) {
+			baseOptions.addOption((Option) option);
+		}
 	}
 
+	@Override
+	public void addGeneralOptions(Options baseOptions) {
+		baseOptions.addOption(APPLICATION_ID);
+	}
 
-	public void getYARNAttachCLIOptions(Options options) {
-		options.addOption(APPLICATION_ID);
+	@Override
+	public YarnClusterClient retrieveCluster(
+			CommandLine cmdLine,
+			Configuration config) throws UnsupportedOperationException {
+
+		// first check for an application id
+		if (cmdLine.hasOption(APPLICATION_ID.getOpt())) {
+			String applicationID = cmdLine.getOptionValue(APPLICATION_ID.getOpt());
+			AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor();
+			yarnDescriptor.setFlinkConfiguration(config);
+			return yarnDescriptor.retrieve(applicationID);
+		// then try to load from yarn properties
+		} else if (resumeFromYarnProperties(config)) {
+			AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor();
+			yarnDescriptor.setFlinkConfiguration(config);
+			return yarnDescriptor.retrieveFromConfig(config);
+		}
+
+		throw new UnsupportedOperationException("Could not resume a Yarn cluster.");
 	}
 
 	@Override
-	public ClusterClient retrieveCluster(Configuration config) throws Exception {
+	public YarnClusterClient createCluster(String applicationName, CommandLine cmdLine, Configuration config) {
+
+		AbstractYarnClusterDescriptor yarnClusterDescriptor = createDescriptor(applicationName, cmdLine);
+		yarnClusterDescriptor.setFlinkConfiguration(config);
 
-		if(resumeFromYarnProperties(config)) {
-			return new StandaloneClusterClient(config);
+		try {
+			return yarnClusterDescriptor.deploy();
+		} catch (Exception e) {
+			throw new RuntimeException("Error deploying the YARN cluster", e);
 		}
 
-		return null;
 	}
 
 	public int run(String[] args) {
@@ -505,7 +499,8 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 		//	Command Line Options
 		//
 		Options options = new Options();
-		addOptions(options);
+		addGeneralOptions(options);
+		addRunOptions(options);
 
 		CommandLineParser parser = new PosixParser();
 		CommandLine cmd;
@@ -519,10 +514,10 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 
 		// Query cluster for metrics
 		if (cmd.hasOption(QUERY.getOpt())) {
-			YarnClusterDescriptor flinkYarnClient = new YarnClusterDescriptor();
+			AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor();
 			String description;
 			try {
-				description = flinkYarnClient.getClusterDescription();
+				description = yarnDescriptor.getClusterDescription();
 			} catch (Exception e) {
 				System.err.println("Error while querying the YARN cluster for available resources: "+e.getMessage());
 				e.printStackTrace(System.err);
@@ -531,56 +526,61 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 			System.out.println(description);
 			return 0;
 		} else if (cmd.hasOption(APPLICATION_ID.getOpt())) {
-			yarnCluster = attachFlinkYarnClient(cmd);
+
+			AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor();
+			try {
+				yarnCluster = yarnDescriptor.retrieve(cmd.getOptionValue(APPLICATION_ID.getOpt()));
+			} catch (Exception e) {
+				throw new RuntimeException("Could not retrieve existing Yarn application", e);
+			}
 
 			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 "+yarnCluster.getApplicationId());
+					"yarn application -kill "+yarnCluster.getClusterIdentifier());
+				yarnCluster.disconnect();
 			} else {
-				runInteractiveCli(yarnCluster);
-
-				if (!yarnCluster.hasBeenStopped()) {
-					LOG.info("Command Line Interface requested session shutdown");
-					yarnCluster.shutdown(false);
-				}
+				runInteractiveCli(yarnCluster, true);
 			}
 		} else {
 
-			YarnClusterDescriptor flinkYarnClient;
+			AbstractYarnClusterDescriptor yarnDescriptor;
 			try {
-				flinkYarnClient = createDescriptor(null, cmd);
+				yarnDescriptor = createDescriptor(null, cmd);
 			} catch (Exception e) {
 				System.err.println("Error while starting the YARN Client. Please check log output!");
 				return 1;
 			}
 
 			try {
-				yarnCluster = flinkYarnClient.deploy();
+				yarnCluster = yarnDescriptor.deploy();
 			} 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().getHostAddress() + ":" + yarnCluster.getJobManagerAddress().getPort();
+			String jobManagerAddress =
+				yarnCluster.getJobManagerAddress().getAddress().getHostAddress() +
+					":" + 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 = new File(getYarnPropertiesLocation(yarnCluster.getFlinkConfiguration()));
+			File yarnPropertiesFile = getYarnPropertiesLocation(yarnCluster.getFlinkConfiguration());
 
 			Properties yarnProps = new Properties();
 			yarnProps.setProperty(YARN_PROPERTIES_JOBMANAGER_KEY, jobManagerAddress);
-			if (flinkYarnClient.getTaskManagerSlots() != -1) {
+			if (yarnDescriptor.getTaskManagerSlots() != -1) {
 				String parallelism =
-						Integer.toString(flinkYarnClient.getTaskManagerSlots() * flinkYarnClient.getTaskManagerCount());
+						Integer.toString(yarnDescriptor.getTaskManagerSlots() * yarnDescriptor.getTaskManagerCount());
 				yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, parallelism);
 			}
 			// add dynamic properties
-			if (flinkYarnClient.getDynamicPropertiesEncoded() != null) {
+			if (yarnDescriptor.getDynamicPropertiesEncoded() != null) {
 				yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING,
-						flinkYarnClient.getDynamicPropertiesEncoded());
+						yarnDescriptor.getDynamicPropertiesEncoded());
 			}
 			writeYarnProperties(yarnProps, yarnPropertiesFile);
 
@@ -592,21 +592,10 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 						"Flink on YARN, use the following command or a YARN web interface to stop it:\n" +
 						"yarn application -kill " + yarnCluster.getClusterIdentifier() + "\n" +
 						"Please also note that the temporary files of the YARN session in {} will not be removed.",
-						flinkYarnClient.getSessionFilesDir());
+						yarnDescriptor.getSessionFilesDir());
 				yarnCluster.disconnect();
 			} else {
 				runInteractiveCli(yarnCluster, acceptInteractiveInput);
-
-				if (!yarnCluster.hasBeenShutdown()) {
-					LOG.info("Command Line Interface requested session shutdown");
-					yarnCluster.shutdown();
-				}
-
-				try {
-					yarnPropertiesFile.delete();
-				} catch (Exception e) {
-					LOG.warn("Exception while deleting the JobManager address file", e);
-				}
 			}
 		}
 		return 0;
@@ -649,11 +638,16 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 		}
 	}
 
-	private static String getYarnPropertiesLocation(Configuration conf) {
+	public static File getYarnPropertiesLocation(Configuration conf) {
 		String defaultPropertiesFileLocation = System.getProperty("java.io.tmpdir");
 		String currentUser = System.getProperty("user.name");
-		String propertiesFileLocation = conf.getString(ConfigConstants.YARN_PROPERTIES_FILE_LOCATION, defaultPropertiesFileLocation);
+		String propertiesFileLocation =
+			conf.getString(ConfigConstants.YARN_PROPERTIES_FILE_LOCATION, defaultPropertiesFileLocation);
+
+		return new File(propertiesFileLocation, YARN_PROPERTIES_FILE + currentUser);
+	}
 
-		return propertiesFileLocation + File.separator + YARN_PROPERTIES_FILE + currentUser;
+	protected AbstractYarnClusterDescriptor getClusterDescriptor() {
+		return new YarnClusterDescriptor();
 	}
 }


[05/10] flink git commit: [FLINK-3667] refactor client communication classes

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
index be2caaf..c291ada 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
@@ -54,7 +54,7 @@ public class ExecutionPlanCreationTest {
 			config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, mockJmAddress.getPort());
 
 			Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config);
-			OptimizedPlan op = (OptimizedPlan) Client.getOptimizedPlan(optimizer, prg, -1);
+			OptimizedPlan op = (OptimizedPlan) ClusterClient.getOptimizedPlan(optimizer, prg, -1);
 			assertNotNull(op);
 			
 			PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator();

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 2541345..6ad250d 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
@@ -36,9 +36,10 @@ import com.google.common.collect.Lists;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.client.program.Client;
+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;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
@@ -205,9 +206,9 @@ public class FlinkClient {
 		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerHost);
 		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
 
-		final Client client;
+		final ClusterClient client;
 		try {
-			client = new Client(configuration);
+			client = new StandaloneClusterClient(configuration);
 		} catch (final IOException e) {
 			throw new RuntimeException("Could not establish a connection to the job manager", e);
 		}
@@ -245,9 +246,9 @@ public class FlinkClient {
 		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, this.jobManagerHost);
 		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, this.jobManagerPort);
 
-		final Client client;
+		final ClusterClient client;
 		try {
-			client = new Client(configuration);
+			client = new StandaloneClusterClient(configuration);
 		} catch (final IOException e) {
 			throw new RuntimeException("Could not establish a connection to the job manager", e);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
index 7962fce..bc5ae09 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java
@@ -117,11 +117,14 @@ public class JobExecutionResult extends JobSubmissionResult {
 		return (Integer) result;
 	}
 
+
 	/**
 	 * Returns a dummy object for wrapping a JobSubmissionResult
 	 * @param result The SubmissionResult
 	 * @return a JobExecutionResult
+	 * @deprecated Will be removed in future versions.
 	 */
+	@Deprecated
 	public static JobExecutionResult fromJobSubmissionResult(JobSubmissionResult result) {
 		return new JobExecutionResult(result.getJobID(), -1, null);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
index 91a838b..4928b25 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java
@@ -25,7 +25,7 @@ import org.apache.flink.annotation.Public;
  */
 @Public
 public class JobSubmissionResult {
-	
+
 	private JobID jobID;
 
 	public JobSubmissionResult(JobID jobID) {
@@ -40,4 +40,26 @@ public class JobSubmissionResult {
 	public JobID getJobID() {
 		return jobID;
 	}
+
+	/**
+	 * Checks if this JobSubmissionResult is also a JobExecutionResult.
+	 * See {@code getJobExecutionResult} to retrieve the JobExecutionResult.
+	 * @return True if this is a JobExecutionResult, false otherwise
+	 */
+	public boolean isJobExecutionResults() {
+		return this instanceof JobExecutionResult;
+	}
+
+	/**
+	 * Returns the JobExecutionResult if available.
+	 * @return The JobExecutionResult
+	 * @throws ClassCastException if this is not a JobExecutionResult
+	 */
+	public JobExecutionResult getJobExecutionResult() {
+		if (isJobExecutionResults()) {
+			return (JobExecutionResult) this;
+		} else {
+			throw new ClassCastException("This JobSubmissionResult is not a JobExecutionResult.");
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-dist/src/main/flink-bin/conf/log4j-cli.properties
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties
index acb9d1a..2aba6af 100644
--- a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties
+++ b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties
@@ -29,7 +29,7 @@ log4j.appender.file.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-
 # Log output from org.apache.flink.yarn to the console. This is used by the
 # CliFrontend class when using a per-job YARN cluster.
 log4j.logger.org.apache.flink.yarn=INFO, console
-log4j.logger.org.apache.flink.client.FlinkYarnSessionCli=INFO, console
+log4j.logger.org.apache.flink.yarn.cli.FlinkYarnSessionCli=INFO, console
 log4j.logger.org.apache.hadoop=INFO, console
 
 log4j.appender.console=org.apache.log4j.ConsoleAppender

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-dist/src/main/flink-bin/yarn-bin/yarn-session.sh
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/flink-bin/yarn-bin/yarn-session.sh b/flink-dist/src/main/flink-bin/yarn-bin/yarn-session.sh
index aa87c89..16f8ab9 100755
--- a/flink-dist/src/main/flink-bin/yarn-bin/yarn-session.sh
+++ b/flink-dist/src/main/flink-bin/yarn-bin/yarn-session.sh
@@ -52,5 +52,5 @@ log_setting="-Dlog.file="$log" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4
 
 export FLINK_CONF_DIR
 
-$JAVA_RUN $JVM_ARGS -classpath $CC_CLASSPATH:$HADOOP_CLASSPATH:$HADOOP_CONF_DIR:$YARN_CONF_DIR  $log_setting org.apache.flink.client.FlinkYarnSessionCli -ship $bin/../lib/ -j $FLINK_LIB_DIR/flink-dist*.jar "$@"
+$JAVA_RUN $JVM_ARGS -classpath $CC_CLASSPATH:$HADOOP_CLASSPATH:$HADOOP_CONF_DIR:$YARN_CONF_DIR  $log_setting org.apache.flink.yarn.cli.FlinkYarnSessionCli -ship $bin/../lib/ -j $FLINK_LIB_DIR/flink-dist*.jar "$@"
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
index 955122f..d79768f 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
@@ -62,7 +62,7 @@ public class DegreesWithExceptionITCase {
 		}
 		catch (Throwable t) {
 			t.printStackTrace();
-			fail("Cluster shutdown caused an exception: " + t.getMessage());
+			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
index 61ef446..56a0a59 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
@@ -66,7 +66,7 @@ public class ReduceOnEdgesWithExceptionITCase {
 		}
 		catch (Throwable t) {
 			t.printStackTrace();
-			fail("Cluster shutdown caused an exception: " + t.getMessage());
+			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
index 6cc0b6a..7458e08 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
@@ -67,7 +67,7 @@ public class ReduceOnNeighborsWithExceptionITCase {
 		}
 		catch (Throwable t) {
 			t.printStackTrace();
-			fail("Cluster shutdown caused an exception: " + t.getMessage());
+			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java
index 03bae4e..9da54c1 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.webmonitor.handlers;
 
 import com.fasterxml.jackson.core.JsonGenerator;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.client.program.Client;
+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;
@@ -92,7 +92,7 @@ public abstract class JarActionHandler implements RequestHandler {
 		ClassLoader classLoader = program.getUserCodeClassLoader();
 
 		Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), new Configuration());
-		FlinkPlan plan = Client.getOptimizedPlan(optimizer, program, parallelism);
+		FlinkPlan plan = ClusterClient.getOptimizedPlan(optimizer, program, parallelism);
 
 		if (plan instanceof StreamingPlan) {
 			graph = ((StreamingPlan) plan).getJobGraph();

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
index cb95040..46a432e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
@@ -205,7 +205,16 @@ public class JobClient {
 		checkNotNull(jobManagerGateway, "The jobManagerGateway must not be null.");
 		checkNotNull(jobGraph, "The jobGraph must not be null.");
 		checkNotNull(timeout, "The timeout must not be null.");
-		
+
+		LOG.info("Checking and uploading JAR files");
+		try {
+			JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout);
+		}
+		catch (IOException e) {
+			throw new JobSubmissionException(jobGraph.getJobID(),
+				"Could not upload the program's JAR files to the JobManager.", e);
+		}
+
 		Object result;
 		try {
 			Future<Object> future = jobManagerGateway.ask(
@@ -214,7 +223,7 @@ public class JobClient {
 					ListeningBehaviour.DETACHED // only receive the Acknowledge for the job submission message
 				),
 				timeout);
-			
+
 			result = Await.result(future, timeout);
 		}
 		catch (TimeoutException e) {
@@ -225,10 +234,10 @@ public class JobClient {
 			throw new JobExecutionException(jobGraph.getJobID(),
 					"Failed to send job to JobManager: " + t.getMessage(), t.getCause());
 		}
-		
+
 		if (result instanceof JobManagerMessages.JobSubmitSuccess) {
 			JobID respondedID = ((JobManagerMessages.JobSubmitSuccess) result).jobId();
-			
+
 			// validate response
 			if (!respondedID.equals(jobGraph.getJobID())) {
 				throw new JobExecutionException(jobGraph.getJobID(),

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java
index 9746cef..b7bb84e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java
@@ -51,4 +51,5 @@ public enum ApplicationStatus {
 	public int processExitCode() {
 		return processExitCode;
 	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java
index 1bcb195..0aaf098 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java
@@ -538,7 +538,7 @@ public abstract class FlinkResourceManager<WorkerType extends ResourceIDRetrieva
 	}
 
 	// ------------------------------------------------------------------------
-	//  Cluster Shutdown
+	//  ClusterClient Shutdown
 	// ------------------------------------------------------------------------
 
 	private void shutdownCluster(ApplicationStatus status, String diagnostics) {

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/messages/GetClusterStatusResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/messages/GetClusterStatusResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/messages/GetClusterStatusResponse.java
index 5146d7b..9ad5ba5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/messages/GetClusterStatusResponse.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/messages/GetClusterStatusResponse.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.clusterframework.messages;
 import java.io.Serializable;
 
 /**
- * The reply to a {@code GetClusterStatus} message sent by the resource manager. Sends over the
+ * The reply to a {@code GetClusterStatus} message sent by the job manager. Sends over the
  * current number of task managers and the available task slots.
  */
 public class GetClusterStatusResponse implements Serializable {

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java
deleted file mode 100644
index c1498c5..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java
+++ /dev/null
@@ -1,143 +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.runtime.yarn;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.hadoop.fs.Path;
-import java.io.File;
-import java.util.List;
-
-/**
- * Abstract interface for an implementation of a Flink on YARN client to deploy.
- *
- * The Client describes the properties of the YARN application to create.
- */
-public abstract class AbstractFlinkYarnClient {
-
-	// ---- Setter for YARN Cluster properties ----- //
-
-	/**
-	 * @param memoryMB The amount of memory for the JobManager (in MB)
-	 */
-	public abstract void setJobManagerMemory(int memoryMB);
-
-	/**
-	 * @param memoryMB The memory per TaskManager (in MB)
-	 */
-	public abstract void setTaskManagerMemory(int memoryMB);
-
-	/**
-	 * Flink configuration
-	 */
-	public abstract void setFlinkConfiguration(org.apache.flink.configuration.Configuration conf);
-
-	public abstract Configuration getFlinkConfiguration();
-
-	/**
-	 *
-	 * @param slots The number of TaskManager slots per TaskManager.
-	 */
-	public abstract void setTaskManagerSlots(int slots);
-
-	/**
-	 * @return the number of TaskManager processing slots per TaskManager.
-	 */
-	public abstract int getTaskManagerSlots();
-
-	/**
-	 * @param queue Name of the YARN queue
-	 */
-	public abstract void setQueue(String queue);
-
-	/**
-	 *
-	 * @param localJarPath Local Path to the Flink uberjar
-	 */
-	public abstract void setLocalJarPath(Path localJarPath);
-
-	/**
-	 *
-	 * @param confPath local path to the Flink configuration file
-	 */
-	public abstract void setConfigurationFilePath(Path confPath);
-
-	/**
-	 *
-	 * @param logConfPath local path to the flink logging configuration
-	 */
-	public abstract void setFlinkLoggingConfigurationPath(Path logConfPath);
-	public abstract Path getFlinkLoggingConfigurationPath();
-
-	/**
-	 *
-	 * @param tmCount number of TaskManagers to start
-	 */
-	public abstract void setTaskManagerCount(int tmCount);
-	public abstract int getTaskManagerCount();
-
-	/**
-	 * @param confDirPath Path to config directory.
-	 */
-	public abstract void setConfigurationDirectory(String confDirPath);
-
-	/**
-	 * List of files to transfer to the YARN containers.
-	 */
-	public abstract void setShipFiles(List<File> shipFiles);
-
-	/**
-	 *
-	 * @param dynamicPropertiesEncoded Encoded String of the dynamic properties (-D configuration values of the Flink configuration)
-	 */
-	public abstract void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded);
-	public abstract String getDynamicPropertiesEncoded();
-
-	// --------------------------------------- Operations on the YARN cluster ----- //
-
-	/**
-	 * Returns a String containing details about the cluster (NodeManagers, available memory, ...)
-	 *
-	 */
-	public abstract String getClusterDescription() throws Exception;
-
-	/**
-	 * Trigger the deployment to YARN.
-	 *
-	 */
-	public abstract AbstractFlinkYarnCluster deploy() throws Exception;
-
-	/**
-	 * @param detachedMode If true, the Flink YARN client is non-blocking. That means it returns
-	 *                        once Flink has been started successfully on YARN.
-	 */
-	public abstract void setDetachedMode(boolean detachedMode);
-
-	public abstract boolean isDetached();
-
-	/**
-	 * @return The string representation of the Path to the YARN session files. This is a temporary
-	 * directory in HDFS that contains the jar files and configuration which is shipped to all the containers.
-	 */
-	public abstract String getSessionFilesDir();
-
-	/**
-	 * Set a name for the YARN application
-	 * @param name
-	 */
-	public abstract void setName(String name);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
deleted file mode 100644
index af015c7..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
+++ /dev/null
@@ -1,123 +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.runtime.yarn;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.List;
-
-/**
- * Abstract class for interacting with a running Flink cluster within YARN.
- */
-public abstract class AbstractFlinkYarnCluster {
-
-	/**
-	 * Get hostname and port of the JobManager.
-	 */
-	public abstract InetSocketAddress getJobManagerAddress();
-
-	/**
-	 * Returns an URL (as a string) to the JobManager web interface, running next to the
-	 * ApplicationMaster and JobManager in a YARN container
-	 */
-	public abstract String getWebInterfaceURL();
-
-	/**
-	 * Request the YARN cluster to shut down.
-	 *
-	 * @param failApplication If true, the application will be marked as failed in YARN
-	 */
-	public abstract void shutdown(boolean failApplication);
-
-	/**
-	 * Boolean indicating whether the cluster has been stopped already
-	 */
-	public abstract boolean hasBeenStopped();
-
-	/**
-	 * Returns the latest cluster status, with number of Taskmanagers and slots
-	 */
-	public abstract GetClusterStatusResponse getClusterStatus();
-
-	/**
-	 * Boolean indicating whether the Flink YARN cluster is in an erronous state.
-	 */
-	public abstract boolean hasFailed();
-
-	/**
-	 * @return Diagnostics if the Cluster is in "failed" state.
-	 */
-	public abstract String getDiagnostics();
-
-	/**
-	 * May return new messages from the cluster.
-	 * Messages can be for example about failed containers or container launch requests.
-	 */
-	public abstract List<String> getNewMessages();
-
-	/**
-	 * Returns a string representation of the ApplicationID assigned by YARN.
-	 */
-	public abstract String getApplicationId();
-
-	/**
-	 * Flink's YARN cluster abstraction has two modes for connecting to the YARN AM.
-	 * In the detached mode, the AM is launched and the Flink YARN client is disconnecting
-	 * afterwards.
-	 * In the non-detached mode, it maintains a connection with the AM to control the cluster.
-	 * @return boolean indicating whether the cluster is a detached cluster
-	 */
-	public abstract boolean isDetached();
-
-	/**
-	 * Connect the FlinkYarnCluster to the ApplicationMaster.
-	 *
-	 * Detached YARN sessions don't need to connect to the ApplicationMaster.
-	 * Detached per job YARN sessions need to connect until the required number of TaskManagers have been started.
-	 *
-	 * @throws IOException
-	 */
-	public abstract void connectToCluster() throws IOException;
-
-	/**
-	 * Disconnect from the ApplicationMaster without stopping the session
-	 * (therefore, use the {@link AbstractFlinkYarnCluster#shutdown(boolean)} method.
-	 *
-	 * @see AbstractFlinkYarnCluster#shutdown(boolean)
-	 */
-	public abstract void disconnect();
-
-	/**
-	 * Tells the ApplicationMaster to monitor the status of JobId and stop itself once the specified
-	 * job has finished.
-	 *
-	 * @param jobID Id of the job
-	 */
-	public abstract void stopAfterJob(JobID jobID);
-
-	/**
-	 * Return the Flink configuration object
-	 * @return The Flink configuration object
-	 */
-	public abstract Configuration getFlinkConfiguration();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 7b80206..1a8870b 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
@@ -19,14 +19,14 @@
 package org.apache.flink.api.scala
 
 import java.io._
-import java.util.Properties
 
-import org.apache.flink.client.{CliFrontend, ClientUtils, FlinkYarnSessionCli}
-import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration}
+import org.apache.flink.client.cli.CliFrontendParser
+import org.apache.flink.client.program.ClusterClient
+import org.apache.flink.client.CliFrontend
+import org.apache.flink.configuration.{ConfigConstants, GlobalConfiguration}
 import org.apache.flink.runtime.minicluster.{FlinkMiniCluster, LocalFlinkMiniCluster}
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster
-import org.apache.hadoop.fs.Path
 
+import scala.collection.mutable.ArrayBuffer
 import scala.tools.nsc.Settings
 import scala.tools.nsc.interpreter._
 
@@ -138,7 +138,7 @@ object FlinkShell {
 
   def fetchConnectionInfo(
     config: Config
-  ): (String, Int, Option[Either[FlinkMiniCluster, AbstractFlinkYarnCluster]]) = {
+  ): (String, Int, Option[Either[FlinkMiniCluster, ClusterClient]]) = {
     config.executionMode match {
       case ExecutionMode.LOCAL => // Local mode
         val config = GlobalConfiguration.getConfiguration()
@@ -217,7 +217,7 @@ object FlinkShell {
       repl.closeInterpreter()
       cluster match {
         case Some(Left(miniCluster)) => miniCluster.stop()
-        case Some(Right(yarnCluster)) => yarnCluster.shutdown(false)
+        case Some(Right(yarnCluster)) => yarnCluster.shutdown()
         case _ =>
       }
     }
@@ -226,71 +226,49 @@ object FlinkShell {
   }
 
   def deployNewYarnCluster(yarnConfig: YarnConfig) = {
-    val yarnClient = FlinkYarnSessionCli.getFlinkYarnClient
-
-    // use flink-dist.jar for scala shell
-    val jarPath = new Path("file://" +
-      s"${yarnClient.getClass.getProtectionDomain.getCodeSource.getLocation.getPath}")
-    yarnClient.setLocalJarPath(jarPath)
-    
-    val confDirPath = CliFrontend.getConfigurationDirectoryFromEnv
-    val flinkConfiguration = GlobalConfiguration.getConfiguration
-    val confFile = new File(confDirPath + File.separator + "flink-conf.yaml")
-    val confPath = new Path(confFile.getAbsolutePath)
-    yarnClient.setFlinkConfiguration(flinkConfiguration)
-    yarnClient.setConfigurationDirectory(confDirPath)
-    yarnClient.setConfigurationFilePath(confPath)
+
+    val args = ArrayBuffer[String](
+      "-m", "yarn-cluster"
+    )
 
     // number of task managers is required.
     yarnConfig.containers match {
-      case Some(containers) => yarnClient.setTaskManagerCount(containers)
+      case Some(containers) => args ++= Seq("-yn", containers.toString)
       case None =>
         throw new IllegalArgumentException("Number of taskmanagers must be specified.")
     }
 
     // set configuration from user input
-    yarnConfig.jobManagerMemory.foreach(yarnClient.setJobManagerMemory)
-    yarnConfig.name.foreach(yarnClient.setName)
-    yarnConfig.queue.foreach(yarnClient.setQueue)
-    yarnConfig.slots.foreach(yarnClient.setTaskManagerSlots)
-    yarnConfig.taskManagerMemory.foreach(yarnClient.setTaskManagerMemory)
-
-    // deploy
-    val cluster = yarnClient.deploy()
+    yarnConfig.jobManagerMemory.foreach((jmMem) => args ++= Seq("-yjm", jmMem.toString))
+    yarnConfig.slots.foreach((tmMem) => args ++= Seq("-ytm", tmMem.toString))
+    yarnConfig.name.foreach((name) => args ++= Seq("-ynm", name.toString))
+    yarnConfig.queue.foreach((queue) => args ++= Seq("-yqu", queue.toString))
+    yarnConfig.slots.foreach((slots) => args ++= Seq("-ys", slots.toString))
+
+    val customCLI = CliFrontendParser.getAllCustomCommandLine.get("yarn-cluster")
+
+    val options = CliFrontendParser.parseRunCommand(args.toArray)
+
+    val cluster = customCLI.createClient("Flink Scala Shell", options.getCommandLine)
+
     val address = cluster.getJobManagerAddress.getAddress.getHostAddress
     val port = cluster.getJobManagerAddress.getPort
-    cluster.connectToCluster()
 
     (address, port, Some(Right(cluster)))
   }
 
   def fetchDeployedYarnClusterInfo() = {
+
     // load configuration
     val globalConfig = GlobalConfiguration.getConfiguration
-    val propertiesLocation = CliFrontend.getYarnPropertiesLocation(globalConfig)
-    val propertiesFile = new File(propertiesLocation)
-
-    // read properties
-    val properties = if (propertiesFile.exists()) {
-      println("Found YARN properties file " + propertiesFile.getAbsolutePath)
-      val properties = new Properties()
-      val inputStream = new FileInputStream(propertiesFile)
-
-      try {
-        properties.load(inputStream)
-      } finally {
-        inputStream.close()
-      }
 
-      properties
-    } else {
-      throw new IllegalArgumentException("Scala Shell cannot fetch YARN properties.")
-    }
+    val customCLI = CliFrontendParser.getAllCustomCommandLine.get("yarn-cluster")
+
+    val cluster = customCLI.retrieveCluster(globalConfig)
 
-    val addressInStr = properties.getProperty(CliFrontend.YARN_PROPERTIES_JOBMANAGER_KEY)
-    val address = ClientUtils.parseHostPortAddress(addressInStr)
+    val jobManager = cluster.getJobManagerAddress
 
-    (address.getHostString, address.getPort, None)
+    (jobManager.getHostString, jobManager.getPort, None)
   }
 
   def ensureYarnConfig(config: Config) = config.yarnConfig match {

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
index 4475bc8..f03cb84 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
@@ -775,7 +775,7 @@ object ExecutionEnvironment {
    * configuration parameters for the Client only; Program parallelism can be set via
    * [[ExecutionEnvironment.setParallelism]].
    *
-   * Cluster configuration has to be done in the remotely running Flink instance.
+   * ClusterClient configuration has to be done in the remotely running Flink instance.
    *
    * @param host The host name or address of the master (JobManager), where the program should be
    *             executed.

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java b/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
index f37969d..bc9bedc 100644
--- a/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
+++ b/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
@@ -230,4 +230,4 @@ public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase {
 			indexer.add(createIndexRequest(element));
 		}
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 03945a0..333f9c0 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
@@ -30,9 +30,10 @@ 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.Client;
+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;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 
@@ -195,9 +196,9 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, host);
 		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port);
 
-		Client client;
+		ClusterClient client;
 		try {
-			client = new Client(configuration);
+			client = new StandaloneClusterClient(configuration);
 			client.setPrintStatusDuringExecution(getConfig().isSysoutLoggingEnabled());
 		}
 		catch (Exception e) {
@@ -205,7 +206,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 		}
 
 		try {
-			return client.runBlocking(streamGraph, jarFiles, globalClasspaths, usercodeClassLoader);
+			return client.run(streamGraph, jarFiles, globalClasspaths, usercodeClassLoader).getJobExecutionResult();
 		}
 		catch (ProgramInvocationException e) {
 			throw e;

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
index 5999143..0332684 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
@@ -63,7 +63,10 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment {
 			((DetachedEnvironment) ctx).setDetachedPlan(streamGraph);
 			return DetachedEnvironment.DetachedJobExecutionResult.INSTANCE;
 		} else {
-			return ctx.getClient().runBlocking(streamGraph, ctx.getJars(), ctx.getClasspaths(), ctx.getUserCodeClassLoader(), ctx.getSavepointPath());
+			return ctx
+				.getClient()
+				.run(streamGraph, ctx.getJars(), ctx.getClasspaths(), ctx.getUserCodeClassLoader(), ctx.getSavepointPath())
+				.getJobExecutionResult();
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
index b1768f0..34a7eed 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
@@ -78,7 +78,7 @@ public class RemoteEnvironmentITCase {
 		}
 		catch (Throwable t) {
 			t.printStackTrace();
-			fail("Cluster shutdown caused an exception: " + t.getMessage());
+			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
index 7dccb7d..09b5e7e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
@@ -69,7 +69,7 @@ public class AutoParallelismITCase {
 		catch (Throwable t) {
 			System.err.println("Error stopping cluster on shutdown");
 			t.printStackTrace();
-			fail("Cluster shutdown caused an exception: " + t.getMessage());
+			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
index 4437db1..0a0f451 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java
@@ -69,7 +69,7 @@ public class SimpleRecoveryITCase {
 		catch (Throwable t) {
 			System.err.println("Error stopping cluster on shutdown");
 			t.printStackTrace();
-			fail("Cluster shutdown caused an exception: " + t.getMessage());
+			fail("ClusterClient shutdown caused an exception: " + t.getMessage());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml
index 239a85a..e849211 100644
--- a/flink-yarn-tests/pom.xml
+++ b/flink-yarn-tests/pom.xml
@@ -92,11 +92,16 @@ under the License.
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<!-- Enforce single threaded execution due to port conflicts with the mini yarn cluster -->
-					<forkCount>1</forkCount>
-					<workingDirectory>../</workingDirectory>
-				</configuration>
+				<executions>
+					<execution>
+						<id>integration-tests</id>
+						<configuration>
+							<!-- Enforce single threaded execution due to port conflicts with the mini yarn cluster -->
+							<forkCount>1</forkCount>
+							<workingDirectory>../</workingDirectory>
+						</configuration>
+					</execution>
+				</executions>
 			</plugin>
 
 			<plugin>

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
new file mode 100644
index 0000000..c6a1ade
--- /dev/null
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
@@ -0,0 +1,220 @@
+/*
+ * 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.CliFrontend;
+import org.apache.flink.client.cli.CommandLineOptions;
+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;
+import org.junit.*;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.lang.reflect.Field;
+import java.net.InetSocketAddress;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests that verify that the CLI client picks up the correct address for the JobManager
+ * from configuration and configs.
+ */
+public class CliFrontendYarnAddressConfigurationTest {
+
+	@Rule
+	public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	private final static PrintStream OUT = System.out;
+	private final static PrintStream ERR = System.err;
+
+	@BeforeClass
+	public static void disableStdOutErr() {
+		class NullPrint extends OutputStream {
+			@Override
+			public void write(int b) {}
+		}
+
+		PrintStream nullPrinter = new PrintStream(new NullPrint());
+		System.setOut(nullPrinter);
+		System.setErr(nullPrinter);
+	}
+
+	@AfterClass
+	public static void restoreAfterwards() {
+		System.setOut(OUT);
+		System.setErr(ERR);
+	}
+
+	@Before
+	public void clearConfig() throws NoSuchFieldException, IllegalAccessException {
+		// reset GlobalConfiguration between tests
+		Field instance = GlobalConfiguration.class.getDeclaredField("SINGLETON");
+		instance.setAccessible(true);
+		instance.set(null, null);
+	}
+
+	private static final String TEST_YARN_JOB_MANAGER_ADDRESS = "22.33.44.55";
+	private static final int TEST_YARN_JOB_MANAGER_PORT = 6655;
+
+	private static final String propertiesFile =
+		"jobManager=" + TEST_YARN_JOB_MANAGER_ADDRESS + ":" + TEST_YARN_JOB_MANAGER_PORT;
+
+
+	private static final String TEST_JOB_MANAGER_ADDRESS = "192.168.1.33";
+	private static final int TEST_JOB_MANAGER_PORT = 55443;
+
+	private static final String flinkConf =
+		"jobmanager.rpc.address: " + TEST_JOB_MANAGER_ADDRESS + "\n" +
+		"jobmanager.rpc.port: " + TEST_JOB_MANAGER_PORT;
+
+
+	private static final String invalidPropertiesFile =
+		"jasfobManager=" + TEST_YARN_JOB_MANAGER_ADDRESS + ":asf" + TEST_YARN_JOB_MANAGER_PORT;
+
+
+	/**
+	 * Test that the CliFrontend is able to pick up the .yarn-properties file from a specified location.
+	 */
+	@Test
+	public void testYarnConfig() {
+		try {
+			File tmpFolder = temporaryFolder.newFolder();
+			String currentUser = System.getProperty("user.name");
+
+			// copy .yarn-properties-<username>
+			File testPropertiesFile = new File(tmpFolder, ".yarn-properties-"+currentUser);
+			Files.write(testPropertiesFile.toPath(), propertiesFile.getBytes(), StandardOpenOption.CREATE);
+
+			// copy reference flink-conf.yaml to temporary test directory and append custom configuration path.
+			String confFile = flinkConf + "\nyarn.properties-file.location: " + tmpFolder;
+			File testConfFile = new File(tmpFolder.getAbsolutePath(), "flink-conf.yaml");
+			Files.write(testConfFile.toPath(), confFile.getBytes(), StandardOpenOption.CREATE);
+
+			// start CLI Frontend
+			TestCLI frontend = new TestCLI(tmpFolder.getAbsolutePath());
+
+			CommandLineOptions options = mock(CommandLineOptions.class);
+
+			frontend.getClient(options, "Program name");
+
+			frontend.updateConfig(options);
+			Configuration config = frontend.getConfiguration();
+
+ 			checkJobManagerAddress(
+					config,
+					TEST_YARN_JOB_MANAGER_ADDRESS,
+					TEST_YARN_JOB_MANAGER_PORT);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	public static class TestCLI extends CliFrontend {
+		TestCLI(String configDir) throws Exception {
+			super(configDir);
+		}
+
+		@Override
+		public ClusterClient getClient(CommandLineOptions options, String programName) throws Exception {
+			return super.getClient(options, programName);
+		}
+
+		@Override
+		public void updateConfig(CommandLineOptions options) {
+			super.updateConfig(options);
+		}
+	}
+
+	@Test
+	public void testInvalidYarnConfig() {
+		try {
+			File tmpFolder = temporaryFolder.newFolder();
+
+			// copy invalid .yarn-properties-<username>
+			File testPropertiesFile = new File(tmpFolder, ".yarn-properties");
+			Files.write(testPropertiesFile.toPath(), invalidPropertiesFile.getBytes(), StandardOpenOption.CREATE);
+
+			// copy reference flink-conf.yaml to temporary test directory and append custom configuration path.
+			String confFile = flinkConf + "\nyarn.properties-file.location: " + tmpFolder;
+			File testConfFile = new File(tmpFolder.getAbsolutePath(), "flink-conf.yaml");
+			Files.write(testConfFile.toPath(), confFile.getBytes(), StandardOpenOption.CREATE);
+
+			TestCLI cli = new TestCLI(tmpFolder.getAbsolutePath());
+
+			CommandLineOptions options = mock(CommandLineOptions.class);
+
+			cli.updateConfig(options);
+
+			Configuration config = cli.getConfiguration();
+
+			checkJobManagerAddress(
+				config,
+				TEST_JOB_MANAGER_ADDRESS,
+				TEST_JOB_MANAGER_PORT);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+
+	@Test
+	public void testManualOptionsOverridesYarn() {
+		try {
+			File emptyFolder = temporaryFolder.newFolder();
+			TestCLI frontend = new TestCLI(emptyFolder.getAbsolutePath());
+
+			CommandLineOptions options = mock(CommandLineOptions.class);
+			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
+
+			frontend.updateConfig(options);
+
+			Configuration config = frontend.getConfiguration();
+
+			InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
+
+			checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+
+	private static void checkJobManagerAddress(Configuration config, String expectedAddress, int expectedPort) {
+		String jobManagerAddress = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
+		int jobManagerPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
+
+		assertEquals(expectedAddress, jobManagerAddress);
+		assertEquals(expectedPort, jobManagerPort);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
index 7197b64..c842bdc 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
@@ -24,8 +24,7 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
 
 import org.apache.flink.client.CliFrontend;
-import org.apache.flink.client.FlinkYarnSessionCli;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
+import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.flink.test.util.TestBaseUtils;
 
 import org.junit.Assert;
@@ -38,6 +37,8 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties;
+
 public class FlinkYarnSessionCliTest {
 
 	@Rule
@@ -54,7 +55,7 @@ public class FlinkYarnSessionCliTest {
 		TestBaseUtils.setEnv(map);
 		Options options = new Options();
 		FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", "", false);
-		cli.getYARNSessionCLIOptions(options);
+		cli.addOptions(options);
 
 		CommandLineParser parser = new PosixParser();
 		CommandLine cmd = null;
@@ -65,11 +66,12 @@ public class FlinkYarnSessionCliTest {
 			Assert.fail("Parsing failed with " + e.getMessage());
 		}
 
-		AbstractFlinkYarnClient flinkYarnClient = cli.createFlinkYarnClient(cmd);
+		YarnClusterDescriptor flinkYarnDescriptor = cli.createDescriptor(null, cmd);
 
-		Assert.assertNotNull(flinkYarnClient);
+		Assert.assertNotNull(flinkYarnDescriptor);
 
-		Map<String, String> dynProperties = CliFrontend.getDynamicProperties(flinkYarnClient.getDynamicPropertiesEncoded());
+		Map<String, String> dynProperties =
+			FlinkYarnSessionCli.getDynamicProperties(flinkYarnDescriptor.getDynamicPropertiesEncoded());
 		Assert.assertEquals(1, dynProperties.size());
 		Assert.assertEquals("5 min", dynProperties.get("akka.ask.timeout"));
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingFlinkYarnClient.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingFlinkYarnClient.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingFlinkYarnClient.java
deleted file mode 100644
index dbfbfe2..0000000
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingFlinkYarnClient.java
+++ /dev/null
@@ -1,71 +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.util.Preconditions;
-
-import java.io.File;
-import java.io.FilenameFilter;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Yarn client which starts a {@link TestingApplicationMaster}. Additionally the client adds the
- * flink-yarn-tests-XXX-tests.jar and the flink-runtime-XXX-tests.jar to the set of files which
- * are shipped to the yarn cluster. This is necessary to load the testing classes.
- */
-public class TestingFlinkYarnClient extends FlinkYarnClientBase {
-
-	public TestingFlinkYarnClient() {
-		List<File> filesToShip = new ArrayList<>();
-
-		File testingJar = YarnTestBase.findFile("..", new TestJarFinder("flink-yarn-tests"));
-		Preconditions.checkNotNull(testingJar, "Could not find the flink-yarn-tests tests jar. " +
-			"Make sure to package the flink-yarn-tests module.");
-
-		File testingRuntimeJar = YarnTestBase.findFile("..", new TestJarFinder("flink-runtime"));
-		Preconditions.checkNotNull(testingRuntimeJar, "Could not find the flink-runtime tests " +
-			"jar. Make sure to package the flink-runtime module.");
-
-		filesToShip.add(testingJar);
-		filesToShip.add(testingRuntimeJar);
-
-		setShipFiles(filesToShip);
-	}
-
-	@Override
-	protected Class<?> getApplicationMasterClass() {
-		return TestingApplicationMaster.class;
-	}
-
-	public static class TestJarFinder implements FilenameFilter {
-
-		private final String jarName;
-
-		public TestJarFinder(final String jarName) {
-			this.jarName = jarName;
-		}
-
-		@Override
-		public boolean accept(File dir, String name) {
-			return name.startsWith(jarName) && name.endsWith("-tests.jar") &&
-				dir.getAbsolutePath().contains(dir.separator + jarName + dir.separator);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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
new file mode 100644
index 0000000..386f48f
--- /dev/null
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.util.Preconditions;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Yarn client which starts a {@link TestingApplicationMaster}. Additionally the client adds the
+ * flink-yarn-tests-XXX-tests.jar and the flink-runtime-XXX-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 TestingYarnClusterDescriptor() {
+		List<File> filesToShip = new ArrayList<>();
+
+		File testingJar = YarnTestBase.findFile("..", new TestJarFinder("flink-yarn-tests"));
+		Preconditions.checkNotNull(testingJar, "Could not find the flink-yarn-tests tests jar. " +
+			"Make sure to package the flink-yarn-tests module.");
+
+		File testingRuntimeJar = YarnTestBase.findFile("..", new TestJarFinder("flink-runtime"));
+		Preconditions.checkNotNull(testingRuntimeJar, "Could not find the flink-runtime tests " +
+			"jar. Make sure to package the flink-runtime module.");
+
+		filesToShip.add(testingJar);
+		filesToShip.add(testingRuntimeJar);
+
+		setShipFiles(filesToShip);
+	}
+
+	@Override
+	protected Class<?> getApplicationMasterClass() {
+		return TestingApplicationMaster.class;
+	}
+
+	public static class TestJarFinder implements FilenameFilter {
+
+		private final String jarName;
+
+		public TestJarFinder(final String jarName) {
+			this.jarName = jarName;
+		}
+
+		@Override
+		public boolean accept(File dir, String name) {
+			return name.startsWith(jarName) && name.endsWith("-tests.jar") &&
+				dir.getAbsolutePath().contains(dir.separator + jarName + dir.separator);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 a93abf0..4d45f16 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
@@ -22,6 +22,7 @@ import akka.actor.ActorSystem;
 import akka.actor.PoisonPill;
 import akka.testkit.JavaTestKit;
 import org.apache.curator.test.TestingServer;
+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;
@@ -33,7 +34,6 @@ import org.apache.flink.runtime.messages.Messages;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.AfterClass;
@@ -97,7 +97,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 	public void testMultipleAMKill() throws Exception {
 		final int numberKillingAttempts = numberApplicationAttempts - 1;
 
-		TestingFlinkYarnClient flinkYarnClient = new TestingFlinkYarnClient();
+		TestingYarnClusterDescriptor flinkYarnClient = new TestingYarnClusterDescriptor();
 
 		Assert.assertNotNull("unable to get yarn client", flinkYarnClient);
 		flinkYarnClient.setTaskManagerCount(1);
@@ -119,13 +119,12 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 			"@@" + ConfigConstants.ZOOKEEPER_RECOVERY_PATH + "=" + fsStateHandlePath + "/recovery");
 		flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml"));
 
-		AbstractFlinkYarnCluster yarnCluster = null;
+		ClusterClient yarnCluster = null;
 
 		final FiniteDuration timeout = new FiniteDuration(2, TimeUnit.MINUTES);
 
 		try {
 			yarnCluster = flinkYarnClient.deploy();
-			yarnCluster.connectToCluster();
 			final Configuration config = yarnCluster.getFlinkConfiguration();
 
 			new JavaTestKit(actorSystem) {{
@@ -169,7 +168,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 			}};
 		} finally {
 			if (yarnCluster != null) {
-				yarnCluster.shutdown(false);
+				yarnCluster.shutdown();
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 38e17a5..826a086 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
@@ -328,7 +328,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	@Test
 	public void testNonexistingQueue() {
 		LOG.info("Starting testNonexistingQueue()");
-		addTestAppender(FlinkYarnClient.class, Level.WARN);
+		addTestAppender(YarnClusterDescriptor.class, Level.WARN);
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
 				"-t", flinkLibFolder.getAbsolutePath(),
 				"-n", "1",
@@ -432,7 +432,9 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 				"-yD", "yarn.heap-cutoff-ratio=0.5", // test if the cutoff is passed correctly
 				"-ytm", "1024",
 				"-ys", "2", // test requesting slots from YARN.
-				"--yarndetached", job, "--input", tmpInFile.getAbsoluteFile().toString(), "--output", tmpOutFolder.getAbsoluteFile().toString()},
+				"--yarndetached", job,
+				"--input", tmpInFile.getAbsoluteFile().toString(),
+				"--output", tmpOutFolder.getAbsoluteFile().toString()},
 			"Job has been submitted with JobID",
 			RunTypes.CLI_FRONTEND);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 cb402a3..fe5400a 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
@@ -18,11 +18,10 @@
 
 package org.apache.flink.yarn;
 
-import org.apache.flink.client.FlinkYarnSessionCli;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -95,8 +94,6 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 
 		checkForLogString("The Flink YARN client has been started in detached mode");
 
-		Assert.assertFalse("The runner should detach.", runner.isAlive());
-
 		LOG.info("Waiting until two containers are running");
 		// wait until two containers are running
 		while(getRunningContainers() < 2) {
@@ -171,7 +168,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	@Ignore("The test is too resource consuming (8.5 GB of memory)")
 	@Test
 	public void testResourceComputation() {
-		addTestAppender(FlinkYarnClient.class, Level.WARN);
+		addTestAppender(YarnClusterDescriptor.class, Level.WARN);
 		LOG.info("Starting testResourceComputation()");
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
 				"-n", "5",
@@ -199,7 +196,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 	@Ignore("The test is too resource consuming (8 GB of memory)")
 	@Test
 	public void testfullAlloc() {
-		addTestAppender(FlinkYarnClient.class, Level.WARN);
+		addTestAppender(YarnClusterDescriptor.class, Level.WARN);
 		LOG.info("Starting testfullAlloc()");
 		runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(),
 				"-n", "2",
@@ -218,7 +215,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 		final int WAIT_TIME = 15;
 		LOG.info("Starting testJavaAPI()");
 
-		AbstractFlinkYarnClient flinkYarnClient = FlinkYarnSessionCli.getFlinkYarnClient();
+		AbstractYarnClusterDescriptor flinkYarnClient = new YarnClusterDescriptor();
 		Assert.assertNotNull("unable to get yarn client", flinkYarnClient);
 		flinkYarnClient.setTaskManagerCount(1);
 		flinkYarnClient.setJobManagerMemory(768);
@@ -231,10 +228,9 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 		flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml"));
 
 		// deploy
-		AbstractFlinkYarnCluster yarnCluster = null;
+		ClusterClient yarnCluster = null;
 		try {
 			yarnCluster = flinkYarnClient.deploy();
-			yarnCluster.connectToCluster();
 		} catch (Exception e) {
 			LOG.warn("Failing test", e);
 			Assert.fail("Error while deploying YARN cluster: "+e.getMessage());
@@ -248,7 +244,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 			}
 			GetClusterStatusResponse status = yarnCluster.getClusterStatus();
 			if(status != null && status.equals(expectedStatus)) {
-				LOG.info("Cluster reached status " + status);
+				LOG.info("ClusterClient reached status " + status);
 				break; // all good, cluster started
 			}
 			if(second > WAIT_TIME) {
@@ -263,7 +259,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 
 		LOG.info("Shutting down cluster. All tests passed");
 		// shutdown cluster
-		yarnCluster.shutdown(false);
+		yarnCluster.shutdown();
 		LOG.info("Finished testJavaAPI()");
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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 03ab647..4de964a 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
@@ -20,7 +20,7 @@ package org.apache.flink.yarn;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.flink.client.CliFrontend;
-import org.apache.flink.client.FlinkYarnSessionCli;
+import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.util.TestLogger;
 import org.apache.hadoop.conf.Configuration;
@@ -368,7 +368,7 @@ public abstract class YarnTestBase extends TestLogger {
 
 			File yarnConfFile = writeYarnSiteConfigXML(conf);
 			map.put("YARN_CONF_DIR", yarnConfFile.getParentFile().getAbsolutePath());
-			map.put("IN_TESTS", "yes we are in tests"); // see FlinkYarnClient() for more infos
+			map.put("IN_TESTS", "yes we are in tests"); // see YarnClusterDescriptor() for more infos
 			TestBaseUtils.setEnv(map);
 
 			Assert.assertTrue(yarnCluster.getServiceState() == Service.STATE.STARTED);


[02/10] flink git commit: [FLINK-3937] implement -yid option to Flink CLI

Posted by mx...@apache.org.
[FLINK-3937] implement -yid option to Flink CLI

- enables to use list, savepoint, cancel and stop subcommands
- adapt FlinkYarnSessionCli to also accept YARN application Id to attach to
- update documentation

This closes #2034


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

Branch: refs/heads/master
Commit: 875d4d238567a0503941488eb4e7d03b38c0fc42
Parents: 9e98424
Author: Sebastian Klemke <se...@researchgate.net>
Authored: Wed May 25 14:28:59 2016 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Fri Jun 17 10:37:58 2016 +0200

----------------------------------------------------------------------
 docs/apis/cli.md                                | 28 ++++++++++
 docs/setup/yarn_setup.md                        | 28 ++++++++++
 .../org/apache/flink/client/CliFrontend.java    |  4 ++
 .../flink/client/cli/CliFrontendParser.java     | 10 ++++
 .../yarn/AbstractYarnClusterDescriptor.java     | 15 ++++++
 .../flink/yarn/cli/FlinkYarnSessionCli.java     | 55 +++++++++++++++++++-
 6 files changed, 139 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/875d4d23/docs/apis/cli.md
----------------------------------------------------------------------
diff --git a/docs/apis/cli.md b/docs/apis/cli.md
index 08a2aa6..511862c 100644
--- a/docs/apis/cli.md
+++ b/docs/apis/cli.md
@@ -105,6 +105,10 @@ The command line can be used to
 
         ./bin/flink list -r
 
+-   List running Flink jobs inside Flink YARN session:
+
+        ./bin/flink list -m yarn-cluster -yid <yarnApplicationID> -r
+
 -   Cancel a job:
 
         ./bin/flink cancel <jobID>
@@ -252,6 +256,12 @@ Action "list" lists running and scheduled programs.
                                    configuration.
      -r,--running                  Show only running programs and their JobIDs
      -s,--scheduled                Show only scheduled programs and their JobIDs
+  Additional arguments if -m yarn-cluster is set:
+     -yid <yarnApplicationId>      YARN application ID of Flink YARN session to
+                                   connect to. Must not be set if JobManager HA
+                                   is used. In this case, JobManager RPC
+                                   location is automatically retrieved from
+                                   Zookeeper.
 
 
 Action "cancel" cancels a running program.
@@ -264,6 +274,12 @@ Action "cancel" cancels a running program.
                                    job. Use this flag to connect to a different
                                    JobManager than the one specified in the
                                    configuration.
+  Additional arguments if -m yarn-cluster is set:
+     -yid <yarnApplicationId>      YARN application ID of Flink YARN session to
+                                   connect to. Must not be set if JobManager HA
+                                   is used. In this case, JobManager RPC
+                                   location is automatically retrieved from
+                                   Zookeeper.
 
 
 Action "stop" stops a running program (streaming jobs only). There are no strong consistency
@@ -275,6 +291,12 @@ guarantees for a stop request.
                                    to connect. Use this flag to connect to a
                                    different JobManager than the one specified
                                    in the configuration.
+  Additional arguments if -m yarn-cluster is set:
+     -yid <yarnApplicationId>      YARN application ID of Flink YARN session to
+                                   connect to. Must not be set if JobManager HA
+                                   is used. In this case, JobManager RPC
+                                   location is automatically retrieved from
+                                   Zookeeper.
 
 
 Action "savepoint" triggers savepoints for a running job or disposes existing ones.
@@ -288,4 +310,10 @@ Action "savepoint" triggers savepoints for a running job or disposes existing on
                                     job. Use this flag to connect to a different
                                     JobManager than the one specified in the
                                     configuration.
+  Additional arguments if -m yarn-cluster is set:
+     -yid <yarnApplicationId>      YARN application ID of Flink YARN session to
+                                   connect to. Must not be set if JobManager HA
+                                   is used. In this case, JobManager RPC
+                                   location is automatically retrieved from
+                                   Zookeeper.
 ~~~

http://git-wip-us.apache.org/repos/asf/flink/blob/875d4d23/docs/setup/yarn_setup.md
----------------------------------------------------------------------
diff --git a/docs/setup/yarn_setup.md b/docs/setup/yarn_setup.md
index aa0f7a4..1e76a58 100644
--- a/docs/setup/yarn_setup.md
+++ b/docs/setup/yarn_setup.md
@@ -143,6 +143,34 @@ Note that in this case its not possible to stop the YARN session using Flink.
 
 Use the YARN utilities (`yarn application -kill <appId>`) to stop the YARN session.
 
+#### Attach to an existing Session
+
+Use the following command to start a session
+
+~~~bash
+./bin/yarn-session.sh
+~~~
+
+This command will show you the following overview:
+
+~~~bash
+Usage:
+   Required
+     -id,--applicationId <yarnAppId> YARN application Id
+~~~
+
+As already mentioned, `YARN_CONF_DIR` or `HADOOP_CONF_DIR` environment variable must be set to read the YARN and HDFS configuration.
+
+**Example:** Issue the following command to attach to running Flink YARN session `application_1463870264508_0029`:
+
+~~~bash
+./bin/yarn-session.sh -id application_1463870264508_0029
+~~~
+
+Attaching to a running session uses YARN ResourceManager to determine Job Manager RPC port.
+
+Stop the YARN session by stopping the unix process (using CTRL+C) or by entering 'stop' into the client.
+
 ### Submit Job to Flink
 
 Use the following command to submit a Flink program to the YARN cluster:

http://git-wip-us.apache.org/repos/asf/flink/blob/875d4d23/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index cf7a8c2..3064f8d 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -800,6 +800,10 @@ public class CliFrontend {
 	 */
 	protected void updateConfig(CommandLineOptions options) {
 		if(options.getJobManagerAddress() != null){
+			if (YARN_DEPLOY_JOBMANAGER.equals(options.getJobManagerAddress())) {
+				jobManagerAddress = CliFrontendParser.getFlinkYarnSessionCli()
+					.attachFlinkYarnClient(options.getCommandLine())
+					.getJobManagerAddress();
 			InetSocketAddress jobManagerAddress = ClientUtils.parseHostPortAddress(options.getJobManagerAddress());
 			writeJobManagerAddressToConfig(config, jobManagerAddress);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/875d4d23/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 f28d1b6..9b935e8 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
@@ -188,6 +188,8 @@ public class CliFrontendParser {
 
 	private static Options getJobManagerAddressOption(Options options) {
 		options.addOption(ADDRESS_OPTION);
+		yarnSessionCLi.getYARNAttachCLIOptions(options);
+
 		return options;
 	}
 
@@ -280,6 +282,10 @@ public class CliFrontendParser {
 		System.out.println("\n  Syntax: info [OPTIONS] <jar-file> <arguments>");
 		formatter.setSyntaxPrefix("  \"info\" action options:");
 		formatter.printHelp(" ", getInfoOptionsWithoutDeprecatedOptions(new Options()));
+		formatter.setSyntaxPrefix("  Additional arguments if -m " + CliFrontend.YARN_DEPLOY_JOBMANAGER + " is set:");
+		Options yarnOpts = new Options();
+		yarnSessionCLi.getYARNSessionCLIOptions(yarnOpts);
+		formatter.printHelp(" ", yarnOpts);
 		System.out.println();
 	}
 
@@ -316,6 +322,10 @@ public class CliFrontendParser {
 		System.out.println("\n  Syntax: cancel [OPTIONS] <Job ID>");
 		formatter.setSyntaxPrefix("  \"cancel\" action options:");
 		formatter.printHelp(" ", getCancelOptions(new Options()));
+		formatter.setSyntaxPrefix("  Additional arguments if -m " + CliFrontend.YARN_DEPLOY_JOBMANAGER + " is set:");
+		Options yarnOpts = new Options();
+		yarnSessionCLi.getYARNSessionCLIOptions(yarnOpts);
+		formatter.printHelp(" ", yarnOpts);
 		System.out.println();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/875d4d23/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 7220a29..c471fa4 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
@@ -48,6 +48,7 @@ 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;
@@ -349,6 +350,20 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		}
 	}
 
+	@Override
+	public AbstractFlinkYarnCluster attach(String appId) throws Exception {
+		// check if required Hadoop environment variables are set. If not, warn user
+		if(System.getenv("HADOOP_CONF_DIR") == null &&
+			System.getenv("YARN_CONF_DIR") == null) {
+			LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." +
+				"The Flink YARN Client needs one of these to be set to properly load the Hadoop " +
+				"configuration for accessing YARN.");
+		}
+
+		final ApplicationId yarnAppId = ConverterUtils.toApplicationId(appId);
+
+		return new FlinkYarnCluster(yarnClient, yarnAppId, conf, flinkConfiguration, sessionFilesDir, detached);
+	}
 	/**
 	 * This method will block until the ApplicationMaster/JobManager have been
 	 * deployed on YARN.

http://git-wip-us.apache.org/repos/asf/flink/blob/875d4d23/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 a2375c5..fdcc858 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
@@ -32,6 +32,7 @@ import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.yarn.YarnClusterDescriptor;
 import org.apache.flink.yarn.YarnClusterClient;
 import org.apache.flink.runtime.clusterframework.ApplicationStatus;
@@ -86,6 +87,8 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 	// the prefix transformation is used by the CliFrontend static constructor.
 	private final Option QUERY;
 	// --- or ---
+	private final Option APPLICATION_ID;
+	// --- or ---
 	private final Option QUEUE;
 	private final Option SHIP_PATH;
 	private final Option FLINK_JAR;
@@ -117,6 +120,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 		this.acceptInteractiveInput = acceptInteractiveInput;
 		
 		QUERY = new Option(shortPrefix + "q", longPrefix + "query", false, "Display available YARN resources (memory, cores)");
+		APPLICATION_ID = new Option(shortPrefix + "id", longPrefix + "applicationId", true, "Attach to running YARN session");
 		QUEUE = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue.");
 		SHIP_PATH = new Option(shortPrefix + "t", longPrefix + "ship", true, "Ship files in the specified directory (t for transfer)");
 		FLINK_JAR = new Option(shortPrefix + "j", longPrefix + "jar", true, "Path to Flink jar file");
@@ -131,6 +135,35 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 	}
 
 	/**
+	 * Attaches a new Yarn Client to running YARN application.
+	 *
+	 */
+	public AbstractFlinkYarnCluster attachFlinkYarnClient(CommandLine cmd) {
+		AbstractFlinkYarnClient flinkYarnClient = getFlinkYarnClient();
+		if (flinkYarnClient == null) {
+			return null;
+		}
+
+		if (!cmd.hasOption(APPLICATION_ID.getOpt())) {
+			LOG.error("Missing required argument " + APPLICATION_ID.getOpt());
+			printUsage();
+			return null;
+		}
+
+		String confDirPath = CliFrontend.getConfigurationDirectoryFromEnv();
+		GlobalConfiguration.loadConfiguration(confDirPath);
+		Configuration flinkConfiguration = GlobalConfiguration.getConfiguration();
+		flinkYarnClient.setFlinkConfiguration(flinkConfiguration);
+		flinkYarnClient.setConfigurationDirectory(confDirPath);
+
+		try {
+			return flinkYarnClient.attach(cmd.getOptionValue(APPLICATION_ID.getOpt()));
+		} catch (Exception e) {
+			LOG.error("Could not attach to YARN session", e);
+			return null;
+		}
+	}
+	/**
 	 * Resumes from a Flink Yarn properties file
 	 * @param flinkConfiguration The flink configuration
 	 * @return True if the properties were loaded, false otherwise
@@ -452,6 +485,11 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 		options.addOption(NAME);
 	}
 
+
+	public void getYARNAttachCLIOptions(Options options) {
+		options.addOption(APPLICATION_ID);
+	}
+
 	@Override
 	public ClusterClient retrieveCluster(Configuration config) throws Exception {
 
@@ -478,7 +516,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 			printUsage();
 			return 1;
 		}
-		
+
 		// Query cluster for metrics
 		if (cmd.hasOption(QUERY.getOpt())) {
 			YarnClusterDescriptor flinkYarnClient = new YarnClusterDescriptor();
@@ -492,6 +530,21 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
 			}
 			System.out.println(description);
 			return 0;
+		} else if (cmd.hasOption(APPLICATION_ID.getOpt())) {
+			yarnCluster = attachFlinkYarnClient(cmd);
+
+			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 "+yarnCluster.getApplicationId());
+			} else {
+				runInteractiveCli(yarnCluster);
+
+				if (!yarnCluster.hasBeenStopped()) {
+					LOG.info("Command Line Interface requested session shutdown");
+					yarnCluster.shutdown(false);
+				}
+			}
 		} else {
 
 			YarnClusterDescriptor flinkYarnClient;


[10/10] flink git commit: [FLINK-3863] Yarn Cluster shutdown may fail if leader changed recently

Posted by mx...@apache.org.
[FLINK-3863] Yarn Cluster shutdown may fail if leader changed recently


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

Branch: refs/heads/master
Commit: 9e9842410a635d183a002d1f25a6f489ce9d6a2f
Parents: f9b52a3
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Jun 1 12:45:52 2016 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Fri Jun 17 10:37:58 2016 +0200

----------------------------------------------------------------------
 .../apache/flink/yarn/ApplicationClient.scala   | 31 ++++++++------------
 1 file changed, 12 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9e984241/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
index aea1aac..e701269 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
@@ -176,29 +176,22 @@ class ApplicationClient(
           }
       }
 
-    case LocalStopYarnSession(status, diagnostics) =>
+    case msg @ LocalStopYarnSession(status, diagnostics) =>
       log.info("Sending StopCluster request to JobManager.")
 
-      val clusterStatus =
-        status match {
-          case FinalApplicationStatus.SUCCEEDED => ApplicationStatus.SUCCEEDED
-          case FinalApplicationStatus.KILLED => ApplicationStatus.CANCELED
-          case FinalApplicationStatus.FAILED => ApplicationStatus.FAILED
-          case _ => ApplicationStatus.UNKNOWN
-        }
-
-      yarnJobManager foreach {
-        // forward to preserve the sender's address
-        _ forward decorateMessage(new StopCluster(clusterStatus, diagnostics))
+      // preserve the original sender so we can reply
+      val originalSender = sender()
+
+      yarnJobManager match {
+        case Some(jm) =>
+          jm.tell(decorateMessage(new StopCluster(status, diagnostics)), originalSender)
+        case None =>
+          context.system.scheduler.scheduleOnce(1 second) {
+            // try once more; we might have been connected in the meantime
+            self.tell(msg, originalSender)
+          }(context.dispatcher)
       }
 
-    case msg: StopClusterSuccessful =>
-      log.info("Remote JobManager has been stopped successfully. " +
-        "Stopping local application client")
-
-      // poison ourselves
-      self ! decorateMessage(PoisonPill)
-
     // handle the responses from the PollYarnClusterStatus messages to the yarn job mgr
     case status: GetClusterStatusResponse =>
       latestClusterStatus = Some(status)


[09/10] flink git commit: [FLINK-3937] programmatic resuming of clusters

Posted by mx...@apache.org.
[FLINK-3937] programmatic resuming of clusters

- integrates with and extends the refactoring of FLINK-3667
- enables to resume from Yarn properties or Yarn application id
- introduces additional StandaloneClusterDescriptor
- introduces DefaultCLI to get rid of standalone mode switches in CliFrontend
- various fixes and improvements
- remove legacy code from CliFrontend
- change activation code of CustomCommandLine interface
- use checked exceptions to signal supported operations
- remove all checked exceptions of type Exception
- fix logging and reduce verbosity of per-job clusters
- print 'id' argument in YarnSessionCli
- minor renaming of methods names
- improve documentation
- deprecate streaming option
- extend CliFrontendYarnAddressConfigurationTest
- move loading of custom CLIs to CliFrontend

This closes #2085


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

Branch: refs/heads/master
Commit: f4ac852275da36ee33aa54ae9097293ccc981afa
Parents: 875d4d2
Author: Maximilian Michels <mx...@apache.org>
Authored: Mon Apr 25 16:28:51 2016 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Fri Jun 17 10:37:58 2016 +0200

----------------------------------------------------------------------
 .../org/apache/flink/client/CliFrontend.java    | 217 +++++------
 .../flink/client/cli/CliFrontendParser.java     | 225 ++++++------
 .../flink/client/cli/CustomCommandLine.java     |  34 +-
 .../org/apache/flink/client/cli/DefaultCLI.java |  77 ++++
 .../client/deployment/ClusterDescriptor.java    |  14 +-
 .../deployment/StandaloneClusterDescriptor.java |  56 +++
 .../flink/client/program/ClusterClient.java     |  16 +-
 .../CliFrontendAddressConfigurationTest.java    |  59 +--
 .../org/apache/flink/api/scala/FlinkShell.scala |  24 +-
 ...CliFrontendYarnAddressConfigurationTest.java | 360 +++++++++++++++----
 .../flink/yarn/FlinkYarnSessionCliTest.java     |  10 +-
 .../yarn/AbstractYarnClusterDescriptor.java     | 220 ++++++++----
 .../apache/flink/yarn/YarnClusterClient.java    | 151 ++++----
 .../flink/yarn/YarnClusterDescriptor.java       |   2 +
 .../flink/yarn/cli/FlinkYarnSessionCli.java     | 226 ++++++------
 15 files changed, 1062 insertions(+), 629 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index 3064f8d..a01ab53 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -18,8 +18,7 @@
 
 package org.apache.flink.client;
 
-import akka.actor.ActorSystem;
-
+import org.apache.commons.cli.CommandLine;
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
@@ -30,6 +29,7 @@ import org.apache.flink.client.cli.CliArgsException;
 import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.cli.CommandLineOptions;
 import org.apache.flink.client.cli.CustomCommandLine;
+import org.apache.flink.client.cli.DefaultCLI;
 import org.apache.flink.client.cli.InfoOptions;
 import org.apache.flink.client.cli.ListOptions;
 import org.apache.flink.client.cli.ProgramOptions;
@@ -39,7 +39,6 @@ import org.apache.flink.client.cli.StopOptions;
 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.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
@@ -56,7 +55,6 @@ import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.client.JobStatusMessage;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
 import org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure;
@@ -67,13 +65,12 @@ import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess;
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.runtime.util.EnvironmentInformation;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.StringUtils;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import scala.Some;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -81,6 +78,8 @@ import scala.concurrent.duration.FiniteDuration;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.net.InetSocketAddress;
 import java.net.URL;
 import java.text.SimpleDateFormat;
@@ -89,6 +88,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Date;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
@@ -102,9 +102,11 @@ import static org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepo
  */
 public class CliFrontend {
 
+	private static final Logger LOG = LoggerFactory.getLogger(CliFrontend.class);
+
 	// actions
-	public static final String ACTION_RUN = "run";
-	public static final String ACTION_INFO = "info";
+	private static final String ACTION_RUN = "run";
+	private static final String ACTION_INFO = "info";
 	private static final String ACTION_LIST = "list";
 	private static final String ACTION_CANCEL = "cancel";
 	private static final String ACTION_STOP = "stop";
@@ -116,19 +118,24 @@ public class CliFrontend {
 	private static final String CONFIG_DIRECTORY_FALLBACK_2 = "conf";
 
 	// --------------------------------------------------------------------------------------------
+
+	private static final List<CustomCommandLine> customCommandLine = new LinkedList<>();
+
+	static {
+		/** command line interface of the YARN session, with a special initialization here
+		 *  to prefix all options with y/yarn. */
+		loadCustomCommandLine("org.apache.flink.yarn.cli.FlinkYarnSessionCli", "y", "yarn");
+		customCommandLine.add(new DefaultCLI());
+	}
+
 	// --------------------------------------------------------------------------------------------
 
-	private static final Logger LOG = LoggerFactory.getLogger(CliFrontend.class);
 
 
 	private final Configuration config;
 
 	private final FiniteDuration clientTimeout;
 
-	private final FiniteDuration lookupTimeout;
-
-	private ActorSystem actorSystem;
-
 	/**
 	 *
 	 * @throws Exception Thrown if the configuration directory was not found, the configuration could not be loaded
@@ -146,6 +153,8 @@ public class CliFrontend {
 		// load the configuration
 		LOG.info("Trying to load configuration file");
 		GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath());
+		System.setProperty("FLINK_CONF_DIR", configDirectory.getAbsolutePath());
+
 		this.config = GlobalConfiguration.getConfiguration();
 
 		try {
@@ -156,7 +165,6 @@ public class CliFrontend {
 		}
 
 		this.clientTimeout = AkkaUtils.getClientTimeout(config);
-		this.lookupTimeout = AkkaUtils.getLookupTimeout(config);
 	}
 
 
@@ -798,19 +806,20 @@ public class CliFrontend {
 	 *
 	 * @param options Command line options
 	 */
-	protected void updateConfig(CommandLineOptions options) {
-		if(options.getJobManagerAddress() != null){
-			if (YARN_DEPLOY_JOBMANAGER.equals(options.getJobManagerAddress())) {
-				jobManagerAddress = CliFrontendParser.getFlinkYarnSessionCli()
-					.attachFlinkYarnClient(options.getCommandLine())
-					.getJobManagerAddress();
-			InetSocketAddress jobManagerAddress = ClientUtils.parseHostPortAddress(options.getJobManagerAddress());
-			writeJobManagerAddressToConfig(config, jobManagerAddress);
+	protected ClusterClient retrieveClient(CommandLineOptions options) {
+		CustomCommandLine customCLI = getActiveCustomCommandLine(options.getCommandLine());
+		try {
+			ClusterClient client = customCLI.retrieveCluster(options.getCommandLine(), config);
+			LOG.info("Using address {} to connect to JobManager.", client.getJobManagerAddressFromConfig());
+			return client;
+		} catch (Exception e) {
+			LOG.error("Couldn't retrieve {} cluster.", customCLI.getId(), e);
+			throw new IllegalConfigurationException("Couldn't retrieve client for cluster", e);
 		}
 	}
 
 	/**
-	 * Retrieves the {@link ActorGateway} for the JobManager. The JobManager address is retrieved
+	 * Retrieves the {@link ActorGateway} for the JobManager. The ClusterClient is retrieved
 	 * from the provided {@link CommandLineOptions}.
 	 *
 	 * @param options CommandLineOptions specifying the JobManager URL
@@ -818,92 +827,41 @@ public class CliFrontend {
 	 * @throws Exception
 	 */
 	protected ActorGateway getJobManagerGateway(CommandLineOptions options) throws Exception {
-		// overwrite config values with given command line options
-		updateConfig(options);
-
-		// start an actor system if needed
-		if (this.actorSystem == null) {
-			LOG.info("Starting actor system to communicate with JobManager");
-			try {
-				scala.Tuple2<String, Object> systemEndpoint = new scala.Tuple2<String, Object>("", 0);
-				this.actorSystem = AkkaUtils.createActorSystem(
-						config,
-						new Some<scala.Tuple2<String, Object>>(systemEndpoint));
-			}
-			catch (Exception e) {
-				throw new IOException("Could not start actor system to communicate with JobManager", e);
-			}
-
-			LOG.info("Actor system successfully started");
-		}
-
-		LOG.info("Trying to lookup the JobManager gateway");
-		// Retrieve the ActorGateway from the LeaderRetrievalService
-		LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config);
-
-		return LeaderRetrievalUtils.retrieveLeaderGateway(lrs, actorSystem, lookupTimeout);
+		return retrieveClient(options).getJobManagerGateway();
 	}
 
 	/**
-	 * Retrieves a {@link ClusterClient} object from the given command line options and other parameters.
-	 *
-	 * @param options Command line options which contain JobManager address
+	 * Creates a {@link ClusterClient} object from the given command line options and other parameters.
+	 * @param options Command line options
 	 * @param programName Program name
 	 * @throws Exception
 	 */
 	protected ClusterClient getClient(
 			CommandLineOptions options,
-			String programName)
-		throws Exception {
-		InetSocketAddress jobManagerAddress;
-
-		// try to get the JobManager address via command-line args
-		if (options.getJobManagerAddress() != null) {
+			String programName) throws Exception {
 
-			// Get the custom command-lines (e.g. Yarn/Mesos)
-			CustomCommandLine<?> activeCommandLine =
-				CliFrontendParser.getActiveCustomCommandLine(options.getJobManagerAddress());
+		// Get the custom command-line (e.g. Standalone/Yarn/Mesos)
+		CustomCommandLine<?> activeCommandLine = getActiveCustomCommandLine(options.getCommandLine());
 
-			if (activeCommandLine != null) {
-				logAndSysout(activeCommandLine.getIdentifier() + " mode detected. Switching Log4j output to console");
-
-				// Default yarn application name to use, if nothing is specified on the command line
+		ClusterClient client;
+		try {
+			client = activeCommandLine.retrieveCluster(options.getCommandLine(), config);
+			logAndSysout("Cluster retrieved");
+		} catch (UnsupportedOperationException e) {
+			try {
 				String applicationName = "Flink Application: " + programName;
-
-				ClusterClient client = activeCommandLine.createClient(applicationName, options.getCommandLine());
-
+				client = activeCommandLine.createCluster(applicationName, options.getCommandLine(), config);
 				logAndSysout("Cluster started");
-				logAndSysout("JobManager web interface address " + client.getWebInterfaceURL());
-
-				return client;
-			} else {
-				// job manager address supplied on the command-line
-				LOG.info("Using address {} to connect to JobManager.", options.getJobManagerAddress());
-				jobManagerAddress = ClientUtils.parseHostPortAddress(options.getJobManagerAddress());
-				writeJobManagerAddressToConfig(config, jobManagerAddress);
-				return new StandaloneClusterClient(config);
-			}
-
-		// try to get the JobManager address via resuming of a cluster
-		} else {
-			for (CustomCommandLine cli : CliFrontendParser.getAllCustomCommandLine().values()) {
-				ClusterClient client = cli.retrieveCluster(config);
-				if (client != null) {
-					LOG.info("Using address {} to connect to JobManager.", client.getJobManagerAddressFromConfig());
-					return client;
-				}
+			} catch (UnsupportedOperationException e2) {
+				throw new IllegalConfigurationException(
+					"The JobManager address is neither provided at the command-line, " +
+						"nor configured in flink-conf.yaml.");
 			}
 		}
 
-		// read JobManager address from the config
-		if (config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null) != null) {
-			return new StandaloneClusterClient(config);
-		// We tried hard but couldn't find a JobManager address
-		} else {
-			throw new IllegalConfigurationException(
-				"The JobManager address is neither provided at the command-line, " +
-					"nor configured in flink-conf.yaml.");
-		}
+		logAndSysout("Using address " + client.getJobManagerAddress() + " to connect to JobManager.");
+		logAndSysout("JobManager web interface address " + client.getWebInterfaceURL());
+		return client;
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -917,7 +875,7 @@ public class CliFrontend {
 	 * @return The return code for the process.
 	 */
 	private int handleArgException(Exception e) {
-		LOG.error("Invalid command line arguments." + (e.getMessage() == null ? "" : e.getMessage()));
+		LOG.error("Invalid command line arguments. " + (e.getMessage() == null ? "" : e.getMessage()));
 
 		System.out.println(e.getMessage());
 		System.out.println();
@@ -1039,14 +997,6 @@ public class CliFrontend {
 		}
 	}
 
-	public void shutdown() {
-		ActorSystem sys = this.actorSystem;
-		if (sys != null) {
-			this.actorSystem = null;
-			sys.shutdown();
-		}
-	}
-
 	/**
 	 * Submits the job based on the arguments
 	 */
@@ -1070,7 +1020,8 @@ public class CliFrontend {
 	// --------------------------------------------------------------------------------------------
 
 	public static String getConfigurationDirectoryFromEnv() {
-		String location = System.getenv(ENV_CONFIG_DIRECTORY);
+		String envLocation = System.getenv(ENV_CONFIG_DIRECTORY);
+		String location = envLocation != null ? envLocation : System.getProperty(ENV_CONFIG_DIRECTORY);
 
 		if (location != null) {
 			if (new File(location).exists()) {
@@ -1102,9 +1053,65 @@ public class CliFrontend {
 	 * @param address Address to write to the configuration
 	 * @param config The config to write to
 	 */
-	public static void writeJobManagerAddressToConfig(Configuration config, InetSocketAddress address) {
+	public static void setJobManagerAddressInConfig(Configuration config, InetSocketAddress address) {
 		config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.getHostName());
 		config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.getPort());
 	}
 
+	// --------------------------------------------------------------------------------------------
+	//  Custom command-line
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Gets the custom command-line for the arguments.
+	 * @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 : customCommandLine) {
+			if (cli.isActive(commandLine, config)) {
+				return cli;
+			}
+		}
+		throw new IllegalStateException("No command-line ran.");
+	}
+
+	/**
+	 * Retrieves the loaded custom command-lines.
+	 * @return An unmodifiyable list of loaded custom command-lines.
+	 */
+	public static List<CustomCommandLine> getCustomCommandLineList() {
+		return Collections.unmodifiableList(customCommandLine);
+	}
+
+	/**
+	 * Loads a class from the classpath that implements the CustomCommandLine interface.
+	 * @param className The fully-qualified class name to load.
+	 * @param params The constructor parameters
+	 */
+	private static void loadCustomCommandLine(String className, Object... params) {
+
+		try {
+			Class<? extends CustomCommandLine> customCliClass =
+				Class.forName(className).asSubclass(CustomCommandLine.class);
+
+			// construct class types from the parameters
+			Class<?>[] types = new Class<?>[params.length];
+			for (int i = 0; i < params.length; i++) {
+				Preconditions.checkNotNull(params[i], "Parameters for custom command-lines may not be null.");
+				types[i] = params[i].getClass();
+			}
+
+			Constructor<? extends CustomCommandLine> constructor = customCliClass.getConstructor(types);
+			final CustomCommandLine cli = constructor.newInstance(params);
+
+			customCommandLine.add(cli);
+
+		} catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException | InstantiationException
+			| InvocationTargetException e) {
+			LOG.warn("Unable to locate custom CLI class {}. " +
+				"Flink is not compiled with support for this class.", className, e);
+		}
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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 9b935e8..c90793d 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
@@ -24,16 +24,10 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.PosixParser;
 
-import org.apache.flink.util.Preconditions;
+import org.apache.flink.client.CliFrontend;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
 
 /**
  * A simple command line parser (based on Apache Commons CLI) that extracts command
@@ -44,16 +38,6 @@ public class CliFrontendParser {
 	private static final Logger LOG = LoggerFactory.getLogger(CliFrontendParser.class);
 
 
-	/** command line interface of the YARN session, with a special initialization here
-	 *  to prefix all options with y/yarn. */
-	private static final Map<String, CustomCommandLine> customCommandLine = new HashMap<>(1);
-
-	static {
-		// we could easily add more here in the future
-		loadCustomCommandLine("org.apache.flink.yarn.cli.FlinkYarnSessionCli", "y", "yarn");
-	}
-
-
 	static final Option HELP_OPTION = new Option("h", "help", false,
 			"Show the help message for the CLI Frontend or the action.");
 
@@ -82,9 +66,8 @@ public class CliFrontendParser {
 	static final Option ARGS_OPTION = new Option("a", "arguments", true,
 			"Program arguments. Arguments can also be added without -a, simply as trailing parameters.");
 
-	static final Option ADDRESS_OPTION = new Option("m", "jobmanager", true,
+	public static final Option ADDRESS_OPTION = new Option("m", "jobmanager", true,
 			"Address of the JobManager (master) to which to connect. " +
-			"Specify " + getCliIdentifierString() +" as the JobManager to deploy a cluster for the job. " +
 			"Use this flag to connect to a different JobManager than the one specified in the configuration.");
 
 	static final Option SAVEPOINT_PATH_OPTION = new Option("s", "fromSavepoint", true,
@@ -146,6 +129,10 @@ public class CliFrontendParser {
 		options.addOption(HELP_OPTION);
 		// backwards compatibility: ignore verbose flag (-v)
 		options.addOption(new Option("v", "verbose", false, "This option is deprecated."));
+		// add general options of all CLIs
+		for (CustomCommandLine customCLI : CliFrontend.getCustomCommandLineList()) {
+			customCLI.addGeneralOptions(options);
+		}
 		return options;
 	}
 
@@ -158,11 +145,6 @@ public class CliFrontendParser {
 		options.addOption(LOGGING_OPTION);
 		options.addOption(DETACHED_OPTION);
 		options.addOption(SAVEPOINT_PATH_OPTION);
-
-		for (CustomCommandLine customCLI : customCommandLine.values()) {
-			customCLI.addOptions(options);
-		}
-
 		return options;
 	}
 
@@ -177,62 +159,85 @@ public class CliFrontendParser {
 	}
 
 	private static Options getRunOptions(Options options) {
-		Options o = getProgramSpecificOptions(options);
-		return getJobManagerAddressOption(o);
+		options = getProgramSpecificOptions(options);
+		options = getJobManagerAddressOption(options);
+		return addCustomCliOptions(options, true);
 	}
 
-	private static Options getRunOptionsWithoutDeprecatedOptions(Options options) {
-		Options o = getProgramSpecificOptionsWithoutDeprecatedOptions(options);
-		return getJobManagerAddressOption(o);
-	}
 
 	private static Options getJobManagerAddressOption(Options options) {
 		options.addOption(ADDRESS_OPTION);
-		yarnSessionCLi.getYARNAttachCLIOptions(options);
-
 		return options;
 	}
 
 	private static Options getInfoOptions(Options options) {
 		options = getProgramSpecificOptions(options);
 		options = getJobManagerAddressOption(options);
-		return options;
+		return addCustomCliOptions(options, false);
+	}
+
+	private static Options getListOptions(Options options) {
+		options.addOption(RUNNING_OPTION);
+		options.addOption(SCHEDULED_OPTION);
+		options = getJobManagerAddressOption(options);
+		return addCustomCliOptions(options, false);
+	}
+
+	private static Options getCancelOptions(Options options) {
+		options = getJobManagerAddressOption(options);
+		return addCustomCliOptions(options, false);
+	}
+
+	private static Options getStopOptions(Options options) {
+		options = getJobManagerAddressOption(options);
+		return addCustomCliOptions(options, false);
+	}
+
+	private static Options getSavepointOptions(Options options) {
+		options = getJobManagerAddressOption(options);
+		options.addOption(SAVEPOINT_DISPOSE_OPTION);
+		return addCustomCliOptions(options, false);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Help
+	// --------------------------------------------------------------------------------------------
+
+	private static Options getRunOptionsWithoutDeprecatedOptions(Options options) {
+		Options o = getProgramSpecificOptionsWithoutDeprecatedOptions(options);
+		return getJobManagerAddressOption(o);
 	}
 
+
 	private static Options getInfoOptionsWithoutDeprecatedOptions(Options options) {
 		options.addOption(CLASS_OPTION);
 		options.addOption(PARALLELISM_OPTION);
-		options = getJobManagerAddressOption(options);
 		return options;
 	}
 
-	private static Options getListOptions(Options options) {
+	private static Options getListOptionsWithoutDeprecatedOptions(Options options) {
 		options.addOption(RUNNING_OPTION);
 		options.addOption(SCHEDULED_OPTION);
 		options = getJobManagerAddressOption(options);
 		return options;
 	}
 
-	private static Options getCancelOptions(Options options) {
+	private static Options getCancelOptionsWithoutDeprecatedOptions(Options options) {
 		options = getJobManagerAddressOption(options);
 		return options;
 	}
 
-	private static Options getStopOptions(Options options) {
+	private static Options getStopOptionsWithoutDeprecatedOptions(Options options) {
 		options = getJobManagerAddressOption(options);
 		return options;
 	}
 
-	private static Options getSavepointOptions(Options options) {
+	private static Options getSavepointOptionsWithoutDeprecatedOptions(Options options) {
 		options = getJobManagerAddressOption(options);
 		options.addOption(SAVEPOINT_DISPOSE_OPTION);
 		return options;
 	}
 
-	// --------------------------------------------------------------------------------------------
-	//  Help
-	// --------------------------------------------------------------------------------------------
-
 	/**
 	 * Prints the help for the client.
 	 */
@@ -261,14 +266,7 @@ public class CliFrontendParser {
 		formatter.setSyntaxPrefix("  \"run\" action options:");
 		formatter.printHelp(" ", getRunOptionsWithoutDeprecatedOptions(new Options()));
 
-		// prints options from all available command-line classes
-		for (Map.Entry<String, CustomCommandLine> entry: customCommandLine.entrySet()) {
-			formatter.setSyntaxPrefix("  Additional arguments if -m " + entry.getKey() + " is set:");
-			Options customOpts = new Options();
-			entry.getValue().addOptions(customOpts);
-			formatter.printHelp(" ", customOpts);
-			System.out.println();
-		}
+		printCustomCliOptions(formatter, true);
 
 		System.out.println();
 	}
@@ -282,10 +280,9 @@ public class CliFrontendParser {
 		System.out.println("\n  Syntax: info [OPTIONS] <jar-file> <arguments>");
 		formatter.setSyntaxPrefix("  \"info\" action options:");
 		formatter.printHelp(" ", getInfoOptionsWithoutDeprecatedOptions(new Options()));
-		formatter.setSyntaxPrefix("  Additional arguments if -m " + CliFrontend.YARN_DEPLOY_JOBMANAGER + " is set:");
-		Options yarnOpts = new Options();
-		yarnSessionCLi.getYARNSessionCLIOptions(yarnOpts);
-		formatter.printHelp(" ", yarnOpts);
+
+		printCustomCliOptions(formatter, false);
+
 		System.out.println();
 	}
 
@@ -297,7 +294,10 @@ public class CliFrontendParser {
 		System.out.println("\nAction \"list\" lists running and scheduled programs.");
 		System.out.println("\n  Syntax: list [OPTIONS]");
 		formatter.setSyntaxPrefix("  \"list\" action options:");
-		formatter.printHelp(" ", getListOptions(new Options()));
+		formatter.printHelp(" ", getListOptionsWithoutDeprecatedOptions(new Options()));
+
+		printCustomCliOptions(formatter, false);
+
 		System.out.println();
 	}
 
@@ -309,7 +309,10 @@ public class CliFrontendParser {
 		System.out.println("\nAction \"stop\" stops a running program (streaming jobs only).");
 		System.out.println("\n  Syntax: stop [OPTIONS] <Job ID>");
 		formatter.setSyntaxPrefix("  \"stop\" action options:");
-		formatter.printHelp(" ", getStopOptions(new Options()));
+		formatter.printHelp(" ", getStopOptionsWithoutDeprecatedOptions(new Options()));
+
+		printCustomCliOptions(formatter, false);
+
 		System.out.println();
 	}
 
@@ -321,11 +324,10 @@ public class CliFrontendParser {
 		System.out.println("\nAction \"cancel\" cancels a running program.");
 		System.out.println("\n  Syntax: cancel [OPTIONS] <Job ID>");
 		formatter.setSyntaxPrefix("  \"cancel\" action options:");
-		formatter.printHelp(" ", getCancelOptions(new Options()));
-		formatter.setSyntaxPrefix("  Additional arguments if -m " + CliFrontend.YARN_DEPLOY_JOBMANAGER + " is set:");
-		Options yarnOpts = new Options();
-		yarnSessionCLi.getYARNSessionCLIOptions(yarnOpts);
-		formatter.printHelp(" ", yarnOpts);
+		formatter.printHelp(" ", getCancelOptionsWithoutDeprecatedOptions(new Options()));
+
+		printCustomCliOptions(formatter, false);
+
 		System.out.println();
 	}
 
@@ -337,10 +339,50 @@ public class CliFrontendParser {
 		System.out.println("\nAction \"savepoint\" triggers savepoints for a running job or disposes existing ones.");
 		System.out.println("\n  Syntax: savepoint [OPTIONS] <Job ID>");
 		formatter.setSyntaxPrefix("  \"savepoint\" action options:");
-		formatter.printHelp(" ", getSavepointOptions(new Options()));
+		formatter.printHelp(" ", getSavepointOptionsWithoutDeprecatedOptions(new Options()));
+
+		printCustomCliOptions(formatter, false);
+
 		System.out.println();
 	}
 
+	/**
+	 * Adds custom cli options
+	 * @param options The options to add options to
+	 * @param runOptions Whether to include run options
+	 * @return Options with additions
+	 */
+	private static Options addCustomCliOptions(Options options, boolean runOptions) {
+		for (CustomCommandLine cli: CliFrontend.getCustomCommandLineList()) {
+			cli.addGeneralOptions(options);
+			if (runOptions) {
+				cli.addRunOptions(options);
+			}
+		}
+		return options;
+	}
+
+	/**
+	 * Prints custom cli options
+	 * @param formatter The formatter to use for printing
+	 * @param runOptions True if the run options should be printed, False to print only general options
+	 */
+	private static void printCustomCliOptions(HelpFormatter formatter, boolean runOptions) {
+		// prints options from all available command-line classes
+		for (CustomCommandLine cli: CliFrontend.getCustomCommandLineList()) {
+			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();
+			}
+		}
+	}
+
 	// --------------------------------------------------------------------------------------------
 	//  Line Parsing
 	// --------------------------------------------------------------------------------------------
@@ -410,63 +452,4 @@ public class CliFrontendParser {
 		}
 	}
 
-	public static Map<String, CustomCommandLine> getAllCustomCommandLine() {
-		if (customCommandLine.isEmpty()) {
-			LOG.warn("No custom command-line classes were loaded.");
-		}
-		return Collections.unmodifiableMap(customCommandLine);
-	}
-
-	private static String getCliIdentifierString() {
-		StringBuilder builder = new StringBuilder();
-		boolean first = true;
-		for (String identifier : customCommandLine.keySet()) {
-			if (!first) {
-				builder.append(", ");
-			}
-			first = false;
-			builder.append("'").append(identifier).append("'");
-		}
-		return builder.toString();
-	}
-
-	/**
-	 * Gets the custom command-line for this identifier.
-	 * @param identifier The unique identifier for this command-line implementation.
-	 * @return CustomCommandLine or null if none was found
-	 */
-	public static CustomCommandLine getActiveCustomCommandLine(String identifier) {
-		return CliFrontendParser.getAllCustomCommandLine().get(identifier);
-	}
-
-	private static void loadCustomCommandLine(String className, Object... params) {
-
-		try {
-			Class<? extends CustomCommandLine> customCliClass =
-				Class.forName(className).asSubclass(CustomCommandLine.class);
-
-			// construct class types from the parameters
-			Class<?>[] types = new Class<?>[params.length];
-			for (int i = 0; i < params.length; i++) {
-				Preconditions.checkNotNull(params[i], "Parameters for custom command-lines may not be null.");
-				types[i] = params[i].getClass();
-			}
-
-			Constructor<? extends CustomCommandLine> constructor = customCliClass.getConstructor(types);
-			final CustomCommandLine cli = constructor.newInstance(params);
-
-			String cliIdentifier = Preconditions.checkNotNull(cli.getIdentifier());
-			CustomCommandLine existing = customCommandLine.put(cliIdentifier, cli);
-
-			if (existing != null) {
-				throw new IllegalStateException("Attempted to register " + cliIdentifier +
-					" but there is already a command-line with this identifier.");
-			}
-		} catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException | InstantiationException
-			| InvocationTargetException e) {
-			LOG.warn("Unable to locate custom CLI class {}. " +
-				"Flink is not compiled with support for this class.", className, e);
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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 cd5e0e6..aecdc7c 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
@@ -29,29 +29,47 @@ import org.apache.flink.configuration.Configuration;
 public interface CustomCommandLine<ClusterType extends ClusterClient> {
 
 	/**
-	 * Returns a unique identifier for this custom command-line.
-	 * @return An unique identifier string
+	 * 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
 	 */
-	String getIdentifier();
+	boolean isActive(CommandLine commandLine, Configuration configuration);
 
 	/**
-	 * Adds custom options to the existing options.
+	 * Gets the unique identifier of this CustomCommandLine
+	 * @return A unique identifier
+	 */
+	String getId();
+
+	/**
+	 * Adds custom options to the existing run options.
+	 * @param baseOptions The existing options.
+	 */
+	void addRunOptions(Options baseOptions);
+
+	/**
+	 * Adds custom options to the existing general options.
 	 * @param baseOptions The existing options.
 	 */
-	void addOptions(Options baseOptions);
+	void addGeneralOptions(Options baseOptions);
 
 	/**
 	 * Retrieves a client for a running cluster
+	 * @param commandLine The command-line parameters from the CliFrontend
 	 * @param config The Flink config
-	 * @return Client if a cluster could be retrieve, null otherwise
+	 * @return Client if a cluster could be retrieved
+	 * @throws UnsupportedOperationException if the operation is not supported
 	 */
-	ClusterClient retrieveCluster(Configuration config) throws Exception;
+	ClusterType retrieveCluster(CommandLine commandLine, Configuration config) throws UnsupportedOperationException;
 
 	/**
 	 * Creates the client for the cluster
 	 * @param applicationName The application name to use
 	 * @param commandLine The command-line options parsed by the CliFrontend
+	 * @param config The Flink config to use
 	 * @return The client to communicate with the cluster which the CustomCommandLine brought up.
+	 * @throws UnsupportedOperationException if the operation is not supported
 	 */
-	ClusterType createClient(String applicationName, CommandLine commandLine) throws Exception;
+	ClusterType createCluster(String applicationName, CommandLine commandLine, Configuration config) throws UnsupportedOperationException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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
new file mode 100644
index 0000000..8bceed7
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -0,0 +1,77 @@
+/*
+ * 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.cli;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.flink.client.ClientUtils;
+import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
+import org.apache.flink.client.program.StandaloneClusterClient;
+import org.apache.flink.configuration.Configuration;
+
+import java.net.InetSocketAddress;
+
+import static org.apache.flink.client.CliFrontend.setJobManagerAddressInConfig;
+
+/**
+ * The default CLI which is used for interaction with standalone clusters.
+ */
+public class DefaultCLI implements CustomCommandLine<StandaloneClusterClient> {
+
+	@Override
+	public boolean isActive(CommandLine commandLine, Configuration configuration) {
+		// always active because we can try to read a JobManager address from the config
+		return true;
+	}
+
+	@Override
+	public String getId() {
+		return null;
+	}
+
+	@Override
+	public void addRunOptions(Options baseOptions) {
+	}
+
+	@Override
+	public void addGeneralOptions(Options baseOptions) {
+	}
+
+	@Override
+	public StandaloneClusterClient retrieveCluster(CommandLine commandLine, Configuration config) {
+
+		if (commandLine.hasOption(CliFrontendParser.ADDRESS_OPTION.getOpt())) {
+			String addressWithPort = commandLine.getOptionValue(CliFrontendParser.ADDRESS_OPTION.getOpt());
+			InetSocketAddress jobManagerAddress = ClientUtils.parseHostPortAddress(addressWithPort);
+			setJobManagerAddressInConfig(config, jobManagerAddress);
+		}
+
+		StandaloneClusterDescriptor descriptor = new StandaloneClusterDescriptor(config);
+		return descriptor.retrieve(null);
+	}
+
+	@Override
+	public StandaloneClusterClient createCluster(
+			String applicationName,
+			CommandLine commandLine,
+			Configuration config) throws UnsupportedOperationException {
+
+		StandaloneClusterDescriptor descriptor = new StandaloneClusterDescriptor(config);
+		return descriptor.deploy();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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 cf0595b..59cece3 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
@@ -30,12 +30,20 @@ public interface ClusterDescriptor<ClientType extends ClusterClient> {
 	 * Returns a String containing details about the cluster (NodeManagers, available memory, ...)
 	 *
 	 */
-	String getClusterDescription() throws Exception;
+	String getClusterDescription();
+
+	/**
+	 * Retrieves an existing Flink Cluster.
+	 * @param applicationID The unique application identifier of the running cluster
+	 * @return Client for the cluster
+	 * @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
+	 */
+	ClientType retrieve(String applicationID) throws UnsupportedOperationException;
 
 	/**
 	 * Triggers deployment of a cluster
 	 * @return Client for the cluster
-	 * @throws Exception
+	 * @throws UnsupportedOperationException if this cluster descriptor doesn't support the operation
 	 */
-	ClientType deploy() throws Exception;
+	ClientType deploy() throws UnsupportedOperationException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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
new file mode 100644
index 0000000..57ccc47
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java
@@ -0,0 +1,56 @@
+/*
+ * 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.client.program.StandaloneClusterClient;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+
+
+/**
+ * A deployment descriptor for an existing cluster
+ */
+public class StandaloneClusterDescriptor implements ClusterDescriptor<StandaloneClusterClient> {
+
+	private final Configuration config;
+
+	public StandaloneClusterDescriptor(Configuration config) {
+		this.config = config;
+	}
+
+	@Override
+	public String getClusterDescription() {
+		String host = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "");
+		int port = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
+		return "Standalone cluster at " + host + ":" + port;
+	}
+
+	@Override
+	public StandaloneClusterClient retrieve(String applicationID) {
+		try {
+			return new StandaloneClusterClient(config);
+		} catch (Exception e) {
+			throw new RuntimeException("Couldn't retrieve standalone cluster", e);
+		}
+	}
+
+	@Override
+	public StandaloneClusterClient deploy() {
+		throw new UnsupportedOperationException("Can't deploy a standalone cluster.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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 b56428d..def9578 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
@@ -76,7 +76,7 @@ import akka.actor.ActorSystem;
  */
 public abstract class ClusterClient {
 
-	private static final Logger LOG = LoggerFactory.getLogger(ClusterClient.class);
+	private final Logger LOG = LoggerFactory.getLogger(getClass());
 
 	/** The optimizer used in the optimization of batch programs */
 	final Optimizer compiler;
@@ -203,9 +203,9 @@ public abstract class ClusterClient {
 	 */
 	public InetSocketAddress getJobManagerAddressFromConfig() {
 		try {
-		String hostName = flinkConfig.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
-		int port = flinkConfig.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
-		return new InetSocketAddress(hostName, port);
+			String hostName = flinkConfig.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
+			int port = flinkConfig.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
+			return new InetSocketAddress(hostName, port);
 		} catch (Exception e) {
 			throw new RuntimeException("Failed to retrieve JobManager address", e);
 		}
@@ -255,11 +255,13 @@ public abstract class ClusterClient {
 	}
 
 	public static OptimizedPlan getOptimizedPlan(Optimizer compiler, Plan p, int parallelism) throws CompilerException {
+		Logger log = LoggerFactory.getLogger(ClusterClient.class);
+
 		if (parallelism > 0 && p.getDefaultParallelism() <= 0) {
-			LOG.debug("Changing plan default parallelism from {} to {}", p.getDefaultParallelism(), parallelism);
+			log.debug("Changing plan default parallelism from {} to {}", p.getDefaultParallelism(), parallelism);
 			p.setDefaultParallelism(parallelism);
 		}
-		LOG.debug("Set parallelism {}, plan default parallelism {}", parallelism, p.getDefaultParallelism());
+		log.debug("Set parallelism {}, plan default parallelism {}", parallelism, p.getDefaultParallelism());
 
 		return compiler.compile(p);
 	}
@@ -603,7 +605,7 @@ public abstract class ClusterClient {
 	 * @return ActorGateway of the current job manager leader
 	 * @throws Exception
 	 */
-	protected ActorGateway getJobManagerGateway() throws Exception {
+	public ActorGateway getJobManagerGateway() throws Exception {
 		LOG.info("Looking up JobManager");
 
 		return LeaderRetrievalUtils.retrieveLeaderGateway(

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
index de85ca8..c6b1111 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java
@@ -23,9 +23,13 @@ import static org.junit.Assert.fail;
 
 import static org.mockito.Mockito.*;
 
+import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.cli.CommandLineOptions;
 
+import org.apache.flink.client.cli.RunOptions;
+import org.apache.flink.client.program.ClusterClient;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -57,14 +61,12 @@ public class CliFrontendAddressConfigurationTest {
 	public void testValidConfig() {
 		try {
 			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir());
+			RunOptions options = CliFrontendParser.parseRunCommand(new String[] {});
 
-			CommandLineOptions options = mock(CommandLineOptions.class);
-
-			frontend.updateConfig(options);
-			Configuration config = frontend.getConfiguration();
+			ClusterClient clusterClient = frontend.retrieveClient(options);
 
 			checkJobManagerAddress(
-					config,
+					clusterClient.getFlinkConfiguration(),
 					CliFrontendTestUtils.TEST_JOB_MANAGER_ADDRESS,
 					CliFrontendTestUtils.TEST_JOB_MANAGER_PORT);
 		}
@@ -74,43 +76,12 @@ public class CliFrontendAddressConfigurationTest {
 			}
 	}
 
-	@Test
-	public void testInvalidConfigAndNoOption() {
-		try {
+	@Test(expected = IllegalConfigurationException.class)
+	public void testInvalidConfigAndNoOption() throws Exception {
 			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir());
-			CommandLineOptions options = mock(CommandLineOptions.class);
-
-			frontend.updateConfig(options);
-			Configuration config = frontend.getConfiguration();
+			RunOptions options = CliFrontendParser.parseRunCommand(new String[] {});
 
-			checkJobManagerAddress(config, null, -1);
-
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testInvalidConfigAndOption() {
-		try {
-			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir());
-
-			CommandLineOptions options = mock(CommandLineOptions.class);
-			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
-
-			frontend.updateConfig(options);
-			Configuration config = frontend.getConfiguration();
-
-			InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
-
-			checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+			frontend.retrieveClient(options);
 	}
 
 	@Test
@@ -118,12 +89,10 @@ public class CliFrontendAddressConfigurationTest {
 		try {
 			CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir());
 
-			CommandLineOptions options = mock(CommandLineOptions.class);
-			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
-
-			frontend.updateConfig(options);
+			RunOptions options = CliFrontendParser.parseRunCommand(new String[] {"-m", "10.221.130.22:7788"});
 
-			Configuration config = frontend.getConfiguration();
+			ClusterClient client = frontend.retrieveClient(options);
+			Configuration config = client.getFlinkConfiguration();
 
 			InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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 1a8870b..f3b3507 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,6 +20,7 @@ package org.apache.flink.api.scala
 
 import java.io._
 
+import org.apache.commons.cli.CommandLine
 import org.apache.flink.client.cli.CliFrontendParser
 import org.apache.flink.client.program.ClusterClient
 import org.apache.flink.client.CliFrontend
@@ -245,11 +246,13 @@ object FlinkShell {
     yarnConfig.queue.foreach((queue) => args ++= Seq("-yqu", queue.toString))
     yarnConfig.slots.foreach((slots) => args ++= Seq("-ys", slots.toString))
 
-    val customCLI = CliFrontendParser.getAllCustomCommandLine.get("yarn-cluster")
 
     val options = CliFrontendParser.parseRunCommand(args.toArray)
+    val frontend = new CliFrontend()
+    val config = frontend.getConfiguration
+    val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
 
-    val cluster = customCLI.createClient("Flink Scala Shell", options.getCommandLine)
+    val cluster = customCLI.createCluster("Flink Scala Shell", options.getCommandLine, config)
 
     val address = cluster.getJobManagerAddress.getAddress.getHostAddress
     val port = cluster.getJobManagerAddress.getPort
@@ -259,12 +262,21 @@ object FlinkShell {
 
   def fetchDeployedYarnClusterInfo() = {
 
-    // load configuration
-    val globalConfig = GlobalConfiguration.getConfiguration
 
-    val customCLI = CliFrontendParser.getAllCustomCommandLine.get("yarn-cluster")
+    val args = ArrayBuffer[String](
+      "-m", "yarn-cluster"
+    )
 
-    val cluster = customCLI.retrieveCluster(globalConfig)
+    val options = CliFrontendParser.parseRunCommand(args.toArray)
+    val frontend = new CliFrontend()
+    val config = frontend.getConfiguration
+    val customCLI = frontend.getActiveCustomCommandLine(options.getCommandLine)
+
+    val cluster = customCLI.retrieveCluster(options.getCommandLine, config)
+
+    if (cluster == null) {
+      throw new RuntimeException("Yarn Cluster could not be retrieved.")
+    }
 
     val jobManager = cluster.getJobManagerAddress
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
index c6a1ade..217ad3d 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
@@ -18,27 +18,45 @@
 
 package org.apache.flink.yarn;
 
+import org.apache.commons.cli.CommandLine;
 import org.apache.flink.client.CliFrontend;
+import org.apache.flink.client.cli.CliFrontendParser;
 import org.apache.flink.client.cli.CommandLineOptions;
+import org.apache.flink.client.cli.CustomCommandLine;
+import org.apache.flink.client.cli.RunOptions;
 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;
-import org.junit.*;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
+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.FinalApplicationStatus;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
 
 import java.io.File;
+import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintStream;
 import java.lang.reflect.Field;
 import java.net.InetSocketAddress;
 import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
+import java.util.LinkedList;
+import java.util.List;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 /**
  * Tests that verify that the CLI client picks up the correct address for the JobManager
@@ -80,8 +98,10 @@ public class CliFrontendYarnAddressConfigurationTest {
 
 	private static final String TEST_YARN_JOB_MANAGER_ADDRESS = "22.33.44.55";
 	private static final int TEST_YARN_JOB_MANAGER_PORT = 6655;
+	private static final ApplicationId TEST_YARN_APPLICATION_ID =
+		ApplicationId.newInstance(System.currentTimeMillis(), 42);
 
-	private static final String propertiesFile =
+	private static final String validPropertiesFile =
 		"jobManager=" + TEST_YARN_JOB_MANAGER_ADDRESS + ":" + TEST_YARN_JOB_MANAGER_PORT;
 
 
@@ -101,110 +121,292 @@ public class CliFrontendYarnAddressConfigurationTest {
 	 * Test that the CliFrontend is able to pick up the .yarn-properties file from a specified location.
 	 */
 	@Test
-	public void testYarnConfig() {
-		try {
-			File tmpFolder = temporaryFolder.newFolder();
-			String currentUser = System.getProperty("user.name");
+	public void testResumeFromYarnPropertiesFile() throws Exception {
 
-			// copy .yarn-properties-<username>
-			File testPropertiesFile = new File(tmpFolder, ".yarn-properties-"+currentUser);
-			Files.write(testPropertiesFile.toPath(), propertiesFile.getBytes(), StandardOpenOption.CREATE);
+		File directoryPath = writeYarnPropertiesFile(validPropertiesFile);
 
-			// copy reference flink-conf.yaml to temporary test directory and append custom configuration path.
-			String confFile = flinkConf + "\nyarn.properties-file.location: " + tmpFolder;
-			File testConfFile = new File(tmpFolder.getAbsolutePath(), "flink-conf.yaml");
-			Files.write(testConfFile.toPath(), confFile.getBytes(), StandardOpenOption.CREATE);
+		// start CLI Frontend
+		TestCLI frontend = new CustomYarnTestCLI(directoryPath.getAbsolutePath());
 
-			// start CLI Frontend
-			TestCLI frontend = new TestCLI(tmpFolder.getAbsolutePath());
+		RunOptions options = CliFrontendParser.parseRunCommand(new String[] {});
 
-			CommandLineOptions options = mock(CommandLineOptions.class);
+		frontend.retrieveClient(options);
+		checkJobManagerAddress(
+			frontend.getConfiguration(),
+			TEST_YARN_JOB_MANAGER_ADDRESS,
+			TEST_YARN_JOB_MANAGER_PORT);
 
-			frontend.getClient(options, "Program name");
+	}
 
-			frontend.updateConfig(options);
-			Configuration config = frontend.getConfiguration();
+	@Test(expected = IllegalConfigurationException.class)
+	public void testResumeFromYarnPropertiesFileWithFinishedApplication() throws Exception {
 
- 			checkJobManagerAddress(
-					config,
-					TEST_YARN_JOB_MANAGER_ADDRESS,
-					TEST_YARN_JOB_MANAGER_PORT);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		File directoryPath = writeYarnPropertiesFile(validPropertiesFile);
+
+		// start CLI Frontend
+		TestCLI frontend = new CustomYarnTestCLI(directoryPath.getAbsolutePath(), FinalApplicationStatus.SUCCEEDED);
+
+		RunOptions options = CliFrontendParser.parseRunCommand(new String[] {});
+
+		frontend.retrieveClient(options);
+		checkJobManagerAddress(
+			frontend.getConfiguration(),
+			TEST_YARN_JOB_MANAGER_ADDRESS,
+			TEST_YARN_JOB_MANAGER_PORT);
 	}
-	public static class TestCLI extends CliFrontend {
-		TestCLI(String configDir) throws Exception {
-			super(configDir);
-		}
 
-		@Override
-		public ClusterClient getClient(CommandLineOptions options, String programName) throws Exception {
-			return super.getClient(options, programName);
-		}
+	@Test(expected = IllegalConfigurationException.class)
+	public void testInvalidYarnPropertiesFile() throws Exception {
 
-		@Override
-		public void updateConfig(CommandLineOptions options) {
-			super.updateConfig(options);
-		}
+		File directoryPath = writeYarnPropertiesFile(invalidPropertiesFile);
+
+		TestCLI frontend = new CustomYarnTestCLI(directoryPath.getAbsolutePath());
+
+		RunOptions options = CliFrontendParser.parseRunCommand(new String[] {});
+
+		frontend.retrieveClient(options);
+		Configuration config = frontend.getConfiguration();
+
+		checkJobManagerAddress(
+			config,
+			TEST_JOB_MANAGER_ADDRESS,
+			TEST_JOB_MANAGER_PORT);
 	}
 
+
 	@Test
-	public void testInvalidYarnConfig() {
-		try {
-			File tmpFolder = temporaryFolder.newFolder();
+	public void testResumeFromYarnID() throws Exception {
+		File directoryPath = writeYarnPropertiesFile(validPropertiesFile);
 
-			// copy invalid .yarn-properties-<username>
-			File testPropertiesFile = new File(tmpFolder, ".yarn-properties");
-			Files.write(testPropertiesFile.toPath(), invalidPropertiesFile.getBytes(), StandardOpenOption.CREATE);
+		// start CLI Frontend
+		TestCLI frontend = new CustomYarnTestCLI(directoryPath.getAbsolutePath());
 
-			// copy reference flink-conf.yaml to temporary test directory and append custom configuration path.
-			String confFile = flinkConf + "\nyarn.properties-file.location: " + tmpFolder;
-			File testConfFile = new File(tmpFolder.getAbsolutePath(), "flink-conf.yaml");
-			Files.write(testConfFile.toPath(), confFile.getBytes(), StandardOpenOption.CREATE);
+		RunOptions options =
+			CliFrontendParser.parseRunCommand(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()});
 
-			TestCLI cli = new TestCLI(tmpFolder.getAbsolutePath());
+		frontend.retrieveClient(options);
 
-			CommandLineOptions options = mock(CommandLineOptions.class);
+		checkJobManagerAddress(
+			frontend.getConfiguration(),
+			TEST_YARN_JOB_MANAGER_ADDRESS,
+			TEST_YARN_JOB_MANAGER_PORT);
+	}
 
-			cli.updateConfig(options);
+	@Test(expected = IllegalConfigurationException.class)
+	public void testResumeFromInvalidYarnID() throws Exception {
+		File directoryPath = writeYarnPropertiesFile(validPropertiesFile);
 
-			Configuration config = cli.getConfiguration();
+		// start CLI Frontend
+		TestCLI frontend = new CustomYarnTestCLI(directoryPath.getAbsolutePath(), FinalApplicationStatus.SUCCEEDED);
 
-			checkJobManagerAddress(
-				config,
-				TEST_JOB_MANAGER_ADDRESS,
-				TEST_JOB_MANAGER_PORT);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
+		RunOptions options =
+			CliFrontendParser.parseRunCommand(new String[] {"-yid", ApplicationId.newInstance(0, 666).toString()});
+
+		frontend.retrieveClient(options);
+		checkJobManagerAddress(
+			frontend.getConfiguration(),
+			TEST_YARN_JOB_MANAGER_ADDRESS,
+			TEST_YARN_JOB_MANAGER_PORT);
+	}
+
+	@Test(expected = IllegalConfigurationException.class)
+	public void testResumeFromYarnIDWithFinishedApplication() throws Exception {
+		File directoryPath = writeYarnPropertiesFile(validPropertiesFile);
+
+		// start CLI Frontend
+		TestCLI frontend = new CustomYarnTestCLI(directoryPath.getAbsolutePath(), FinalApplicationStatus.SUCCEEDED);
+
+		RunOptions options =
+			CliFrontendParser.parseRunCommand(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()});
+
+		frontend.retrieveClient(options);
+
+		checkJobManagerAddress(
+			frontend.getConfiguration(),
+			TEST_YARN_JOB_MANAGER_ADDRESS,
+			TEST_YARN_JOB_MANAGER_PORT);
 	}
 
 
 	@Test
-	public void testManualOptionsOverridesYarn() {
-		try {
-			File emptyFolder = temporaryFolder.newFolder();
-			TestCLI frontend = new TestCLI(emptyFolder.getAbsolutePath());
+	public void testYarnIDOverridesPropertiesFile() throws Exception {
+		File directoryPath = writeYarnPropertiesFile(invalidPropertiesFile);
+
+		// start CLI Frontend
+		TestCLI frontend = new CustomYarnTestCLI(directoryPath.getAbsolutePath());
+
+		RunOptions options =
+			CliFrontendParser.parseRunCommand(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()});
+
+		frontend.retrieveClient(options);
+
+		checkJobManagerAddress(
+			frontend.getConfiguration(),
+			TEST_YARN_JOB_MANAGER_ADDRESS,
+			TEST_YARN_JOB_MANAGER_PORT);
+	}
+
+
+	@Test
+	public void testManualOptionsOverridesYarn() throws Exception {
+
+		File emptyFolder = temporaryFolder.newFolder();
+		File testConfFile = new File(emptyFolder.getAbsolutePath(), "flink-conf.yaml");
+		Files.createFile(testConfFile.toPath());
 
-			CommandLineOptions options = mock(CommandLineOptions.class);
-			when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788");
+		TestCLI frontend = new TestCLI(emptyFolder.getAbsolutePath());
 
-			frontend.updateConfig(options);
+		RunOptions options = CliFrontendParser.parseRunCommand(new String[] {"-m", "10.221.130.22:7788"});
 
-			Configuration config = frontend.getConfiguration();
+		frontend.retrieveClient(options);
 
-			InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
+		Configuration config = frontend.getConfiguration();
 
-			checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort());
+		InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788);
+
+		checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort());
+
+	}
+
+
+	///////////
+	// Utils //
+	///////////
+
+	private File writeYarnPropertiesFile(String contents) throws IOException {
+		File tmpFolder = temporaryFolder.newFolder();
+		String currentUser = System.getProperty("user.name");
+
+		// copy .yarn-properties-<username>
+		File testPropertiesFile = new File(tmpFolder, ".yarn-properties-"+currentUser);
+		Files.write(testPropertiesFile.toPath(), contents.getBytes(), StandardOpenOption.CREATE);
+
+		// copy reference flink-conf.yaml to temporary test directory and append custom configuration path.
+		String confFile = flinkConf + "\nyarn.properties-file.location: " + tmpFolder;
+		File testConfFile = new File(tmpFolder.getAbsolutePath(), "flink-conf.yaml");
+		Files.write(testConfFile.toPath(), confFile.getBytes(), StandardOpenOption.CREATE);
+
+		return tmpFolder.getAbsoluteFile();
+	}
+
+	private static class TestCLI extends CliFrontend {
+		TestCLI(String configDir) throws Exception {
+			super(configDir);
+		}
+
+		@Override
+		// make method public
+		public ClusterClient getClient(CommandLineOptions options, String programName) throws Exception {
+			return super.getClient(options, programName);
+		}
+
+		@Override
+		// make method public
+		public ClusterClient retrieveClient(CommandLineOptions options) {
+			return super.retrieveClient(options);
+		}
+	}
+
+
+	/**
+	 * Injects an extended FlinkYarnSessionCli that deals with mocking Yarn communication
+	 */
+	private static class CustomYarnTestCLI extends TestCLI {
+
+		// the default application status for yarn applications to be retrieved
+		private final FinalApplicationStatus finalApplicationStatus;
+
+		CustomYarnTestCLI(String configDir) throws Exception {
+			this(configDir, FinalApplicationStatus.UNDEFINED);
 		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
+
+		CustomYarnTestCLI(String configDir, FinalApplicationStatus finalApplicationStatus) throws Exception {
+			super(configDir);
+			this.finalApplicationStatus = finalApplicationStatus;
+		}
+
+		@Override
+		public CustomCommandLine getActiveCustomCommandLine(CommandLine commandLine) {
+			// inject the testing FlinkYarnSessionCli
+			return new TestingYarnSessionCli();
+		}
+
+		/**
+		 * Testing FlinkYarnSessionCli which returns a modified cluster descriptor for testing.
+		 */
+		private class TestingYarnSessionCli extends FlinkYarnSessionCli {
+			TestingYarnSessionCli() {
+				super("y", "yarn");
+			}
+
+			@Override
+			// override cluster descriptor to replace the YarnClient
+			protected AbstractYarnClusterDescriptor getClusterDescriptor() {
+				return new TestingYarnClusterDescriptor();
+			}
+
+			/**
+			 * Replace the YarnClient for this test.
+			 */
+			private class TestingYarnClusterDescriptor extends YarnClusterDescriptor {
+
+				@Override
+				protected YarnClient getYarnClient() {
+					return new TestYarnClient();
+				}
+
+				@Override
+				protected YarnClusterClient createYarnClusterClient(
+						AbstractYarnClusterDescriptor descriptor,
+						YarnClient yarnClient,
+						ApplicationReport report,
+						Configuration flinkConfiguration,
+						Path sessionFilesDir,
+						boolean perJobCluster) throws IOException, YarnException {
+
+					return Mockito.mock(YarnClusterClient.class);
+				}
+
+
+				private class TestYarnClient extends YarnClientImpl {
+
+					private final List<ApplicationReport> reports = new LinkedList<>();
+
+					TestYarnClient() {
+						{   // a report that of our Yarn application we want to resume from
+							ApplicationReport report = Mockito.mock(ApplicationReport.class);
+							Mockito.when(report.getHost()).thenReturn(TEST_YARN_JOB_MANAGER_ADDRESS);
+							Mockito.when(report.getRpcPort()).thenReturn(TEST_YARN_JOB_MANAGER_PORT);
+							Mockito.when(report.getApplicationId()).thenReturn(TEST_YARN_APPLICATION_ID);
+							Mockito.when(report.getFinalApplicationStatus()).thenReturn(finalApplicationStatus);
+							this.reports.add(report);
+						}
+						{   // a second report, just for noise
+							ApplicationReport report = Mockito.mock(ApplicationReport.class);
+							Mockito.when(report.getHost()).thenReturn("1.2.3.4");
+							Mockito.when(report.getRpcPort()).thenReturn(-123);
+							Mockito.when(report.getApplicationId()).thenReturn(ApplicationId.newInstance(0, 0));
+							Mockito.when(report.getFinalApplicationStatus()).thenReturn(finalApplicationStatus);
+							this.reports.add(report);
+						}
+					}
+
+					@Override
+					public List<ApplicationReport> getApplications() throws YarnException, IOException {
+						return reports;
+					}
+
+					@Override
+					public ApplicationReport getApplicationReport(ApplicationId appId) throws YarnException, IOException {
+						for (ApplicationReport report : reports) {
+							if (report.getApplicationId().equals(appId)) {
+								return report;
+							}
+						}
+						throw new YarnException();
+					}
+				}
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
index c842bdc..f71dd63 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java
@@ -23,7 +23,6 @@ import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
 
-import org.apache.flink.client.CliFrontend;
 import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.flink.test.util.TestBaseUtils;
 
@@ -37,8 +36,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties;
-
 public class FlinkYarnSessionCliTest {
 
 	@Rule
@@ -53,9 +50,10 @@ public class FlinkYarnSessionCliTest {
 		fakeConf.createNewFile();
 		map.put("FLINK_CONF_DIR", tmpFolder.getAbsolutePath());
 		TestBaseUtils.setEnv(map);
-		Options options = new Options();
 		FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", "", false);
-		cli.addOptions(options);
+		Options options = new Options();
+		cli.addGeneralOptions(options);
+		cli.addRunOptions(options);
 
 		CommandLineParser parser = new PosixParser();
 		CommandLine cmd = null;
@@ -66,7 +64,7 @@ public class FlinkYarnSessionCliTest {
 			Assert.fail("Parsing failed with " + e.getMessage());
 		}
 
-		YarnClusterDescriptor flinkYarnDescriptor = cli.createDescriptor(null, cmd);
+		AbstractYarnClusterDescriptor flinkYarnDescriptor = cli.createDescriptor(null, cmd);
 
 		Assert.assertNotNull(flinkYarnDescriptor);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f4ac8522/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 c471fa4..aebb14d 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
@@ -22,6 +22,7 @@ import org.apache.flink.client.CliFrontend;
 import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.jobmanager.RecoveryMode;
 
@@ -37,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -73,18 +75,8 @@ import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_
 import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties;
 
 /**
-* All classes in this package contain code taken from
-* https://github.com/apache/hadoop-common/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java?source=cc
-* and
-* https://github.com/hortonworks/simple-yarn-app
-* and
-* https://github.com/yahoo/storm-yarn/blob/master/src/main/java/com/yahoo/storm/yarn/StormOnYarn.java
-*
-* The Flink jar is uploaded to HDFS by this client.
-* The application master and all the TaskManager containers get the jar file downloaded
-* by YARN into their local fs.
-*
-*/
+ * The descriptor with deployment information for spwaning or resuming a {@link YarnClusterClient}.
+ */
 public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor<YarnClusterClient> {
 	private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class);
 
@@ -132,7 +124,8 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 
 	private boolean detached;
 
-	private String customName = null;
+	private String customName;
+
 
 	public AbstractYarnClusterDescriptor() {
 		// for unit tests only
@@ -321,49 +314,112 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	 * Gets a Hadoop Yarn client
 	 * @return Returns a YarnClient which has to be shutdown manually
 	 */
-	public static YarnClient getYarnClient(Configuration conf) {
+	protected YarnClient getYarnClient() {
 		YarnClient yarnClient = YarnClient.createYarnClient();
 		yarnClient.init(conf);
 		yarnClient.start();
 		return yarnClient;
 	}
 
-	@Override
-	public YarnClusterClient deploy() throws Exception {
+	/**
+	 * Retrieves the Yarn application and cluster from the config
+	 * @param config The config with entries to retrieve the cluster
+	 * @return YarnClusterClient
+	 * @deprecated This should be removed in the future
+	 */
+	public YarnClusterClient retrieveFromConfig(org.apache.flink.configuration.Configuration config)
+			throws UnsupportedOperationException {
+		String jobManagerHost = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
+		int jobManagerPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1);
 
-		UserGroupInformation.setConfiguration(conf);
-		UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+		if (jobManagerHost != null && jobManagerPort != -1) {
 
-		if (UserGroupInformation.isSecurityEnabled()) {
-			if (!ugi.hasKerberosCredentials()) {
-				throw new YarnDeploymentException("In secure mode. Please provide Kerberos credentials in order to authenticate. " +
-					"You may use kinit to authenticate and request a TGT from the Kerberos server.");
+			YarnClient yarnClient = getYarnClient();
+			final List<ApplicationReport> applicationReports;
+			try {
+				applicationReports = yarnClient.getApplications();
+			} catch (Exception e) {
+				throw new RuntimeException("Couldn't get Yarn application reports", e);
 			}
-			return ugi.doAs(new PrivilegedExceptionAction<YarnClusterClient>() {
-				@Override
-				public YarnClusterClient run() throws Exception {
-					return deployInternal();
+			for (ApplicationReport report : applicationReports) {
+				if (report.getHost().equals(jobManagerHost) && report.getRpcPort() == jobManagerPort) {
+					LOG.info("Found application '{}' " +
+						"with JobManager host name '{}' and port '{}' from Yarn properties file.",
+						report.getApplicationId(), jobManagerHost, jobManagerPort);
+					return retrieve(report.getApplicationId().toString());
 				}
-			});
-		} else {
-			return deployInternal();
+			}
+
 		}
+
+		LOG.warn("Couldn't retrieve Yarn cluster from Flink configuration using JobManager address '{}:{}'",
+			jobManagerHost, jobManagerPort);
+
+		throw new IllegalConfigurationException("Could not resume Yarn cluster from config.");
 	}
 
 	@Override
-	public AbstractFlinkYarnCluster attach(String appId) throws Exception {
-		// check if required Hadoop environment variables are set. If not, warn user
-		if(System.getenv("HADOOP_CONF_DIR") == null &&
-			System.getenv("YARN_CONF_DIR") == null) {
-			LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." +
-				"The Flink YARN Client needs one of these to be set to properly load the Hadoop " +
-				"configuration for accessing YARN.");
+	public YarnClusterClient retrieve(String applicationID) {
+
+		try {
+			// check if required Hadoop environment variables are set. If not, warn user
+			if (System.getenv("HADOOP_CONF_DIR") == null &&
+				System.getenv("YARN_CONF_DIR") == null) {
+				LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." +
+					"The Flink YARN Client needs one of these to be set to properly load the Hadoop " +
+					"configuration for accessing YARN.");
+			}
+
+			final ApplicationId yarnAppId = ConverterUtils.toApplicationId(applicationID);
+			final YarnClient yarnClient = getYarnClient();
+			final ApplicationReport appReport = yarnClient.getApplicationReport(yarnAppId);
+
+			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.");
+			}
+
+			LOG.info("Found application JobManager host name '{}' and port '{}' from supplied application id '{}'",
+				appReport.getHost(), appReport.getRpcPort(), applicationID);
+
+			flinkConfiguration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, appReport.getHost());
+			flinkConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, appReport.getRpcPort());
+
+			return createYarnClusterClient(this, yarnClient, appReport, flinkConfiguration, sessionFilesDir, false);
+		} catch (Exception e) {
+			throw new RuntimeException("Couldn't retrieve Yarn cluster", e);
 		}
+	}
 
-		final ApplicationId yarnAppId = ConverterUtils.toApplicationId(appId);
+	@Override
+	public YarnClusterClient deploy() {
 
-		return new FlinkYarnCluster(yarnClient, yarnAppId, conf, flinkConfiguration, sessionFilesDir, detached);
+		try {
+
+			UserGroupInformation.setConfiguration(conf);
+			UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
+			if (UserGroupInformation.isSecurityEnabled()) {
+				if (!ugi.hasKerberosCredentials()) {
+					throw new YarnDeploymentException("In secure mode. Please provide Kerberos credentials in order to authenticate. " +
+						"You may use kinit to authenticate and request a TGT from the Kerberos server.");
+				}
+				return ugi.doAs(new PrivilegedExceptionAction<YarnClusterClient>() {
+					@Override
+					public YarnClusterClient run() throws Exception {
+						return deployInternal();
+					}
+				});
+			} else {
+				return deployInternal();
+			}
+		} catch (Exception e) {
+			throw new RuntimeException("Couldn't deploy Yarn cluster", e);
+		}
 	}
+
 	/**
 	 * This method will block until the ApplicationMaster/JobManager have been
 	 * deployed on YARN.
@@ -377,7 +433,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		LOG.info("\tTaskManager memory = {}", taskManagerMemoryMb);
 
 		// Create application via yarnClient
-		final YarnClient yarnClient = getYarnClient(conf);
+		final YarnClient yarnClient = getYarnClient();
 		final YarnClientApplication yarnApplication = yarnClient.createApplication();
 		GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse();
 
@@ -726,7 +782,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 		flinkConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port);
 
 		// the Flink cluster is deployed in YARN. Represent cluster
-		return new YarnClusterClient(this, yarnClient, report, flinkConfiguration, sessionFilesDir);
+		return createYarnClusterClient(this, yarnClient, report, flinkConfiguration, sessionFilesDir, true);
 	}
 
 	/**
@@ -780,40 +836,44 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	}
 
 	@Override
-	public String getClusterDescription() throws Exception {
+	public String getClusterDescription() {
 
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		PrintStream ps = new PrintStream(baos);
+		try {
+			ByteArrayOutputStream baos = new ByteArrayOutputStream();
+			PrintStream ps = new PrintStream(baos);
 
-		YarnClient yarnClient = getYarnClient(conf);
-		YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics();
+			YarnClient yarnClient = getYarnClient();
+			YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics();
 
-		ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers());
-		List<NodeReport> nodes = yarnClient.getNodeReports(NodeState.RUNNING);
-		final String format = "|%-16s |%-16s %n";
-		ps.printf("|Property         |Value          %n");
-		ps.println("+---------------------------------------+");
-		int totalMemory = 0;
-		int totalCores = 0;
-		for(NodeReport rep : nodes) {
-			final Resource res = rep.getCapability();
-			totalMemory += res.getMemory();
-			totalCores += res.getVirtualCores();
-			ps.format(format, "NodeID", rep.getNodeId());
-			ps.format(format, "Memory", res.getMemory() + " MB");
-			ps.format(format, "vCores", res.getVirtualCores());
-			ps.format(format, "HealthReport", rep.getHealthReport());
-			ps.format(format, "Containers", rep.getNumContainers());
+			ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers());
+			List<NodeReport> nodes = yarnClient.getNodeReports(NodeState.RUNNING);
+			final String format = "|%-16s |%-16s %n";
+			ps.printf("|Property         |Value          %n");
 			ps.println("+---------------------------------------+");
+			int totalMemory = 0;
+			int totalCores = 0;
+			for (NodeReport rep : nodes) {
+				final Resource res = rep.getCapability();
+				totalMemory += res.getMemory();
+				totalCores += res.getVirtualCores();
+				ps.format(format, "NodeID", rep.getNodeId());
+				ps.format(format, "Memory", res.getMemory() + " MB");
+				ps.format(format, "vCores", res.getVirtualCores());
+				ps.format(format, "HealthReport", rep.getHealthReport());
+				ps.format(format, "Containers", rep.getNumContainers());
+				ps.println("+---------------------------------------+");
+			}
+			ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores);
+			List<QueueInfo> qInfo = yarnClient.getAllQueues();
+			for (QueueInfo q : qInfo) {
+				ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " +
+					q.getMaximumCapacity() + " Applications: " + q.getApplications().size());
+			}
+			yarnClient.stop();
+			return baos.toString();
+		} catch (Exception e) {
+			throw new RuntimeException("Couldn't get cluster description", e);
 		}
-		ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores);
-		List<QueueInfo> qInfo = yarnClient.getAllQueues();
-		for(QueueInfo q : qInfo) {
-			ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " +
-				q.getMaximumCapacity() + " Applications: " + q.getApplications().size());
-		}
-		yarnClient.stop();
-		return baos.toString();
 	}
 
 	public String getSessionFilesDir() {
@@ -918,9 +978,6 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 	private static class YarnDeploymentException extends RuntimeException {
 		private static final long serialVersionUID = -812040641215388943L;
 
-		public YarnDeploymentException() {
-		}
-
 		public YarnDeploymentException(String message) {
 			super(message);
 		}
@@ -954,5 +1011,24 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
 			}
 		}
 	}
+
+	/**
+	 * Creates a YarnClusterClient; may be overriden in tests
+	 */
+	protected YarnClusterClient createYarnClusterClient(
+			AbstractYarnClusterDescriptor descriptor,
+			YarnClient yarnClient,
+			ApplicationReport report,
+			org.apache.flink.configuration.Configuration flinkConfiguration,
+			Path sessionFilesDir,
+			boolean perJobCluster) throws IOException, YarnException {
+		return new YarnClusterClient(
+			descriptor,
+			yarnClient,
+			report,
+			flinkConfiguration,
+			sessionFilesDir,
+			perJobCluster);
+	}
 }
 


[03/10] flink git commit: [FLINK-3667] refactor client communication classes

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
deleted file mode 100644
index 3b48228..0000000
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java
+++ /dev/null
@@ -1,559 +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 akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-
-import static akka.pattern.Patterns.ask;
-
-import akka.actor.PoisonPill;
-import akka.actor.Props;
-import akka.pattern.Patterns;
-import akka.util.Timeout;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
-import org.apache.flink.runtime.clusterframework.messages.InfoMessage;
-import org.apache.flink.runtime.net.ConnectionUtils;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
-import org.apache.flink.util.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.service.Service;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.None$;
-import scala.Option;
-import scala.Some;
-import scala.Tuple2;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-/**
- * Java representation of a running Flink cluster within YARN.
- */
-public class FlinkYarnCluster extends AbstractFlinkYarnCluster {
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnCluster.class);
-
-	private static final int POLLING_THREAD_INTERVAL_MS = 1000;
-
-	private YarnClient yarnClient;
-	private Thread actorRunner;
-	private Thread clientShutdownHook = new ClientShutdownHook();
-	private PollingThread pollingRunner;
-	private final Configuration hadoopConfig;
-	// (HDFS) location of the files required to run on YARN. Needed here to delete them on shutdown.
-	private final Path sessionFilesDir;
-	private final InetSocketAddress jobManagerAddress;
-
-	//---------- Class internal fields -------------------
-
-	private ActorSystem actorSystem;
-	private ActorRef applicationClient;
-	private ApplicationReport intialAppReport;
-	private final FiniteDuration akkaDuration;
-	private final Timeout akkaTimeout;
-	private final ApplicationId applicationId;
-	private final boolean detached;
-	private final org.apache.flink.configuration.Configuration flinkConfig;
-	private final ApplicationId appId;
-
-	private boolean isConnected = false;
-
-
-	/**
-	 * Create a new Flink on YARN cluster.
-	 *
-	 * @param yarnClient Client to talk to YARN
-	 * @param appId the YARN application ID
-	 * @param hadoopConfig Hadoop configuration
-	 * @param flinkConfig Flink configuration
-	 * @param sessionFilesDir Location of files required for YARN session
-	 * @param detached Set to true if no actor system or RPC communication with the cluster should be established
-	 * @throws IOException
-	 * @throws YarnException
-	 */
-	public FlinkYarnCluster(
-			final YarnClient yarnClient,
-			final ApplicationId appId,
-			Configuration hadoopConfig,
-			org.apache.flink.configuration.Configuration flinkConfig,
-			Path sessionFilesDir,
-			boolean detached) throws IOException, YarnException {
-		this.akkaDuration = AkkaUtils.getTimeout(flinkConfig);
-		this.akkaTimeout = Timeout.durationToTimeout(akkaDuration);
-		this.yarnClient = yarnClient;
-		this.hadoopConfig = hadoopConfig;
-		this.sessionFilesDir = sessionFilesDir;
-		this.applicationId = appId;
-		this.detached = detached;
-		this.flinkConfig = flinkConfig;
-		this.appId = appId;
-
-		// get one application report manually
-		intialAppReport = yarnClient.getApplicationReport(appId);
-		String jobManagerHost = intialAppReport.getHost();
-		int jobManagerPort = intialAppReport.getRpcPort();
-		this.jobManagerAddress = new InetSocketAddress(jobManagerHost, jobManagerPort);
-	}
-
-	/**
-	 * Connect the FlinkYarnCluster to the ApplicationMaster.
-	 *
-	 * Detached YARN sessions don't need to connect to the ApplicationMaster.
-	 * Detached per job YARN sessions need to connect until the required number of TaskManagers have been started.
-	 * 
-	 * @throws IOException
-	 */
-	public void connectToCluster() throws IOException {
-		if(isConnected) {
-			throw new IllegalStateException("Can not connect to the cluster again");
-		}
-
-		// start actor system
-		LOG.info("Start actor system.");
-		// find name of own public interface, able to connect to the JM
-		// try to find address for 2 seconds. log after 400 ms.
-		InetAddress ownHostname = ConnectionUtils.findConnectingAddress(jobManagerAddress, 2000, 400);
-		actorSystem = AkkaUtils.createActorSystem(flinkConfig,
-				new Some<Tuple2<String, Object>>(new Tuple2<String, Object>(ownHostname.getCanonicalHostName(), 0)));
-
-		// Create the leader election service
-		flinkConfig.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, this.jobManagerAddress.getHostName());
-		flinkConfig.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, this.jobManagerAddress.getPort());
-
-		LeaderRetrievalService leaderRetrievalService;
-
-		try {
-			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
-		} catch (Exception e) {
-			throw new IOException("Could not create the leader retrieval service.", e);
-		}
-
-		// start application client
-		LOG.info("Start application client.");
-
-		applicationClient = actorSystem.actorOf(
-			Props.create(
-				ApplicationClient.class,
-				flinkConfig,
-				leaderRetrievalService),
-			"applicationClient");
-
-		actorRunner = new Thread(new Runnable() {
-			@Override
-			public void run() {
-				// blocks until ApplicationClient has been stopped
-				actorSystem.awaitTermination();
-
-				// get final application report
-				try {
-					ApplicationReport appReport = yarnClient.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:\n"
-								+ "\tyarn logs -applicationId " + appReport.getApplicationId() + "\n"
-								+ "(It sometimes takes a few seconds until the logs are aggregated)");
-					}
-				} catch (Exception e) {
-					LOG.warn("Error while getting final application report", e);
-				}
-			}
-		});
-		actorRunner.setDaemon(true);
-		actorRunner.start();
-
-		pollingRunner = new PollingThread(yarnClient, appId);
-		pollingRunner.setDaemon(true);
-		pollingRunner.start();
-
-		Runtime.getRuntime().addShutdownHook(clientShutdownHook);
-
-		isConnected = true;
-	}
-
-	@Override
-	public void disconnect() {
-		if(!isConnected) {
-			throw new IllegalStateException("Can not disconnect from an unconnected cluster.");
-		}
-		LOG.info("Disconnecting FlinkYarnCluster from ApplicationMaster");
-
-		if(!Runtime.getRuntime().removeShutdownHook(clientShutdownHook)) {
-			LOG.warn("Error while removing the shutdown hook. The YARN session might be killed unintentionally");
-		}
-		// tell the actor to shut down.
-		applicationClient.tell(PoisonPill.getInstance(), applicationClient);
-
-		try {
-			actorRunner.join(1000); // wait for 1 second
-		} catch (InterruptedException e) {
-			LOG.warn("Shutdown of the actor runner was interrupted", e);
-			Thread.currentThread().interrupt();
-		}
-		try {
-			pollingRunner.stopRunner();
-			pollingRunner.join(1000);
-		} catch(InterruptedException e) {
-			LOG.warn("Shutdown of the polling runner was interrupted", e);
-			Thread.currentThread().interrupt();
-		}
-		isConnected = false;
-	}
-
-
-	// -------------------------- Interaction with the cluster ------------------------
-
-	/*
-	 * This call blocks until the message has been recevied.
-	 */
-	@Override
-	public void stopAfterJob(JobID jobID) {
-		Preconditions.checkNotNull(jobID, "The job id must not be null");
-		Future<Object> messageReceived = ask(applicationClient, new YarnMessages.LocalStopAMAfterJob(jobID), akkaTimeout);
-		try {
-			Await.result(messageReceived, akkaDuration);
-		} catch (Exception e) {
-			throw new RuntimeException("Unable to tell application master to stop once the specified job has been finised", e);
-		}
-	}
-
-	@Override
-	public org.apache.flink.configuration.Configuration getFlinkConfiguration() {
-		return flinkConfig;
-	}
-
-	@Override
-	public InetSocketAddress getJobManagerAddress() {
-		return jobManagerAddress;
-	}
-
-	@Override
-	public String getWebInterfaceURL() {
-		String url = this.intialAppReport.getTrackingUrl();
-		// there seems to be a difference between HD 2.2.0 and 2.6.0
-		if(!url.startsWith("http://")) {
-			url = "http://" + url;
-		}
-		return url;
-	}
-
-	@Override
-	public String getApplicationId() {
-		return applicationId.toString();
-	}
-
-	@Override
-	public boolean isDetached() {
-		return this.detached;
-	}
-
-	/**
-	 * This method is only available if the cluster hasn't been started in detached mode.
-	 */
-	@Override
-	public GetClusterStatusResponse getClusterStatus() {
-		if(!isConnected) {
-			throw new IllegalStateException("The cluster is not connected to the ApplicationMaster.");
-		}
-		if(hasBeenStopped()) {
-			throw new RuntimeException("The FlinkYarnCluster has already been stopped");
-		}
-		Future<Object> clusterStatusOption = ask(applicationClient, YarnMessages.getLocalGetyarnClusterStatus(), akkaTimeout);
-		Object clusterStatus;
-		try {
-			clusterStatus = Await.result(clusterStatusOption, akkaDuration);
-		} catch (Exception e) {
-			throw new RuntimeException("Unable to get Cluster status from Application Client", e);
-		}
-		if(clusterStatus instanceof None$) {
-			return null;
-		} else if(clusterStatus instanceof Some) {
-			return (GetClusterStatusResponse) (((Some) clusterStatus).get());
-		} else {
-			throw new RuntimeException("Unexpected type: " + clusterStatus.getClass().getCanonicalName());
-		}
-	}
-
-	@Override
-	public boolean hasFailed() {
-		if(!isConnected) {
-			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
-		}
-		if(pollingRunner == null) {
-			LOG.warn("FlinkYarnCluster.hasFailed() has been called on an uninitialized cluster." +
-					"The system might be in an erroneous state");
-		}
-		ApplicationReport lastReport = pollingRunner.getLastReport();
-		if(lastReport == null) {
-			LOG.warn("FlinkYarnCluster.hasFailed() has been called on a cluster that didn't receive a status so far." +
-					"The system might be in an erroneous state");
-			return false;
-		} else {
-			YarnApplicationState appState = lastReport.getYarnApplicationState();
-			boolean status = (appState == YarnApplicationState.FAILED ||
-					appState == YarnApplicationState.KILLED);
-			if(status) {
-				LOG.warn("YARN reported application state {}", appState);
-				LOG.warn("Diagnostics: {}", lastReport.getDiagnostics());
-			}
-			return status;
-		}
-	}
-
-
-	@Override
-	public String getDiagnostics() {
-		if(!isConnected) {
-			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
-		}
-
-		if (!hasFailed()) {
-			LOG.warn("getDiagnostics() called for cluster which is not in failed state");
-		}
-		ApplicationReport lastReport = pollingRunner.getLastReport();
-		if (lastReport == null) {
-			LOG.warn("Last report is null");
-			return null;
-		} else {
-			return lastReport.getDiagnostics();
-		}
-	}
-
-	@Override
-	public List<String> getNewMessages() {
-		if(!isConnected) {
-			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
-		}
-
-		if(hasBeenStopped()) {
-			throw new RuntimeException("The FlinkYarnCluster has already been stopped");
-		}
-		List<String> ret = new ArrayList<String>();
-
-		// get messages from ApplicationClient (locally)
-		while(true) {
-			Object result;
-			try {
-				Future<Object> response = Patterns.ask(applicationClient,
-						YarnMessages.getLocalGetYarnMessage(), new Timeout(akkaDuration));
-
-				result = Await.result(response, akkaDuration);
-			} catch(Exception ioe) {
-				LOG.warn("Error retrieving the YARN messages locally", ioe);
-				break;
-			}
-
-			if(!(result instanceof Option)) {
-				throw new RuntimeException("LocalGetYarnMessage requires a response of type " +
-						"Option. Instead the response is of type " + result.getClass() + ".");
-			} else {
-				Option messageOption = (Option) result;
-				LOG.debug("Received message option {}", messageOption);
-				if(messageOption.isEmpty()) {
-					break;
-				} else {
-					Object obj = messageOption.get();
-
-					if(obj instanceof InfoMessage) {
-						InfoMessage msg = (InfoMessage) obj;
-						ret.add("[" + msg.date() + "] " + msg.message());
-					} else {
-						LOG.warn("LocalGetYarnMessage returned unexpected type: " + messageOption);
-					}
-				}
-			}
-		}
-		return ret;
-	}
-
-	// -------------------------- Shutdown handling ------------------------
-
-	private AtomicBoolean hasBeenShutDown = new AtomicBoolean(false);
-
-	/**
-	 * Shutdown the YARN cluster.
-	 * @param failApplication whether we should fail the YARN application (in case of errors in Flink)
-	 */
-	@Override
-	public void shutdown(boolean failApplication) {
-		if(!isConnected) {
-			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
-		}
-
-		if(hasBeenShutDown.getAndSet(true)) {
-			return;
-		}
-
-		try {
-			Runtime.getRuntime().removeShutdownHook(clientShutdownHook);
-		} catch (IllegalStateException e) {
-			// we are already in the shutdown hook
-		}
-
-		if(actorSystem != null){
-			LOG.info("Sending shutdown request to the Application Master");
-			if(applicationClient != ActorRef.noSender()) {
-				try {
-					FinalApplicationStatus finalStatus;
-					if (failApplication) {
-						finalStatus = FinalApplicationStatus.FAILED;
-					} else {
-						finalStatus = FinalApplicationStatus.SUCCEEDED;
-					}
-					Future<Object> response = Patterns.ask(applicationClient,
-							new YarnMessages.LocalStopYarnSession(finalStatus,
-									"Flink YARN Client requested shutdown"),
-							new Timeout(akkaDuration));
-					Await.ready(response, akkaDuration);
-				} catch(Exception e) {
-					LOG.warn("Error while stopping YARN Application Client", e);
-				}
-			}
-
-			actorSystem.shutdown();
-			actorSystem.awaitTermination();
-
-			actorSystem = null;
-		}
-
-		LOG.info("Deleting files in " + sessionFilesDir );
-		try {
-			FileSystem shutFS = FileSystem.get(hadoopConfig);
-			shutFS.delete(sessionFilesDir, true); // delete conf and jar file.
-			shutFS.close();
-		}catch(IOException e){
-			LOG.error("Could not delete the Flink jar and configuration files in HDFS..", e);
-		}
-
-		try {
-			actorRunner.join(1000); // wait for 1 second
-		} catch (InterruptedException e) {
-			LOG.warn("Shutdown of the actor runner was interrupted", e);
-			Thread.currentThread().interrupt();
-		}
-		try {
-			pollingRunner.stopRunner();
-			pollingRunner.join(1000);
-		} catch(InterruptedException e) {
-			LOG.warn("Shutdown of the polling runner was interrupted", e);
-			Thread.currentThread().interrupt();
-		}
-
-		LOG.info("YARN Client is shutting down");
-		yarnClient.stop(); // actorRunner is using the yarnClient.
-		yarnClient = null; // set null to clearly see if somebody wants to access it afterwards.
-	}
-
-	@Override
-	public boolean hasBeenStopped() {
-		return hasBeenShutDown.get();
-	}
-
-
-	public class ClientShutdownHook extends Thread {
-		@Override
-		public void run() {
-			LOG.info("Shutting down FlinkYarnCluster from the client shutdown hook");
-			shutdown(false);
-		}
-	}
-
-	// -------------------------- Polling ------------------------
-
-	public static class PollingThread extends Thread {
-
-		AtomicBoolean running = new AtomicBoolean(true);
-		private YarnClient yarnClient;
-		private ApplicationId appId;
-
-		// ------- status information stored in the polling thread
-		private final Object lock = new Object();
-		private ApplicationReport lastReport;
-
-
-		public PollingThread(YarnClient yarnClient, ApplicationId appId) {
-			this.yarnClient = yarnClient;
-			this.appId = appId;
-		}
-
-		public void stopRunner() {
-			if(!running.get()) {
-				LOG.warn("Polling thread was already stopped");
-			}
-			running.set(false);
-		}
-
-		public ApplicationReport getLastReport() {
-			synchronized (lock) {
-				return lastReport;
-			}
-		}
-
-		@Override
-		public void run() {
-			while (running.get() && yarnClient.isInState(Service.STATE.STARTED)) {
-				try {
-					ApplicationReport report = yarnClient.getApplicationReport(appId);
-					synchronized (lock) {
-						lastReport = report;
-					}
-				} catch (Exception e) {
-					LOG.warn("Error while getting application report", e);
-				}
-				try {
-					Thread.sleep(FlinkYarnCluster.POLLING_THREAD_INTERVAL_MS);
-				} catch (InterruptedException e) {
-					LOG.error("Polling thread got interrupted", e);
-					Thread.currentThread().interrupt(); // pass interrupt.
-				}
-			}
-			if(running.get() && !yarnClient.isInState(Service.STATE.STARTED)) {
-				// == if the polling thread is still running but the yarn client is stopped.
-				LOG.warn("YARN client is unexpected in state " + yarnClient.getServiceState());
-			}
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
index 1d0afc4..24b5a35 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
@@ -22,7 +22,6 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 import akka.actor.Props;
 
-import org.apache.flink.client.CliFrontend;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
@@ -39,6 +38,7 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils;
 import org.apache.flink.runtime.util.SignalHandler;
 import org.apache.flink.runtime.webmonitor.WebMonitor;
 
+import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.Credentials;
@@ -192,7 +192,7 @@ public class YarnApplicationMasterRunner {
 
 			// Flink configuration
 			final Map<String, String> dynamicProperties =
-				CliFrontend.getDynamicProperties(ENV.get(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES));
+				FlinkYarnSessionCli.getDynamicProperties(ENV.get(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES));
 			LOG.debug("YARN dynamic properties: {}", dynamicProperties);
 
 			final Configuration config = createConfiguration(currDir, dynamicProperties);
@@ -292,8 +292,7 @@ public class YarnApplicationMasterRunner {
 			// 3: Flink's Yarn ResourceManager
 			LOG.debug("Starting YARN Flink Resource Manager");
 
-			// we need the leader retrieval service here to be informed of new
-			// leader session IDs, even though there can be only one leader ever
+			// we need the leader retrieval service here to be informed of new leaders and session IDs
 			LeaderRetrievalService leaderRetriever = 
 				LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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
new file mode 100644
index 0000000..a5b8af7
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java
@@ -0,0 +1,577 @@
+/*
+ * 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 akka.actor.ActorRef;
+
+import static akka.pattern.Patterns.ask;
+
+import akka.actor.PoisonPill;
+import akka.actor.Props;
+import akka.pattern.Patterns;
+import akka.util.Timeout;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobSubmissionResult;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
+import org.apache.flink.runtime.clusterframework.messages.InfoMessage;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.service.Service;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.None$;
+import scala.Option;
+import scala.Some;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Java representation of a running Flink cluster within YARN.
+ */
+public class YarnClusterClient extends ClusterClient {
+
+	private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClient.class);
+
+	private static final int POLLING_THREAD_INTERVAL_MS = 1000;
+
+	private YarnClient yarnClient;
+	private Thread actorRunner;
+	private Thread clientShutdownHook = new ClientShutdownHook();
+	private PollingThread pollingRunner;
+	private final Configuration hadoopConfig;
+	// (HDFS) location of the files required to run on YARN. Needed here to delete them on shutdown.
+	private final Path sessionFilesDir;
+
+	/** The leader retrieval service for connecting to the cluster and finding the active leader. */
+	private final LeaderRetrievalService leaderRetrievalService;
+
+	//---------- Class internal fields -------------------
+
+	private final AbstractYarnClusterDescriptor clusterDescriptor;
+	private final ActorRef applicationClient;
+	private final FiniteDuration akkaDuration;
+	private final Timeout akkaTimeout;
+	private final ApplicationReport applicationId;
+	private final ApplicationId appId;
+	private final String trackingURL;
+
+	private boolean isConnected = false;
+
+
+	/**
+	 * Create a new Flink on YARN cluster.
+	 *
+	 * @param clusterDescriptor The descriptor used at cluster creation
+	 * @param yarnClient Client to talk to YARN
+	 * @param appReport the YARN application ID
+	 * @param flinkConfig Flink configuration
+	 * @param sessionFilesDir Location of files required for YARN session
+	 * @throws IOException
+	 * @throws YarnException
+	 */
+	public YarnClusterClient(
+		final AbstractYarnClusterDescriptor clusterDescriptor,
+		final YarnClient yarnClient,
+		final ApplicationReport appReport,
+		org.apache.flink.configuration.Configuration flinkConfig,
+		Path sessionFilesDir) throws IOException, YarnException {
+
+		super(flinkConfig);
+
+		this.akkaDuration = AkkaUtils.getTimeout(flinkConfig);
+		this.akkaTimeout = Timeout.durationToTimeout(akkaDuration);
+		this.clusterDescriptor = clusterDescriptor;
+		this.yarnClient = yarnClient;
+		this.hadoopConfig = yarnClient.getConfig();
+		this.sessionFilesDir = sessionFilesDir;
+		this.applicationId = appReport;
+		this.appId = appReport.getApplicationId();
+		this.trackingURL = appReport.getTrackingUrl();
+
+		try {
+			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
+		} catch (Exception e) {
+			throw new IOException("Could not create the leader retrieval service.", e);
+		}
+
+
+		if (isConnected) {
+			throw new IllegalStateException("Already connected to the cluster.");
+		}
+
+		// start application client
+		LOG.info("Start application client.");
+
+		applicationClient = actorSystem.actorOf(
+			Props.create(
+				ApplicationClient.class,
+				flinkConfig,
+				leaderRetrievalService),
+			"applicationClient");
+
+		actorRunner = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				// blocks until ApplicationClient has been stopped
+				actorSystem.awaitTermination();
+
+				// get final application report
+				try {
+					ApplicationReport appReport = yarnClient.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:\n"
+							+ "\tyarn logs -applicationId " + appReport.getApplicationId() + "\n"
+							+ "(It sometimes takes a few seconds until the logs are aggregated)");
+					}
+				} catch (Exception e) {
+					LOG.warn("Error while getting final application report", e);
+				}
+			}
+		});
+		actorRunner.setDaemon(true);
+		actorRunner.start();
+
+		pollingRunner = new PollingThread(yarnClient, appId);
+		pollingRunner.setDaemon(true);
+		pollingRunner.start();
+
+		Runtime.getRuntime().addShutdownHook(clientShutdownHook);
+
+		isConnected = true;
+
+		logAndSysout("Waiting until all TaskManagers have connected");
+
+		while(true) {
+			GetClusterStatusResponse status = getClusterStatus();
+			if (status != null) {
+				if (status.numRegisteredTaskManagers() < clusterDescriptor.getTaskManagerCount()) {
+					logAndSysout("TaskManager status (" + status.numRegisteredTaskManagers() + "/"
+						+ clusterDescriptor.getTaskManagerCount() + ")");
+				} else {
+					logAndSysout("All TaskManagers are connected");
+					break;
+				}
+			} else {
+				logAndSysout("No status updates from the YARN cluster received so far. Waiting ...");
+			}
+
+			try {
+				Thread.sleep(500);
+			} catch (InterruptedException e) {
+				LOG.error("Interrupted while waiting for TaskManagers");
+				System.err.println("Thread is interrupted");
+				throw new IOException("Interrupted while waiting for TaskManagers", e);
+			}
+		}
+	}
+
+	public void disconnect() {
+		if(!isConnected) {
+			throw new IllegalStateException("Can not disconnect from an unconnected cluster.");
+		}
+		LOG.info("Disconnecting YarnClusterClient from ApplicationMaster");
+
+		if(!Runtime.getRuntime().removeShutdownHook(clientShutdownHook)) {
+			LOG.warn("Error while removing the shutdown hook. The YARN session might be killed unintentionally");
+		}
+		// tell the actor to shut down.
+		applicationClient.tell(PoisonPill.getInstance(), applicationClient);
+
+		try {
+			actorRunner.join(1000); // wait for 1 second
+		} catch (InterruptedException e) {
+			LOG.warn("Shutdown of the actor runner was interrupted", e);
+			Thread.currentThread().interrupt();
+		}
+		try {
+			pollingRunner.stopRunner();
+			pollingRunner.join(1000);
+		} catch(InterruptedException e) {
+			LOG.warn("Shutdown of the polling runner was interrupted", e);
+			Thread.currentThread().interrupt();
+		}
+		isConnected = false;
+	}
+
+
+	// -------------------------- Interaction with the cluster ------------------------
+
+	/*
+	 * Tells the Cluster to monitor the status of JobId and stop itself once the specified job has finished.
+	 */
+	private void stopAfterJob(JobID jobID) {
+		Preconditions.checkNotNull(jobID, "The job id must not be null");
+		Future<Object> messageReceived = ask(applicationClient, new YarnMessages.LocalStopAMAfterJob(jobID), akkaTimeout);
+		try {
+			Await.result(messageReceived, akkaDuration);
+		} catch (Exception e) {
+			throw new RuntimeException("Unable to tell application master to stop once the specified job has been finised", e);
+		}
+	}
+
+	@Override
+	public org.apache.flink.configuration.Configuration getFlinkConfiguration() {
+		return flinkConfig;
+	}
+
+	@Override
+	public int getMaxSlots() {
+		int maxSlots = clusterDescriptor.getTaskManagerCount() * clusterDescriptor.getTaskManagerSlots();
+		return maxSlots > 0 ? maxSlots : -1;
+	}
+
+	@Override
+	protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException {
+		if (isDetached()) {
+			JobSubmissionResult result = super.runDetached(jobGraph, classLoader);
+			stopAfterJob(jobGraph.getJobID());
+			return result;
+		} else {
+			return super.run(jobGraph, classLoader);
+		}
+	}
+
+	@Override
+	public String getWebInterfaceURL() {
+		// there seems to be a difference between HD 2.2.0 and 2.6.0
+		if(!trackingURL.startsWith("http://")) {
+			return "http://" + trackingURL;
+		} else {
+			return trackingURL;
+		}
+	}
+
+	@Override
+	public String getClusterIdentifier() {
+		return applicationId.getApplicationId().toString();
+	}
+
+	/**
+	 * This method is only available if the cluster hasn't been started in detached mode.
+	 */
+	@Override
+	public GetClusterStatusResponse getClusterStatus() {
+		if(!isConnected) {
+			throw new IllegalStateException("The cluster is not connected to the ApplicationMaster.");
+		}
+		if(hasBeenShutdown()) {
+			throw new RuntimeException("The YarnClusterClient has already been stopped");
+		}
+		Future<Object> clusterStatusOption = ask(applicationClient, YarnMessages.getLocalGetyarnClusterStatus(), akkaTimeout);
+		Object clusterStatus;
+		try {
+			clusterStatus = Await.result(clusterStatusOption, akkaDuration);
+		} catch (Exception e) {
+			throw new RuntimeException("Unable to get ClusterClient status from Application Client", e);
+		}
+		if(clusterStatus instanceof None$) {
+			return null;
+		} else if(clusterStatus instanceof Some) {
+			return (GetClusterStatusResponse) (((Some) clusterStatus).get());
+		} else {
+			throw new RuntimeException("Unexpected type: " + clusterStatus.getClass().getCanonicalName());
+		}
+	}
+
+	public ApplicationStatus getApplicationStatus() {
+		if(!isConnected) {
+			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
+		}
+		if(pollingRunner == null) {
+			LOG.warn("YarnClusterClient.getApplicationStatus() has been called on an uninitialized cluster." +
+					"The system might be in an erroneous state");
+		}
+		ApplicationReport lastReport = pollingRunner.getLastReport();
+		if(lastReport == null) {
+			LOG.warn("YarnClusterClient.getApplicationStatus() has been called on a cluster that didn't receive a status so far." +
+					"The system might be in an erroneous state");
+			return ApplicationStatus.UNKNOWN;
+		} else {
+			YarnApplicationState appState = lastReport.getYarnApplicationState();
+			ApplicationStatus status =
+				(appState == YarnApplicationState.FAILED || appState == YarnApplicationState.KILLED) ?
+					ApplicationStatus.FAILED : ApplicationStatus.SUCCEEDED;
+			if(status != ApplicationStatus.SUCCEEDED) {
+				LOG.warn("YARN reported application state {}", appState);
+				LOG.warn("Diagnostics: {}", lastReport.getDiagnostics());
+			}
+			return status;
+		}
+	}
+
+
+	private String getDiagnostics() {
+		if(!isConnected) {
+			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
+		}
+
+		if (getApplicationStatus() == ApplicationStatus.SUCCEEDED) {
+			LOG.warn("getDiagnostics() called for cluster which is not in failed state");
+		}
+		ApplicationReport lastReport = pollingRunner.getLastReport();
+		if (lastReport == null) {
+			LOG.warn("Last report is null");
+			return null;
+		} else {
+			return lastReport.getDiagnostics();
+		}
+	}
+
+	@Override
+	public List<String> getNewMessages() {
+		if(!isConnected) {
+			throw new IllegalStateException("The cluster has been connected to the ApplicationMaster.");
+		}
+
+		if(hasBeenShutdown()) {
+			throw new RuntimeException("The YarnClusterClient has already been stopped");
+		}
+		List<String> ret = new ArrayList<String>();
+
+		// get messages from ApplicationClient (locally)
+		while(true) {
+			Object result;
+			try {
+				Future<Object> response = Patterns.ask(applicationClient,
+						YarnMessages.getLocalGetYarnMessage(), new Timeout(akkaDuration));
+
+				result = Await.result(response, akkaDuration);
+			} catch(Exception ioe) {
+				LOG.warn("Error retrieving the YARN messages locally", ioe);
+				break;
+			}
+
+			if(!(result instanceof Option)) {
+				throw new RuntimeException("LocalGetYarnMessage requires a response of type " +
+						"Option. Instead the response is of type " + result.getClass() + ".");
+			} else {
+				Option messageOption = (Option) result;
+				LOG.debug("Received message option {}", messageOption);
+				if(messageOption.isEmpty()) {
+					break;
+				} else {
+					Object obj = messageOption.get();
+
+					if(obj instanceof InfoMessage) {
+						InfoMessage msg = (InfoMessage) obj;
+						ret.add("[" + msg.date() + "] " + msg.message());
+					} else {
+						LOG.warn("LocalGetYarnMessage returned unexpected type: " + messageOption);
+					}
+				}
+			}
+		}
+		return ret;
+	}
+
+	// -------------------------- Shutdown handling ------------------------
+
+	private AtomicBoolean hasBeenShutDown = new AtomicBoolean(false);
+
+	/**
+	 * Shuts down or disconnects from the YARN cluster.
+	 */
+	@Override
+	public void finalizeCluster() {
+
+		if (!isConnected) {
+			throw new IllegalStateException("The cluster has been not been connected to the ApplicationMaster.");
+		}
+
+		if (isDetached()) {
+			// only disconnect if we are running detached
+			disconnect();
+			return;
+		}
+
+		// show cluster status
+
+		List<String> msgs = getNewMessages();
+		if (msgs != null && msgs.size() > 1) {
+
+			logAndSysout("The following messages were created by the YARN cluster while running the Job:");
+			for (String msg : msgs) {
+				logAndSysout(msg);
+			}
+		}
+		if (getApplicationStatus() != ApplicationStatus.SUCCEEDED) {
+			logAndSysout("YARN cluster is in non-successful state " + getApplicationStatus());
+			logAndSysout("YARN Diagnostics: " + getDiagnostics());
+		}
+
+
+		if(hasBeenShutDown.getAndSet(true)) {
+			return;
+		}
+
+		try {
+			Runtime.getRuntime().removeShutdownHook(clientShutdownHook);
+		} catch (IllegalStateException e) {
+			// we are already in the shutdown hook
+		}
+
+		if(actorSystem != null){
+			LOG.info("Sending shutdown request to the Application Master");
+			if(applicationClient != ActorRef.noSender()) {
+				try {
+					Future<Object> response = Patterns.ask(applicationClient,
+							new YarnMessages.LocalStopYarnSession(getApplicationStatus(),
+									"Flink YARN Client requested shutdown"),
+							new Timeout(akkaDuration));
+					Await.ready(response, akkaDuration);
+				} catch(Exception e) {
+					LOG.warn("Error while stopping YARN Application Client", e);
+				}
+			}
+
+			actorSystem.shutdown();
+			actorSystem.awaitTermination();
+		}
+
+		LOG.info("Deleting files in " + sessionFilesDir);
+		try {
+			FileSystem shutFS = FileSystem.get(hadoopConfig);
+			shutFS.delete(sessionFilesDir, true); // delete conf and jar file.
+			shutFS.close();
+		}catch(IOException e){
+			LOG.error("Could not delete the Flink jar and configuration files in HDFS..", e);
+		}
+
+		try {
+			actorRunner.join(1000); // wait for 1 second
+		} catch (InterruptedException e) {
+			LOG.warn("Shutdown of the actor runner was interrupted", e);
+			Thread.currentThread().interrupt();
+		}
+		try {
+			pollingRunner.stopRunner();
+			pollingRunner.join(1000);
+		} catch(InterruptedException e) {
+			LOG.warn("Shutdown of the polling runner was interrupted", e);
+			Thread.currentThread().interrupt();
+		}
+
+		LOG.info("YARN Client is shutting down");
+		yarnClient.stop(); // actorRunner is using the yarnClient.
+		yarnClient = null; // set null to clearly see if somebody wants to access it afterwards.
+	}
+
+	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");
+			shutdown();
+		}
+	}
+
+	// -------------------------- Polling ------------------------
+
+	private static class PollingThread extends Thread {
+
+		AtomicBoolean running = new AtomicBoolean(true);
+		private YarnClient yarnClient;
+		private ApplicationId appId;
+
+		// ------- status information stored in the polling thread
+		private final Object lock = new Object();
+		private ApplicationReport lastReport;
+
+
+		public PollingThread(YarnClient yarnClient, ApplicationId appId) {
+			this.yarnClient = yarnClient;
+			this.appId = appId;
+		}
+
+		public void stopRunner() {
+			if(!running.get()) {
+				LOG.warn("Polling thread was already stopped");
+			}
+			running.set(false);
+		}
+
+		public ApplicationReport getLastReport() {
+			synchronized (lock) {
+				return lastReport;
+			}
+		}
+
+		@Override
+		public void run() {
+			while (running.get() && yarnClient.isInState(Service.STATE.STARTED)) {
+				try {
+					ApplicationReport report = yarnClient.getApplicationReport(appId);
+					synchronized (lock) {
+						lastReport = report;
+					}
+				} catch (Exception e) {
+					LOG.warn("Error while getting application report", e);
+				}
+				try {
+					Thread.sleep(YarnClusterClient.POLLING_THREAD_INTERVAL_MS);
+				} catch (InterruptedException e) {
+					LOG.error("Polling thread got interrupted", e);
+					Thread.currentThread().interrupt(); // pass interrupt.
+					stopRunner();
+				}
+			}
+			if(running.get() && !yarnClient.isInState(Service.STATE.STARTED)) {
+				// == if the polling thread is still running but the yarn client is stopped.
+				LOG.warn("YARN client is unexpected in state " + yarnClient.getServiceState());
+			}
+		}
+	}
+
+	@Override
+	public boolean isDetached() {
+		// either we have set detached mode using the general '-d' flag or using the Yarn CLI flag 'yd'
+		return super.isDetached() || clusterDescriptor.isDetachedMode();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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
new file mode 100644
index 0000000..43e7c7b
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * Default implementation of {@link AbstractYarnClusterDescriptor} which starts an {@link YarnApplicationMasterRunner}.
+ */
+public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor {
+	@Override
+	protected Class<?> getApplicationMasterClass() {
+		return YarnApplicationMasterRunner.class;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/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
new file mode 100644
index 0000000..a2375c5
--- /dev/null
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -0,0 +1,606 @@
+/*
+ * 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.cli;
+
+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.flink.client.CliFrontend;
+import org.apache.flink.client.ClientUtils;
+import org.apache.flink.client.cli.CliFrontendParser;
+import org.apache.flink.client.cli.CustomCommandLine;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.StandaloneClusterClient;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.yarn.YarnClusterDescriptor;
+import org.apache.flink.yarn.YarnClusterClient;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Class handling the command line interface to the YARN session.
+ */
+public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient> {
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnSessionCli.class);
+
+	//------------------------------------ Constants   -------------------------
+
+	public static final String CONFIG_FILE_LOGBACK_NAME = "logback.xml";
+	public static final String CONFIG_FILE_LOG4J_NAME = "log4j.properties";
+
+	private static final int CLIENT_POLLING_INTERVALL = 3;
+
+	/** The id for the CommandLine interface */
+	private static final String ID = "yarn-cluster";
+
+	// YARN-session related constants
+	private static final String YARN_PROPERTIES_FILE = ".yarn-properties-";
+	private static final String YARN_PROPERTIES_JOBMANAGER_KEY = "jobManager";
+	private static final String YARN_PROPERTIES_PARALLELISM = "parallelism";
+	private static final String YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING = "dynamicPropertiesString";
+
+	private static final String YARN_DYNAMIC_PROPERTIES_SEPARATOR = "@@"; // this has to be a regex for String.split()
+
+	//------------------------------------ Command Line argument options -------------------------
+	// the prefix transformation is used by the CliFrontend static constructor.
+	private final Option QUERY;
+	// --- or ---
+	private final Option QUEUE;
+	private final Option SHIP_PATH;
+	private final Option FLINK_JAR;
+	private final Option JM_MEMORY;
+	private final Option TM_MEMORY;
+	private final Option CONTAINER;
+	private final Option SLOTS;
+	private final Option DETACHED;
+	private final Option STREAMING;
+	private final Option NAME;
+
+	/**
+	 * Dynamic properties allow the user to specify additional configuration values with -D, such as
+	 *  -Dfs.overwrite-files=true  -Dtaskmanager.network.numberOfBuffers=16368
+	 */
+	private final Option DYNAMIC_PROPERTIES;
+
+	private final boolean acceptInteractiveInput;
+	
+	//------------------------------------ Internal fields -------------------------
+	private YarnClusterClient yarnCluster = null;
+	private boolean detachedMode = false;
+
+	public FlinkYarnSessionCli(String shortPrefix, String longPrefix) {
+		this(shortPrefix, longPrefix, true);
+	}
+
+	public FlinkYarnSessionCli(String shortPrefix, String longPrefix, boolean acceptInteractiveInput) {
+		this.acceptInteractiveInput = acceptInteractiveInput;
+		
+		QUERY = new Option(shortPrefix + "q", longPrefix + "query", false, "Display available YARN resources (memory, cores)");
+		QUEUE = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue.");
+		SHIP_PATH = new Option(shortPrefix + "t", longPrefix + "ship", true, "Ship files in the specified directory (t for transfer)");
+		FLINK_JAR = new Option(shortPrefix + "j", longPrefix + "jar", true, "Path to Flink jar file");
+		JM_MEMORY = new Option(shortPrefix + "jm", longPrefix + "jobManagerMemory", true, "Memory for JobManager Container [in MB]");
+		TM_MEMORY = 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");
+		DYNAMIC_PROPERTIES = new Option(shortPrefix + "D", true, "Dynamic properties");
+		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");
+	}
+
+	/**
+	 * Resumes from a Flink Yarn properties file
+	 * @param flinkConfiguration The flink configuration
+	 * @return True if the properties were loaded, false otherwise
+	 */
+	private boolean resumeFromYarnProperties(Configuration flinkConfiguration) {
+		// load the YARN properties
+		File propertiesFile = new File(getYarnPropertiesLocation(flinkConfiguration));
+		if (!propertiesFile.exists()) {
+			return false;
+		}
+
+		logAndSysout("Found YARN properties file " + propertiesFile.getAbsolutePath());
+
+		Properties yarnProperties = new Properties();
+		try {
+			try (InputStream is = new FileInputStream(propertiesFile)) {
+				yarnProperties.load(is);
+			}
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Cannot read the YARN properties file", e);
+		}
+
+		// 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);
+			}
+			catch (NumberFormatException e) {
+				throw new RuntimeException("Error while parsing the YARN properties: " +
+					"Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer.");
+			}
+		}
+
+		// get the JobManager address from the YARN properties
+		String address = yarnProperties.getProperty(YARN_PROPERTIES_JOBMANAGER_KEY);
+		InetSocketAddress jobManagerAddress;
+		if (address != null) {
+			try {
+				jobManagerAddress = ClientUtils.parseHostPortAddress(address);
+				// store address in config from where it is retrieved by the retrieval service
+				CliFrontend.writeJobManagerAddressToConfig(flinkConfiguration, jobManagerAddress);
+			}
+			catch (Exception e) {
+				throw new RuntimeException("YARN properties contain an invalid entry for JobManager address.", e);
+			}
+
+			logAndSysout("Using JobManager address from YARN properties " + jobManagerAddress);
+		}
+
+		// 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 true;
+	}
+
+	public YarnClusterDescriptor createDescriptor(String defaultApplicationName, CommandLine cmd) {
+
+
+		YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor();
+
+		if (!cmd.hasOption(CONTAINER.getOpt())) { // number of containers is required option!
+			LOG.error("Missing required argument {}", CONTAINER.getOpt());
+			printUsage();
+			throw new IllegalArgumentException("Missing required argument " + CONTAINER.getOpt());
+		}
+		yarnClusterDescriptor.setTaskManagerCount(Integer.valueOf(cmd.getOptionValue(CONTAINER.getOpt())));
+
+		// Jar Path
+		Path localJarPath;
+		if (cmd.hasOption(FLINK_JAR.getOpt())) {
+			String userPath = cmd.getOptionValue(FLINK_JAR.getOpt());
+			if(!userPath.startsWith("file://")) {
+				userPath = "file://" + userPath;
+			}
+			localJarPath = new Path(userPath);
+		} else {
+			LOG.info("No path for the flink jar passed. Using the location of "
+				+ yarnClusterDescriptor.getClass() + " to locate the jar");
+			localJarPath = new Path("file://" +
+				yarnClusterDescriptor.getClass().getProtectionDomain().getCodeSource().getLocation().getPath());
+		}
+
+		yarnClusterDescriptor.setLocalJarPath(localJarPath);
+
+		List<File> shipFiles = new ArrayList<>();
+		// path to directory to ship
+		if (cmd.hasOption(SHIP_PATH.getOpt())) {
+			String shipPath = cmd.getOptionValue(SHIP_PATH.getOpt());
+			File shipDir = new File(shipPath);
+			if (shipDir.isDirectory()) {
+				shipFiles = new ArrayList<>(Arrays.asList(shipDir.listFiles(new FilenameFilter() {
+					@Override
+					public boolean accept(File dir, String name) {
+						return !(name.equals(".") || name.equals(".."));
+					}
+				})));
+			} else {
+				LOG.warn("Ship directory is not a directory. Ignoring it.");
+			}
+		}
+
+		yarnClusterDescriptor.setShipFiles(shipFiles);
+
+		// queue
+		if (cmd.hasOption(QUEUE.getOpt())) {
+			yarnClusterDescriptor.setQueue(cmd.getOptionValue(QUEUE.getOpt()));
+		}
+
+		// JobManager Memory
+		if (cmd.hasOption(JM_MEMORY.getOpt())) {
+			int jmMemory = Integer.valueOf(cmd.getOptionValue(JM_MEMORY.getOpt()));
+			yarnClusterDescriptor.setJobManagerMemory(jmMemory);
+		}
+
+		// Task Managers memory
+		if (cmd.hasOption(TM_MEMORY.getOpt())) {
+			int tmMemory = Integer.valueOf(cmd.getOptionValue(TM_MEMORY.getOpt()));
+			yarnClusterDescriptor.setTaskManagerMemory(tmMemory);
+		}
+
+		if (cmd.hasOption(SLOTS.getOpt())) {
+			int slots = Integer.valueOf(cmd.getOptionValue(SLOTS.getOpt()));
+			yarnClusterDescriptor.setTaskManagerSlots(slots);
+		}
+
+		String[] dynamicProperties = null;
+		if (cmd.hasOption(DYNAMIC_PROPERTIES.getOpt())) {
+			dynamicProperties = cmd.getOptionValues(DYNAMIC_PROPERTIES.getOpt());
+		}
+		String dynamicPropertiesEncoded = StringUtils.join(dynamicProperties, YARN_DYNAMIC_PROPERTIES_SEPARATOR);
+
+		yarnClusterDescriptor.setDynamicPropertiesEncoded(dynamicPropertiesEncoded);
+
+		if (cmd.hasOption(DETACHED.getOpt()) || cmd.hasOption(CliFrontendParser.DETACHED_OPTION.getOpt())) {
+			this.detachedMode = true;
+			yarnClusterDescriptor.setDetachedMode(true);
+		}
+
+		if(cmd.hasOption(NAME.getOpt())) {
+			yarnClusterDescriptor.setName(cmd.getOptionValue(NAME.getOpt()));
+		} else {
+			// set the default application name, if none is specified
+			if(defaultApplicationName != null) {
+				yarnClusterDescriptor.setName(defaultApplicationName);
+			}
+		}
+
+		// ----- Convenience -----
+
+		// the number of slots available from YARN:
+		int yarnTmSlots = yarnClusterDescriptor.getTaskManagerSlots();
+		if (yarnTmSlots == -1) {
+			yarnTmSlots = 1;
+		}
+
+		int maxSlots = yarnTmSlots * yarnClusterDescriptor.getTaskManagerCount();
+		int userParallelism = Integer.valueOf(cmd.getOptionValue(CliFrontendParser.PARALLELISM_OPTION.getOpt(), "-1"));
+		if (userParallelism != -1) {
+			int slotsPerTM = userParallelism / yarnClusterDescriptor.getTaskManagerCount();
+			String message = "The YARN cluster has " + maxSlots + " slots available, " +
+				"but the user requested a parallelism of " + userParallelism + " on YARN. " +
+				"Each of the " + yarnClusterDescriptor.getTaskManagerCount() + " TaskManagers " +
+				"will get "+slotsPerTM+" slots.";
+			logAndSysout(message);
+			yarnClusterDescriptor.setTaskManagerSlots(slotsPerTM);
+		}
+
+		return yarnClusterDescriptor;
+	}
+
+	@Override
+	public YarnClusterClient createClient(String applicationName, CommandLine cmdLine) throws Exception {
+
+		YarnClusterDescriptor yarnClusterDescriptor = createDescriptor(applicationName, cmdLine);
+
+		try {
+			return yarnClusterDescriptor.deploy();
+		} catch (Exception e) {
+			throw new RuntimeException("Error deploying the YARN cluster", e);
+		}
+
+	}
+
+	private void printUsage() {
+		System.out.println("Usage:");
+		HelpFormatter formatter = new HelpFormatter();
+		formatter.setWidth(200);
+		formatter.setLeftPadding(5);
+		formatter.setSyntaxPrefix("   Required");
+		Options req = new Options();
+		req.addOption(CONTAINER);
+		formatter.printHelp(" ", req);
+
+		formatter.setSyntaxPrefix("   Optional");
+		Options opt = new Options();
+		opt.addOption(JM_MEMORY);
+		opt.addOption(TM_MEMORY);
+		opt.addOption(QUERY);
+		opt.addOption(QUEUE);
+		opt.addOption(SLOTS);
+		opt.addOption(DYNAMIC_PROPERTIES);
+		opt.addOption(DETACHED);
+		opt.addOption(STREAMING);
+		opt.addOption(NAME);
+		formatter.printHelp(" ", opt);
+	}
+
+	private static void writeYarnProperties(Properties properties, File propertiesFile) {
+		try {
+			OutputStream out = new FileOutputStream(propertiesFile);
+			properties.store(out, "Generated YARN properties file");
+			out.close();
+		} catch (IOException e) {
+			throw new RuntimeException("Error writing the properties file", e);
+		}
+		propertiesFile.setReadable(true, false); // readable for all.
+	}
+
+	public static void runInteractiveCli(YarnClusterClient yarnCluster, boolean readConsoleInput) {
+		final String HELP = "Available commands:\n" +
+				"help - show these commands\n" +
+				"stop - stop the YARN session";
+		int numTaskmanagers = 0;
+		try {
+			BufferedReader in = new BufferedReader(new InputStreamReader(System.in));
+			label:
+			while (true) {
+				// ------------------ check if there are updates by the cluster -----------
+
+				GetClusterStatusResponse status = yarnCluster.getClusterStatus();
+				LOG.debug("Received status message: {}", status);
+
+				if (status != null && numTaskmanagers != status.numRegisteredTaskManagers()) {
+					System.err.println("Number of connected TaskManagers changed to " +
+							status.numRegisteredTaskManagers() + ". " +
+						"Slots available: " + status.totalNumberOfSlots());
+					numTaskmanagers = status.numRegisteredTaskManagers();
+				}
+
+				List<String> messages = yarnCluster.getNewMessages();
+				if (messages != null && messages.size() > 0) {
+					System.err.println("New messages from the YARN cluster: ");
+					for (String msg : messages) {
+						System.err.println(msg);
+					}
+				}
+
+				if (yarnCluster.getApplicationStatus() != ApplicationStatus.SUCCEEDED) {
+					System.err.println("The YARN cluster has failed");
+					yarnCluster.shutdown();
+				}
+
+				// wait until CLIENT_POLLING_INTERVAL is over or the user entered something.
+				long startTime = System.currentTimeMillis();
+				while ((System.currentTimeMillis() - startTime) < CLIENT_POLLING_INTERVALL * 1000
+						&& (!readConsoleInput || !in.ready()))
+				{
+					Thread.sleep(200);
+				}
+				//------------- handle interactive command by user. ----------------------
+				
+				if (readConsoleInput && in.ready()) {
+					String command = in.readLine();
+					switch (command) {
+						case "quit":
+						case "stop":
+							break label;
+
+						case "help":
+							System.err.println(HELP);
+							break;
+						default:
+							System.err.println("Unknown command '" + command + "'. Showing help: \n" + HELP);
+							break;
+					}
+				}
+				
+				if (yarnCluster.hasBeenShutdown()) {
+					LOG.info("Stopping interactive command line interface, YARN cluster has been stopped.");
+					break;
+				}
+			}
+		} catch(Exception e) {
+			LOG.warn("Exception while running the interactive command line interface", e);
+		}
+	}
+
+	public static void main(String[] args) {
+		FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", ""); // no prefix for the YARN session
+		System.exit(cli.run(args));
+	}
+
+	@Override
+	public String getIdentifier() {
+		return ID;
+	}
+
+	public void addOptions(Options options) {
+		options.addOption(FLINK_JAR);
+		options.addOption(JM_MEMORY);
+		options.addOption(TM_MEMORY);
+		options.addOption(CONTAINER);
+		options.addOption(QUEUE);
+		options.addOption(QUERY);
+		options.addOption(SHIP_PATH);
+		options.addOption(SLOTS);
+		options.addOption(DYNAMIC_PROPERTIES);
+		options.addOption(DETACHED);
+		options.addOption(STREAMING);
+		options.addOption(NAME);
+	}
+
+	@Override
+	public ClusterClient retrieveCluster(Configuration config) throws Exception {
+
+		if(resumeFromYarnProperties(config)) {
+			return new StandaloneClusterClient(config);
+		}
+
+		return null;
+	}
+
+	public int run(String[] args) {
+		//
+		//	Command Line Options
+		//
+		Options options = new Options();
+		addOptions(options);
+
+		CommandLineParser parser = new PosixParser();
+		CommandLine cmd;
+		try {
+			cmd = parser.parse(options, args);
+		} catch(Exception e) {
+			System.out.println(e.getMessage());
+			printUsage();
+			return 1;
+		}
+		
+		// Query cluster for metrics
+		if (cmd.hasOption(QUERY.getOpt())) {
+			YarnClusterDescriptor flinkYarnClient = new YarnClusterDescriptor();
+			String description;
+			try {
+				description = flinkYarnClient.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 {
+
+			YarnClusterDescriptor flinkYarnClient;
+			try {
+				flinkYarnClient = createDescriptor(null, cmd);
+			} catch (Exception e) {
+				System.err.println("Error while starting the YARN Client. Please check log output!");
+				return 1;
+			}
+
+			try {
+				yarnCluster = flinkYarnClient.deploy();
+			} 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().getHostAddress() + ":" + 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 = new File(getYarnPropertiesLocation(yarnCluster.getFlinkConfiguration()));
+
+			Properties yarnProps = new Properties();
+			yarnProps.setProperty(YARN_PROPERTIES_JOBMANAGER_KEY, jobManagerAddress);
+			if (flinkYarnClient.getTaskManagerSlots() != -1) {
+				String parallelism =
+						Integer.toString(flinkYarnClient.getTaskManagerSlots() * flinkYarnClient.getTaskManagerCount());
+				yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, parallelism);
+			}
+			// add dynamic properties
+			if (flinkYarnClient.getDynamicPropertiesEncoded() != null) {
+				yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING,
+						flinkYarnClient.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.getClusterIdentifier() + "\n" +
+						"Please also note that the temporary files of the YARN session in {} will not be removed.",
+						flinkYarnClient.getSessionFilesDir());
+				yarnCluster.disconnect();
+			} else {
+				runInteractiveCli(yarnCluster, acceptInteractiveInput);
+
+				if (!yarnCluster.hasBeenShutdown()) {
+					LOG.info("Command Line Interface requested session shutdown");
+					yarnCluster.shutdown();
+				}
+
+				try {
+					yarnPropertiesFile.delete();
+				} catch (Exception e) {
+					LOG.warn("Exception while deleting the JobManager address file", e);
+				}
+			}
+		}
+		return 0;
+	}
+
+	/**
+	 * Utility method for tests.
+	 */
+	public void stop() {
+		if (yarnCluster != null) {
+			LOG.info("Command line interface is shutting down the yarnCluster");
+			yarnCluster.shutdown();
+		}
+	}
+
+	private void logAndSysout(String message) {
+		LOG.info(message);
+		System.out.println(message);
+	}
+
+	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;
+				}
+
+				String[] kv = propLine.split("=");
+				if (kv.length >= 2 && kv[0] != null && kv[1] != null && kv[0].length() > 0) {
+					properties.put(kv[0], kv[1]);
+				}
+			}
+			return properties;
+		}
+		else {
+			return Collections.emptyMap();
+		}
+	}
+
+	private static String getYarnPropertiesLocation(Configuration conf) {
+		String defaultPropertiesFileLocation = System.getProperty("java.io.tmpdir");
+		String currentUser = System.getProperty("user.name");
+		String propertiesFileLocation = conf.getString(ConfigConstants.YARN_PROPERTIES_FILE_LOCATION, defaultPropertiesFileLocation);
+
+		return propertiesFileLocation + File.separator + YARN_PROPERTIES_FILE + currentUser;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
index 2876309..aea1aac 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala
@@ -23,12 +23,10 @@ import java.util.UUID
 import akka.actor._
 import grizzled.slf4j.Logger
 import org.apache.flink.configuration.Configuration
-import org.apache.flink.runtime.clusterframework.ApplicationStatus
 import org.apache.flink.runtime.clusterframework.messages._
 import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService}
 import org.apache.flink.runtime.{LeaderSessionMessageFilter, FlinkActor, LogMessages}
 import org.apache.flink.yarn.YarnMessages._
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus
 import scala.collection.mutable
 import scala.concurrent.duration._
 
@@ -36,7 +34,7 @@ import scala.language.postfixOps
 
 /** Actor which is responsible to repeatedly poll the Yarn cluster status from the ResourceManager.
   *
-  * This class represents the bridge between the [[FlinkYarnCluster]] and the
+  * This class represents the bridge between the [[YarnClusterClient]] and the
   * [[YarnApplicationMasterRunner]].
   *
   * @param flinkConfig Configuration object
@@ -135,9 +133,9 @@ class ApplicationClient(
       }
 
     case msg: RegisterInfoMessageListenerSuccessful =>
+      // The job manager acts as a proxy between the client and the resource managert
       val jm = sender()
-
-      log.info(s"Successfully registered at the ResourceManager $jm")
+      log.info(s"Successfully registered at the ResourceManager using JobManager $jm")
 
       yarnJobManager = Some(jm)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f9b52a31/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnMessages.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnMessages.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnMessages.scala
index 8645581..da1917b 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnMessages.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnMessages.scala
@@ -18,12 +18,13 @@
 
 package org.apache.flink.yarn
 
-import java.util.{List => JavaList, UUID, Date}
+import java.util.{Date, UUID, List => JavaList}
 
 import org.apache.flink.api.common.JobID
+import org.apache.flink.runtime.clusterframework.ApplicationStatus
 import org.apache.flink.runtime.messages.RequiresLeaderSessionID
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.yarn.api.records.{ContainerStatus, Container, FinalApplicationStatus}
+import org.apache.hadoop.yarn.api.records.{Container, ContainerStatus, FinalApplicationStatus}
 
 import scala.concurrent.duration.{Deadline, FiniteDuration}
 
@@ -31,7 +32,7 @@ object YarnMessages {
 
   case class ApplicationMasterStatus(numTaskManagers: Int, numSlots: Int)
 
-  case class LocalStopYarnSession(status: FinalApplicationStatus, diagnostics: String)
+  case class LocalStopYarnSession(status: ApplicationStatus, diagnostics: String)
 
   /**
     * Entry point to start a new YarnSession.