Launching Spark on YARN

Support for running on YARN (Hadoop NextGen) was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0.

Building a YARN-Enabled Assembly JAR

We need a consolidated Spark JAR (which bundles all the required dependencies) to run Spark jobs on a YARN cluster. This can be built by setting the Hadoop version and SPARK_YARN environment variable, as follows:

SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly

The assembled JAR will be something like this: ./assembly/target/scala-2.10/spark-assembly_0.9.2-hadoop2.0.5.jar.

The build process now also supports new YARN versions (2.2.x). See below.

Preparations

If you want to test out the YARN deployment mode, you can use the current Spark examples. A spark-examples_2.10-0.9.2 file can be generated by running sbt/sbt assembly. NOTE: since the documentation you’re reading is for Spark version 0.9.2, we are assuming here that you have downloaded Spark 0.9.2 or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different.

Configuration

Most of the configs are the same for Spark on YARN as other deploys. See the Configuration page for more information on those. These are configs that are specific to SPARK on YARN.

Environment variables:

System Properties:

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. This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager.

There are two scheduler mode that can be used to launch spark application on YARN.

Launch spark application by YARN Client with yarn-standalone mode.

The command to launch the YARN Client is as follows:

SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./bin/spark-class org.apache.spark.deploy.yarn.Client \
  --jar <YOUR_APP_JAR_FILE> \
  --class <APP_MAIN_CLASS> \
  --args <APP_MAIN_ARGUMENTS> \
  --num-workers <NUMBER_OF_WORKER_MACHINES> \
  --master-class <ApplicationMaster_CLASS>
  --master-memory <MEMORY_FOR_MASTER> \
  --worker-memory <MEMORY_PER_WORKER> \
  --worker-cores <CORES_PER_WORKER> \
  --name <application_name> \
  --queue <queue_name> \
  --addJars <any_local_files_used_in_SparkContext.addJar> \
  --files <files_for_distributed_cache> \
  --archives <archives_for_distributed_cache>

For example:

# Build the Spark assembly JAR and the Spark examples JAR
$ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly

# Configure logging
$ cp conf/log4j.properties.template conf/log4j.properties

# Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example
$ SPARK_JAR=./assembly/target/scala-2.10/spark-assembly-0.9.2-hadoop2.0.5-alpha.jar \
    ./bin/spark-class org.apache.spark.deploy.yarn.Client \
      --jar examples/target/scala-2.10/spark-examples-assembly-0.9.2.jar \
      --class org.apache.spark.examples.SparkPi \
      --args yarn-standalone \
      --num-workers 3 \
      --master-memory 4g \
      --worker-memory 2g \
      --worker-cores 1

# Examine the output (replace $YARN_APP_ID in the following with the "application identifier" output by the previous command)
# (Note: YARN_APP_LOGS_DIR is usually /tmp/logs or $HADOOP_HOME/logs/userlogs depending on the Hadoop version.)
$ cat $YARN_APP_LOGS_DIR/$YARN_APP_ID/container*_000001/stdout
Pi is roughly 3.13794

The above starts a YARN Client programs which start the default Application Master. Then SparkPi will be run as a child thread of Application Master, YARN Client will periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running.

With this mode, your application is actually run on the remote machine where the Application Master is run upon. Thus application that involve local interaction will not work well, e.g. spark-shell.

Launch spark application with yarn-client mode.

With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use “yarn-client” instead. And you also need to export the env value for SPARK_JAR and SPARK_YARN_APP_JAR. If you are using spark-shell with secure HDFS you also need to export SPARK_YARN_MODE=true.

Configuration in yarn-client mode:

In order to tune worker core/number/memory etc. You need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options.

For example:

SPARK_JAR=./assembly/target/scala-2.10/spark-assembly-0.9.2-hadoop2.0.5-alpha.jar \
SPARK_YARN_APP_JAR=examples/target/scala-2.10/spark-examples-assembly-0.9.2.jar \
./bin/run-example org.apache.spark.examples.SparkPi yarn-client

SPARK_YARN_MODE=true \
SPARK_JAR=./assembly/target/scala-2.10/spark-assembly-0.9.2-hadoop2.0.5-alpha.jar \
SPARK_YARN_APP_JAR=examples/target/scala-2.10/spark-examples-assembly-0.9.2.jar \
MASTER=yarn-client ./bin/spark-shell

Building Spark for Hadoop/YARN 2.2.x

See Building Spark with Maven for instructions on how to build Spark using the Maven process.

Important Notes