spark-instrumented-optimizer/docs/cluster-overview.md
Patrick Wendell fb98488fc8 Clean up and simplify Spark configuration
Over time as we've added more deployment modes, this have gotten a bit unwieldy with user-facing configuration options in Spark. Going forward we'll advise all users to run `spark-submit` to launch applications. This is a WIP patch but it makes the following improvements:

1. Improved `spark-env.sh.template` which was missing a lot of things users now set in that file.
2. Removes the shipping of SPARK_CLASSPATH, SPARK_JAVA_OPTS, and SPARK_LIBRARY_PATH to the executors on the cluster. This was an ugly hack. Instead it introduces config variables spark.executor.extraJavaOpts, spark.executor.extraLibraryPath, and spark.executor.extraClassPath.
3. Adds ability to set these same variables for the driver using `spark-submit`.
4. Allows you to load system properties from a `spark-defaults.conf` file when running `spark-submit`. This will allow setting both SparkConf options and other system properties utilized by `spark-submit`.
5. Made `SPARK_LOCAL_IP` an environment variable rather than a SparkConf property. This is more consistent with it being set on each node.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #299 from pwendell/config-cleanup and squashes the following commits:

127f301 [Patrick Wendell] Improvements to testing
a006464 [Patrick Wendell] Moving properties file template.
b4b496c [Patrick Wendell] spark-defaults.properties -> spark-defaults.conf
0086939 [Patrick Wendell] Minor style fixes
af09e3e [Patrick Wendell] Mention config file in docs and clean-up docs
b16e6a2 [Patrick Wendell] Cleanup of spark-submit script and Scala quick start guide
af0adf7 [Patrick Wendell] Automatically add user jar
a56b125 [Patrick Wendell] Responses to Tom's review
d50c388 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into config-cleanup
a762901 [Patrick Wendell] Fixing test failures
ffa00fe [Patrick Wendell] Review feedback
fda0301 [Patrick Wendell] Note
308f1f6 [Patrick Wendell] Properly escape quotes and other clean-up for YARN
e83cd8f [Patrick Wendell] Changes to allow re-use of test applications
be42f35 [Patrick Wendell] Handle case where SPARK_HOME is not set
c2a2909 [Patrick Wendell] Test compile fixes
4ee6f9d [Patrick Wendell] Making YARN doc changes consistent
afc9ed8 [Patrick Wendell] Cleaning up line limits and two compile errors.
b08893b [Patrick Wendell] Additional improvements.
ace4ead [Patrick Wendell] Responses to review feedback.
b72d183 [Patrick Wendell] Review feedback for spark env file
46555c1 [Patrick Wendell] Review feedback and import clean-ups
437aed1 [Patrick Wendell] Small fix
761ebcd [Patrick Wendell] Library path and classpath for drivers
7cc70e4 [Patrick Wendell] Clean up terminology inside of spark-env script
5b0ba8e [Patrick Wendell] Don't ship executor envs
84cc5e5 [Patrick Wendell] Small clean-up
1f75238 [Patrick Wendell] SPARK_JAVA_OPTS --> SPARK_MASTER_OPTS for master settings
4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH
6eaf7d0 [Patrick Wendell] executorJavaOpts
0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN
ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS
2014-04-21 10:26:33 -07:00

9.5 KiB

layout title
global Cluster Mode Overview

This document gives a short overview of how Spark runs on clusters, to make it easier to understand the components involved.

Components

