You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2013/09/10 21:40:22 UTC

[08/50] git commit: More fair scheduler docs and property names.

More fair scheduler docs and property names.

Also changed uses of "job" terminology to "application" when they
referred to an entire Spark program, to avoid confusion.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/651a96ad
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/651a96ad
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/651a96ad

Branch: refs/heads/branch-0.8
Commit: 651a96adf7b53085bd810e153f8eabf52eed1994
Parents: 98fb698
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Sat Sep 7 00:34:12 2013 -0400
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Sun Sep 8 00:29:11 2013 -0700

----------------------------------------------------------------------
 conf/fairscheduler.xml.template                 |  18 ++--
 .../scala/org/apache/spark/SparkContext.scala   |   5 +-
 .../scheduler/cluster/SchedulableBuilder.scala  |   6 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |   2 +-
 .../spark/ui/jobs/JobProgressListener.scala     |   2 +-
 .../cluster/ClusterSchedulerSuite.scala         |   6 +-
 .../scheduler/local/LocalSchedulerSuite.scala   |   4 +-
 docs/ec2-scripts.md                             |   8 +-
 docs/job-scheduling.md                          | 101 +++++++++++++++++--
 docs/python-programming-guide.md                |  12 +--
 docs/quick-start.md                             |  70 ++++++-------
 docs/running-on-mesos.md                        |  19 ++--
 docs/scala-programming-guide.md                 |   4 +-
 docs/spark-standalone.md                        |  26 ++---
 14 files changed, 185 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/conf/fairscheduler.xml.template
----------------------------------------------------------------------
diff --git a/conf/fairscheduler.xml.template b/conf/fairscheduler.xml.template
index 04a6b41..acf59e2 100644
--- a/conf/fairscheduler.xml.template
+++ b/conf/fairscheduler.xml.template
@@ -1,15 +1,13 @@
 <?xml version="1.0"?>
 <allocations>
-<pool name="production">
-    <minShare>2</minShare>
-    <weight>1</weight>
+  <pool name="production">
     <schedulingMode>FAIR</schedulingMode>
-</pool>
-<pool name="test">
-    <minShare>3</minShare>
-    <weight>2</weight>
+    <weight>1</weight>
+    <minShare>2</minShare>
+  </pool>
+  <pool name="test">
     <schedulingMode>FIFO</schedulingMode>
-</pool>
-<pool name="data">
-</pool>
+    <weight>2</weight>
+    <minShare>3</minShare>
+  </pool>
 </allocations>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 8931871..edf71c9 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -260,7 +260,7 @@ class SparkContext(
   private val localProperties = new DynamicVariable[Properties](null)
 
   def initLocalProperties() {
-      localProperties.value = new Properties()
+    localProperties.value = new Properties()
   }
 
   def setLocalProperty(key: String, value: String) {
@@ -723,7 +723,8 @@ class SparkContext(
     val callSite = Utils.formatSparkCallSite
     logInfo("Starting job: " + callSite)
     val start = System.nanoTime
-    val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler, localProperties.value)
+    val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler,
+      localProperties.value)
     logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s")
     rdd.doCheckpoint()
     result

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala
index d04eeb6..f808233 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala
@@ -51,8 +51,8 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool)
 private[spark] class FairSchedulableBuilder(val rootPool: Pool)
   extends SchedulableBuilder with Logging {
 
-  val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file")
-  val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool"
+  val schedulerAllocFile = System.getProperty("spark.scheduler.allocation.file")
+  val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool"
   val DEFAULT_POOL_NAME = "default"
   val MINIMUM_SHARES_PROPERTY = "minShare"
   val SCHEDULING_MODE_PROPERTY = "schedulingMode"
@@ -60,7 +60,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool)
   val POOL_NAME_PROPERTY = "@name"
   val POOLS_PROPERTY = "pool"
   val DEFAULT_SCHEDULING_MODE = SchedulingMode.FIFO
