-
Josh Rosen authoredJosh Rosen authored
layout: global
title: Running Spark on YARN
Support for running on YARN (Hadoop NextGen) was added to Spark in version 0.6.0, and improved in subsequent releases.
Preparations
Running Spark-on-YARN requires a binary distribution of Spark which is built with YARN support. Binary distributions can be downloaded from the Spark project website. To build Spark yourself, refer to the building with Maven guide.
Configuration
Most of the configs are the same for Spark on YARN as for other deployment modes. See the configuration page for more information on those. These are configs that are specific to Spark on YARN.
Spark Properties
Property Name | Default | Meaning |
---|---|---|
spark.yarn.applicationMaster.waitTries |
10 | Set the number of times the ApplicationMaster waits for the the Spark master and then also the number of tries it waits for the SparkContext to be initialized |
spark.yarn.submit.file.replication |
3 | HDFS replication level for the files uploaded into HDFS for the application. These include things like the Spark jar, the app jar, and any distributed cache files/archives. |
spark.yarn.preserve.staging.files |
false | Set to true to preserve the staged files (Spark jar, app jar, distributed cache files) at the end of the job rather then delete them. |
spark.yarn.scheduler.heartbeat.interval-ms |
5000 | The interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. |
spark.yarn.max.executor.failures |
numExecutors * 2, with minimum of 3 | The maximum number of executor failures before failing the application. |
spark.yarn.historyServer.address |
(none) | The address of the Spark history server (i.e. host.com:18080). The address should not contain a scheme (http://). Defaults to not being set since the history server is an optional service. This address is given to the YARN ResourceManager when the Spark application finishes to link the application from the ResourceManager UI to the Spark history server UI. |
spark.yarn.dist.archives |
(none) | Comma separated list of archives to be extracted into the working directory of each executor. |
spark.yarn.dist.files |
(none) | Comma-separated list of files to be placed in the working directory of each executor. |
spark.yarn.executor.memoryOverhead |
384 | The amount of off heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. |
spark.yarn.driver.memoryOverhead |
384 | The amount of off heap memory (in megabytes) to be allocated per driver. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. |
spark.yarn.jar |
(none) | The location of the Spark jar file, in case overriding the default location is desired. By default, Spark on YARN will use a Spark jar installed locally, but the Spark jar can also be in a world-readable location on HDFS. This allows YARN to cache it on nodes so that it doesn't need to be distributed each time an application runs. To point to a jar on HDFS, for example, set this configuration to "hdfs:///some/path". |
spark.yarn.access.namenodes |
(none) | A list of secure HDFS namenodes your Spark application is going to access. For example, `spark.yarn.access.namenodes=hdfs://nn1.com:8032,hdfs://nn2.com:8032`. The Spark application must have acess to the namenodes listed and Kerberos must be properly configured to be able to access them (either in the same realm or in a trusted realm). Spark acquires security tokens for each of the namenodes so that the Spark application can access those remote HDFS clusters. |
spark.yarn.appMasterEnv.[EnvironmentVariableName] |
(none) |
Add the environment variable specified by EnvironmentVariableName to the
Application Master process launched on YARN. The user can specify multiple of
these and to set multiple environment variables. In yarn-cluster mode this controls
the environment of the SPARK driver and in yarn-client mode it only controls
the environment of the executor launcher.
|
Launching Spark on YARN
Ensure that HADOOP_CONF_DIR
or YARN_CONF_DIR
points to the directory which contains the (client side) configuration files for the Hadoop cluster.
These configs are used to write to the dfs and connect to the YARN ResourceManager.
There are two deploy modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN.
Unlike in Spark standalone and Mesos mode, in which the master's address is specified in the "master" parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration. Thus, the master parameter is simply "yarn-client" or "yarn-cluster".
To launch a Spark application in yarn-cluster mode:
./bin/spark-submit --class path.to.your.Class --master yarn-cluster [options] <app jar> [app options]
For example:
$ ./bin/spark-submit --class org.apache.spark.examples.SparkPi \
--master yarn-cluster \
--num-executors 3 \
--driver-memory 4g \
--executor-memory 2g \
--executor-cores 1 \
lib/spark-examples*.jar \
10
The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Viewing Logs" section below for how to see driver and executor logs.
To launch a Spark application in yarn-client mode, do the same, but replace "yarn-cluster" with "yarn-client". To run spark-shell:
$ ./bin/spark-shell --master yarn-client