Spark applications run as independent sets of processes on a cluster, coordinated by the SparkContext object in your main program (called the driver program). Specifically, to run on a cluster, the SparkContext can connect to several types of cluster managers (either Spark's own standalone cluster manager or Mesos/YARN), which allocate resources across applications. Once connected, Spark acquires executors on nodes in the cluster, which are processes that run computations and store data for your application. Next, it sends your application code (defined by JAR or Python files passed to SparkContext) to the executors. Finally, SparkContext sends tasks for the executors to run.

Spark cluster components

There are several useful things to note about this architecture:

  1. Each application gets its own executor processes, which stay up for the duration of the whole application and run tasks in multiple threads. This has the benefit of isolating applications from each other, on both the scheduling side (each driver schedules its own tasks) and executor side (tasks from different applications run in different JVMs). However, it also means that data cannot be shared across different Spark applications (instances of SparkContext) without writing it to an external storage system.
  2. Spark is agnostic to the underlying cluster manager. As long as it can acquire executor processes, and these communicate with each other, it is relatively easy to run it even on a cluster manager that also supports other applications (e.g. Mesos/YARN).
  3. Because the driver schedules tasks on the cluster, it should be run close to the worker nodes, preferably on the same local area network. If you'd like to send requests to the cluster remotely, it's better to open an RPC to the driver and have it submit operations from nearby than to run a driver far away from the worker nodes.

Cluster Manager Types

The system currently supports three cluster managers:

  • Standalone -- a simple cluster manager included with Spark that makes it easy to set up a cluster.
  • Apache Mesos -- a general cluster manager that can also run Hadoop MapReduce and service applications.
  • Hadoop YARN -- the resource manager in Hadoop 2.

In addition, Spark's EC2 launch scripts make it easy to launch a standalone cluster on Amazon EC2.

Bundling and Launching Applications

Bundling Your Application's Dependencies

If your code depends on other projects, you will need to package them alongside your application in order to distribute the code to a Spark cluster. To do this, to create an assembly jar (or "uber" jar) containing your code and its dependencies. Both sbt and Maven have assembly plugins. When creating assembly jars, list Spark and Hadoop as provided dependencies; these need not be bundled since they are provided by the cluster manager at runtime. Once you have an assembled jar you can call the bin/spark-submit script as shown here while passing your jar.

For Python, you can use the pyFiles argument of SparkContext or its addPyFile method to add .py, .zip or .egg files to be distributed.

Launching Applications with ./bin/spark-submit

Once a user application is bundled, it can be launched using the spark-submit script located in the bin directory. This script takes care of setting up the classpath with Spark and its dependencies, and can support different cluster managers and deploy modes that Spark supports. It's usage is

./bin/spark-submit <app jar> --class path.to.your.Class [other options..]

To enumerate all options available to spark-submit run it with the --help flag. Here are a few examples of common options:

{% highlight bash %}

Run application locally

./bin/spark-submit my-app.jar
--class my.main.ClassName --master local[8]

Run on a Spark cluster

./bin/spark-submit my-app.jar
--class my.main.ClassName --master spark://mycluster:7077
--executor-memory 20G
--total-executor-cores 100

Run on a YARN cluster

HADOOP_CONF_DIR=XX /bin/spark-submit my-app.jar
--class my.main.ClassName --master yarn-cluster \ # can also be yarn-client for client mode --executor-memory 20G
--num-executors 50 {% endhighlight %}

Loading Configurations from a File

The spark-submit script can load default SparkConf values from a properties file and pass them onto your application. By default it will read configuration options from conf/spark-defaults.conf. Any values specified in the file will be passed on to the application when run. They can obviate the need for certain flags to spark-submit: for instance, if spark.master property is set, you can safely omit the --master flag from spark-submit. In general, configuration values explicitly set on a SparkConf take the highest precedence, then flags passed to spark-submit, then values in the defaults file.

If you are ever unclear where configuration options are coming from. fine-grained debugging information can be printed by adding the --verbose option to ./spark-submit.

Advanced Dependency Management

When using ./bin/spark-submit jars will be automatically transferred to the cluster. For many users this is sufficient. However, advanced users can add jars by calling addFile or addJar on an existing SparkContext. This can be used to distribute JAR files (Java/Scala) or .egg and .zip libraries (Python) to executors. Spark uses the following URL scheme to allow different strategies for disseminating jars:

  • file: - Absolute paths and file:/ URIs are served by the driver's HTTP file server, and every executor pulls the file from the driver HTTP server
  • hdfs:, http:, https:, ftp: - these pull down files and JARs from the URI as expected
  • local: - a URI starting with local:/ is expected to exist as a local file on each worker node. This means that no network IO will be incurred, and works well for large files/JARs that are pushed to each worker, or shared via NFS, GlusterFS, etc.

Note that JARs and files are copied to the working directory for each SparkContext on the executor nodes. Over time this can use up a significant amount of space and will need to be cleaned up.

Monitoring

Each driver program has a web UI, typically on port 4040, that displays information about running tasks, executors, and storage usage. Simply go to http://<driver-node>:4040 in a web browser to access this UI. The monitoring guide also describes other monitoring options.

Job Scheduling

Spark gives control over resource allocation both across applications (at the level of the cluster manager) and within applications (if multiple computations are happening on the same SparkContext). The job scheduling overview describes this in more detail.

Glossary

The following table summarizes terms you'll see used to refer to cluster concepts:

TermMeaning
Application User program built on Spark. Consists of a driver program and executors on the cluster.
Application jar A jar containing the user's Spark application. In some cases users will want to create an "uber jar" containing their application along with its dependencies. The user's jar should never include Hadoop or Spark libraries, however, these will be added at runtime.
Driver program The process running the main() function of the application and creating the SparkContext
Cluster manager An external service for acquiring resources on the cluster (e.g. standalone manager, Mesos, YARN)
Deploy mode Distinguishes where the driver process runs. In "cluster" mode, the framework launches the driver inside of the cluster. In "client" mode, the submitter launches the driver outside of the cluster.
Worker node Any node that can run application code in the cluster
Executor A process launched for an application on a worker node, that runs tasks and keeps data in memory or disk storage across them. Each application has its own executors.
Task A unit of work that will be sent to one executor
Job A parallel computation consisting of multiple tasks that gets spawned in response to a Spark action (e.g. save, collect); you'll see this term used in the driver's logs.
Stage Each job gets divided into smaller sets of tasks called stages that depend on each other (similar to the map and reduce stages in MapReduce); you'll see this term used in the driver's logs.