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.