From 2af658673f877a7e0fd73fdd2907f88824d793a5 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 1 Dec 2014 18:59:49 +0100 Subject: [PATCH] [FLINK-1295][FLINK-883] Allow to deploy 'job only' YARN cluster. Add tests to YARN - users can now also deploy Flink on YARN for executing a single job. - The flink-yarn project has been moved out of the flink-addons module - the MiniYARNCluster is used for testing Flink on YARN - There is now a (undocumented) Java interface Flink's YARN client, allowing users to manually control the Yarn session. - ALL ports used by Flink when running on YARN are automatically determined. In the past users reported problems with blocked ports (YARN is telling the client the RPC address of the application master) - The checks before deployment have been improved to give better error messages if the user is requesting too many resources for a YARN session --- docs/cli.md | 8 +- docs/config.md | 3 +- docs/yarn_setup.md | 32 +- .../java/org/apache/flink/yarn/Client.java | 713 ------------------ .../apache/flink/yarn/ApplicationClient.scala | 158 ---- flink-addons/pom.xml | 13 - flink-clients/pom.xml | 10 +- .../org/apache/flink/client/CliFrontend.java | 153 +++- .../flink/client/FlinkYarnSessionCli.java | 429 +++++++++++ .../apache/flink/client/program/Client.java | 1 - .../flink/client/CliFrontendInfoTest.java | 2 +- .../CliFrontendJobManagerConnectionTest.java | 13 +- .../client/CliFrontendListCancelTest.java | 8 +- .../flink/client/CliFrontendTestUtils.java | 10 +- .../flink/configuration/ConfigConstants.java | 14 + .../org/apache/flink/core/fs/FileSystem.java | 1 - flink-dist/pom.xml | 4 +- flink-dist/src/main/flink-bin/bin/flink | 3 +- .../main/flink-bin/yarn-bin/yarn-session.sh | 2 +- .../jobmanager/web/SetupInfoServlet.java | 16 +- .../runtime/jobmanager/web/WebInfoServer.java | 9 +- .../apache/flink/runtime/net/NetUtils.java | 4 +- .../runtime/yarn/AbstractFlinkYarnClient.java | 50 ++ .../yarn/AbstractFlinkYarnCluster.java | 44 ++ .../runtime/yarn/FlinkYarnClusterStatus.java | 86 +++ .../apache/flink/runtime/akka/AkkaUtils.scala | 3 + .../flink/runtime/client/JobClient.scala | 3 +- .../flink/runtime/jobmanager/JobManager.scala | 7 + flink-yarn-tests/pom.xml | 121 +++ .../java/org/apache/flink/yarn/UtilsTest.java | 42 ++ .../yarn/YARNSessionCapacitySchedulerIT.java | 72 ++ .../apache/flink/yarn/YARNSessionFIFOIT.java | 225 ++++++ .../org/apache/flink/yarn/YarnTestBase.java | 379 ++++++++++ .../src/test/resources/log4j-test.properties | 25 + .../flink-yarn => flink-yarn}/pom.xml | 4 +- .../apache/flink/yarn/FlinkYarnClient.java | 653 ++++++++++++++++ .../apache/flink/yarn/FlinkYarnCluster.java | 363 +++++++++ .../java/org/apache/flink/yarn/Utils.java | 78 +- .../yarn/appMaster/YarnTaskManagerRunner.java | 4 +- .../apache/flink/yarn/ApplicationClient.scala | 127 ++++ .../apache/flink/yarn/ApplicationMaster.scala | 79 +- .../org/apache/flink/yarn/Messages.scala | 13 +- .../apache/flink/yarn/YarnJobManager.scala | 47 +- .../apache/flink/yarn/YarnTaskManager.scala | 1 + .../org/apache/flink/yarn/YarnUtils.scala | 4 + .../org/apache/flink/yarn/UtilsTests.java | 33 + pom.xml | 33 +- 47 files changed, 3005 insertions(+), 1097 deletions(-) delete mode 100644 flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/Client.java delete mode 100644 flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala create mode 100644 flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/yarn/FlinkYarnClusterStatus.java create mode 100644 flink-yarn-tests/pom.xml create mode 100644 flink-yarn-tests/src/test/java/org/apache/flink/yarn/UtilsTest.java create mode 100644 flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerIT.java create mode 100644 flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOIT.java create mode 100644 flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java create mode 100644 flink-yarn-tests/src/test/resources/log4j-test.properties rename {flink-addons/flink-yarn => flink-yarn}/pom.xml (98%) create mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java create mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java rename {flink-addons/flink-yarn => flink-yarn}/src/main/java/org/apache/flink/yarn/Utils.java (78%) rename {flink-addons/flink-yarn => flink-yarn}/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java (95%) create mode 100644 flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala rename {flink-addons/flink-yarn => flink-yarn}/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala (64%) rename {flink-addons/flink-yarn => flink-yarn}/src/main/scala/org/apache/flink/yarn/Messages.scala (71%) rename {flink-addons/flink-yarn => flink-yarn}/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala (87%) rename {flink-addons/flink-yarn => flink-yarn}/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala (94%) rename {flink-addons/flink-yarn => flink-yarn}/src/main/scala/org/apache/flink/yarn/YarnUtils.scala (97%) create mode 100644 flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTests.java diff --git a/docs/cli.md b/docs/cli.md index ceab6e42edd34..8d6048aa3124c 100644 --- a/docs/cli.md +++ b/docs/cli.md @@ -33,7 +33,8 @@ started from the same installation directory. A prerequisite to using the command line interface is that the Flink master (JobManager) has been started (via `/bin/start- -local.sh` or `/bin/start-cluster.sh`). +local.sh` or `/bin/start-cluster.sh`) or that a YARN +environment is available. The command line can be used to @@ -64,6 +65,11 @@ The command line can be used to ./examples/flink-java-examples-{{ site.FLINK_VERSION_SHORT }}-WordCount.jar \ -file:///home/user/hamlet.txt file:///home/user/wordcount_out +- Run example program using a [per-job YARN cluster](yarn_setup.html#run-a-single-flink-job-on-hadoop-yarn) with 2 TaskManagers: + + ./bin/flink run -m yarn-cluster -yn 2 \ + ./examples/flink-java-examples-{{ site.FLINK_VERSION_STABLE }}-WordCount.jar \ + -file:///home/user/hamlet.txt file:///home/user/wordcount_out - Display the expected arguments for the WordCount example program: diff --git a/docs/config.md b/docs/config.md index dab6c2b0c6c6c..edaefdf74a71a 100644 --- a/docs/config.md +++ b/docs/config.md @@ -304,8 +304,7 @@ to avoid duplicate port allocations when running multiple YARN sessions in paral So if `yarn.am.rpc.port` is configured to `10245` and the session's application ID is `application_1406629969999_0002`, then the actual port being used is 10245 + 2 = 10247 -- `yarn.am.rpc.port`: The port that is being opened by the Application Master (AM) to -let the YARN client connect for an RPC serice. (DEFAULT: Port 10245) +- `yarn.heap-cutoff-ratio`: Percentage of heap space to remove from containers started by YARN. ## Background diff --git a/docs/yarn_setup.md b/docs/yarn_setup.md index 3374500bcda33..af036af1482e6 100644 --- a/docs/yarn_setup.md +++ b/docs/yarn_setup.md @@ -36,14 +36,14 @@ cd flink-yarn-{{ site.FLINK_VERSION_SHORT }}/ Specify the `-s` flag for the number of processing slots per Task Manager. We recommend to set the number of slots to the number of processors per machine. -## Introducing YARN +## Apache Flink on Hadoop YARN using a YARN Session Apache [Hadoop YARN](http://hadoop.apache.org/) is a cluster resource management framework. It allows to run various distributed applications on top of a cluster. Flink runs on YARN next to other applications. Users do not have to setup or install anything if there is already a YARN setup. **Requirements** - Apache Hadoop 2.2 -- HDFS (Hadoop Distributed File System) +- HDFS (Hadoop Distributed File System) (or another distributed file system supported by Hadoop) If you have troubles using the Flink YARN client, have a look in the [FAQ section](faq.html). @@ -80,16 +80,14 @@ This command will show you the following overview: ~~~bash Usage: Required - -n,--container Number of Yarn container to allocate (=Number of Task Managers) + -n,--container Number of YARN container to allocate (=Number of Task Managers) Optional - -D Dynamic Properties + -D Dynamic properties -jm,--jobManagerMemory Memory for JobManager Container [in MB] -q,--query Display available YARN resources (memory, cores) -qu,--queue Specify YARN queue. -s,--slots Number of slots per TaskManager -tm,--taskManagerMemory Memory per TaskManager Container [in MB] - -tmc,--taskManagerCores Virtual CPU cores per TaskManager - -v,--verbose Verbose debug mode ~~~ Please note that the Client requires the `HADOOP_HOME` (or `YARN_CONF_DIR` or `HADOOP_CONF_DIR`) environment variable to be set to read the YARN and HDFS configuration. @@ -118,7 +116,7 @@ The client has to remain open to keep the deployment running. We suggest to use 4. Use `screen -r` to resume again. -## Submit Job to Flink +### Submit Job to Flink Use the following command to submit a Flink program to the YARN cluster: @@ -173,6 +171,24 @@ You can check the number of TaskManagers in the JobManager web interface. The ad If the TaskManagers do not show up after a minute, you should investigate the issue using the log files. +## Run a single Flink job on Hadoop YARN + +The documentation above describes how to start a Flink cluster within a Hadoop YARN environment. +It is also possible to launch Flink within YARN only for executing a single job. + +To deploy a job to a per-job YARN cluster, set the master name to `yarn-cluster`. +Please note that the client then expects the `-n` value to be set (number of TaskManagers). + +***Example:*** + +~~~bash +./bin/flink run -m yarn-cluster -yn 2 ./examples/flink-java-examples-{{site.FLINK_VERSION_STABLE }}-WordCount.jar +~~~ + +The command line options of the YARN session are also available with the `./bin/flink` tool. They are prefixed with a `y` or `yarn` (for the long argument options). + + + ## Debugging a failed YARN session There are many reasons why a Flink YARN session deployment can fail. A misconfigured Hadoop setup (HDFS permissions, YARN configuration), version incompatibilities (running Flink with vanilla Hadoop dependencies on Cloudera Hadoop) or other errors. @@ -203,7 +219,7 @@ It allows to access log files for running YARN applications and shows diagnostic Users using Hadoop distributions from companies like Hortonworks, Cloudera or MapR might have to build Flink against their specific versions of Hadoop (HDFS) and YARN. Please read the [build instructions](building.html) for more details. -## Background +## Background / Internals This section briefly describes how Flink and YARN interact. diff --git a/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/Client.java b/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/Client.java deleted file mode 100644 index 1de61a81fb6d7..0000000000000 --- a/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/Client.java +++ /dev/null @@ -1,713 +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 java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.FilenameFilter; -import java.io.IOException; -import java.io.InputStream; -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.concurrent.TimeUnit; -import java.util.jar.JarFile; - -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.actor.Props; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.HelpFormatter; -import org.apache.commons.cli.MissingOptionException; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.PosixParser; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.flink.client.CliFrontend; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.GlobalConfiguration; -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.exceptions.YarnException; -import org.apache.hadoop.yarn.util.Records; -import scala.concurrent.duration.FiniteDuration; - -/** - * 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 class Client { - private static final Logger LOG = LoggerFactory.getLogger(Client.class); - - /** - * Command Line argument options - */ - private static final Option QUERY = new Option("q","query",false, "Display available YARN resources (memory, cores)"); - // --- or --- - private static final Option VERBOSE = new Option("v","verbose",false, "Verbose debug mode"); - private static final Option GEN_CONF = new Option("g","generateConf",false, "Place default configuration file in current directory"); - private static final Option QUEUE = new Option("qu","queue",true, "Specify YARN queue."); - private static final Option SHIP_PATH = new Option("t","ship",true, "Ship files in the specified directory (t for transfer)"); - private static final Option FLINK_CONF_DIR = new Option("c","confDir",true, "Path to Flink configuration directory"); - private static final Option FLINK_JAR = new Option("j","jar",true, "Path to Flink jar file"); - private static final Option JM_MEMORY = new Option("jm","jobManagerMemory",true, "Memory for JobManager Container [in MB]"); - private static final Option TM_MEMORY = new Option("tm","taskManagerMemory",true, "Memory per TaskManager Container [in MB]"); - private static final Option TM_CORES = new Option("tmc","taskManagerCores",true, "Virtual CPU cores per TaskManager"); - private static final Option CONTAINER = new Option("n","container",true, "Number of Yarn container to allocate (=Number of" - + " Task Managers)"); - private static final Option SLOTS = new Option("s","slots",true, "Number of slots per TaskManager"); - /** - * Dynamic properties allow the user to specify additional configuration values with -D, such as - * -Dfs.overwrite-files=true -Dtaskmanager.network.numberOfBuffers=16368 - */ - private static final Option DYNAMIC_PROPERTIES = new Option("D", true, "Dynamic properties"); - - /** - * Constants, - * all starting with ENV_ are used as environment variables to pass values from the Client - * to the Application Master. - */ - public final static String ENV_TM_MEMORY = "_CLIENT_TM_MEMORY"; - public final static String ENV_TM_CORES = "_CLIENT_TM_CORES"; - public final static String ENV_TM_COUNT = "_CLIENT_TM_COUNT"; - public final static String ENV_APP_ID = "_APP_ID"; - public final static String ENV_APP_NUMBER = "_APP_NUMBER"; - public final static String FLINK_JAR_PATH = "_FLINK_JAR_PATH"; // the Flink jar resource location (in HDFS). - public static final String ENV_CLIENT_HOME_DIR = "_CLIENT_HOME_DIR"; - public static final String ENV_CLIENT_SHIP_FILES = "_CLIENT_SHIP_FILES"; - public static final String ENV_CLIENT_USERNAME = "_CLIENT_USERNAME"; - public static final String ENV_SLOTS = "_SLOTS"; - public static final String ENV_DYNAMIC_PROPERTIES = "_DYNAMIC_PROPERTIES"; - - private static final String CONFIG_FILE_NAME = "flink-conf.yaml"; - - /** - * Minimum memory requirements, checked by the Client. - */ - private static final int MIN_JM_MEMORY = 128; - private static final int MIN_TM_MEMORY = 128; - - private Configuration conf; - private YarnClient yarnClient; - - private ActorSystem actorSystem; - - private ActorRef applicationClient = ActorRef.noSender(); - - private File yarnPropertiesFile; - - /** - * 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; - - public void run(String[] args) throws Exception { - - if(UserGroupInformation.isSecurityEnabled()) { - throw new RuntimeException("Flink YARN client does not have security support right now." - + "File a bug, we will fix it asap"); - } - //Utils.logFilesInCurrentDirectory(LOG); - // - // Command Line Options - // - Options options = new Options(); - options.addOption(VERBOSE); - options.addOption(FLINK_CONF_DIR); - options.addOption(FLINK_JAR); - options.addOption(JM_MEMORY); - options.addOption(TM_MEMORY); - options.addOption(TM_CORES); - options.addOption(CONTAINER); - options.addOption(GEN_CONF); - options.addOption(QUEUE); - options.addOption(QUERY); - options.addOption(SHIP_PATH); - options.addOption(SLOTS); - options.addOption(DYNAMIC_PROPERTIES); - - CommandLineParser parser = new PosixParser(); - CommandLine cmd = null; - try { - cmd = parser.parse( options, args); - } catch(MissingOptionException moe) { - System.out.println(moe.getMessage()); - printUsage(); - System.exit(1); - } - - // 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 { - localJarPath = new Path("file://"+Client.class.getProtectionDomain().getCodeSource().getLocation().getPath()); - } - - if(cmd.hasOption(GEN_CONF.getOpt())) { - LOG.info("Placing default configuration in current directory"); - File outFile = generateDefaultConf(localJarPath); - LOG.info("File written to "+outFile.getAbsolutePath()); - System.exit(0); - } - - // Conf Path - Path confPath = null; - String confDirPath = ""; - if(cmd.hasOption(FLINK_CONF_DIR.getOpt())) { - confDirPath = cmd.getOptionValue(FLINK_CONF_DIR.getOpt())+"/"; - File confFile = new File(confDirPath+CONFIG_FILE_NAME); - if(!confFile.exists()) { - LOG.error("Unable to locate configuration file in "+confFile); - System.exit(1); - } - confPath = new Path(confFile.getAbsolutePath()); - } else { - System.out.println("No configuration file has been specified"); - - // no configuration path given. - // -> see if there is one in the current directory - File currDir = new File("."); - File[] candidates = currDir.listFiles(new FilenameFilter() { - @Override - public boolean accept(final File dir, final String name) { - return name != null && name.endsWith(".yaml"); - } - }); - if(candidates == null || candidates.length == 0) { - System.out.println("No configuration file has been found in current directory.\n" - + "Copying default."); - File outFile = generateDefaultConf(localJarPath); - confPath = new Path(outFile.toURI()); - } else { - if(candidates.length > 1) { - System.out.println("Multiple .yaml configuration files were found in the current directory\n" - + "Please specify one explicitly"); - System.exit(1); - } else if(candidates.length == 1) { - confPath = new Path(candidates[0].toURI()); - } - } - } - List 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!"); - } - } - boolean hasLogback = false; - boolean hasLog4j = false; - //check if there is a logback or log4j file - if(confDirPath.length() > 0) { - File logback = new File(confDirPath+"/logback.xml"); - if(logback.exists()) { - shipFiles.add(logback); - hasLogback = true; - } - File log4j = new File(confDirPath+"/log4j.properties"); - if(log4j.exists()) { - shipFiles.add(log4j); - hasLog4j = true; - } - } - - // queue - String queue = "default"; - if(cmd.hasOption(QUEUE.getOpt())) { - queue = cmd.getOptionValue(QUEUE.getOpt()); - } - - // JobManager Memory - int jmMemory = 512; - if(cmd.hasOption(JM_MEMORY.getOpt())) { - jmMemory = Integer.valueOf(cmd.getOptionValue(JM_MEMORY.getOpt())); - } - if(jmMemory < MIN_JM_MEMORY) { - System.out.println("The JobManager memory is below the minimum required memory amount " - + "of "+MIN_JM_MEMORY+" MB"); - System.exit(1); - } - // Task Managers memory - int tmMemory = 1024; - if(cmd.hasOption(TM_MEMORY.getOpt())) { - tmMemory = Integer.valueOf(cmd.getOptionValue(TM_MEMORY.getOpt())); - } - if(tmMemory < MIN_TM_MEMORY) { - System.out.println("The TaskManager memory is below the minimum required memory amount " - + "of "+MIN_TM_MEMORY+" MB"); - System.exit(1); - } - - if(cmd.hasOption(SLOTS.getOpt())) { - slots = Integer.valueOf(cmd.getOptionValue(SLOTS.getOpt())); - } - - 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); - - // Task Managers vcores - int tmCores = 1; - if(cmd.hasOption(TM_CORES.getOpt())) { - tmCores = Integer.valueOf(cmd.getOptionValue(TM_CORES.getOpt())); - } - Utils.getFlinkConfiguration(confPath.toUri().getPath()); - int jmPort = GlobalConfiguration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0); - if(jmPort == 0) { - LOG.warn("Unable to find job manager port in configuration!"); - jmPort = ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT; - } - FiniteDuration timeout = new FiniteDuration(GlobalConfiguration.getInteger - (ConfigConstants.AKKA_ASK_TIMEOUT, ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT), - TimeUnit.SECONDS); - - conf = Utils.initializeYarnConfiguration(); - - // intialize HDFS - LOG.info("Copy App Master jar from local filesystem and add to local environment"); - // 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 sytem is using the default Hadoop configuration values." - + "The Flink YARN client needs to store its files in a distributed file system"); - } - - // Create yarnClient - yarnClient = YarnClient.createYarnClient(); - yarnClient.init(conf); - yarnClient.start(); - - // Query cluster for metrics - if(cmd.hasOption(QUERY.getOpt())) { - showClusterMetrics(yarnClient); - } - if(!cmd.hasOption(CONTAINER.getOpt())) { - LOG.error("Missing required argument "+CONTAINER.getOpt()); - printUsage(); - yarnClient.stop(); - System.exit(1); - } - - // TM Count - final int taskManagerCount = Integer.valueOf(cmd.getOptionValue(CONTAINER.getOpt())); - - System.out.println("Using values:"); - System.out.println("\tContainer Count = "+taskManagerCount); - System.out.println("\tJar Path = "+localJarPath.toUri().getPath()); - System.out.println("\tConfiguration file = "+confPath.toUri().getPath()); - System.out.println("\tJobManager memory = "+jmMemory); - System.out.println("\tTaskManager memory = "+tmMemory); - System.out.println("\tTaskManager cores = "+tmCores); - - // Create application via yarnClient - YarnClientApplication app = yarnClient.createApplication(); - GetNewApplicationResponse appResponse = app.getNewApplicationResponse(); - Resource maxRes = appResponse.getMaximumResourceCapability(); - if(tmMemory > maxRes.getMemory() || tmCores > maxRes.getVirtualCores()) { - LOG.error("The cluster does not have the requested resources for the TaskManagers available!\n" - + "Maximum Memory: "+maxRes.getMemory() +", Maximum Cores: "+tmCores); - yarnClient.stop(); - System.exit(1); - } - if(jmMemory > maxRes.getMemory() ) { - LOG.error("The cluster does not have the requested resources for the JobManager available!\n" - + "Maximum Memory: "+maxRes.getMemory()); - yarnClient.stop(); - System.exit(1); - } - int totalMemoryRequired = jmMemory + tmMemory * taskManagerCount; - ClusterResourceDescription freeClusterMem = getCurrentFreeClusterResources(yarnClient); - if(freeClusterMem.totalFreeMemory < totalMemoryRequired) { - LOG.error("This YARN session requires "+totalMemoryRequired+"MB of memory in the cluster. " - + "There are currently only "+freeClusterMem.totalFreeMemory+"MB available."); - yarnClient.stop(); - System.exit(1); - } - if( tmMemory > freeClusterMem.containerLimit) { - LOG.error("The requested amount of memory for the TaskManagers ("+tmMemory+"MB) is more than " - + "the largest possible YARN container: "+freeClusterMem.containerLimit); - yarnClient.stop(); - System.exit(1); - } - if( jmMemory > freeClusterMem.containerLimit) { - LOG.error("The requested amount of memory for the JobManager ("+jmMemory+"MB) is more than " - + "the largest possible YARN container: "+freeClusterMem.containerLimit); - yarnClient.stop(); - System.exit(1); - } - - // respect custom JVM options in the YAML file - final String javaOpts = GlobalConfiguration.getString(ConfigConstants.FLINK_JVM_OPTIONS, ""); - - // 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(jmMemory)+"M " +javaOpts; - if(hasLogback || hasLog4j) { - amCommand += " -Dlog.file=\""+ApplicationConstants.LOG_DIR_EXPANSION_VAR +"/jobmanager-main.log\""; - } - if(hasLogback) { - amCommand += " -Dlogback.configurationFile=file:logback.xml"; - } - if(hasLog4j) { - amCommand += " -Dlog4j.configuration=file:log4j.properties"; - } - - amCommand += " "+ApplicationMaster.class.getName()+" " - + " 1>" - + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager-stdout.log" - + " 2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager-stderr.log"; - amContainer.setCommands(Collections.singletonList(amCommand)); - - System.err.println("amCommand="+amCommand); - - // Set-up ApplicationSubmissionContext for the application - ApplicationSubmissionContext appContext = app.getApplicationSubmissionContext(); - final ApplicationId appId = appContext.getApplicationId(); - /** - * All network ports are offsetted by the application number - * to avoid version port clashes when running multiple Flink sessions - * in parallel - */ - int appNumber = appId.getId(); - - jmPort = Utils.offsetPort(jmPort, appNumber); - - // Setup jar for ApplicationMaster - LocalResource appMasterJar = Records.newRecord(LocalResource.class); - LocalResource flinkConf = Records.newRecord(LocalResource.class); - Path remotePathJar = Utils.setupLocalResource(conf, fs, appId.toString(), localJarPath, appMasterJar, fs.getHomeDirectory()); - Path remotePathConf = Utils.setupLocalResource(conf, fs, appId.toString(), confPath, flinkConf, fs.getHomeDirectory()); - Map 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[3 + shipFiles.size()]; - StringBuffer envShipFileList = new StringBuffer(); - // 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[3 + i] = Utils.setupLocalResource(conf, fs, appId.toString(), - shipLocalPath, shipResources, fs.getHomeDirectory()); - localResources.put(shipFile.getName(), shipResources); - - envShipFileList.append(paths[3 + 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.ALL, FsAction.ALL); - fs.setPermission(sessionFilesDir, permission); // set permission for path. - Utils.setTokensFor(amContainer, paths, this.conf); - - - amContainer.setLocalResources(localResources); - fs.close(); - - // Setup CLASSPATH for ApplicationMaster - Map appMasterEnv = new HashMap(); - Utils.setupEnv(conf, appMasterEnv); - // set configuration values - appMasterEnv.put(Client.ENV_TM_COUNT, String.valueOf(taskManagerCount)); - appMasterEnv.put(Client.ENV_TM_CORES, String.valueOf(tmCores)); - appMasterEnv.put(Client.ENV_TM_MEMORY, String.valueOf(tmMemory)); - appMasterEnv.put(Client.FLINK_JAR_PATH, remotePathJar.toString() ); - appMasterEnv.put(Client.ENV_APP_ID, appId.toString()); - appMasterEnv.put(Client.ENV_CLIENT_HOME_DIR, fs.getHomeDirectory().toString()); - appMasterEnv.put(Client.ENV_CLIENT_SHIP_FILES, envShipFileList.toString() ); - appMasterEnv.put(Client.ENV_CLIENT_USERNAME, UserGroupInformation.getCurrentUser().getShortUserName()); - appMasterEnv.put(Client.ENV_SLOTS, String.valueOf(slots)); - appMasterEnv.put(Client.ENV_APP_NUMBER, String.valueOf(appNumber)); - if(dynamicPropertiesEncoded != null) { - appMasterEnv.put(Client.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded); - } - - amContainer.setEnvironment(appMasterEnv); - - // Set up resource type requirements for ApplicationMaster - Resource capability = Records.newRecord(Resource.class); - capability.setMemory(jmMemory); - capability.setVirtualCores(1); - - appContext.setApplicationName("Flink"); // application name - appContext.setAMContainerSpec(amContainer); - appContext.setResource(capability); - appContext.setQueue(queue); - - // file that we write into the conf/ dir containing the jobManager address and the dop. - yarnPropertiesFile = new File(confDirPath + CliFrontend.YARN_PROPERTIES_FILE); - - LOG.info("Submitting application master " + appId); - yarnClient.submitApplication(appContext); - - Runtime.getRuntime().addShutdownHook(new ClientShutdownHook()); - - // start actor system - LOG.info("Start actor system."); - actorSystem = YarnUtils.createActorSystem(); - - // start application client - LOG.info("Start application client."); - applicationClient = actorSystem.actorOf(Props.create(ApplicationClient.class, appId, jmPort, - yarnClient, confDirPath, slots, taskManagerCount, dynamicPropertiesEncoded, - timeout)); - - actorSystem.awaitTermination(); - - actorSystem = null; - - 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 retreive " - + "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)"); - } - } - - private void stopSession() { - if(actorSystem != null){ - LOG.info("Sending shutdown request to the Application Master"); - if(applicationClient != ActorRef.noSender()) { - applicationClient.tell(new Messages.StopYarnSession(FinalApplicationStatus.KILLED), - ActorRef.noSender()); - applicationClient = ActorRef.noSender(); - } - - actorSystem.shutdown(); - actorSystem.awaitTermination(); - - actorSystem = null; - } - - try { - FileSystem shutFS = FileSystem.get(conf); - shutFS.delete(sessionFilesDir, true); // delete conf and jar file. - shutFS.close(); - }catch(IOException e){ - LOG.error("Could not delete the conf and jar files.", e); - } - - try { - yarnPropertiesFile.delete(); - } catch (Exception e) { - LOG.warn("Exception while deleting the JobManager address file", e); - } - LOG.info("YARN Client is shutting down"); - yarnClient.stop(); - - LOG.info("Deleting files in "+sessionFilesDir ); - } - - public class ClientShutdownHook extends Thread { - @Override - public void run() { - stopSession(); - } - } - - private static class ClusterResourceDescription { - public int totalFreeMemory; - public int containerLimit; - } - - private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) throws YarnException, IOException { - ClusterResourceDescription crd = new ClusterResourceDescription(); - crd.totalFreeMemory = 0; - crd.containerLimit = 0; - List nodes = yarnClient.getNodeReports(NodeState.RUNNING); - for(NodeReport rep : nodes) { - int free = rep.getCapability().getMemory() - (rep.getUsed() != null ? rep.getUsed().getMemory() : 0 ); - crd.totalFreeMemory += free; - if(free > crd.containerLimit) { - crd.containerLimit = free; - } - } - return crd; - } - - 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(VERBOSE); - opt.addOption(JM_MEMORY); - opt.addOption(TM_MEMORY); - opt.addOption(TM_CORES); - opt.addOption(QUERY); - opt.addOption(QUEUE); - opt.addOption(SLOTS); - opt.addOption(DYNAMIC_PROPERTIES); - formatter.printHelp(" ", opt); - } - - private void showClusterMetrics(YarnClient yarnClient) - throws YarnException, IOException { - YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics(); - System.out.println("NodeManagers in the Cluster " + metrics.getNumNodeManagers()); - List nodes = yarnClient.getNodeReports(NodeState.RUNNING); - final String format = "|%-16s |%-16s %n"; - System.out.printf("|Property |Value %n"); - System.out.println("+---------------------------------------+"); - int totalMemory = 0; - int totalCores = 0; - for(NodeReport rep : nodes) { - final Resource res = rep.getCapability(); - totalMemory += res.getMemory(); - totalCores += res.getVirtualCores(); - System.out.format(format, "NodeID", rep.getNodeId()); - System.out.format(format, "Memory", res.getMemory()+" MB"); - System.out.format(format, "vCores", res.getVirtualCores()); - System.out.format(format, "HealthReport", rep.getHealthReport()); - System.out.format(format, "Containers", rep.getNumContainers()); - System.out.println("+---------------------------------------+"); - } - System.out.println("Summary: totalMemory "+totalMemory+" totalCores "+totalCores); - List qInfo = yarnClient.getAllQueues(); - for(QueueInfo q : qInfo) { - System.out.println("Queue: "+q.getQueueName()+", Current Capacity: "+q.getCurrentCapacity()+" Max Capacity: "+q.getMaximumCapacity()+" Applications: "+q.getApplications().size()); - } - yarnClient.stop(); - System.exit(0); - } - - private File generateDefaultConf(Path localJarPath) throws IOException, - FileNotFoundException { - JarFile jar = null; - try { - jar = new JarFile(localJarPath.toUri().getPath()); - } catch(FileNotFoundException fne) { - LOG.error("Unable to access jar file. Specify jar file or configuration file.", fne); - System.exit(1); - } - InputStream confStream = jar.getInputStream(jar.getEntry("flink-conf.yaml")); - - if(confStream == null) { - LOG.warn("Given jar file does not contain yaml conf."); - confStream = this.getClass().getResourceAsStream("flink-conf.yaml"); - if(confStream == null) { - throw new RuntimeException("Unable to find flink-conf in jar file"); - } - } - File outFile = new File("flink-conf.yaml"); - if(outFile.exists()) { - throw new RuntimeException("File unexpectedly exists"); - } - FileOutputStream outputStream = new FileOutputStream(outFile); - int read = 0; - byte[] bytes = new byte[1024]; - while ((read = confStream.read(bytes)) != -1) { - outputStream.write(bytes, 0, read); - } - confStream.close(); outputStream.close(); jar.close(); - return outFile; - } - - public static void main(String[] args) throws Exception { - Client c = new Client(); - c.run(args); - } -} diff --git a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala b/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala deleted file mode 100644 index 4a6e8cbbb16b6..0000000000000 --- a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala +++ /dev/null @@ -1,158 +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 java.io.{File, FileOutputStream} -import java.util.Properties - -import akka.actor._ -import akka.camel.{Consumer, CamelMessage} -import org.apache.flink.client.CliFrontend -import org.apache.flink.runtime.ActorLogMessages -import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.jobmanager.JobManager -import org.apache.flink.yarn.Messages._ -import org.apache.hadoop.yarn.api.records.{FinalApplicationStatus, YarnApplicationState, -ApplicationId} -import org.apache.hadoop.yarn.client.api.YarnClient -import scala.concurrent.duration._ - -class ApplicationClient(appId: ApplicationId, port: Int, yarnClient: YarnClient, - confDirPath: String, slots: Int, numTaskManagers: Int, - dynamicPropertiesEncoded: String, timeout: FiniteDuration) - extends Actor with Consumer with ActorLogMessages with ActorLogging { - import context._ - - val INITIAL_POLLING_DELAY = 0 seconds - val WAIT_FOR_YARN_INTERVAL = 500 milliseconds - val POLLING_INTERVAL = 3 seconds - - val waitingChars = Array[Char]('/', '|', '\\', '-') - - var jobManager: Option[ActorRef] = None - var pollingTimer: Option[Cancellable] = None - var running = false - var waitingCharsIndex = 0 - - def endpointUri = "stream:in" - - override def preStart(): Unit = { - super.preStart() - pollingTimer = Some(context.system.scheduler.schedule(INITIAL_POLLING_DELAY, - WAIT_FOR_YARN_INTERVAL, self, PollYarnReport)) - } - - override def postStop(): Unit = { - log.info("Stopped Application client.") - pollingTimer foreach { - _.cancel() - } - - pollingTimer = None - } - - override def receiveWithLogMessages: Receive = { - case PollYarnReport => { - val report = yarnClient.getApplicationReport(appId) - - report.getYarnApplicationState match { - case YarnApplicationState.FINISHED | YarnApplicationState.KILLED | YarnApplicationState - .FAILED => { - log.info(s"Terminate polling.") - - context.system.shutdown() - } - case YarnApplicationState.RUNNING if !running => { - val address = s"${report.getHost}:$port" - log.info(s"Flink JobManager is now running on $address") - log.info(s"JobManager Web Interface: ${report.getTrackingUrl}") - - writeYarnProperties(address) - - jobManager = Some(AkkaUtils.getReference(JobManager.getRemoteAkkaURL(address))(system, - timeout)) - jobManager.get ! RegisterMessageListener - - pollingTimer foreach { - _.cancel() - } - - pollingTimer = Some(context.system.scheduler.schedule(INITIAL_POLLING_DELAY, - POLLING_INTERVAL, self, PollYarnReport)) - - running = true - } - case _ => - } - - if(!running){ - print(waitingChars(waitingCharsIndex) + "\r") - waitingCharsIndex += 1 - - if(waitingCharsIndex >= waitingChars.length){ - waitingCharsIndex = 0 - } - } - } - case msg: YarnMessage => { - println(msg) - } - case msg: StopYarnSession => { - log.info("Stop yarn session.") - jobManager foreach { - _ forward msg - } - } - case msg: CamelMessage => { - msg.bodyAs[String] match { - case "stop" | "quit" | "exit" => self ! StopYarnSession(FinalApplicationStatus.KILLED) - case "help" => printHelp - case msg => println(s"Unknown command ${msg}.") - } - } - } - - def printHelp: Unit = { - println( - """Available commands: - |stop : Stop the YARN session - """.stripMargin) - } - - def writeYarnProperties(address: String): Unit = { - val yarnProps = new Properties() - yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_JOBMANAGER_KEY, address) - - if(slots > 0){ - yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_DOP, (slots * numTaskManagers).toString ) - } - - if(dynamicPropertiesEncoded != null){ - yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING, - dynamicPropertiesEncoded) - } - - val yarnPropertiesFile = new File(confDirPath + CliFrontend.YARN_PROPERTIES_FILE) - - val out = new FileOutputStream(yarnPropertiesFile) - yarnProps.store(out, "Generated YARN properties file") - out.close() - yarnPropertiesFile.setReadable(true, false) - } -} diff --git a/flink-addons/pom.xml b/flink-addons/pom.xml index a20a37589630f..ea45cdb10d678 100644 --- a/flink-addons/pom.xml +++ b/flink-addons/pom.xml @@ -59,19 +59,6 @@ under the License. flink-tachyon - - - include-yarn - - - - !hadoop.profile - - - - flink-yarn - - diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml index 85506b2aa398d..1a96c9c5330dd 100644 --- a/flink-clients/pom.xml +++ b/flink-clients/pom.xml @@ -36,11 +36,6 @@ under the License. jar - - commons-cli - commons-cli - - org.apache.flink flink-core @@ -108,6 +103,11 @@ under the License. compile + + commons-cli + commons-cli + + com.typesafe.akka akka-testkit_2.10 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 8092513687840..358783a053c33 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 @@ -56,6 +56,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobStatus; @@ -63,7 +64,12 @@ import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; import org.apache.flink.runtime.messages.JobManagerMessages.RequestRunningJobs$; import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobs; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; +import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus; import org.apache.flink.util.StringUtils; +import org.apache.log4j.ConsoleAppender; +import org.apache.log4j.LogManager; +import org.apache.log4j.PatternLayout; import scala.concurrent.duration.FiniteDuration; /** @@ -71,34 +77,40 @@ */ public class CliFrontend { + // run 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"; + + // command line interface of the YARN session, with a special initialization here to prefix all options with y/yarn. + private static FlinkYarnSessionCli yarnSessionCLi = new FlinkYarnSessionCli("y", "yarn"); + //actions 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"; - + // general options private static final Option HELP_OPTION = new Option("h", "help", false, "Show the help for the CLI Frontend."); private static final Option VERBOSE_OPTION = new Option("v", "verbose", false, "Print more detailed error messages."); - + // program (jar file) specific options private static final Option JAR_OPTION = new Option("j", "jarfile", true, "Flink program JAR file."); private 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."); private 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."); private static final Option ARGS_OPTION = new Option("a", "arguments", true, "Program arguments. Arguments can also be added without -a, simply as trailing parameters."); - - private static final Option ADDRESS_OPTION = new Option("m", "jobmanager", true, "Address of the JobManager (master) to which to connect. Use this flag to connect to a different JobManager than the one specified in the configuration."); - + + private static final Option ADDRESS_OPTION = new Option("m", "jobmanager", true, "Address of the JobManager (master) to which to connect. Specify '"+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."); + // info specific options private static final Option PLAN_OPTION = new Option("e", "executionplan", false, "Show optimized execution plan of the program (JSON)"); - + // list specific options private static final Option RUNNING_OPTION = new Option("r", "running", false, "Show running programs and their JobIDs"); private static final Option SCHEDULED_OPTION = new Option("s", "scheduled", false, "Show scheduled prorgrams and their JobIDs"); - + // canceling private static final Option ID_OPTION = new Option("i", "jobid", true, "JobID of program to cancel"); - + static { initOptions(); } @@ -126,6 +138,8 @@ public class CliFrontend { public static final String YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING = "dynamicPropertiesString"; // this has to be a regex for String.split() public static final String YARN_DYNAMIC_PROPERTIES_SEPARATOR = "@@"; + private static final String DEFAULT_LOG4J_PATTERN_LAYOUT = "%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n"; + private CommandLineParser parser; @@ -139,6 +153,15 @@ public class CliFrontend { private Properties yarnProperties; + // this flag indicates if the given Job is executed using a YARN cluster, + // started for this purpose. + private boolean runInYarnCluster = false; + + private AbstractFlinkYarnCluster yarnCluster = null; + + protected String configurationDirectory = null; + + /** * Initializes the class */ @@ -193,6 +216,9 @@ static Options getProgramSpecificOptions(Options options) { options.addOption(CLASS_OPTION); options.addOption(PARALLELISM_OPTION); options.addOption(ARGS_OPTION); + + // also add the YARN options so that the parser can parse them + yarnSessionCLi.getYARNSessionCLIOptions(options); return options; } @@ -309,7 +335,7 @@ protected int run(String[] args) { return 1; } - Client client = getClient(line, program.getUserCodeClassLoader()); + Client client = getClient(line, program.getUserCodeClassLoader(), program.getMainClassName()); if (client == null) { printHelpForRun(); return 1; @@ -332,8 +358,25 @@ protected int run(String[] args) { return 1; } } - - return executeProgram(program, client, parallelism); + int programResult = executeProgram(program, client, parallelism); + // check if the program has been executed in a "job only" YARN cluster. + if(runInYarnCluster) { + List msgs = yarnCluster.getNewMessages(); + if(msgs != null && msgs.size() > 1) { + System.out.println("The following messages were created by the YARN cluster while running the Job:"); + for(String msg : msgs) { + System.out.println(msg); + } + } + if(yarnCluster.hasFailed()) { + System.out.println("YARN cluster is in failed state!"); + System.out.println("YARN Diagnostics: " + yarnCluster.getDiagnostics()); + } + System.out.println("Shutting down YARN cluster"); + yarnCluster.shutdown(); + } + + return programResult; } catch (Throwable t) { return handleError(t); @@ -443,7 +486,7 @@ protected int info(String[] args) { try { // check for json plan request if (plan) { - Client client = getClient(line, program.getUserCodeClassLoader()); + Client client = getClient(line, program.getUserCodeClassLoader(), program.getMainClassName()); String jsonPlan = client.getOptimizedPlanAsJson(program, parallelism); if (jsonPlan != null) { @@ -693,14 +736,13 @@ else if (!jarFile.isFile()) { } } - protected InetSocketAddress getJobManagerAddress(CommandLine line) throws IOException { + protected String getJobManagerAddressString(CommandLine line) throws IOException { Configuration configuration = getGlobalConfiguration(); // first, check if the address comes from the command line option if (line.hasOption(ADDRESS_OPTION.getOpt())) { try { - String address = line.getOptionValue(ADDRESS_OPTION.getOpt()); - return RemoteExecutor.getInetFromHostport(address); + return line.getOptionValue(ADDRESS_OPTION.getOpt()); } catch (Exception e) { System.out.println("Error: The JobManager address has an invalid format. " + e.getMessage()); @@ -714,9 +756,9 @@ protected InetSocketAddress getJobManagerAddress(CommandLine line) throws IOExce String address = yarnProps.getProperty(YARN_PROPERTIES_JOBMANAGER_KEY); System.out.println("Found a yarn properties file (" + YARN_PROPERTIES_FILE + ") file, " + "using \""+address+"\" to connect to the JobManager"); - return RemoteExecutor.getInetFromHostport(address); + return address; } catch (Exception e) { - System.out.println("Found a yarn properties " + YARN_PROPERTIES_FILE + " file, but could not read the JobManager address from the file. " + System.out.println("Found a yarn properties " + YARN_PROPERTIES_FILE + " file, but could not read the JobManager address from the file. " + e.getMessage()); return null; } @@ -726,7 +768,7 @@ protected InetSocketAddress getJobManagerAddress(CommandLine line) throws IOExce // verify that there is a jobmanager address and port in the configuration if (jobManagerAddress == null) { - System.out.println("Error: Found no configuration in the config directory '" + + System.out.println("Error: Found no configuration in the config directory '" + getConfigurationDirectory() + "' that specifies the JobManager address."); return null; } @@ -741,29 +783,37 @@ protected InetSocketAddress getJobManagerAddress(CommandLine line) throws IOExce } if (jobManagerPort == -1) { - System.out.println("Error: Found no configuration in the config directory '" + + System.out.println("Error: Found no configuration in the config directory '" + getConfigurationDirectory() + "' that specifies the JobManager port."); return null; } - return new InetSocketAddress(jobManagerAddress, jobManagerPort); + return jobManagerAddress + ":" + jobManagerPort; } } } protected ActorRef getJobManager(CommandLine line) throws IOException { - InetSocketAddress jobManagerAddress = getJobManagerAddress(line); - if (jobManagerAddress == null) { + //TODO: Get ActorRef from YarnCluster if we are in YARN mode. + String jobManagerAddressStr = getJobManagerAddressString(line); + if (jobManagerAddressStr == null) { return null; } - return JobManager.getJobManager(jobManagerAddress, + return JobManager.getJobManager(RemoteExecutor.getInetFromHostport(jobManagerAddressStr), ActorSystem.create("CliFrontendActorSystem", AkkaUtils .getDefaultActorSystemConfig()),getAkkaTimeout()); } - - protected String getConfigurationDirectory() { + + public String getConfigurationDirectory() { + if(configurationDirectory == null) { + configurationDirectory = getConfigurationDirectoryFromEnv(); + } + return configurationDirectory; + } + + public static String getConfigurationDirectoryFromEnv() { String location = null; if (System.getenv(ENV_CONFIG_DIRECTORY) != null) { location = System.getenv(ENV_CONFIG_DIRECTORY); @@ -860,8 +910,50 @@ protected Properties getYarnProperties() throws IOException { return yarnProperties; } - protected Client getClient(CommandLine line, ClassLoader classLoader) throws IOException { - return new Client(getJobManagerAddress(line), getGlobalConfiguration(), classLoader); + protected Client getClient(CommandLine line, ClassLoader classLoader, String programName) throws IOException { + String jmAddrString = getJobManagerAddressString(line); + InetSocketAddress jobManagerAddress = null; + if(jmAddrString.equals(YARN_DEPLOY_JOBMANAGER)) { + System.out.println("YARN cluster mode detected. Switching Log4j output to console"); + LogManager.getRootLogger().addAppender(new ConsoleAppender(new PatternLayout(DEFAULT_LOG4J_PATTERN_LAYOUT))); + + this.runInYarnCluster = true; + // user wants to run Flink in YARN cluster. + AbstractFlinkYarnClient flinkYarnClient = yarnSessionCLi.createFlinkYarnClient(line); + if(flinkYarnClient == null) { + throw new RuntimeException("Unable to create Flink YARN Client. Check previous log messages"); + } + try { + yarnCluster = flinkYarnClient.deploy("Flink Application: "+programName); + } catch(Exception e) { + throw new RuntimeException("Error deploying the YARN cluster", e); + } + jobManagerAddress = yarnCluster.getJobManagerAddress(); + System.out.println("YARN cluster started"); + System.out.println("JobManager web interface address "+yarnCluster.getWebInterfaceURL()); + System.out.println("Waiting until all TaskManagers have connected"); + while(true) { + FlinkYarnClusterStatus status = yarnCluster.getClusterStatus(); + if(status != null) { + if (status.getNumberOfTaskManagers() < flinkYarnClient.getTaskManagerCount()) { + System.out.println("TaskManager status (" + status.getNumberOfTaskManagers()+"/"+flinkYarnClient.getTaskManagerCount()+")"); + } else { + System.out.println("Enough TaskManagers are connected"); + break; + } + } else { + System.out.println("No status updates from YARN cluster received so far. Waiting ..."); + } + try { + Thread.sleep(500); + } catch (InterruptedException e) { + System.err.println("Thread as interrupted"); Thread.currentThread().interrupt(); + } + } + } else { + jobManagerAddress = RemoteExecutor.getInetFromHostport(jmAddrString); + } + return new Client(jobManagerAddress, getGlobalConfiguration(), classLoader); } /** @@ -891,6 +983,10 @@ private void printHelpForRun() { System.out.println("\n Syntax: run [OPTIONS] "); formatter.setSyntaxPrefix(" \"run\" action arguments:"); formatter.printHelp(" ", getRunOptionsWithoutDeprecatedOptions(new Options())); + formatter.setSyntaxPrefix(" additional arguments if -m "+YARN_DEPLOY_JOBMANAGER+" is set:"); + Options yarnOpts = new Options(); + yarnSessionCLi.getYARNSessionCLIOptions(yarnOpts); + formatter.printHelp(" ", yarnOpts); } private void printHelpForInfo() { @@ -990,14 +1086,15 @@ public int parseParameters(String[] args) { } } - /** * Submits the job based on the arguments */ public static void main(String[] args) throws ParseException { + CliFrontend cli = new CliFrontend(); int retCode = cli.parseParameters(args); System.exit(retCode); } + } 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 new file mode 100644 index 0000000000000..6546ef0ac7759 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java @@ -0,0 +1,429 @@ +/* + * 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.GlobalConfiguration; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; +import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus; +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; + + /** + * 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 AbstractFlinkYarnCluster yarnCluster = null; + + public FlinkYarnSessionCli(String shortPrefix, String longPrefix) { + 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"); + } + + 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()); + if(!localJarPath.toString().contains("uberjar")) { + // we need to have a proper uberjar because otherwise we don't have the required classes available on the cluster. + // most likely the user did try to start yarn in a regular hadoop2 flink build (not a yarn package) (using ./bin/flink -m yarn-cluster) + LOG.error("The detected jar file '"+localJarPath+"' is not a uberjar."); + return null; + } + } + + flinkYarnClient.setLocalJarPath(localJarPath); + + // Conf Path + String confDirPath = CliFrontend.getConfigurationDirectoryFromEnv(); + GlobalConfiguration.loadConfiguration(confDirPath); + 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 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.setConfigurationFilePath(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.error("The configuration directory ('"+confDirPath+"') contains both LOG4J and Logback configuration files." + + "Please delete or rename one of them."); + return null; + } // else + flinkYarnClient.setConfigurationFilePath(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); + + 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); + formatter.printHelp(" ", opt); + } + + public static AbstractFlinkYarnClient getFlinkYarnClient() { + AbstractFlinkYarnClient yarnClient = null; + try { + Class yarnClientClass = (Class) Class.forName("org.apache.flink.yarn.FlinkYarnClient"); + 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) { + 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)); + while (true) { + // ------------------ check if there are updates by the cluster ----------- + + FlinkYarnClusterStatus status = yarnCluster.getClusterStatus(); + if(status != null && numTaskmanagers != status.getNumberOfTaskManagers()) { + System.err.println("Number of connected TaskManagers changed to "+status.getNumberOfTaskManagers()+". " + + "Slots available: "+status.getNumberOfSlots()); + numTaskmanagers = status.getNumberOfTaskManagers(); + } + + List 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"); + } + + // wait until CLIENT_POLLING_INTERVALL is over or the user entered something. + long startTime = System.currentTimeMillis(); + while ((System.currentTimeMillis() - startTime) < CLIENT_POLLING_INTERVALL * 1000 + && !in.ready()) { + Thread.sleep(200); + } + //------------- handle interactive command by user. ---------------------- + + if (in.ready()) { + String command = in.readLine(); + if(command.equals("quit") || command.equals("stop")) { + break; // leave loop, cli will stop cluster. + } else if(command.equals("help")) { + System.err.println(HELP); + } else { + System.err.println("Unknown command '"+command+"'. Showing help: \n"+HELP); + } + } + 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); + return; + } + } + + public static void main(String[] args) { + FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", ""); // 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); + } + + public int run(String[] args) { + + // + // Command Line Options + // + Options options = new Options(); + getYARNSessionCLIOptions(options); + + CommandLineParser parser = new PosixParser(); + CommandLine cmd = null; + 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 = null; + 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(null); + } 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().getHostName() + ":" +yarnCluster.getJobManagerAddress().getPort(); + System.err.println("Flink JobManager is now running on " + jobManagerAddress); + System.err.println("JobManager Web Interface: " + yarnCluster.getWebInterfaceURL()); + // file that we write into the conf/ dir containing the jobManager address and the dop. + String confDirPath = CliFrontend.getConfigurationDirectoryFromEnv(); + File yarnPropertiesFile = new File(confDirPath + File.separator + CliFrontend.YARN_PROPERTIES_FILE); + + Properties yarnProps = new Properties(); + yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_JOBMANAGER_KEY, jobManagerAddress); + if(flinkYarnClient.getTaskManagerSlots() != -1) { + yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_DOP, Integer.toString(flinkYarnClient.getTaskManagerSlots() * flinkYarnClient.getTaskManagerCount()) ); + } + // 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 ------------ + + runInteractiveCli(yarnCluster); + + 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(); + } + } +} + 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 index 00fba957953a4..d8f1bf784fefe 100644 --- 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 @@ -317,7 +317,6 @@ public JobExecutionResult run(JobGraph jobGraph, boolean wait) throws ProgramInv } try { - if (wait) { return JobClient.submitJobAndWait(jobGraph, printStatusDuringExecution, client, timeout); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java index b6d4542717473..1bc533f1418ed 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java @@ -113,7 +113,7 @@ public InfoTestCliFrontend(int expectedDop) { } @Override - protected Client getClient(CommandLine line, ClassLoader loader) throws IOException { + protected Client getClient(CommandLine line, ClassLoader loader, String programName) throws IOException { try { return new TestClient(expectedDop); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendJobManagerConnectionTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendJobManagerConnectionTest.java index 6a59019516a7a..ef7dff625ad71 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendJobManagerConnectionTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendJobManagerConnectionTest.java @@ -29,7 +29,6 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.flink.client.CliFrontend; import org.apache.flink.client.CliFrontendTestUtils.TestingCliFrontend; import org.junit.Before; import org.junit.BeforeClass; @@ -55,7 +54,7 @@ public void testInvalidConfig() { TestingCliFrontend frontend = new TestingCliFrontend(CliFrontendTestUtils.getInvalidConfigDir()); - assertTrue(frontend.getJobManagerAddress(line) == null); + assertTrue(frontend.getJobManagerAddressString(line) == null); } catch (Exception e) { System.err.println(e.getMessage()); @@ -72,7 +71,7 @@ public void testValidConfig() { TestingCliFrontend frontend = new TestingCliFrontend(CliFrontendTestUtils.getConfigDir()); - InetSocketAddress address = frontend.getJobManagerAddress(line); + InetSocketAddress address = RemoteExecutor.getInetFromHostport(frontend.getJobManagerAddressString(line)); assertNotNull(address); assertEquals(CliFrontendTestUtils.TEST_JOB_MANAGER_ADDRESS, address.getAddress().getHostAddress()); @@ -93,7 +92,7 @@ public void testYarnConfig() { TestingCliFrontend frontend = new TestingCliFrontend(CliFrontendTestUtils.getConfigDirWithYarnFile()); - InetSocketAddress address = frontend.getJobManagerAddress(line); + InetSocketAddress address = RemoteExecutor.getInetFromHostport(frontend.getJobManagerAddressString(line)); assertNotNull(address); assertEquals(CliFrontendTestUtils.TEST_YARN_JOB_MANAGER_ADDRESS, address.getAddress().getHostAddress()); @@ -114,7 +113,7 @@ public void testInvalidYarnConfig() { TestingCliFrontend frontend = new TestingCliFrontend(CliFrontendTestUtils.getConfigDirWithInvalidYarnFile()); - assertTrue(frontend.getJobManagerAddress(line) == null); + assertTrue(frontend.getJobManagerAddressString(line) == null); } catch (Exception e) { System.err.println(e.getMessage()); @@ -131,7 +130,7 @@ public void testManualOptionsOverridesConfig() { TestingCliFrontend frontend = new TestingCliFrontend(CliFrontendTestUtils.getConfigDir()); - InetSocketAddress address = frontend.getJobManagerAddress(line); + InetSocketAddress address = RemoteExecutor.getInetFromHostport(frontend.getJobManagerAddressString(line)); assertNotNull(address); assertEquals("10.221.130.22", address.getAddress().getHostAddress()); @@ -152,7 +151,7 @@ public void testManualOptionsOverridesYarn() { TestingCliFrontend frontend = new TestingCliFrontend(CliFrontendTestUtils.getConfigDirWithYarnFile()); - InetSocketAddress address = frontend.getJobManagerAddress(line); + InetSocketAddress address = RemoteExecutor.getInetFromHostport(frontend.getJobManagerAddressString(line)); assertNotNull(address); assertEquals("10.221.130.22", address.getAddress().getHostAddress()); diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java index b9af927eca334..0cd7104f21dcc 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java @@ -62,7 +62,7 @@ public void testCancel() { // test unrecognized option { String[] parameters = {"-v", "-l"}; - CliFrontend testFrontend = new CliFrontend(); + CliFrontend testFrontend = new CliFrontendTestUtils.TestingCliFrontend(); int retCode = testFrontend.cancel(parameters); assertTrue(retCode == 2); } @@ -70,7 +70,7 @@ public void testCancel() { // test missing job id { String[] parameters = {}; - CliFrontend testFrontend = new CliFrontend(); + CliFrontend testFrontend = new CliFrontendTestUtils.TestingCliFrontend(); int retCode = testFrontend.cancel(parameters); assertTrue(retCode != 0); } @@ -104,7 +104,7 @@ public void testList() { // test unrecognized option { String[] parameters = {"-v", "-k"}; - CliFrontend testFrontend = new CliFrontend(); + CliFrontend testFrontend = new CliFrontendTestUtils.TestingCliFrontend(); int retCode = testFrontend.list(parameters); assertTrue(retCode == 2); } @@ -112,7 +112,7 @@ public void testList() { // test missing flags { String[] parameters = {}; - CliFrontend testFrontend = new CliFrontend(); + CliFrontend testFrontend = new CliFrontendTestUtils.TestingCliFrontend(); int retCode = testFrontend.list(parameters); assertTrue(retCode != 0); } 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 9d4c6aed7655c..95f6cb855ba80 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 @@ -113,19 +113,13 @@ public static void clearGlobalConfiguration() { public static class TestingCliFrontend extends CliFrontend { - public final String configDir; - + public TestingCliFrontend() { this(getConfigDir()); } public TestingCliFrontend(String configDir) { - this.configDir = configDir; - } - - @Override - protected String getConfigurationDirectory() { - return this.configDir; + this.configurationDirectory = configDir; } } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index d482e3cadd943..969329efa56cf 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -193,6 +193,20 @@ public final class ConfigConstants { */ public static final String JOBCLIENT_POLLING_INTERVAL_KEY = "jobclient.polling.interval"; + // ------------------------ YARN Configuration ------------------------ + + /** + * Percentage of heap space to remove from containers started by YARN. + */ + public static final String YARN_HEAP_CUTOFF_RATIO = "yarn.heap-cutoff-ratio"; + + /** + * Upper bound for heap cutoff on YARN. + * The "yarn.heap-cutoff-ratio" is removing a certain ratio from the heap. + * This value is limiting this cutoff to a absolute value. + */ + public static final String YARN_HEAP_LIMIT_CAP = "yarn.heap-limit-cap"; + // ------------------------ Hadoop Configuration ------------------------ /** diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java index fb45466f596a5..c3c7ae89c3b2b 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java @@ -242,7 +242,6 @@ public static FileSystem get(URI uri) throws IOException { // by now we know that the HadoopFileSystem wrapper can wrap the file system. fs = instantiateHadoopFileSystemWrapper(wrapperClass); fs.initialize(uri); - System.out.println("Initializing new instance of wrapper for "+wrapperClass); CACHE.put(wrappedKey, fs); } else { diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index e753a0529e94f..91359c2f13a89 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -197,10 +197,12 @@ under the License. implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer"> reference.conf + + - org.apache.flink.yarn.Client + org.apache.flink.yarn.FlinkYarnClient diff --git a/flink-dist/src/main/flink-bin/bin/flink b/flink-dist/src/main/flink-bin/bin/flink index e5dd3c696d4df..12dd6b7c42afd 100755 --- a/flink-dist/src/main/flink-bin/bin/flink +++ b/flink-dist/src/main/flink-bin/bin/flink @@ -50,4 +50,5 @@ log_setting="-Dlog.file="$log" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4 export FLINK_CONF_DIR -$JAVA_RUN $JVM_ARGS $log_setting -classpath $CC_CLASSPATH org.apache.flink.client.CliFrontend $* +# Add HADOOP_CLASSPATH to allow the usage of Hadoop file systems +$JAVA_RUN $JVM_ARGS $log_setting -classpath $CC_CLASSPATH:$HADOOP_CLASSPATH org.apache.flink.client.CliFrontend $* 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 21da505dc662b..bf0775f772c8c 100644 --- 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 $log_setting org.apache.flink.yarn.Client -ship $bin/../ship/ -confDir $FLINK_CONF_DIR -j $FLINK_LIB_DIR/*yarn-uberjar.jar $* +$JAVA_RUN $JVM_ARGS -classpath $CC_CLASSPATH:$HADOOP_CLASSPATH $log_setting org.apache.flink.client.FlinkYarnSessionCli -ship $bin/../ship/ -j $FLINK_LIB_DIR/*yarn-uberjar.jar $* diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java index 9e0a55bb1d841..026758de8435c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java @@ -34,7 +34,6 @@ import akka.actor.ActorRef; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.instance.Instance; @@ -60,13 +59,13 @@ public class SetupInfoServlet extends HttpServlet { private static final Logger LOG = LoggerFactory.getLogger(SetupInfoServlet.class); - final private Configuration globalC; + final private Configuration configuration; final private ActorRef jobmanager; final private FiniteDuration timeout; - public SetupInfoServlet(ActorRef jm, FiniteDuration timeout) { - globalC = GlobalConfiguration.getConfiguration(); + public SetupInfoServlet(Configuration conf, ActorRef jm, FiniteDuration timeout) { + configuration = conf; this.jobmanager = jm; this.timeout = timeout; } @@ -74,7 +73,6 @@ public SetupInfoServlet(ActorRef jm, FiniteDuration timeout) { @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - resp.setStatus(HttpServletResponse.SC_OK); resp.setContentType("application/json"); @@ -86,15 +84,15 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp) } private void writeGlobalConfiguration(HttpServletResponse resp) throws IOException { - - Set keys = globalC.keySet(); + Set keys = configuration.keySet(); List list = new ArrayList(keys); Collections.sort(list); JSONObject obj = new JSONObject(); for (String k : list) { try { - obj.put(k, globalC.getString(k, "")); + + obj.put(k, configuration.getString(k, "")); } catch (JSONException e) { LOG.warn("Json object creation failed", e); } @@ -151,7 +149,7 @@ private void writeTaskmanagers(HttpServletResponse resp) throws IOException { private static final Comparator INSTANCE_SORTER = new Comparator() { @Override public int compare(Instance o1, Instance o2) { - return o1.getInstanceConnectionInfo().compareTo(o2.getInstanceConnectionInfo()); + return o1.getInstanceConnectionInfo().compareTo(o2.getInstanceConnectionInfo()); } }; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java index 24dbaf79df2ff..2b92f9f7b105f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java @@ -30,7 +30,6 @@ import org.slf4j.LoggerFactory; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; import org.eclipse.jetty.http.security.Constraint; import org.eclipse.jetty.security.ConstraintMapping; import org.eclipse.jetty.security.ConstraintSecurityHandler; @@ -88,7 +87,7 @@ public WebInfoServer(Configuration config, ActorRef jobmanager, // if no explicit configuration is given, use the global configuration if (config == null) { - config = GlobalConfiguration.getConfiguration(); + throw new IllegalArgumentException("No Configuration has been passed to the web server"); } this.port = config.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, @@ -133,7 +132,7 @@ public WebInfoServer(Configuration config, ActorRef jobmanager, servletContext.addServlet(new ServletHolder(new JobmanagerInfoServlet(jobmanager, archive, timeout)), "/jobsInfo"); servletContext.addServlet(new ServletHolder(new LogfileInfoServlet(logDirFiles)), "/logInfo"); - servletContext.addServlet(new ServletHolder(new SetupInfoServlet(jobmanager, timeout)), + servletContext.addServlet(new ServletHolder(new SetupInfoServlet(config, jobmanager, timeout)), "/setupInfo"); servletContext.addServlet(new ServletHolder(new MenuServlet()), "/menu"); @@ -206,4 +205,8 @@ public void stop() throws Exception { server.stop(); } + public Server getServer() { + return server; + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java index ec2633c8bb7d2..5a5f51571dbca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java @@ -76,7 +76,7 @@ public static InetAddress resolveAddress(InetSocketAddress jobManagerAddress) th case ADDRESS: if (hasCommonPrefix(jobManagerAddress.getAddress().getAddress(), i.getAddress())) { if (tryToConnect(i, jobManagerAddress, strategy.getTimeout())) { - LOG.info("Determined " + i + " as the TaskTracker's own IP address"); + LOG.info("Determined " + i + " as the machine's own IP address"); return i; } } @@ -86,7 +86,7 @@ public static InetAddress resolveAddress(InetSocketAddress jobManagerAddress) th case SLOW_CONNECT: boolean correct = tryToConnect(i, jobManagerAddress, strategy.getTimeout()); if (correct) { - LOG.info("Determined " + i + " as the TaskTracker's own IP address"); + LOG.info("Determined " + i + " as the machine's own IP address"); return i; } break; 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 new file mode 100644 index 0000000000000..7f2b14eabeae7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java @@ -0,0 +1,50 @@ +/* + * 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.hadoop.fs.Path; +import java.io.File; +import java.util.List; + +public abstract class AbstractFlinkYarnClient { + + // ---- Setter for YARN Cluster properties ----- // + public abstract void setJobManagerMemory(int memoryMB); + public abstract void setTaskManagerMemory(int memoryMB); + public abstract void setTaskManagerSlots(int slots); + public abstract int getTaskManagerSlots(); + public abstract void setQueue(String queue); + public abstract void setLocalJarPath(Path localJarPath); + public abstract void setConfigurationFilePath(Path confPath); + public abstract void setFlinkLoggingConfigurationPath(Path logConfPath); + public abstract Path getFlinkLoggingConfigurationPath(); + public abstract void setTaskManagerCount(int tmCount); + public abstract int getTaskManagerCount(); + public abstract void setConfigurationDirectory(String confDirPath); + // List of files to transfer to the YARN containers. + public abstract void setShipFiles(List shipFiles); + public abstract void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded); + public abstract String getDynamicPropertiesEncoded(); + + // ---- Operations on the YARN cluster ----- // + public abstract String getClusterDescription() throws Exception; + + public abstract AbstractFlinkYarnCluster deploy(String clusterName) throws Exception; + + +} 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 new file mode 100644 index 0000000000000..58eaf1d8f1a24 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java @@ -0,0 +1,44 @@ +/* + * 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 java.net.InetSocketAddress; +import java.util.List; + +public abstract class AbstractFlinkYarnCluster { + + public abstract InetSocketAddress getJobManagerAddress(); + + public abstract String getWebInterfaceURL(); + + public abstract void shutdown(); + + public abstract boolean hasBeenStopped(); + + public abstract FlinkYarnClusterStatus getClusterStatus(); + + public abstract boolean hasFailed(); + + /** + * @return Diagnostics if the Cluster is in "failed" state. + */ + public abstract String getDiagnostics(); + + public abstract List getNewMessages(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/FlinkYarnClusterStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/FlinkYarnClusterStatus.java new file mode 100644 index 0000000000000..2aaaaa0a55bcc --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/FlinkYarnClusterStatus.java @@ -0,0 +1,86 @@ +/* + * 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 java.io.Serializable; + + +public class FlinkYarnClusterStatus implements Serializable { + private int numberOfTaskManagers; + private int numberOfSlots; + + public FlinkYarnClusterStatus() { + } + + public FlinkYarnClusterStatus(int numberOfTaskManagers, int numberOfSlots) { + this.numberOfTaskManagers = numberOfTaskManagers; + this.numberOfSlots = numberOfSlots; + } + + public int getNumberOfTaskManagers() { + return numberOfTaskManagers; + } + + public void setNumberOfTaskManagers(int numberOfTaskManagers) { + this.numberOfTaskManagers = numberOfTaskManagers; + } + + public int getNumberOfSlots() { + return numberOfSlots; + } + + public void setNumberOfSlots(int numberOfSlots) { + this.numberOfSlots = numberOfSlots; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + FlinkYarnClusterStatus that = (FlinkYarnClusterStatus) o; + + if (numberOfSlots != that.numberOfSlots) { + return false; + } + if (numberOfTaskManagers != that.numberOfTaskManagers) { + return false; + } + + return true; + } + + @Override + public int hashCode() { + int result = numberOfTaskManagers; + result = 31 * result + numberOfSlots; + return result; + } + + @Override + public String toString() { + return "FlinkYarnClusterStatus{" + + "numberOfTaskManagers=" + numberOfTaskManagers + + ", numberOfSlots=" + numberOfSlots + + '}'; + } +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index cf678b0c40afa..1f2791ce8e955 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -53,6 +53,9 @@ object AkkaUtils { } def createActorSystem(akkaConfig: Config): ActorSystem = { + if(LOG.isDebugEnabled) { + LOG.debug(s"Using akka config to create actor system: $akkaConfig") + } ActorSystem.create("flink", akkaConfig) } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala index 6a4beedc9e356..195a0b6828983 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/client/JobClient.scala @@ -76,7 +76,8 @@ class JobClientListener(client: ActorRef) extends Actor with ActorLogMessages wi client ! Failure(new JobExecutionException(msg, false)) self ! PoisonPill case msg => - println(msg.toString) + // we have to use System.out.println here to avoid erroneous behavior for output redirection + System.out.println(msg.toString) } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index cd1119d32fb00..37a41a535e1aa 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -437,6 +437,13 @@ class JobManager(val configuration: Configuration) } } + /** + * Handle unmatched messages with an exception. + */ + override def unhandled(message: Any): Unit = { + throw new RuntimeException("Received unknown message " + message) + } + private def removeJob(jobID: JobID): Unit = { currentJobs.remove(jobID) match { case Some((eg, _)) => archive ! ArchiveExecutionGraph(jobID, eg) diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml new file mode 100644 index 0000000000000..0cccf3a5e83ae --- /dev/null +++ b/flink-yarn-tests/pom.xml @@ -0,0 +1,121 @@ + + + 4.0.0 + + + org.apache.flink + flink-parent + 0.9-SNAPSHOT + .. + + + + + flink-yarn-tests + flink-yarn-tests + jar + + + + + org.apache.flink + flink-runtime + ${project.version} + + + hadoop-core + org.apache.hadoop + + + + + + org.apache.flink + flink-clients + ${project.version} + + + + org.apache.flink + flink-yarn + ${project.version} + + + + org.apache.hadoop + hadoop-yarn-client + test + + + + + org.apache.hadoop + hadoop-yarn-common + test + + + + org.apache.hadoop + hadoop-yarn-server-tests + test + + + + org.apache.hadoop + hadoop-minicluster + test + + + + org.apache.hadoop + hadoop-mapreduce-client-core + test + + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + 1 + ../ + + + + org.apache.maven.plugins + maven-surefire-plugin + + ../ + + + + + diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/UtilsTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/UtilsTest.java new file mode 100644 index 0000000000000..9fd2541e39bb6 --- /dev/null +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/UtilsTest.java @@ -0,0 +1,42 @@ +/** + * 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.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.util.Arrays; +import java.util.List; + +public class UtilsTest { + + @Test + public void testUberjarLocator() { + File dir = YarnTestBase.findFile(".", new YarnTestBase.RootDirFilenameFilter()); + Assert.assertNotNull(dir); + dir = dir.getParentFile().getParentFile(); // from uberjar to lib to root + Assert.assertTrue(dir.exists()); + Assert.assertTrue(dir.isDirectory()); + Assert.assertTrue(dir.toString().contains("flink-dist")); + List files = Arrays.asList(dir.list()); + Assert.assertTrue(files.contains("lib")); + Assert.assertTrue(files.contains("bin")); + Assert.assertTrue(files.contains("conf")); + } +} diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerIT.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerIT.java new file mode 100644 index 0000000000000..25e1aa2b49afd --- /dev/null +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerIT.java @@ -0,0 +1,72 @@ +/** + * 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.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This test starts a MiniYARNCluster with a CapacityScheduler. + * Is has, by default a queue called "default". The configuration here adds another queue: "qa-team". + */ +public class YARNSessionCapacitySchedulerIT extends YarnTestBase { + private static final Logger LOG = LoggerFactory.getLogger(YARNSessionCapacitySchedulerIT.class); + + @BeforeClass + public static void setup() { + yarnConfiguration.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); + yarnConfiguration.set("yarn.scheduler.capacity.root.queues", "default,qa-team"); + yarnConfiguration.setInt("yarn.scheduler.capacity.root.default.capacity", 40); + yarnConfiguration.setInt("yarn.scheduler.capacity.root.qa-team.capacity", 60); + startYARNWithConfig(yarnConfiguration); + } + + /** + * Test regular operation, including command line parameter parsing. + */ + @Test + public void testClientStartup() { + runWithArgs(new String[] {"-j", flinkUberjar.getAbsolutePath(), + "-n", "1", + "-jm", "512", + "-tm", "1024", "-qu", "qa-team"}, + "Number of connected TaskManagers changed to 1. Slots available: 1", RunTypes.YARN_SESSION); + } + + + /** + * Test deployment to non-existing queue. (user-reported error) + * Deployment to the queue is possible because there are no queues, so we don't check. + */ + @Test + public void testNonexistingQueue() { + runWithArgs(new String[] {"-j", flinkUberjar.getAbsolutePath(), + "-n", "1", + "-jm", "512", + "-tm", "1024", + "-qu", "doesntExist"}, "Error while deploying YARN cluster: The specified queue 'doesntExist' does not exist. Available queues: default, qa-team, ", RunTypes.YARN_SESSION); + } +} diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOIT.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOIT.java new file mode 100644 index 0000000000000..5f8ae87f0079f --- /dev/null +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOIT.java @@ -0,0 +1,225 @@ +/** + * 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.FlinkYarnSessionCli; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; +import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; + + +/** + * This test starts a MiniYARNCluster with a FIFO scheudler. + * There are no queues for that scheduler. + */ +public class YARNSessionFIFOIT extends YarnTestBase { + private static final Logger LOG = LoggerFactory.getLogger(YARNSessionFIFOIT.class); + + /* + Override init with FIFO scheduler. + */ + @BeforeClass + public static void setup() { + yarnConfiguration.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class, ResourceScheduler.class); + startYARNWithConfig(yarnConfiguration); + } + /** + * Test regular operation, including command line parameter parsing. + */ + @Test + public void testClientStartup() { + LOG.info("Starting testClientStartup()"); + runWithArgs(new String[] {"-j", flinkUberjar.getAbsolutePath(), + "-n", "1", + "-jm", "512", + "-tm", "1024"}, + "Number of connected TaskManagers changed to 1. Slots available: 1", RunTypes.YARN_SESSION); + LOG.info("Finished testClientStartup()"); + } + + /** + * Test querying the YARN cluster. + * + * This test validates through 666*2 cores in the "cluster". + */ + @Test + public void testQueryCluster() { + LOG.info("Starting testQueryCluster()"); + runWithArgs(new String[] {"-q"}, "Summary: totalMemory 8192 totalCores 1332", RunTypes.YARN_SESSION); // we have 666*2 cores. + LOG.info("Finished testQueryCluster()"); + } + + /** + * Test deployment to non-existing queue. (user-reported error) + * Deployment to the queue is possible because there are no queues, so we don't check. + */ + @Test + public void testNonexistingQueue() { + LOG.info("Starting testNonexistingQueue()"); + runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), + "-n", "1", + "-jm", "512", + "-tm", "1024", + "-qu", "doesntExist"}, "Number of connected TaskManagers changed to 1. Slots available: 1", RunTypes.YARN_SESSION); + LOG.info("Finished testNonexistingQueue()"); + } + + /** + * Test requesting more resources than available. + */ + @Test + public void testMoreNodesThanAvailable() { + LOG.info("Starting testMoreNodesThanAvailable()"); + runWithArgs(new String[] {"-j", flinkUberjar.getAbsolutePath(), + "-n", "10", + "-jm", "512", + "-tm", "1024"}, "Error while deploying YARN cluster: This YARN session requires 10752MB of memory in the cluster. There are currently only 8192MB available.", RunTypes.YARN_SESSION); + LOG.info("Finished testMoreNodesThanAvailable()"); + } + + /** + * The test cluster has the following resources: + * - 2 Nodes with 4096 MB each. + * - RM_SCHEDULER_MINIMUM_ALLOCATION_MB is 512 + * + * We allocate: + * 1 JobManager with 256 MB (will be automatically upgraded to 512 due to min alloc mb) + * 5 TaskManagers with 1585 MB + * + * user sees a total request of: 8181 MB (fits) + * system sees a total request of: 8437 (doesn't fit due to min alloc mb) + */ + @Test + public void testResourceComputation() { + LOG.info("Starting testResourceComputation()"); + runWithArgs(new String[] {"-j", flinkUberjar.getAbsolutePath(), + "-n", "5", + "-jm", "256", + "-tm", "1585"}, "Error while deploying YARN cluster: This YARN session requires 8437MB of memory in the cluster. There are currently only 8192MB available.", RunTypes.YARN_SESSION); + LOG.info("Finished testResourceComputation()"); + } + + /** + * The test cluster has the following resources: + * - 2 Nodes with 4096 MB each. + * - RM_SCHEDULER_MINIMUM_ALLOCATION_MB is 512 + * + * We allocate: + * 1 JobManager with 256 MB (will be automatically upgraded to 512 due to min alloc mb) + * 2 TaskManagers with 3840 MB + * + * the user sees a total request of: 7936 MB (fits) + * the system sees a request of: 8192 MB (fits) + * HOWEVER: one machine is going to need 3840 + 512 = 4352 MB, which doesn't fit. + * + * --> check if the system properly rejects allocating this session. + */ + @Test + public void testfullAlloc() { + LOG.info("Starting testfullAlloc()"); + runWithArgs(new String[] {"-j", flinkUberjar.getAbsolutePath(), + "-n", "2", + "-jm", "256", + "-tm", "3840"}, "Error while deploying YARN cluster: There is not enough memory available in the YARN cluster. The TaskManager(s) require 3840MB each. NodeManagers available: [4096, 4096]\n" + + "After allocating the JobManager (512MB) and (1/2) TaskManagers, the following NodeManagers are available: [3584, 256]", RunTypes.YARN_SESSION); + LOG.info("Finished testfullAlloc()"); + } + + /** + * Test per-job yarn cluster + * + * This also tests the prefixed CliFrontend options for the YARN case + */ + @Test + public void perJobYarnCluster() { + LOG.info("Starting perJobYarnCluster()"); + File exampleJarLocation = YarnTestBase.findFile(".", new ContainsName("-WordCount.jar", "streaming")); // exclude streaming wordcount here. + runWithArgs(new String[] {"run", "-m", "yarn-cluster", + "-yj", flinkUberjar.getAbsolutePath(), + "-yn", "1", + "-yjm", "512", + "-ytm", "1024", exampleJarLocation.getAbsolutePath()}, "Job execution switched to status FINISHED.", RunTypes.CLI_FRONTEND); + LOG.info("Finished perJobYarnCluster()"); + } + + /** + * Test the YARN Java API + */ + @Test + public void testJavaAPI() { + final int WAIT_TIME = 15; + LOG.info("Starting testJavaAPI()"); + + AbstractFlinkYarnClient flinkYarnClient = FlinkYarnSessionCli.getFlinkYarnClient(); + flinkYarnClient.setTaskManagerCount(1); + flinkYarnClient.setJobManagerMemory(512); + flinkYarnClient.setTaskManagerMemory(512); + flinkYarnClient.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); + String confDirPath = System.getenv("FLINK_CONF_DIR"); + flinkYarnClient.setConfigurationDirectory(confDirPath); + flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml")); + + // deploy + AbstractFlinkYarnCluster yarnCluster = null; + try { + yarnCluster = flinkYarnClient.deploy(null); + } catch (Exception e) { + System.err.println("Error while deploying YARN cluster: "+e.getMessage()); + e.printStackTrace(System.err); + Assert.fail(); + } + FlinkYarnClusterStatus expectedStatus = new FlinkYarnClusterStatus(1, 1); + for(int second = 0; second < WAIT_TIME * 2; second++) { // run "forever" + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + LOG.warn("Interrupted", e); + Thread.interrupted(); + } + FlinkYarnClusterStatus status = yarnCluster.getClusterStatus(); + if(status != null && status.equals(expectedStatus)) { + LOG.info("Cluster reached status " + status); + break; // all good, cluster started + } + if(second > WAIT_TIME) { + // we waited for 15 seconds. cluster didn't come up correctly + Assert.fail("The custer didn't start after " + WAIT_TIME + " seconds"); + } + } + + // use the cluster + Assert.assertNotNull(yarnCluster.getJobManagerAddress()); + Assert.assertNotNull(yarnCluster.getWebInterfaceURL()); + + LOG.info("Shutting down cluster. All tests passed"); + // shutdown cluster + yarnCluster.shutdown(); + LOG.info("Finished testJavaAPI()"); + } +} 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 new file mode 100644 index 0000000000000..b12952aa73d5e --- /dev/null +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -0,0 +1,379 @@ +/** + * 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.hadoop.conf.Configuration; +import org.apache.hadoop.service.Service; +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.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.MiniYARNCluster; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileWriter; +import java.io.FilenameFilter; +import java.io.IOException; +import java.io.PrintStream; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +/** + * This base class allows to use the MiniYARNCluster. + * The cluster is re-used for all tests. + * + * This class is located in a different package which is build after flink-dist. This way, + * we can use the YARN uberjar of flink to start a Flink YARN session. + */ +public abstract class YarnTestBase { + private static final Logger LOG = LoggerFactory.getLogger(YARNSessionFIFOIT.class); + + private final static PrintStream originalStdout = System.out; + private final static PrintStream originalStderr = System.err; + + + // Temp directory which is deleted after the unit test. + private static TemporaryFolder tmp = new TemporaryFolder(); + + protected static MiniYARNCluster yarnCluster = null; + + protected static File flinkUberjar; + + protected static final Configuration yarnConfiguration; + static { + yarnConfiguration = new YarnConfiguration(); + yarnConfiguration.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512); + yarnConfiguration.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 4096); // 4096 is the available memory anyways + yarnConfiguration.setBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, true); + yarnConfiguration.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true); + yarnConfiguration.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); + yarnConfiguration.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2); + yarnConfiguration.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600); + yarnConfiguration.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true); + yarnConfiguration.setInt(YarnConfiguration.NM_VCORES, 666); // memory is overwritten in the MiniYARNCluster. + // so we have to change the number of cores for testing. + } + + // This code is taken from: http://stackoverflow.com/a/7201825/568695 + // it changes the environment variables of this JVM. Use only for testing purposes! + private static void setEnv(Map newenv) { + try { + Class processEnvironmentClass = Class.forName("java.lang.ProcessEnvironment"); + Field theEnvironmentField = processEnvironmentClass.getDeclaredField("theEnvironment"); + theEnvironmentField.setAccessible(true); + Map env = (Map) theEnvironmentField.get(null); + env.putAll(newenv); + Field theCaseInsensitiveEnvironmentField = processEnvironmentClass.getDeclaredField("theCaseInsensitiveEnvironment"); + theCaseInsensitiveEnvironmentField.setAccessible(true); + Map cienv = (Map) theCaseInsensitiveEnvironmentField.get(null); + cienv.putAll(newenv); + } catch (NoSuchFieldException e) { + try { + Class[] classes = Collections.class.getDeclaredClasses(); + Map env = System.getenv(); + for (Class cl : classes) { + if ("java.util.Collections$UnmodifiableMap".equals(cl.getName())) { + Field field = cl.getDeclaredField("m"); + field.setAccessible(true); + Object obj = field.get(env); + Map map = (Map) obj; + map.clear(); + map.putAll(newenv); + } + } + } catch (Exception e2) { + throw new RuntimeException(e2); + } + } catch (Exception e1) { + throw new RuntimeException(e1); + } + } + + /** + * Sleep a bit between the tests (we are re-using the YARN cluster for the tests) + */ + @After + public void sleep() { + try { + Thread.sleep(500); + } catch (InterruptedException e) { + Assert.fail("Should not happen"); + } + } + + @Before + public void checkClusterEmpty() throws IOException, YarnException { + YarnClient yarnClient = YarnClient.createYarnClient(); + yarnClient.init(yarnConfiguration); + yarnClient.start(); + List apps = yarnClient.getApplications(); + for(ApplicationReport app : apps) { + if(app.getYarnApplicationState() != YarnApplicationState.FINISHED) { + Assert.fail("There is at least one application on the cluster is not finished"); + } + } + } + + /** + * Locate a file or diretory directory + */ + public static File findFile(String startAt, FilenameFilter fnf) { + File root = new File(startAt); + String[] files = root.list(); + if(files == null) { + return null; + } + for(String file : files) { + + File f = new File(startAt + File.separator + file); + if(f.isDirectory()) { + File r = findFile(f.getAbsolutePath(), fnf); + if(r != null) { + return r; + } + } else if (fnf.accept(f.getParentFile(), f.getName())) { + return f; + } + + } + return null; + } + + /** + * Filter to find root dir of the flink-yarn dist. + */ + public static class RootDirFilenameFilter implements FilenameFilter { + @Override + public boolean accept(File dir, String name) { + return name.endsWith("yarn-uberjar.jar") && dir.toString().contains("/lib"); + } + } + public static class ContainsName implements FilenameFilter { + private String name; + private String excludeInPath = null; + + public ContainsName(String name) { + this.name = name; + } + + public ContainsName(String name, String excludeInPath) { + this.name = name; + this.excludeInPath = excludeInPath; + } + + @Override + public boolean accept(File dir, String name) { + if(excludeInPath == null) { + return name.contains(this.name); + } else { + return name.contains(this.name) && !dir.toString().contains(excludeInPath); + } + } + } + + public static File writeYarnSiteConfigXML(Configuration yarnConf) throws IOException { + tmp.create(); + File yarnSiteXML = new File(tmp.newFolder().getAbsolutePath() + "/yarn-site.xml"); + + FileWriter writer = new FileWriter(yarnSiteXML); + yarnConf.writeXml(writer); + writer.flush(); + writer.close(); + return yarnSiteXML; + } + + public static void startYARNWithConfig(Configuration conf) { + flinkUberjar = findFile(".", new RootDirFilenameFilter()); + Assert.assertNotNull(flinkUberjar); + String flinkDistRootDir = flinkUberjar.getParentFile().getParent(); + + if (!flinkUberjar.exists()) { + Assert.fail("Unable to locate yarn-uberjar.jar"); + } + + try { + LOG.info("Starting up MiniYARN cluster"); + if (yarnCluster == null) { + yarnCluster = new MiniYARNCluster(YARNSessionFIFOIT.class.getName(), 2, 1, 1); + + yarnCluster.init(conf); + yarnCluster.start(); + } + + Map map = new HashMap(System.getenv()); + File flinkConfFilePath = findFile(flinkDistRootDir, new ContainsName("flink-conf.yaml")); + Assert.assertNotNull(flinkConfFilePath); + map.put("FLINK_CONF_DIR", flinkConfFilePath.getParent()); + File yarnConfFile = writeYarnSiteConfigXML(conf); + map.put("YARN_CONF_DIR", yarnConfFile.getParentFile().getAbsolutePath()); + setEnv(map); + + Assert.assertTrue(yarnCluster.getServiceState() == Service.STATE.STARTED); + } catch (Exception ex) { + ex.printStackTrace(); + LOG.error("setup failure", ex); + Assert.fail(); + } + } + + /** + * Default @BeforeClass impl. Overwrite this for passing a different configuration + */ + @BeforeClass + public static void setup() { + startYARNWithConfig(yarnConfiguration); + } + + // -------------------------- Runner -------------------------- // + + private static ByteArrayOutputStream outContent; + private static ByteArrayOutputStream errContent; + enum RunTypes { + YARN_SESSION, CLI_FRONTEND + } + + protected void runWithArgs(String[] args, String expect, RunTypes type) { + LOG.info("Running with args "+ Arrays.toString(args)); + + outContent = new ByteArrayOutputStream(); + errContent = new ByteArrayOutputStream(); + System.setOut(new PrintStream(outContent)); + System.setErr(new PrintStream(errContent)); + + + final int START_TIMEOUT_SECONDS = 60; + + Runner runner = new Runner(args, type); + runner.start(); + + boolean expectedStringSeen = false; + for(int second = 0; second < START_TIMEOUT_SECONDS; second++) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + Assert.fail("Interruption not expected"); + } + // check output for correct TaskManager startup. + if(outContent.toString().contains(expect) + || errContent.toString().contains(expect) ) { + expectedStringSeen = true; + LOG.info("Found expected output in redirected streams"); + // send "stop" command to command line interface + runner.sendStop(); + // wait for the thread to stop + try { + runner.join(1000); + } catch (InterruptedException e) { + LOG.warn("Interrupted while stopping runner", e); + } + LOG.warn("stopped"); + break; + } + // check if thread died + if(!runner.isAlive()) { + sendOutput(); + Assert.fail("Runner thread died before the test was finished. Return value = "+runner.getReturnValue()); + } + } + + sendOutput(); + Assert.assertTrue("During the timeout period of " + START_TIMEOUT_SECONDS + " seconds the " + + "expected string did not show up", expectedStringSeen); + LOG.info("Test was successful"); + } + + private static void sendOutput() { + System.setOut(originalStdout); + System.setErr(originalStderr); + + LOG.info("Sending stdout content through logger: \n\n"+outContent.toString()+"\n\n"); + LOG.info("Sending stderr content through logger: \n\n"+errContent.toString()+"\n\n"); + } + + public static class Runner extends Thread { + private final String[] args; + private int returnValue; + private RunTypes type; + private FlinkYarnSessionCli yCli; + + public Runner(String[] args, RunTypes type) { + this.args = args; + this.type = type; + } + + public int getReturnValue() { + return returnValue; + } + + @Override + public void run() { + switch(type) { + case YARN_SESSION: + yCli = new FlinkYarnSessionCli("", ""); + returnValue = yCli.run(args); + break; + case CLI_FRONTEND: + CliFrontend cli = new CliFrontend(); + returnValue = cli.parseParameters(args); + break; + default: + throw new RuntimeException("Unknown type " + type); + } + + if(returnValue != 0) { + Assert.fail("The YARN session returned with non-null value="+returnValue); + } + } + + public void sendStop() { + if(yCli != null) { + yCli.stop(); + } + } + } + + // -------------------------- Tear down -------------------------- // + + @AfterClass + public static void tearDown() { + //shutdown YARN cluster + if (yarnCluster != null) { + LOG.info("shutdown MiniYarn cluster"); + yarnCluster.stop(); + yarnCluster = null; + } + } +} diff --git a/flink-yarn-tests/src/test/resources/log4j-test.properties b/flink-yarn-tests/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..26d6a123a0853 --- /dev/null +++ b/flink-yarn-tests/src/test/resources/log4j-test.properties @@ -0,0 +1,25 @@ +################################################################################ +# 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. +################################################################################ + +log4j.rootLogger=INFO, file + +# Log all infos in the given file +log4j.appender.file=org.apache.log4j.ConsoleAppender +log4j.appender.file.append=false +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n \ No newline at end of file diff --git a/flink-addons/flink-yarn/pom.xml b/flink-yarn/pom.xml similarity index 98% rename from flink-addons/flink-yarn/pom.xml rename to flink-yarn/pom.xml index 42167ee8c96a8..1569f15ace7d4 100644 --- a/flink-addons/flink-yarn/pom.xml +++ b/flink-yarn/pom.xml @@ -22,7 +22,7 @@ under the License. org.apache.flink - flink-addons + flink-parent 0.9-SNAPSHOT .. @@ -218,7 +218,7 @@ under the License. false ${basedir}/src/main/scala ${basedir}/src/test/scala - ${project.basedir}/../../tools/maven/scalastyle-config.xml + ${project.basedir}/../tools/maven/scalastyle-config.xml ${project.basedir}/scalastyle-output.xml UTF-8 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 new file mode 100644 index 0000000000000..c922963f880ad --- /dev/null +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java @@ -0,0 +1,653 @@ +/* + * 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 java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +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 org.apache.flink.client.FlinkYarnSessionCli; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.GlobalConfiguration; +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.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.YarnClusterMetrics; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.Records; + +/** + * 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 class FlinkYarnClient extends AbstractFlinkYarnClient { + private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnClient.class); + + /** + * Constants, + * all starting with ENV_ are used as environment variables to pass values from the Client + * to the Application Master. + */ + public final static String ENV_TM_MEMORY = "_CLIENT_TM_MEMORY"; + public final static String ENV_TM_COUNT = "_CLIENT_TM_COUNT"; + public final static String ENV_APP_ID = "_APP_ID"; + public final static String FLINK_JAR_PATH = "_FLINK_JAR_PATH"; // the Flink jar resource location (in HDFS). + public static final String ENV_CLIENT_HOME_DIR = "_CLIENT_HOME_DIR"; + public static final String ENV_CLIENT_SHIP_FILES = "_CLIENT_SHIP_FILES"; + public static final String ENV_CLIENT_USERNAME = "_CLIENT_USERNAME"; + public static final String ENV_SLOTS = "_SLOTS"; + public static final String ENV_DYNAMIC_PROPERTIES = "_DYNAMIC_PROPERTIES"; + + private static final String DEFAULT_QUEUE_NAME = "default"; + + + /** + * Minimum memory requirements, checked by the Client. + */ + private static final int MIN_JM_MEMORY = 128; + private static final int MIN_TM_MEMORY = 128; + + private Configuration conf; + private YarnClient yarnClient; + private YarnClientApplication yarnApplication; + + + /** + * 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 = 512; + + private int taskManagerMemoryMb = 512; + + private int taskManagerCount = 1; + + private String yarnQueue = DEFAULT_QUEUE_NAME; + + private String configurationDirectory; + + private Path flinkConfigurationPath; + + private Path flinkLoggingConfigurationPath; // optional + + private Path flinkJarPath; + + private String dynamicPropertiesEncoded; + + private List shipFiles = new ArrayList(); + + + public FlinkYarnClient() { + // Check if security is enabled + if(UserGroupInformation.isSecurityEnabled()) { + throw new RuntimeException("Flink YARN client does not have security support right now." + + "File a bug, we will fix it asap"); + } + conf = Utils.initializeYarnConfiguration(); + if(this.yarnClient == null) { + // Create yarnClient + yarnClient = YarnClient.createYarnClient(); + yarnClient.init(conf); + yarnClient.start(); + } + } + + @Override + public void setJobManagerMemory(int memoryMb) { + if(memoryMb < MIN_JM_MEMORY) { + throw new IllegalArgumentException("The JobManager memory 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 is below the minimum required memory amount " + + "of "+MIN_TM_MEMORY+" MB"); + } + this.taskManagerMemoryMb = memoryMb; + } + + @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; + } + + 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 shipFiles) { + this.shipFiles.addAll(shipFiles); + } + + public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) { + this.dynamicPropertiesEncoded = dynamicPropertiesEncoded; + } + + @Override + public String getDynamicPropertiesEncoded() { + return this.dynamicPropertiesEncoded; + } + + + public void isReadyForDepoyment() 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"); + } + + } + + 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; + } + + /** + * This method will block until the ApplicationMaster/JobManager have been + * deployed on YARN. + */ + @Override + public AbstractFlinkYarnCluster deploy(String clusterName) throws Exception { + isReadyForDepoyment(); + + 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(); + + // ------------------ Check if the specified queue exists -------------- + + List queues = yarnClient.getAllQueues(); + if(queues.size() > 0) { // check only if there are queues configured. + 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() + ", "; + } + throw new YarnDeploymentException("The specified queue '" + this.yarnQueue + "' does not exist. " + + "Available queues: " + queueNames); + } + } else { + LOG.debug("The YARN cluster does not have any queues configured"); + } + + // ------------------ 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); + } + + + int totalMemoryRequired = jobManagerMemoryMb + taskManagerMemoryMb * taskManagerCount; + ClusterResourceDescription freeClusterMem = getCurrentFreeClusterResources(yarnClient); + if(freeClusterMem.totalFreeMemory < totalMemoryRequired) { + failSessionDuringDeployment(); + throw new YarnDeploymentException("This YARN session requires " + totalMemoryRequired + "MB of memory in the cluster. " + + "There are currently only " + freeClusterMem.totalFreeMemory+"MB available."); + + } + if( taskManagerMemoryMb > freeClusterMem.containerLimit) { + failSessionDuringDeployment(); + throw new YarnDeploymentException("The requested amount of memory for the TaskManagers ("+taskManagerMemoryMb+"MB) is more than " + + "the largest possible YARN container: "+freeClusterMem.containerLimit); + } + if( jobManagerMemoryMb > freeClusterMem.containerLimit) { + failSessionDuringDeployment(); + throw new YarnDeploymentException("The requested amount of memory for the JobManager ("+jobManagerMemoryMb+"MB) is more than " + + "the largest possible YARN container: "+freeClusterMem.containerLimit); + } + + // ----------------- 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)) { + failSessionDuringDeployment(); + throw new YarnDeploymentException("Unable to find a NodeManager that can fit the JobManager/Application master. " + + "The JobManager requires " + jobManagerMemoryMb + "MB. NodeManagers available: "+Arrays.toString(freeClusterMem.nodeManagersFree)); + } + // allocate TaskManagers + for(int i = 0; i < taskManagerCount; i++) { + if(!allocateResource(nmFree, taskManagerMemoryMb)) { + failSessionDuringDeployment(); + throw new YarnDeploymentException("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) ); + } + } + + // ------------------ Prepare Application Master Container ------------------------------ + + // respect custom JVM options in the YAML file + final String javaOpts = GlobalConfiguration.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)+"M " +javaOpts; + + if(hasLogback || hasLog4j) { + amCommand += " -Dlog.file=\""+ApplicationConstants.LOG_DIR_EXPANSION_VAR +"/jobmanager-main.log\""; + } + + if(hasLogback) { + amCommand += " -Dlogback.configurationFile=file:" + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME; + } + if(hasLog4j) { + amCommand += " -Dlog4j.configuration=file:" + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME; + } + + amCommand += " "+ApplicationMaster.class.getName()+" " + + " 1>" + + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager-stdout.log" + + " 2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager-stderr.log"; + 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 sytem 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(); + 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(conf, fs, appId.toString(), flinkJarPath, appMasterJar, fs.getHomeDirectory()); + Path remotePathConf = Utils.setupLocalResource(conf, fs, appId.toString(), flinkConfigurationPath, flinkConf, fs.getHomeDirectory()); + Map 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[3 + shipFiles.size()]; + StringBuffer envShipFileList = new StringBuffer(); + // 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[3 + i] = Utils.setupLocalResource(conf, fs, appId.toString(), + shipLocalPath, shipResources, fs.getHomeDirectory()); + localResources.put(shipFile.getName(), shipResources); + + envShipFileList.append(paths[3 + 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.ALL, FsAction.ALL); + fs.setPermission(sessionFilesDir, permission); // set permission for path. + Utils.setTokensFor(amContainer, paths, this.conf); + + + amContainer.setLocalResources(localResources); + fs.close(); + + // Setup CLASSPATH for ApplicationMaster + Map appMasterEnv = new HashMap(); + Utils.setupEnv(conf, appMasterEnv); + // set configuration values + appMasterEnv.put(FlinkYarnClient.ENV_TM_COUNT, String.valueOf(taskManagerCount)); + appMasterEnv.put(FlinkYarnClient.ENV_TM_MEMORY, String.valueOf(taskManagerMemoryMb)); + appMasterEnv.put(FlinkYarnClient.FLINK_JAR_PATH, remotePathJar.toString() ); + appMasterEnv.put(FlinkYarnClient.ENV_APP_ID, appId.toString()); + appMasterEnv.put(FlinkYarnClient.ENV_CLIENT_HOME_DIR, fs.getHomeDirectory().toString()); + appMasterEnv.put(FlinkYarnClient.ENV_CLIENT_SHIP_FILES, envShipFileList.toString() ); + appMasterEnv.put(FlinkYarnClient.ENV_CLIENT_USERNAME, UserGroupInformation.getCurrentUser().getShortUserName()); + appMasterEnv.put(FlinkYarnClient.ENV_SLOTS, String.valueOf(slots)); + if(dynamicPropertiesEncoded != null) { + appMasterEnv.put(FlinkYarnClient.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); + + if(clusterName == null) { + clusterName = "Flink session with "+taskManagerCount+" TaskManagers"; + } + + appContext.setApplicationName(clusterName); // application name + appContext.setApplicationType("Apache Flink"); + appContext.setAMContainerSpec(amContainer); + appContext.setResource(capability); + appContext.setQueue(yarnQueue); + + + 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 = yarnClient.getApplicationReport(appId); + 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 invesitage 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); + } + // the Flink cluster is deployed in YARN. Represent cluster + return new FlinkYarnCluster(yarnClient, appId, conf, sessionFilesDir); + } + + /** + * 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 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); + } + + + + 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 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 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 static class YarnDeploymentException extends RuntimeException { + public YarnDeploymentException() { + } + + public YarnDeploymentException(String message) { + super(message); + } + + public YarnDeploymentException(String message, Throwable cause) { + super(message, cause); + } + } + +} 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 new file mode 100644 index 0000000000000..98abd5ed09176 --- /dev/null +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java @@ -0,0 +1,363 @@ +/* + * 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.Props; +import akka.util.Timeout; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.net.NetUtils; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; +import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus; +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.Some; +import scala.concurrent.Await; +import scala.concurrent.Awaitable; +import scala.concurrent.Future; +import scala.concurrent.duration.Duration; +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.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + + +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 Configuration hadoopConfig; + // (HDFS) location of the files required to run on YARN. Needed here to delete them on shutdown. + private Path sessionFilesDir; + private InetSocketAddress jobManagerAddress; + + //---------- Class internal fields ------------------- + + private ActorSystem actorSystem; + private ActorRef applicationClient; + private ApplicationReport intialAppReport; + private static FiniteDuration akkaDuration = Duration.apply(5, TimeUnit.SECONDS); + private static Timeout akkaTimeout = Timeout.durationToTimeout(akkaDuration); + + public FlinkYarnCluster(final YarnClient yarnClient, final ApplicationId appId, + Configuration hadoopConfig, Path sessionFilesDir) throws IOException, YarnException { + this.yarnClient = yarnClient; + this.hadoopConfig = hadoopConfig; + this.sessionFilesDir = sessionFilesDir; + + // get one application report manually + intialAppReport = yarnClient.getApplicationReport(appId); + String jobManagerHost = intialAppReport.getHost(); + int jobManagerPort = intialAppReport.getRpcPort(); + this.jobManagerAddress = new InetSocketAddress(jobManagerHost, jobManagerPort); + + // start actor system + LOG.info("Start actor system."); + InetAddress ownHostname = NetUtils.resolveAddress(jobManagerAddress); // find name of own public interface, able to connect to the JM + actorSystem = YarnUtils.createActorSystem(ownHostname.getCanonicalHostName(), 0, GlobalConfiguration.getConfiguration()); // set port automatically. + + // start application client + LOG.info("Start application client."); + + applicationClient = actorSystem.actorOf(Props.create(ApplicationClient.class)); + + // instruct ApplicationClient to start a periodical status polling + applicationClient.tell(new Messages.LocalRegisterClient(jobManagerHost + ":" + jobManagerPort), applicationClient); + + + // add hook to ensure proper shutdown + Runtime.getRuntime().addShutdownHook(clientShutdownHook); + + actorRunner = new Thread(new Runnable() { + @Override + public void run() { + // blocks until ApplicationMaster 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(); + } + + // -------------------------- Interaction with the cluster ------------------------ + + @Override + public InetSocketAddress getJobManagerAddress() { + return jobManagerAddress; + } + + @Override + public String getWebInterfaceURL() { + return this.intialAppReport.getTrackingUrl(); + } + + + @Override + public FlinkYarnClusterStatus getClusterStatus() { + if(hasBeenStopped()) { + throw new RuntimeException("The FlinkYarnCluster has alread been stopped"); + } + Future clusterStatusOption = ask(applicationClient, Messages.LocalGetYarnClusterStatus$.MODULE$, akkaTimeout); + Object clusterStatus = awaitUtil(clusterStatusOption, "Unable to get Cluster status from Application Client"); + if(clusterStatus instanceof None$) { + return null; + } else if(clusterStatus instanceof Some) { + return (FlinkYarnClusterStatus) (((Some) clusterStatus).get()); + } else { + throw new RuntimeException("Unexpected type: "+clusterStatus.getClass().getCanonicalName()); + } + } + + @Override + public boolean hasFailed() { + 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 { + return (lastReport.getYarnApplicationState() == YarnApplicationState.FAILED || + lastReport.getYarnApplicationState() == YarnApplicationState.KILLED); + } + } + + @Override + public String getDiagnostics() { + 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 getNewMessages() { + if(hasBeenStopped()) { + throw new RuntimeException("The FlinkYarnCluster has alread been stopped"); + } + List ret = new ArrayList(); + // get messages from ApplicationClient (locally) + while(true) { + Future messageOptionFuture = ask(applicationClient, Messages.LocalGetYarnMessage$.MODULE$, akkaTimeout); + Object messageOption = awaitUtil(messageOptionFuture, "Error getting new messages from Appliation Client"); + if(messageOption instanceof None$) { + break; + } else if(messageOption instanceof org.apache.flink.yarn.Messages.YarnMessage) { + Messages.YarnMessage msg = (Messages.YarnMessage) messageOption; + ret.add("["+msg.date()+"] "+msg.message()); + } else { + LOG.warn("LocalGetYarnMessage returned unexpected type: "+messageOption); + } + } + return ret; + } + + private static T awaitUtil(Awaitable awaitable, String message) { + try { + return Await.result(awaitable, akkaDuration); + } catch (Exception e) { + throw new RuntimeException(message, e); + } + } + + // -------------------------- Shutdown handling ------------------------ + + private AtomicBoolean hasBeenShutDown = new AtomicBoolean(false); + @Override + public void shutdown() { + shutdownInternal(true); + } + + private void shutdownInternal(boolean removeShutdownHook) { + if(hasBeenShutDown.getAndSet(true)) { + return; + } + // the session is being stopped explicitly. + if(removeShutdownHook) { + Runtime.getRuntime().removeShutdownHook(clientShutdownHook); + } + if(actorSystem != null){ + LOG.info("Sending shutdown request to the Application Master"); + if(applicationClient != ActorRef.noSender()) { + Future future = ask(applicationClient, new Messages.StopYarnSession(FinalApplicationStatus.SUCCEEDED), akkaTimeout); + awaitUtil(future, "Error while stopping YARN Application Client"); + } + + 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() { + shutdownInternal(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 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); + // TODO: do more here. + } + 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()); + } + } + } + +} diff --git a/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java similarity index 78% rename from flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java rename to flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index bd5659a9d8995..8bb26681b410a 100644 --- a/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -15,24 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - package org.apache.flink.yarn; import java.io.File; -import java.io.FileOutputStream; import java.io.FilenameFilter; import java.io.IOException; -import java.io.InputStream; import java.net.MalformedURLException; import java.net.URL; import java.net.URLClassLoader; import java.nio.ByteBuffer; import java.util.Collection; -import java.util.Enumeration; import java.util.Map; -import java.util.jar.JarEntry; -import java.util.jar.JarFile; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,40 +55,10 @@ public class Utils { private static final Logger LOG = LoggerFactory.getLogger(Utils.class); - private static final int HEAP_LIMIT_CAP = 500; - - public static void copyJarContents(String prefix, String pathToJar) throws IOException { - LOG.info("Copying jar (location: "+pathToJar+") to prefix "+prefix); - - JarFile jar = null; - jar = new JarFile(pathToJar); - Enumeration enumr = jar.entries(); - byte[] bytes = new byte[1024]; - while(enumr.hasMoreElements()) { - JarEntry entry = enumr.nextElement(); - if(entry.getName().startsWith(prefix)) { - if(entry.isDirectory()) { - File cr = new File(entry.getName()); - cr.mkdirs(); - continue; - } - InputStream inStream = jar.getInputStream(entry); - File outFile = new File(entry.getName()); - if(outFile.exists()) { - throw new RuntimeException("File unexpectedly exists"); - } - FileOutputStream outputStream = new FileOutputStream(outFile); - int read = 0; - while ((read = inStream.read(bytes)) != -1) { - outputStream.write(bytes, 0, read); - } - inStream.close(); outputStream.close(); - } - } - jar.close(); - } - + private static final int DEFAULT_HEAP_LIMIT_CAP = 500; + private static final float DEFAULT_YARN_HEAP_CUTOFF_RATIO = 0.8f; + /** * Calculate the heap size for the JVMs to start in the containers. * Since JVMs are allocating more than just the heap space, and YARN is very @@ -106,17 +69,16 @@ public static void copyJarContents(String prefix, String pathToJar) throws IOExc * */ public static int calculateHeapSize(int memory) { - int heapLimit = (int)((float)memory*0.80); - if( (memory - heapLimit) > HEAP_LIMIT_CAP) { - heapLimit = memory-HEAP_LIMIT_CAP; + float memoryCutoffRatio = GlobalConfiguration.getFloat(ConfigConstants.YARN_HEAP_CUTOFF_RATIO, DEFAULT_YARN_HEAP_CUTOFF_RATIO); + int heapLimitCap = GlobalConfiguration.getInteger(ConfigConstants.YARN_HEAP_LIMIT_CAP, DEFAULT_HEAP_LIMIT_CAP); + + int heapLimit = (int)((float)memory * memoryCutoffRatio); + if( (memory - heapLimit) > heapLimitCap) { + heapLimit = memory-heapLimitCap; } return heapLimit; } - public static void getFlinkConfiguration(String confDir) { - GlobalConfiguration.loadConfiguration(confDir); - } - private static void addPathToConfig(Configuration conf, File path) { // chain-in a new classloader URL fileUrl = null; @@ -163,8 +125,7 @@ public static Configuration initializeYarnConfiguration() { try { hadoopHome = Shell.getHadoopHome(); } catch (IOException e) { - LOG.error("Unable to get hadoop home. Please set HADOOP_HOME variable!", e); - System.exit(1); + throw new RuntimeException("Unable to get hadoop home. Please set HADOOP_HOME variable!", e); } File tryConf = new File(hadoopHome+"/etc/hadoop"); if(tryConf.exists()) { @@ -266,23 +227,4 @@ public static void addToEnvironment(Map environment, environment.put(StringInterner.weakIntern(variable), StringInterner.weakIntern(val)); } - - /** - * Valid ports are 1024 - 65535. - * We offset the incoming port by the applicationId to avoid port collisions if YARN allocates two ApplicationMasters - * on the same physical hardware - */ - public static int offsetPort(int port, int appId) { - if(port > 65535) { - LOG.warn("The specified YARN RPC port ("+port+") is above the maximum possible port 65535." - + "Setting it to "+64535); - port = 64535; - } - if(port + (appId % 1000) > 65535) { - LOG.warn("The specified YARN RPC port ("+port+") is, when offsetted by the ApplicationID ("+appId+") above " - + "the maximum possible port 65535. Setting it to "+64535); - port = port - 1000; - } - return port + (appId % 1000); - } } diff --git a/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java similarity index 95% rename from flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java rename to flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java index 33a8942b1ccfb..3f1cc23c48c3f 100644 --- a/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/YarnTaskManagerRunner.java @@ -28,7 +28,7 @@ import org.apache.flink.yarn.YarnUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.flink.yarn.Client; +import org.apache.flink.yarn.FlinkYarnClient; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; @@ -42,7 +42,7 @@ public class YarnTaskManagerRunner { public static void main(final String[] args) throws IOException { Map envs = System.getenv(); - final String yarnClientUsername = envs.get(Client.ENV_CLIENT_USERNAME); + final String yarnClientUsername = envs.get(FlinkYarnClient.ENV_CLIENT_USERNAME); final String localDirs = envs.get(Environment.LOCAL_DIRS.key()); // configure local directory 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 new file mode 100644 index 0000000000000..22f4c02f71014 --- /dev/null +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala @@ -0,0 +1,127 @@ +/* + * 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 java.util.concurrent.TimeUnit + +import akka.actor._ +import org.apache.flink.configuration.{ConfigConstants, GlobalConfiguration} +import org.apache.flink.runtime.ActorLogMessages +import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.jobmanager.JobManager +import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus +import org.apache.flink.yarn.Messages._ +import scala.collection.mutable +import scala.concurrent.duration._ + +class ApplicationClient + + extends Actor with ActorLogMessages with ActorLogging { + import context._ + + val INITIAL_POLLING_DELAY = 0 seconds + val WAIT_FOR_YARN_INTERVAL = 2 seconds + val POLLING_INTERVAL = 3 seconds + + var yarnJobManager: Option[ActorRef] = None + var pollingTimer: Option[Cancellable] = None + implicit var timeout: FiniteDuration = 0 seconds + var running = false + var messagesQueue : mutable.Queue[YarnMessage] = mutable.Queue[YarnMessage]() + var latestClusterStatus : Option[FlinkYarnClusterStatus] = None + var stopMessageReceiver : Option[ActorRef] = None + + override def preStart(): Unit = { + super.preStart() + + timeout = new FiniteDuration(GlobalConfiguration.getInteger(ConfigConstants.AKKA_ASK_TIMEOUT, + ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT), TimeUnit.SECONDS) + } + + override def postStop(): Unit = { + log.info("Stopped Application client.") + pollingTimer foreach { + _.cancel() + } + + pollingTimer = None + } + + override def receiveWithLogMessages: Receive = { + // ----------------------------- Registration -> Status updates -> shutdown ---------------- + case LocalRegisterClient(address: String) => { + val jmAkkaUrl = JobManager.getRemoteAkkaURL(address) + + yarnJobManager = Some(AkkaUtils.getReference(jmAkkaUrl)(system, timeout)) + yarnJobManager match { + case Some(jm) => { + // the message came from the FlinkYarnCluster. We send the message to the JobManager. + // it is important not to forward the message because the JobManager is storing the + // sender as the Application Client (this class). + jm ! RegisterClient + + // schedule a periodic status report from the JobManager + // request the number of task managers and slots from the job manager + pollingTimer = Some(context.system.scheduler.schedule(INITIAL_POLLING_DELAY, + WAIT_FOR_YARN_INTERVAL, yarnJobManager.get, PollYarnClusterStatus)) + } + case None => throw new RuntimeException("Registration at JobManager/ApplicationMaster " + + "failed. Job Manager RPC connection has not properly been initialized"); + } + } + case msg: StopYarnSession => { + log.info("Stop yarn session.") + stopMessageReceiver = Some(sender()) + yarnJobManager foreach { + _ forward msg + } + } + case JobManagerStopped => { + log.info("Remote JobManager has been stopped successfully. " + + "Stopping local application client") + stopMessageReceiver foreach { + _ ! JobManagerStopped + } + // stop ourselves + context.system.shutdown() + } + + // handle the responses from the PollYarnClusterStatus messages to the yarn job mgr + case status: FlinkYarnClusterStatus => { + latestClusterStatus = Some(status) + } + + // locally get cluster status + case LocalGetYarnClusterStatus => { + sender() ! latestClusterStatus + } + + // ----------------- handle messages from the cluster ------------------- + // receive remote messages + case msg: YarnMessage => { + messagesQueue.enqueue(msg) + } + // locally forward messages + case LocalGetYarnMessage => { + sender() ! (if( messagesQueue.size == 0) None else messagesQueue.dequeue) + } + case _ => + } + +} diff --git a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala similarity index 64% rename from flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala rename to flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala index 64db0add04c20..fd67b016fc6e2 100644 --- a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala @@ -32,7 +32,7 @@ import org.slf4j.LoggerFactory import scala.io.Source -object ApplicationMaster{ +object ApplicationMaster { import scala.collection.JavaConversions._ val LOG = LoggerFactory.getLogger(this.getClass) @@ -41,9 +41,9 @@ object ApplicationMaster{ val MODIFIED_CONF_FILE = "flink-conf-modified.yaml" def main(args: Array[String]): Unit ={ - val yarnClientUsername = System.getenv(Client.ENV_CLIENT_USERNAME) - LOG.info(s"YARN daemon runs as ${UserGroupInformation.getCurrentUser.getShortUserName} " + - s"' setting user to execute Flink ApplicationMaster/JobManager to ${yarnClientUsername}'") + val yarnClientUsername = System.getenv(FlinkYarnClient.ENV_CLIENT_USERNAME) + LOG.info(s"YARN daemon runs as ${UserGroupInformation.getCurrentUser.getShortUserName}" + + s" setting user to execute Flink ApplicationMaster/JobManager to ${yarnClientUsername}") val ugi = UserGroupInformation.createRemoteUser(yarnClientUsername) @@ -61,42 +61,45 @@ object ApplicationMaster{ val env = System.getenv() + if(LOG.isDebugEnabled) { + LOG.debug("All environment variables: " + env.toString) + } + val currDir = env.get(Environment.PWD.key()) require(currDir != null, "Current directory unknown.") val logDirs = env.get(Environment.LOG_DIRS.key()) + // Note that we use the "ownHostname" given by YARN here, to make sure + // we use the hostnames given by YARN consitently throuout akka. + // for akka "localhost" and "localhost.localdomain" are different actors. val ownHostname = env.get(Environment.NM_HOST.key()) require(ownHostname != null, s"Own hostname not set.") - val taskManagerCount = env.get(Client.ENV_TM_COUNT).toInt - val slots = env.get(Client.ENV_SLOTS).toInt - val dynamicPropertiesEncodedString = env.get(Client.ENV_DYNAMIC_PROPERTIES) - - val appNumber = env.get(Client.ENV_APP_NUMBER).toInt - - val jobManagerPort = GlobalConfiguration.getInteger( - ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT) match { - case x if x <= 0 => x - case x => x + appNumber - } - - val jobManagerWebPort = GlobalConfiguration.getInteger(ConfigConstants - .JOB_MANAGER_WEB_PORT_KEY, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT) + val taskManagerCount = env.get(FlinkYarnClient.ENV_TM_COUNT).toInt + val slots = env.get(FlinkYarnClient.ENV_SLOTS).toInt + val dynamicPropertiesEncodedString = env.get(FlinkYarnClient.ENV_DYNAMIC_PROPERTIES) - generateConfigurationFile(currDir, ownHostname, jobManagerPort, jobManagerWebPort, - logDirs, slots, taskManagerCount, dynamicPropertiesEncodedString) + val jobManagerWebPort = 0 // automatic assignment. - val (system, actor) = startJobManager(currDir) + val (system, actor) = startJobManager(currDir, ownHostname,dynamicPropertiesEncodedString) actorSystem = system jobManager = actor + val extActor = system.asInstanceOf[ExtendedActorSystem] + val jobManagerPort = extActor.provider.getDefaultAddress.port.get + + // generate configuration file for TaskManagers + generateConfigurationFile(s"$currDir/$MODIFIED_CONF_FILE", currDir, ownHostname, + jobManagerPort, jobManagerWebPort, logDirs, slots, taskManagerCount, + dynamicPropertiesEncodedString) + + // send "start yarn session" message to YarnJobManager. LOG.info("Start yarn session on job manager.") - jobManager ! StartYarnSession(conf) + jobManager ! StartYarnSession(conf, jobManagerPort) - LOG.info("Await termination of actor system.") + LOG.info("Application Master properly initiated. Await termination of actor system.") actorSystem.awaitTermination() }catch{ case t: Throwable => @@ -116,13 +119,14 @@ object ApplicationMaster{ } - def generateConfigurationFile(currDir: String, ownHostname: String, jobManagerPort: Int, + def generateConfigurationFile(fileName: String, currDir: String, ownHostname: String, + jobManagerPort: Int, jobManagerWebPort: Int, logDirs: String, slots: Int, taskManagerCount: Int, dynamicPropertiesEncodedString: String) : Unit = { LOG.info("Generate configuration file for application master.") val output = new PrintWriter(new BufferedWriter( - new FileWriter(s"$currDir/$MODIFIED_CONF_FILE")) + new FileWriter(fileName)) ) for (line <- Source.fromFile(s"$currDir/$CONF_FILE").getLines() if !(line.contains @@ -132,9 +136,11 @@ object ApplicationMaster{ output.println(s"${ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY}: $ownHostname") output.println(s"${ConfigConstants.JOB_MANAGER_IPC_PORT_KEY}: $jobManagerPort") + output.println(s"${ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY}: $logDirs") output.println(s"${ConfigConstants.JOB_MANAGER_WEB_PORT_KEY}: $jobManagerWebPort") + if(slots != -1){ output.println(s"${ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS}: $slots") output.println( @@ -153,15 +159,24 @@ object ApplicationMaster{ output.close() } - def startJobManager(currDir: String): (ActorSystem, ActorRef) = { + def startJobManager(currDir: String, hostname: String, dynamicPropertiesEncodedString: String): + (ActorSystem, ActorRef) = { LOG.info("Start job manager for yarn") - val pathToConfig = s"$currDir/$MODIFIED_CONF_FILE" - val args = Array[String]("--configDir", pathToConfig) + val args = Array[String]("--configDir", currDir) - LOG.info(s"Config path: ${pathToConfig}.") - val (hostname, port, configuration, _) = JobManager.parseArgs(args) + LOG.info(s"Config path: ${currDir}.") + val (_, _, configuration, _) = JobManager.parseArgs(args) + + // add dynamic properties to JobManager configuration. + val dynamicProperties = CliFrontend.getDynamicProperties(dynamicPropertiesEncodedString) + import scala.collection.JavaConverters._ + for(property <- dynamicProperties.asScala){ + configuration.setString(property.f0, property.f1) + } + GlobalConfiguration.getConfiguration.addAll(configuration) // make part of globalConf. - implicit val jobManagerSystem = YarnUtils.createActorSystem(hostname, port, configuration) + // set port to 0 to let Akka automatically determine the port. + implicit val jobManagerSystem = YarnUtils.createActorSystem(hostname, port = 0, configuration) LOG.info("Start job manager actor.") (jobManagerSystem, JobManager.startActor(Props(new JobManager(configuration) with diff --git a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala similarity index 71% rename from flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala rename to flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala index cc92165242104..47ce7820690f7 100644 --- a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/Messages.scala @@ -26,12 +26,19 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus object Messages { case class YarnMessage(message: String, date: Date = new Date()) case class ApplicationMasterStatus(numTaskManagers: Int, numSlots: Int) - case object RegisterMessageListener + case object RegisterClient case class StopYarnSession(status: FinalApplicationStatus) - case class StartYarnSession(configuration: Configuration) + case object JobManagerStopped + case class StartYarnSession(configuration: Configuration, actorSystemPort: Int) case object PollContainerCompletion - case object PollYarnReport + case object PollYarnClusterStatus // see org.apache.flink.runtime.yarn.FlinkYarnClusterStatus for + // the response case object CheckForUserCommand + + // Client-local messages + case class LocalRegisterClient(jobManagerAddress: String) + case object LocalGetYarnMessage // request new message + case object LocalGetYarnClusterStatus // request the latest cluster status } diff --git a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala similarity index 87% rename from flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala rename to flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala index 63c9b712eae93..7e9570e920150 100644 --- a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala @@ -22,10 +22,11 @@ import java.io.{IOException, File} import java.nio.ByteBuffer import java.util.{ Collections} -import akka.actor.{ActorRef} +import akka.actor.ActorRef import org.apache.flink.configuration.ConfigConstants import org.apache.flink.runtime.ActorLogMessages -import org.apache.flink.runtime.jobmanager.JobManager +import org.apache.flink.runtime.jobmanager.{WithWebServer, JobManager} +import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus import org.apache.flink.yarn.Messages._ import org.apache.flink.yarn.appMaster.YarnTaskManagerRunner import org.apache.hadoop.conf.Configuration @@ -43,7 +44,7 @@ import scala.concurrent.duration._ trait YarnJobManager extends ActorLogMessages { - that: JobManager => + that: JobManager with WithWebServer => import context._ import scala.collection.JavaConverters._ @@ -67,7 +68,7 @@ trait YarnJobManager extends ActorLogMessages { def receiveYarnMessages: Receive = { case StopYarnSession(status) => - log.info("Stopping Yarn Session.") + log.info("Stopping YARN Session.") instanceManager.getAllRegisteredInstances.asScala foreach { instance => @@ -87,34 +88,38 @@ trait YarnJobManager extends ActorLogMessages { } nmClientOption = None - + messageListener foreach { + _ ! JobManagerStopped + } context.system.shutdown() - case RegisterMessageListener => - messageListener = Some(sender) + case RegisterClient => + messageListener = Some(sender()) + + case PollYarnClusterStatus => + sender() ! new FlinkYarnClusterStatus(instanceManager.getNumberOfRegisteredTaskManagers, + instanceManager.getTotalNumberOfSlots) - case StartYarnSession(conf) => { + case StartYarnSession(conf, actorSystemPort: Int) => { log.info("Start yarn session.") - val memoryPerTaskManager = env.get(Client.ENV_TM_MEMORY).toInt + val memoryPerTaskManager = env.get(FlinkYarnClient.ENV_TM_MEMORY).toInt val heapLimit = Utils.calculateHeapSize(memoryPerTaskManager) val applicationMasterHost = env.get(Environment.NM_HOST.key) require(applicationMasterHost != null, s"Application master (${Environment.NM_HOST} not set.") - numTaskManager = env.get(Client.ENV_TM_COUNT).toInt + numTaskManager = env.get(FlinkYarnClient.ENV_TM_COUNT).toInt log.info(s"Requesting ${numTaskManager} task managers.") - val coresPerTaskManager = env.get(Client.ENV_TM_CORES).toInt - val remoteFlinkJarPath = env.get(Client.FLINK_JAR_PATH) + val remoteFlinkJarPath = env.get(FlinkYarnClient.FLINK_JAR_PATH) val fs = FileSystem.get(conf) - val appId = env.get(Client.ENV_APP_ID) + val appId = env.get(FlinkYarnClient.ENV_APP_ID) val currDir = env.get(Environment.PWD.key()) - val clientHomeDir = env.get(Client.ENV_CLIENT_HOME_DIR) - val shipListString = env.get(Client.ENV_CLIENT_SHIP_FILES) - val yarnClientUsername = env.get(Client.ENV_CLIENT_USERNAME) + val clientHomeDir = env.get(FlinkYarnClient.ENV_CLIENT_HOME_DIR) + val shipListString = env.get(FlinkYarnClient.ENV_CLIENT_SHIP_FILES) + val yarnClientUsername = env.get(FlinkYarnClient.ENV_CLIENT_USERNAME) - val jobManagerWebPort = configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, - ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT) + val jobManagerWebPort = that.webServer.getServer.getConnectors()(0).getLocalPort; val rm = AMRMClient.createAMRMClient[ContainerRequest]() rm.init(conf) @@ -132,7 +137,7 @@ trait YarnJobManager extends ActorLogMessages { // Register with ResourceManager val url = s"http://$applicationMasterHost:$jobManagerWebPort" log.info(s"Registering ApplicationMaster with tracking url $url.") - rm.registerApplicationMaster(applicationMasterHost, 0, url) + rm.registerApplicationMaster(applicationMasterHost, actorSystemPort, url) // Priority for worker containers - priorities are intra-application @@ -142,7 +147,7 @@ trait YarnJobManager extends ActorLogMessages { // Resource requirements for worker containers val capability = Records.newRecord(classOf[Resource]) capability.setMemory(memoryPerTaskManager) - capability.setVirtualCores(coresPerTaskManager) + capability.setVirtualCores(1) // hard-code that number (YARN is not accunting for CPUs) // Make container requests to ResourceManager for (i <- 0 until numTaskManager) { @@ -284,7 +289,7 @@ trait YarnJobManager extends ActorLogMessages { // Setup classpath for container ( = TaskManager ) val containerEnv = new java.util.HashMap[String, String]() Utils.setupEnv(yarnConf, containerEnv) - containerEnv.put(Client.ENV_CLIENT_USERNAME, yarnClientUsername) + containerEnv.put(FlinkYarnClient.ENV_CLIENT_USERNAME, yarnClientUsername) ctx.setEnvironment(containerEnv) val user = UserGroupInformation.getCurrentUser diff --git a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala similarity index 94% rename from flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala rename to flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala index 1652705999247..b59694646b20a 100644 --- a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala @@ -31,6 +31,7 @@ trait YarnTaskManager extends ActorLogMessages { def receiveYarnMessages: Receive = { case StopYarnSession(status) => { + log.info(s"Stopping YARN TaskManager with final application status $status") context.system.shutdown() } } diff --git a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnUtils.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnUtils.scala similarity index 97% rename from flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnUtils.scala rename to flink-yarn/src/main/scala/org/apache/flink/yarn/YarnUtils.scala index 245651db6e513..185190d8ebcef 100644 --- a/flink-addons/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnUtils.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnUtils.scala @@ -23,8 +23,12 @@ import com.typesafe.config.ConfigFactory import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.taskmanager.TaskManager +import org.slf4j.LoggerFactory object YarnUtils { + + val LOG = LoggerFactory.getLogger(this.getClass) + def createActorSystem(hostname: String, port: Int, configuration: Configuration): ActorSystem = { val akkaConfig = ConfigFactory.parseString(AkkaUtils.getConfigString(hostname, port, configuration) + getConfigString) diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTests.java b/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTests.java new file mode 100644 index 0000000000000..c8d639a3f0303 --- /dev/null +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/UtilsTests.java @@ -0,0 +1,33 @@ +/* + * 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.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Test; + +public class UtilsTests { + + @Test + public void testHeapCutoff() { + + // ASSUMES DEFAULT Configuration values. + Assert.assertEquals(800, Utils.calculateHeapSize(1000) ); + Assert.assertEquals(9500, Utils.calculateHeapSize(10000) ); + } +} diff --git a/pom.xml b/pom.xml index bcff76f6466bd..e792a3a246989 100644 --- a/pom.xml +++ b/pom.xml @@ -349,7 +349,8 @@ under the License. - + + org.apache.hadoop hadoop-common @@ -573,6 +574,23 @@ under the License. + + org.apache.hadoop + hadoop-yarn-common + ${hadoop.version} + + + org.apache.hadoop + hadoop-yarn-server-tests + test + ${hadoop.version} + + + org.apache.hadoop + hadoop-minicluster + test + ${hadoop.version} + @@ -602,6 +620,19 @@ under the License. + + include-yarn + + + + !hadoop.profile + + + + flink-yarn + flink-yarn-tests + + hadoop-2.0.0-alpha