aboutsummaryrefslogtreecommitdiff
path: root/docs
diff options
context:
space:
mode:
authorMatei Zaharia <matei@eecs.berkeley.edu>2013-09-06 00:29:37 -0400
committerMatei Zaharia <matei@eecs.berkeley.edu>2013-09-08 00:29:11 -0700
commit98fb69822cf780160bca51abeaab7c82e49fab54 (patch)
tree524a1e75519f6a5cc65d004501d6237228db97f2 /docs
parent38488aca8a67d5d2749b82e3fd5f3dc50873d09a (diff)
downloadspark-98fb69822cf780160bca51abeaab7c82e49fab54.tar.gz
spark-98fb69822cf780160bca51abeaab7c82e49fab54.tar.bz2
spark-98fb69822cf780160bca51abeaab7c82e49fab54.zip
Work in progress:
- Add job scheduling docs - Rename some fair scheduler properties - Organize intro page better - Link to Apache wiki for "contributing to Spark"
Diffstat (limited to 'docs')
-rwxr-xr-xdocs/_layouts/global.html4
-rw-r--r--docs/cluster-overview.md5
-rw-r--r--docs/configuration.md32
-rw-r--r--docs/contributing-to-spark.md24
-rw-r--r--docs/index.md26
-rw-r--r--docs/job-scheduling.md81
6 files changed, 128 insertions, 44 deletions
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 90928c8021..5034111ecb 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -86,6 +86,7 @@
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown">Deploying<b class="caret"></b></a>
<ul class="dropdown-menu">
+ <li><a href="cluster-overview.html">Overview</a></li>
<li><a href="ec2-scripts.html">Amazon EC2</a></li>
<li><a href="spark-standalone.html">Standalone Mode</a></li>
<li><a href="running-on-mesos.html">Mesos</a></li>
@@ -101,8 +102,9 @@
<li><a href="tuning.html">Tuning Guide</a></li>
<li><a href="hadoop-third-party-distributions.html">Running with CDH/HDP</a></li>
<li><a href="hardware-provisioning.html">Hardware Provisioning</a></li>
+ <li><a href="job-scheduling.html">Job Scheduling</a></li>
<li><a href="building-with-maven.html">Building Spark with Maven</a></li>
- <li><a href="contributing-to-spark.html">Contributing to Spark</a></li>
+ <li><a href="https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark">Contributing to Spark</a></li>
</ul>
</li>
</ul>
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
new file mode 100644
index 0000000000..9e781bbf1f
--- /dev/null
+++ b/docs/cluster-overview.md
@@ -0,0 +1,5 @@
+---
+layout: global
+title: Cluster Mode Overview
+---
+
diff --git a/docs/configuration.md b/docs/configuration.md
index 310e78a9eb..d4f85538b2 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -82,17 +82,6 @@ Apart from these, the following properties are also available, and may be useful
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
<tr>
- <td>spark.mesos.coarse</td>
- <td>false</td>
- <td>
- If set to "true", runs over Mesos clusters in
- <a href="running-on-mesos.html#mesos-run-modes">"coarse-grained" sharing mode</a>,
- where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per Spark task.
- This gives lower-latency scheduling for short queries, but leaves resources in use for the whole
- duration of the Spark job.
- </td>
-</tr>
-<tr>
<td>spark.default.parallelism</td>
<td>8</td>
<td>
@@ -110,6 +99,17 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
+ <td>spark.mesos.coarse</td>
+ <td>false</td>
+ <td>
+ If set to "true", runs over Mesos clusters in
+ <a href="running-on-mesos.html#mesos-run-modes">"coarse-grained" sharing mode</a>,
+ where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per Spark task.
+ This gives lower-latency scheduling for short queries, but leaves resources in use for the whole
+ duration of the Spark job.
+ </td>
+</tr>
+<tr>
<td>spark.ui.port</td>
<td>3030</td>
<td>
@@ -161,6 +161,16 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
+ <td>spark.scheduler.mode</td>
+ <td>FIFO</td>
+ <td>
+ The <a href="job-scheduling.html#scheduling-within-an-application">scheduling mode</a> between
+ jobs submitted to the same SparkContext. Can be set to <code>FAIR</code>
+ to use fair sharing instead of queueing jobs one after another. Useful for
+ multi-user services.
+ </td>
+</tr>
+<tr>
<td>spark.reducer.maxMbInFlight</td>
<td>48</td>
<td>
diff --git a/docs/contributing-to-spark.md b/docs/contributing-to-spark.md
index 50feeb2d6c..ef1b3ad6da 100644
--- a/docs/contributing-to-spark.md
+++ b/docs/contributing-to-spark.md
@@ -3,24 +3,6 @@ layout: global
title: Contributing to Spark
---
-The Spark team welcomes contributions in the form of GitHub pull requests. Here are a few tips to get your contribution in:
-
-- Break your work into small, single-purpose patches if possible. It's much harder to merge in a large change with a lot of disjoint features.
-- Submit the patch as a GitHub pull request. For a tutorial, see the GitHub guides on [forking a repo](https://help.github.com/articles/fork-a-repo) and [sending a pull request](https://help.github.com/articles/using-pull-requests).
-- Follow the style of the existing codebase. Specifically, we use [standard Scala style guide](http://docs.scala-lang.org/style/), but with the following changes:
- * Maximum line length of 100 characters.
- * Always import packages using absolute paths (e.g. `scala.collection.Map` instead of `collection.Map`).
- * No "infix" syntax for methods other than operators. For example, don't write `table containsKey myKey`; replace it with `table.containsKey(myKey)`.
-- Make sure that your code passes the unit tests. You can run the tests with `sbt/sbt test` in the root directory of Spark.
- But first, make sure that you have [configured a spark-env.sh](configuration.html) with at least
- `SCALA_HOME`, as some of the tests try to spawn subprocesses using this.
-- Add new unit tests for your code. We use [ScalaTest](http://www.scalatest.org/) for testing. Just add a new Suite in `core/src/test`, or methods to an existing Suite.
-- If you'd like to report a bug but don't have time to fix it, you can still post it to our [issue tracker]({{site.SPARK_ISSUE_TRACKER_URL}}), or email the [mailing list](http://www.spark-project.org/mailing-lists.html).
-
-# Licensing of Contributions
-
-Contributions via GitHub pull requests are gladly accepted from their original author. Along with any pull requests, please
-state that the contribution is your original work and that you license the work to the project under the project's open source
-license. *Whether or not you state this explicitly, by submitting any copyrighted material via pull request, email, or other
-means you agree to license the material under the project's open source license and warrant that you have the legal authority
-to do so.*
+The Spark team welcomes all forms of contributions, including bug reports, documentation or patches.
+For the newest information on how to contribute to the project, please read the
+[wiki page on contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark).
diff --git a/docs/index.md b/docs/index.md
index d3aacc629f..1814cb19c8 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -21,7 +21,7 @@ Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with
For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VERSION}}. If you write applications in Scala, you will need to use this same version of Scala in your own program -- newer major versions may not work. You can get the right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/).
-# Testing the Build
+# Running the Examples and Shell
Spark comes with several sample programs in the `examples` directory.
To run one of the samples, use `./run-example <class> <params>` in the top-level Spark directory
@@ -34,14 +34,16 @@ to connect to. This can be a [URL for a distributed cluster](scala-programming-g
or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using
`local` for testing.
-Finally, Spark can be used interactively through modified versions of the Scala shell (`./spark-shell`) or
-Python interpreter (`./pyspark`). These are a great way to learn Spark.
+Finally, you can run Spark interactively through modified versions of the Scala shell (`./spark-shell`) or
+Python interpreter (`./pyspark`). These are a great way to learn the framework.
-# Running on a Cluster
+# Launching on a Cluster
-Spark supports several options for deployment:
+The Spark [cluster mode overview](cluster-overview.html) explains the key concepts in running on a cluster.
+Spark can run both by itself, or over several existing cluster managers. It currently provides several
+options for deployment:
-* [Amazon EC2](ec2-scripts.html): our scripts let you launch a cluster in about 5 minutes
+* [Amazon EC2](ec2-scripts.html): our EC2 scripts let you launch a cluster in about 5 minutes
* [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy Spark on a private cluster
* [Apache Mesos](running-on-mesos.html)
* [Hadoop YARN](running-on-yarn.html)
@@ -91,19 +93,21 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set
**Deployment guides:**
-* [Running Spark on Amazon EC2](ec2-scripts.html): scripts that let you launch a cluster on EC2 in about 5 minutes
+* [Cluster Overview](cluster-overview.html): overview of concepts and components when running on a cluster
+* [Amazon EC2](ec2-scripts.html): scripts that let you launch a cluster on EC2 in about 5 minutes
* [Standalone Deploy Mode](spark-standalone.html): launch a standalone cluster quickly without a third-party cluster manager
-* [Running Spark on Mesos](running-on-mesos.html): deploy a private cluster using
+* [Mesos](running-on-mesos.html): deploy a private cluster using
[Apache Mesos](http://incubator.apache.org/mesos)
-* [Running Spark on YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN)
+* [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN)
**Other documents:**
* [Configuration](configuration.html): customize Spark via its configuration system
* [Tuning Guide](tuning.html): best practices to optimize performance and memory use
* [Hardware Provisioning](hardware-provisioning.html): recommendations for cluster hardware
-* [Building Spark with Maven](building-with-maven.html): Build Spark using the Maven build tool
-* [Contributing to Spark](contributing-to-spark.html)
+* [Job Scheduling](job-scheduling.html): scheduling resources across and within Spark applications
+* [Building Spark with Maven](building-with-maven.html): build Spark using the Maven system
+* [Contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark)
**External resources:**
diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md
new file mode 100644
index 0000000000..44182571d5
--- /dev/null
+++ b/docs/job-scheduling.md
@@ -0,0 +1,81 @@
+---
+layout: global
+title: Job Scheduling
+---
+
+Spark has several facilities for scheduling resources between jobs. First, recall that, as described
+in the [cluster mode overview](cluster-overview.html), each Spark application (instance of SparkContext)
+runs an independent set of executor processes. The cluster managers that Spark runs on provide
+facilities for [scheduling across applications](#scheduling-across-applications). Second,
+_within_ each Spark application, multiple jobs may be running concurrently if they were submitted
+from different threads. This is common if your application is serving requests over the network; for
+example, the [Shark](http://shark.cs.berkeley.edu) server works this way. Spark includes a
+[fair scheduler](#scheduling-within-an-application) to schedule between these jobs.
+
+# Scheduling Across Applications
+
+When running on a cluster, each Spark application gets an independent set of executor JVMs that only
+run tasks and store data for that application. If multiple users need to share your cluster, there are
+different options to manage allocation, depending on the cluster manager.
+
+The simplest option, available on all cluster managers, is _static partitioning_ of resources. With
+this approach, each application is given a maximum amount of resources it can use, and holds onto them
+for its whole duration. This is the only approach available in Spark's [standalone](spark-standalone.html)
+and [YARN](running-on-yarn.html) modes, as well as the
+[coarse-grained Mesos mode](running-on-mesos.html#mesos-run-modes).
+Resource allocation can be configured as follows, based on the cluster type:
+
+* **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in
+ FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit
+ the number of nodes an application uses by setting the `spark.cores.max` system property in it. This
+ will allow multiple users/applications to run concurrently. For example, you might launch a long-running
+ server that uses 10 cores, and allow users to launch shells that use 20 cores each.
+ Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls
+ its memory use.
+* **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` system property to `true`,
+ and optionally set `spark.cores.max` to limit each application's resource share as in the standalone mode.
+ You should also set `spark.executor.memory` to control the executor memory.
+* **YARN:** The `--num-workers` option to the Spark YARN client controls how many workers it will allocate
+ on the cluster, while `--worker-memory` and `--worker-cores` control the resources per worker.
+
+A second option available on Mesos is _dynamic sharing_ of CPU cores. In this mode, each Spark application
+still has a fixed and independent memory allocation (set by `spark.executor.memory`), but when the
+application is not running tasks on a machine, other applications may run tasks on those cores. This mode
+is useful when you expect large numbers of not overly active applications, such as shell sessions from
+separate users. However, it comes with a risk of less predictable latency, because it may take a while for
+an application to gain back cores on one node when it has work to do. To use this mode, simply use a
+`mesos://` URL without setting `spark.mesos.coarse` to true.
+
+Note that none of the modes currently provide memory sharing across applications. If you would like to share
+data this way, we recommend running a single server application that can serve multiple requests by querying
+the same RDDs. For example, the [Shark](http://shark.cs.berkeley.edu) JDBC server works this way for SQL
+queries. In future releases, in-memory storage systems such as [Tachyon](http://tachyon-project.org) will
+provide another approach to share RDDs.
+
+
+# Scheduling Within an Application
+
+Inside a given Spark application (SparkContext instance), multiple parallel jobs can run simultaneously if
+they were submitted from separate threads. By "job", in this section, we mean a Spark action (e.g. `save`,
+`collect`) and any tasks that need to run to evaluate that action. Spark's scheduler is fully thread-safe
+and supports this use case to enable applications that serve multiple requests (e.g. queries for
+multiple users).
+
+By default, Spark's scheduler runs jobs in FIFO fashion. Each job is divided into "stages" (e.g. map and
+reduce phases), and the first job gets priority on all available resources while its stages have tasks to
+launch, then the second job gets priority, etc. If the jobs at the head of the queue don't need to use
+the whole cluster, later jobs can start to run right away, but if the jobs at the head of the queue are
+large, then later jobs may be delayed significantly.
+
+Starting in Spark 0.8, it is also possible to configure fair sharing between jobs. Under fair sharing,
+Spark assigns tasks between jobs in a "round robin" fashion, so that all jobs get a roughly equal share
+of cluster resources. This means that short jobs submitted while a long job is running can start receiving
+resources right away and still get good response times, without waiting for the long job to finish. This
+mode is best for multi-user settings.
+
+To enable the fair scheduler, simply set the `spark.scheduler.mode` to `FAIR` before creating
+a SparkContext:
+
+ System.setProperty("spark.scheduler.mode", "FAIR")
+
+The fair scheduler also supports