-  val DEFAULT_MINIMUM_SHARE = 2
+  val DEFAULT_MINIMUM_SHARE = 0
   val DEFAULT_WEIGHT = 1
 
   override def buildPools() {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index 2ae23cd..3ec9760 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -49,7 +49,7 @@ private[spark] object UIWorkloadGenerator {
 
     def setProperties(s: String) = {
       if(schedulingMode == SchedulingMode.FAIR) {
-        sc.setLocalProperty("spark.scheduler.cluster.fair.pool", s)
+        sc.setLocalProperty("spark.scheduler.pool", s)
       }
       sc.setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, s)
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index e2bcd98..5d46f38 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -95,7 +95,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
     activeStages += stage
 
     val poolName = Option(stageSubmitted.properties).map {
-      p => p.getProperty("spark.scheduler.cluster.fair.pool", DEFAULT_POOL_NAME)
+      p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME)
     }.getOrElse(DEFAULT_POOL_NAME)
     stageToPool(stage) = poolName
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
index 92ad9f0..2b0d90e 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
@@ -166,7 +166,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
     val taskSet = new TaskSet(tasks.toArray,0,0,0,null)
 
     val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
-    System.setProperty("spark.fairscheduler.allocation.file", xmlPath)
+    System.setProperty("spark.scheduler.allocation.file", xmlPath)
     val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
     val schedulableBuilder = new FairSchedulableBuilder(rootPool)
     schedulableBuilder.buildPools()
@@ -183,9 +183,9 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
     assert(rootPool.getSchedulableByName("3").weight === 1)
 
     val properties1 = new Properties()
-    properties1.setProperty("spark.scheduler.cluster.fair.pool","1")
+    properties1.setProperty("spark.scheduler.pool","1")
     val properties2 = new Properties()
-    properties2.setProperty("spark.scheduler.cluster.fair.pool","2")
+    properties2.setProperty("spark.scheduler.pool","2")
 
     val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, clusterScheduler, taskSet)
     val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, clusterScheduler, taskSet)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
index ca9c590..af76c84 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
@@ -73,7 +73,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext {
     TaskThreadInfo.threadToStarted(threadIndex) = new CountDownLatch(1)
     new Thread {
       if (poolName != null) {
-        sc.setLocalProperty("spark.scheduler.cluster.fair.pool", poolName)
+        sc.setLocalProperty("spark.scheduler.pool", poolName)
       }
       override def run() {
         val ans = nums.map(number => {
@@ -152,7 +152,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext {
     val sem = new Semaphore(0)
     System.setProperty("spark.scheduler.mode", "FAIR")
     val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
-    System.setProperty("spark.fairscheduler.allocation.file", xmlPath)
+    System.setProperty("spark.scheduler.allocation.file", xmlPath)
 
     createThread(10,"1",sc,sem)
     TaskThreadInfo.threadToStarted(10).await()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/docs/ec2-scripts.md
----------------------------------------------------------------------
diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md
index da0c06e..1e5575d 100644
--- a/docs/ec2-scripts.md
+++ b/docs/ec2-scripts.md
@@ -80,7 +80,7 @@ another.
 permissions on your private key file, you can run `launch` with the
 `--resume` option to restart the setup process on an existing cluster.
 
-# Running Jobs
+# Running Applications
 
 -   Go into the `ec2` directory in the release of Spark you downloaded.
 -   Run `./spark-ec2 -k <keypair> -i <key-file> login <cluster-name>` to
@@ -90,7 +90,7 @@ permissions on your private key file, you can run `launch` with the
 -   To deploy code or data within your cluster, you can log in and use the
     provided script `~/spark-ec2/copy-dir`, which,
     given a directory path, RSYNCs it to the same location on all the slaves.
--   If your job needs to access large datasets, the fastest way to do
+-   If your application needs to access large datasets, the fastest way to do
     that is to load them from Amazon S3 or an Amazon EBS device into an
     instance of the Hadoop Distributed File System (HDFS) on your nodes.
     The `spark-ec2` script already sets up a HDFS instance for you. It's
@@ -103,8 +103,8 @@ permissions on your private key file, you can run `launch` with the
     (about 3 GB), but you can use the `--ebs-vol-size` option to
     `spark-ec2` to attach a persistent EBS volume to each node for
     storing the persistent HDFS.
--   Finally, if you get errors while running your jobs, look at the slave's logs
-    for that job inside of the scheduler work directory (/root/spark/work). You can
+-   Finally, if you get errors while running your application, look at the slave's logs
+    for that application inside of the scheduler work directory (/root/spark/work). You can
     also view the status of the cluster using the web UI: `http://<master-hostname>:8080`.
 
 # Configuration

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/docs/job-scheduling.md
----------------------------------------------------------------------
diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md
index 4418257..11b7331 100644
--- a/docs/job-scheduling.md
+++ b/docs/job-scheduling.md
@@ -3,14 +3,19 @@ layout: global
 title: Job Scheduling
 ---
 
-Spark has several facilities for scheduling resources between jobs. First, recall that, as described
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+# Overview
+
+Spark has several facilities for scheduling resources between computations. 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.
+_within_ each Spark application, multiple "jobs" (Spark actions) may be running concurrently
+if they were submitted by 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 resources within each SparkContext.
 
 # Scheduling Across Applications
 
@@ -76,6 +81,88 @@ 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")
+{% highlight scala %}
+System.setProperty("spark.scheduler.mode", "FAIR")
+{% endhighlight %}
+
+## Fair Scheduler Pools
+
+The fair scheduler also supports grouping jobs into _pools_, and setting different scheduling options
+(e.g. weight) for each pool. This can be useful to create a "high-priority" pool for more important jobs,
+for example, or to group the jobs of each user together and give _users_ equal shares regardless of how
+many concurrent jobs they have instead of giving _jobs_ equal shares. This approach is modeled after the
+[Hadoop Fair Scheduler](http://hadoop.apache.org/docs/stable/fair_scheduler.html).
+
+Without any intervention, newly submitted jobs go into a _default pool_, but jobs' pools can be set by
+adding the `spark.scheduler.pool` "local property" to the SparkContext in the thread that's submitting them.
+This is done as follows:
+
+{% highlight scala %}
+// Assuming context is your SparkContext variable
+context.setLocalProperty("spark.scheduler.pool", "pool1")
+{% endhighlight %}
+
+After setting this local property, _all_ jobs submitted within this thread (by calls in this thread
+to `RDD.save`, `count`, `collect`, etc) will use this pool name. The setting is per-thread to make
+it easy to have a thread run multiple jobs on behalf of the same user. If you'd like to clear the
+pool that a thread is associated with, simply call:
+
+{% highlight scala %}
+context.setLocalProperty("spark.scheduler.pool", null)
+{% endhighlight %}
+
+## Default Behavior of Pools
+
+By default, each pool gets an equal share of the cluster (also equal in share to each job in the default
+pool), but inside each pool, jobs run in FIFO order. For example, if you create one pool per user, this
+means that each user will get an equal share of the cluster, and that each user's queries will run in
+order instead of later queries taking resources from that user's earlier ones.
+
+## Configuring Pool Properties
+
+Specific pools' properties can also be modified through a configuration file. Each pool supports three
+properties:
+
+* `schedulingMode`: This can be FIFO or FAIR, to control whether jobs within the pool queue up behind
+  each other (the default) or share the pool's resources fairly.
+* `weight`: This controls the pool's share of the cluster relative to other pools. By default, all pools
+  have a weight of 1. If you give a specific pool a weight of 2, for example, it will get 2x more
+  resources as other active pools. Setting a high weight such as 1000 also makes it possible to implement
+  _priority_ between pools---in essence, the weight-1000 pool will always get to launch tasks first
+  whenever it has jobs active.
+* `minShare`: Apart from an overall weight, each pool can be given a _minimum shares_ (as a number of
+  CPU cores) that the administrator would like it to have. The fair scheduler always attempts to meet
+  all active pools' minimum shares before redistributing extra resources according to the weights.
+  The `minShare` property can therefore be another way to ensure that a pool can always get up to a
+  certain number of resources (e.g. 10 cores) quickly without giving it a high priority for the rest
+  of the cluster. By default, each pool's `minShare` is 0.
+
+The pool properties can be set by creating an XML file, similar to `conf/fairscheduler.xml.template`,
+and setting the `spark.scheduler.allocation.file` property:
+
+{% highlight scala %}
+System.setProperty("spark.scheduler.allocation.file", "/path/to/file")
+{% endhighlight %}
+
+The format of the XML file is simply a `<pool>` element for each pool, with different elements
+within it for the various settings. For example:
+
+{% highlight xml %}
+<?xml version="1.0"?>
+<allocations>
+  <pool name="production">
+    <schedulingMode>FAIR</schedulingMode>
+    <weight>1</weight>
+    <minShare>2</minShare>
+  </pool>
+  <pool name="test">
+    <schedulingMode>FIFO</schedulingMode>
+    <weight>2</weight>
+    <minShare>3</minShare>
+  </pool>
+</allocations>
+{% endhighlight %}
 
-The fair scheduler also supports
+A full example is also available in `conf/fairscheduler.xml.template`. Note that any pools not
+configured in the XML file will simply get default values for all settings (scheduling mode FIFO,
+weight 1, and minShare 0).

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/docs/python-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 8c33a95..5662e7d 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -53,20 +53,20 @@ In addition, PySpark fully supports interactive use---simply run `./pyspark` to
 # Installing and Configuring PySpark
 
 PySpark requires Python 2.6 or higher.
-PySpark jobs are executed using a standard CPython interpreter in order to support Python modules that use C extensions.
+PySpark applications are executed using a standard CPython interpreter in order to support Python modules that use C extensions.
 We have not tested PySpark with Python 3 or with alternative Python interpreters, such as [PyPy](http://pypy.org/) or [Jython](http://www.jython.org/).
 
 By default, PySpark requires `python` to be available on the system `PATH` and use it to run programs; an alternate Python executable may be specified by setting the `PYSPARK_PYTHON` environment variable in `conf/spark-env.sh` (or `.cmd` on Windows).
 
 All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported.
 
-Standalone PySpark jobs should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`.
+Standalone PySpark applications should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`.
 The script automatically adds the `pyspark` package to the `PYTHONPATH`.
 
 
 # Interactive Use
 
-The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
+The `pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
 
 {% highlight bash %}
 $ sbt/sbt assembly
@@ -82,7 +82,7 @@ The Python shell can be used explore data interactively and is a simple way to l
 >>> help(pyspark) # Show all pyspark functions
 {% endhighlight %}
 
-By default, the `pyspark` shell creates SparkContext that runs jobs locally on a single core.
+By default, the `pyspark` shell creates SparkContext that runs applications locally on a single core.
 To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable.
 For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
 
@@ -119,13 +119,13 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env
 # Standalone Programs
 
 PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `pyspark`.
-The Quick Start guide includes a [complete example](quick-start.html#a-standalone-job-in-python) of a standalone Python job.
+The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application.
 
 Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor:
 
 {% highlight python %}
 from pyspark import SparkContext
-sc = SparkContext("local", "Job Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg'])
+sc = SparkContext("local", "App Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg'])
 {% endhighlight %}
 
 Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/docs/quick-start.md
----------------------------------------------------------------------
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 70c3df8..1b069ce 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -6,7 +6,7 @@ title: Quick Start
 * This will become a table of contents (this text will be scraped).
 {:toc}
 
-This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will not need much for this), then show how to write standalone jobs in Scala, Java, and Python.
+This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will not need much for this), then show how to write standalone applications in Scala, Java, and Python.
 See the [programming guide](scala-programming-guide.html) for a more complete reference.
 
 To follow along with this guide, you only need to have successfully built Spark on one machine. Simply go into your Spark directory and run:
@@ -36,7 +36,7 @@ scala> textFile.count() // Number of items in this RDD
 res0: Long = 74
 
 scala> textFile.first() // First item in this RDD
-res1: String = # Spark
+res1: String = Welcome to the Spark documentation!
 {% endhighlight %}
 
 Now let's use a transformation. We will use the [`filter`](scala-programming-guide.html#transformations) transformation to return a new RDD with a subset of the items in the file.
@@ -101,20 +101,20 @@ res9: Long = 15
 
 It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
 
-# A Standalone Job in Scala
-Now say we wanted to write a standalone job using the Spark API. We will walk through a simple job in both Scala (with sbt) and Java (with maven). If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
+# A Standalone App in Scala
+Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
 
-We'll create a very simple Spark job in Scala. So simple, in fact, that it's named `SimpleJob.scala`:
+We'll create a very simple Spark application in Scala. So simple, in fact, that it's named `SimpleApp.scala`:
 
 {% highlight scala %}
-/*** SimpleJob.scala ***/
+/*** SimpleApp.scala ***/
 import org.apache.spark.SparkContext
 import org.apache.spark.SparkContext._
 
-object SimpleJob {
+object SimpleApp {
   def main(args: Array[String]) {
     val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system
-    val sc = new SparkContext("local", "Simple Job", "YOUR_SPARK_HOME",
+    val sc = new SparkContext("local", "Simple App", "YOUR_SPARK_HOME",
       List("target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar"))
     val logData = sc.textFile(logFile, 2).cache()
     val numAs = logData.filter(line => line.contains("a")).count()
@@ -124,7 +124,7 @@ object SimpleJob {
 }
 {% endhighlight %}
 
-This job simply counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the job. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the job, the directory where Spark is installed, and a name for the jar file containing the job's sources. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
+This program simply counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the proogram. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
 
 This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on:
 
@@ -146,7 +146,7 @@ If you also wish to read data from Hadoop's HDFS, you will also need to add a de
 libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "<your-hdfs-version>"
 {% endhighlight %}
 
-Finally, for sbt to work correctly, we'll need to layout `SimpleJob.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the job's code, then use `sbt run` to execute our example job.
+Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt run` to execute our program.
 
 {% highlight bash %}
 $ find .
@@ -155,7 +155,7 @@ $ find .
 ./src
 ./src/main
 ./src/main/scala
-./src/main/scala/SimpleJob.scala
+./src/main/scala/SimpleApp.scala
 
 $ sbt package
 $ sbt run
@@ -163,20 +163,20 @@ $ sbt run
 Lines with a: 46, Lines with b: 23
 {% endhighlight %}
 
-# A Standalone Job In Java
-Now say we wanted to write a standalone job using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
+# A Standalone App in Java
+Now say we wanted to write a standalone application using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
 
-We'll create a very simple Spark job, `SimpleJob.java`:
+We'll create a very simple Spark application, `SimpleApp.java`:
 
 {% highlight java %}
-/*** SimpleJob.java ***/
+/*** SimpleApp.java ***/
 import org.apache.spark.api.java.*;
 import org.apache.spark.api.java.function.Function;
 
-public class SimpleJob {
+public class SimpleApp {
   public static void main(String[] args) {
     String logFile = "$YOUR_SPARK_HOME/README.md"; // Should be some file on your system
-    JavaSparkContext sc = new JavaSparkContext("local", "Simple Job",
+    JavaSparkContext sc = new JavaSparkContext("local", "Simple App",
       "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"});
     JavaRDD<String> logData = sc.textFile(logFile).cache();
 
@@ -193,9 +193,9 @@ public class SimpleJob {
 }
 {% endhighlight %}
 
-This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
+This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
 
-To build the job, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version.
+To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version.
 
 {% highlight xml %}
 <project>
@@ -238,29 +238,29 @@ $ find .
 ./src
 ./src/main
 ./src/main/java
-./src/main/java/SimpleJob.java
+./src/main/java/SimpleApp.java
 {% endhighlight %}
 
-Now, we can execute the job using Maven:
+Now, we can execute the application using Maven:
 
 {% highlight bash %}
 $ mvn package
-$ mvn exec:java -Dexec.mainClass="SimpleJob"
+$ mvn exec:java -Dexec.mainClass="SimpleApp"
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight %}
 
-# A Standalone Job In Python
-Now we will show how to write a standalone job using the Python API (PySpark).
+# A Standalone App in Python
+Now we will show how to write a standalone application using the Python API (PySpark).
 
-As an example, we'll create a simple Spark job, `SimpleJob.py`:
+As an example, we'll create a simple Spark application, `SimpleApp.py`:
 
 {% highlight python %}
-"""SimpleJob.py"""
+"""SimpleApp.py"""
 from pyspark import SparkContext
 
 logFile = "$YOUR_SPARK_HOME/README.md"  # Should be some file on your system
-sc = SparkContext("local", "Simple job")
+sc = SparkContext("local", "Simple App")
 logData = sc.textFile(logFile).cache()
 
 numAs = logData.filter(lambda s: 'a' in s).count()
@@ -270,25 +270,25 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs)
 {% endhighlight %}
 
 
-This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file.
+This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file.
 Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. 
 As with the Scala and Java examples, we use a SparkContext to create RDDs.
 We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference.
-For jobs that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html).
-`SimpleJob` is simple enough that we do not need to specify any code dependencies.
+For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html).
+`SimpleApp` is simple enough that we do not need to specify any code dependencies.
 
-We can run this job using the `pyspark` script:
+We can run this application using the `pyspark` script:
 
 {% highlight python %}
 $ cd $SPARK_HOME
-$ ./pyspark SimpleJob.py
+$ ./pyspark SimpleApp.py
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight python %}
 
-# Running Jobs on a Cluster
+# Running on a Cluster
 
-There are a few additional considerations when running jobs on a 
+There are a few additional considerations when running applicaitons on a 
 [Spark](spark-standalone.html), [YARN](running-on-yarn.html), or 
 [Mesos](running-on-mesos.html) cluster.
 
@@ -306,7 +306,7 @@ your dependent jars one-by-one when creating a SparkContext.
 
 ### Setting Configuration Options
 Spark includes several configuration options which influence the behavior
-of your job. These should be set as 
+of your application. These should be set as 
 [JVM system properties](configuration.html#system-properties) in your 
 program. The options will be captured and shipped to all slave nodes.
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/docs/running-on-mesos.md
----------------------------------------------------------------------
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
index b31f78e..eee7a45 100644
--- a/docs/running-on-mesos.md
+++ b/docs/running-on-mesos.md
@@ -17,10 +17,10 @@ Spark can run on private clusters managed by the [Apache Mesos](http://incubator
    * On all nodes, edit `<prefix>/var/mesos/conf/mesos.conf` and add the line `master=HOST:5050`, where HOST is your master node.
    * Run `<prefix>/sbin/mesos-start-cluster.sh` on your master to start Mesos. If all goes well, you should see Mesos's web UI on port 8080 of the master machine.
    * See Mesos's README file for more information on deploying it.
-8. To run a Spark job against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:5050` as the first parameter, where `HOST` is the machine running your Mesos master. In addition, pass the location of Spark on your nodes as the third parameter, and a list of JAR files containing your JAR's code as the fourth (these will automatically get copied to the workers). For example:
+8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:5050` as the first parameter, where `HOST` is the machine running your Mesos master. In addition, pass the location of Spark on your nodes as the third parameter, and a list of JAR files containing your JAR's code as the fourth (these will automatically get copied to the workers). For example:
 
 {% highlight scala %}
-new SparkContext("mesos://HOST:5050", "My Job Name", "/home/user/spark", List("my-job.jar"))
+new SparkContext("mesos://HOST:5050", "My App Name", "/home/user/spark", List("my-app.jar"))
 {% endhighlight %}
 
 If you want to run Spark on Amazon EC2, you can use the Spark [EC2 launch scripts](ec2-scripts.html), which provide an easy way to launch a cluster with Mesos, Spark, and HDFS pre-configured. This will get you a cluster in about five minutes without any configuration on your part.
@@ -28,24 +28,23 @@ If you want to run Spark on Amazon EC2, you can use the Spark [EC2 launch script
 # Mesos Run Modes
 
 Spark can run over Mesos in two modes: "fine-grained" and "coarse-grained". In fine-grained mode, which is the default,
-each Spark task runs as a separate Mesos task. This allows multiple instances of Spark (and other applications) to share
-machines at a very fine granularity, where each job gets more or fewer machines as it ramps up, but it comes with an
-additional overhead in launching each task, which may be inappropriate for low-latency applications that aim for
-sub-second Spark operations (e.g. interactive queries or serving web requests). The coarse-grained mode will instead
+each Spark task runs as a separate Mesos task. This allows multiple instances of Spark (and other frameworks) to share
+machines at a very fine granularity, where each application gets more or fewer machines as it ramps up, but it comes with an
+additional overhead in launching each task, which may be inappropriate for low-latency applications (e.g. interactive queries or serving web requests). The coarse-grained mode will instead
 launch only *one* long-running Spark task on each Mesos machine, and dynamically schedule its own "mini-tasks" within
 it. The benefit is much lower startup overhead, but at the cost of reserving the Mesos resources for the complete duration
-of the job.
+of the application.
 
 To run in coarse-grained mode, set the `spark.mesos.coarse` system property to true *before* creating your SparkContext:
 
 {% highlight scala %}
 System.setProperty("spark.mesos.coarse", "true")
-val sc = new SparkContext("mesos://HOST:5050", "Job Name", ...)
+val sc = new SparkContext("mesos://HOST:5050", "App Name", ...)
 {% endhighlight %}
 
 In addition, for coarse-grained mode, you can control the maximum number of resources Spark will acquire. By default,
-it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just a single
-job at a time. You can cap the maximum number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
+it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just one
+application at a time. You can cap the maximum number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
 Again, this must be done *before* initializing a SparkContext.
 
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/docs/scala-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index f7768e5..03647a2 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -87,10 +87,10 @@ For running on YARN, Spark launches an instance of the standalone deploy cluster
 
 ### Deploying Code on a Cluster
 
-If you want to run your job on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code:
+If you want to run your application on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code:
 
 * `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them).
-* `jars`: A list of JAR files on the local machine containing your job's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your job into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies.
+* `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies.
 
 If you run `spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it.  This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed.
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/651a96ad/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 69e1291..81cdbef 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -48,11 +48,11 @@ Finally, the following configuration options can be passed to the master and wor
   </tr>
   <tr>
     <td><code>-c CORES</code>, <code>--cores CORES</code></td>
-    <td>Total CPU cores to allow Spark jobs to use on the machine (default: all available); only on worker</td>
+    <td>Total CPU cores to allow Spark applicatons to use on the machine (default: all available); only on worker</td>
   </tr>
   <tr>
     <td><code>-m MEM</code>, <code>--memory MEM</code></td>
-    <td>Total amount of memory to allow Spark jobs to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
+    <td>Total amount of memory to allow Spark applicatons to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
   </tr>
   <tr>
     <td><code>-d DIR</code>, <code>--work-dir DIR</code></td>
@@ -98,15 +98,15 @@ You can optionally configure the cluster further by setting environment variable
   </tr>
   <tr>
     <td><code>SPARK_WORKER_DIR</code></td>
-    <td>Directory to run jobs in, which will include both logs and scratch space (default: SPARK_HOME/work).</td>
+    <td>Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work).</td>
   </tr>
   <tr>
     <td><code>SPARK_WORKER_CORES</code></td>
-    <td>Total number of cores to allow Spark jobs to use on the machine (default: all available cores).</td>
+    <td>Total number of cores to allow Spark applications to use on the machine (default: all available cores).</td>
   </tr>
   <tr>
     <td><code>SPARK_WORKER_MEMORY</code></td>
-    <td>Total amount of memory to allow Spark jobs to use on the machine, e.g. <code>1000m</code>, <code>2g</code> (default: total memory minus 1 GB); note that each job's <i>individual</i> memory is configured using its <code>spark.executor.memory</code> property.</td>
+    <td>Total amount of memory to allow Spark applications to use on the machine, e.g. <code>1000m</code>, <code>2g</code> (default: total memory minus 1 GB); note that each application's <i>individual</i> memory is configured using its <code>spark.executor.memory</code> property.</td>
   </tr>
   <tr>
     <td><code>SPARK_WORKER_WEBUI_PORT</code></td>
@@ -133,9 +133,9 @@ You can optionally configure the cluster further by setting environment variable
 
 **Note:** The launch scripts do not currently support Windows. To run a Spark cluster on Windows, start the master and workers by hand.
 
-# Connecting a Job to the Cluster
+# Connecting an Application to the Cluster
 
-To run a job on the Spark cluster, simply pass the `spark://IP:PORT` URL of the master as to the [`SparkContext`
+To run an application on the Spark cluster, simply pass the `spark://IP:PORT` URL of the master as to the [`SparkContext`
 constructor](scala-programming-guide.html#initializing-spark).
 
 To run an interactive Spark shell against the cluster, run the following command:
@@ -147,12 +147,14 @@ automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` vari
 
 You can also pass an option `-c <numCores>` to control the number of cores that spark-shell uses on the cluster.
 
-# Job Scheduling
+# Resource Scheduling
 
-The standalone cluster mode currently only supports a simple FIFO scheduler across jobs.
-However, to allow multiple concurrent jobs, you can control the maximum number of resources each Spark job will acquire.
-By default, it will acquire *all* the cores in the cluster, which only makes sense if you run just a single
-job at a time. You can cap the number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
+The standalone cluster mode currently only supports a simple FIFO scheduler across applications.
+However, to allow multiple concurrent users, you can control the maximum number of resources each
+application will acquire.
+By default, it will acquire *all* cores in the cluster, which only makes sense if you just run one
+application at a time. You can cap the number of cores using
+`System.setProperty("spark.cores.max", "10")` (for example).
 This value must be set *before* initializing your SparkContext.