Repository: spark
Updated Branches:
  refs/heads/master 3a390bfd8 -> fb98488fc


http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/docs/quick-start.md
----------------------------------------------------------------------
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 60e8b1b..6b4f4ba 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -99,23 +99,32 @@ scala> linesWithSpark.count()
 res9: Long = 15
 {% endhighlight %}
 
-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 `bin/spark-shell` to a cluster, as 
described in the [programming 
guide](scala-programming-guide.html#initializing-spark).
+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 
`bin/spark-shell` to
+a cluster, as described in the [programming 
guide](scala-programming-guide.html#initializing-spark).
 
-# 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.
+# A Standalone Application
+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.
 
-We'll create a very simple Spark application in Scala. So simple, in fact, 
that it's named `SimpleApp.scala`:
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
+
+We'll create a very simple Spark application in Scala. So simple, in fact, 
that it's
+named `SimpleApp.scala`:
 
 {% highlight scala %}
 /*** SimpleApp.scala ***/
 import org.apache.spark.SparkContext
 import org.apache.spark.SparkContext._
+import org.apache.spark.SparkConf
 
 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 App", "YOUR_SPARK_HOME",
-      
List("target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar"))
+    val logFile = "YOUR_SPARK_HOME/README.md" // Should be some file on your 
system
+    val conf = new SparkConf().setAppName("Simple Application")
+    val sc = new SparkContext(conf)
     val logData = sc.textFile(logFile, 2).cache()
     val numAs = logData.filter(line => line.contains("a")).count()
     val numBs = logData.filter(line => line.contains("b")).count()
@@ -124,9 +133,17 @@ object SimpleApp {
 }
 {% endhighlight %}
 
-This program just 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 program. 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 program just 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 program.
+
+We pass the SparkContext constructor a SparkConf object which contains 
information about our
+application. We also call sc.addJar to make sure that when our application is 
launched in cluster
+mode, the jar file containing it will be shipped automatically to worker 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:
+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:
 
 {% highlight scala %}
 name := "Simple Project"
@@ -140,15 +157,12 @@ libraryDependencies += "org.apache.spark" %% "spark-core" 
% "{{site.SPARK_VERSIO
 resolvers += "Akka Repository" at "http://repo.akka.io/releases/";
 {% endhighlight %}
 
-If you also wish to read data from Hadoop's HDFS, you will also need to add a 
dependency on `hadoop-client` for your version of HDFS:
-
-{% highlight scala %}
-libraryDependencies += "org.apache.hadoop" % "hadoop-client" % 
"<your-hdfs-version>"
-{% endhighlight %}
-
-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/sbt run` to execute our program.
+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 the `spark-submit` script to run 
our program.
 
 {% highlight bash %}
+# Your directory layout should look like this
 $ find .
 .
 ./simple.sbt
@@ -157,27 +171,36 @@ $ find .
 ./src/main/scala
 ./src/main/scala/SimpleApp.scala
 
-$ sbt/sbt package
-$ sbt/sbt run
+# Package a jar containing your application
+$ sbt package
+...
+[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar
+
+# Use spark-submit to run your application
+$ YOUR_SPARK_HOME/bin/spark-submit 
target/scala-2.10/simple-project_2.10-1.0.jar \
+  --class "SimpleApp" \
+  --master local[4]
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight %}
 
-# 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.
+</div>
+<div data-lang="java" markdown="1">
+This example will use Maven to compile an application jar, but any similar 
build system will work.
 
 We'll create a very simple Spark application, `SimpleApp.java`:
 
 {% highlight java %}
 /*** SimpleApp.java ***/
 import org.apache.spark.api.java.*;
+import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.function.Function;
 
 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 App",
-      "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"});
+    String logFile = "YOUR_SPARK_HOME/README.md"; // Should be some file on 
your system
+    SparkConf conf = new SparkConf().setAppName("Simple Application");
+    JavaSparkContext sc = new JavaSparkContext(conf);
     JavaRDD<String> logData = sc.textFile(logFile).cache();
 
     long numAs = logData.filter(new Function<String, Boolean>() {
@@ -193,9 +216,16 @@ public class SimpleApp {
 }
 {% endhighlight %}
 
-This program just counts the number of lines containing 'a' and the number 
containing 'b' in a text 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 just counts the number of lines containing 'a' and the number 
containing 'b' in a text
+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 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.
+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>
@@ -221,16 +251,6 @@ To build the program, we also write a Maven `pom.xml` file 
that lists Spark as a
 </project>
 {% endhighlight %}
 
-If you also wish to read data from Hadoop's HDFS, you will also need to add a 
dependency on `hadoop-client` for your version of HDFS:
-
-{% highlight xml %}
-<dependency>
-  <groupId>org.apache.hadoop</groupId>
-  <artifactId>hadoop-client</artifactId>
-  <version>...</version>
-</dependency>
-{% endhighlight %}
-
 We lay out these files according to the canonical Maven directory structure:
 {% highlight bash %}
 $ find .
@@ -241,16 +261,25 @@ $ find .
 ./src/main/java/SimpleApp.java
 {% endhighlight %}
 
-Now, we can execute the application using Maven:
+Now, we can package the application using Maven and execute it with 
`./bin/spark-submit`.
 
 {% highlight bash %}
+# Package a jar containing your application
 $ mvn package
-$ mvn exec:java -Dexec.mainClass="SimpleApp"
+...
+[INFO] Building jar: {..}/{..}/target/simple-project-1.0.jar
+
+# Use spark-submit to run your application
+$ YOUR_SPARK_HOME/bin/spark-submit target/simple-project-1.0.jar \
+  --class "SimpleApp" \
+  --master local[4]
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight %}
 
-# A Standalone App in Python
+</div>
+<div data-lang="python" markdown="1">
+
 Now we will show how to write a standalone application using the Python API 
(PySpark).
 
 As an example, we'll create a simple Spark application, `SimpleApp.py`:
@@ -259,7 +288,7 @@ As an example, we'll create a simple Spark application, 
`SimpleApp.py`:
 """SimpleApp.py"""
 from pyspark import SparkContext
 
-logFile = "$YOUR_SPARK_HOME/README.md"  # Should be some file on your system
+logFile = "YOUR_SPARK_HOME/README.md"  # Should be some file on your system
 sc = SparkContext("local", "Simple App")
 logData = sc.textFile(logFile).cache()
 
@@ -270,11 +299,15 @@ print "Lines with a: %i, lines with b: %i" % (numAs, 
numBs)
 {% endhighlight %}
 
 
-This program just counts the number of lines containing 'a' and the number 
containing 'b' in a text file.
-Note that you'll need to replace $YOUR_SPARK_HOME with the location where 
Spark is installed. 
+This program just counts the number of lines containing 'a' and the number 
containing 'b' in a
+text 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 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).
+We can pass Python functions to Spark, which are automatically serialized 
along with any variables
+that they reference.
+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 application using the `bin/pyspark` script:
@@ -286,57 +319,12 @@ $ ./bin/pyspark SimpleApp.py
 Lines with a: 46, Lines with b: 23
 {% endhighlight python %}
 
-# Running on a Cluster
-
-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.
-
-### Including Your Dependencies
-If your code depends on other projects, you will need to ensure they are also
-present on the slave nodes. A popular approach is to create an
-assembly jar (or "uber" jar) containing your code and its dependencies. Both
-[sbt](https://github.com/sbt/sbt-assembly) and 
-[Maven](http://maven.apache.org/plugins/maven-assembly-plugin/) 
-have assembly plugins. When creating assembly jars, list Spark 
-itself as a `provided` dependency; it need not be bundled since it is 
-already present on the slaves. Once you have an assembled jar, 
-add it to the SparkContext as shown here. It is also possible to add
-your dependent jars one-by-one using the `addJar` method of `SparkContext`.
-
-For Python, you can use the `pyFiles` argument of SparkContext
-or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be 
distributed.
-
-### Setting Configuration Options
-Spark includes several [configuration 
options](configuration.html#spark-properties)
-that influence the behavior of your application.
-These should be set by building a 
[SparkConf](api/core/index.html#org.apache.spark.SparkConf)
-object and passing it to the SparkContext constructor.
-For example, in Java and Scala, you can do:
-
-{% highlight scala %}
-import org.apache.spark.{SparkConf, SparkContext}
-val conf = new SparkConf()
-             .setMaster("local")
-             .setAppName("My application")
-             .set("spark.executor.memory", "1g")
-val sc = new SparkContext(conf)
-{% endhighlight %}
-
-Or in Python:
-
-{% highlight scala %}
-from pyspark import SparkConf, SparkContext
-conf = SparkConf()
-conf.setMaster("local")
-conf.setAppName("My application")
-conf.set("spark.executor.memory", "1g"))
-sc = SparkContext(conf = conf)
-{% endhighlight %}
+</div>
+</div>
 
-### Accessing Hadoop Filesystems
+# Where to go from here
+Congratulations on running your first Spark application!
 
-The examples here access a local file. To read data from a distributed
-filesystem, such as HDFS, include 
-[Hadoop version information](index.html#a-note-about-hadoop-versions)
-in your build file. By default, Spark builds against HDFS 1.0.4.
+* For an in-depth overview of the API see "Programming Guides" menu section.
+* For running applications on a cluster head to the [deployment 
overview](cluster-overview.html).
+* For configuration options available to Spark applications see the 
[configuration page](configuration.html).
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/docs/scala-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 2b0a51e..4431da0 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -25,36 +25,43 @@ To write a Spark application, you need to add a dependency 
on Spark. If you use
     artifactId = spark-core_{{site.SCALA_BINARY_VERSION}}
     version = {{site.SPARK_VERSION}}
 
-In addition, if you wish to access an HDFS cluster, you need to add a 
dependency on `hadoop-client` for your version of HDFS:
+In addition, if you wish to access an HDFS cluster, you need to add a 
dependency on
+`hadoop-client` for your version of HDFS. Some common HDFS version tags are 
listed on the
+[third party distributions](hadoop-third-party-distributions.html) page.
 
     groupId = org.apache.hadoop
     artifactId = hadoop-client
     version = <your-hdfs-version>
 
-For other build systems, you can run `sbt/sbt assembly` to pack Spark and its 
dependencies into one JAR 
(`assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`),
 then add this to your CLASSPATH. Set the HDFS version as described 
[here](index.html#a-note-about-hadoop-versions).
-
 Finally, you need to import some Spark classes and implicit conversions into 
your program. Add the following lines:
 
 {% highlight scala %}
 import org.apache.spark.SparkContext
 import org.apache.spark.SparkContext._
+import org.apache.spark.SparkConf
 {% endhighlight %}
 
 # Initializing Spark
 
-The first thing a Spark program must do is to create a `SparkContext` object, 
which tells Spark how to access a cluster.
-This is done through the following constructor:
+The first thing a Spark program must do is to create a `SparkContext` object, 
which tells Spark
+how to access a cluster. To create a `SparkContext` you first need to build a 
`SparkConf` object
+that contains information about your application.
 
 {% highlight scala %}
-new SparkContext(master, appName, [sparkHome], [jars])
+val conf = new SparkConf().setAppName(<app name>).setMaster(<master>)
+new SparkContext(conf)
 {% endhighlight %}
 
-or through `new SparkContext(conf)`, which takes a 
[SparkConf](api/core/index.html#org.apache.spark.SparkConf)
-object for more advanced configuration.
-
-The `master` parameter is a string specifying a [Spark or Mesos cluster 
URL](#master-urls) to connect to, or a special "local" string to run in local 
mode, as described below. `appName` is a name for your application, which will 
be shown in the cluster web UI. Finally, the last two parameters are needed to 
deploy your code to a cluster if running in distributed mode, as described 
later.
+The `<master>` parameter is a string specifying a [Spark, Mesos or YARN 
cluster URL](#master-urls)
+to connect to, or a special "local" string to run in local mode, as described 
below. `<app name>` is
+a name for your application, which will be shown in the cluster web UI. It's 
also possible to set
+these variables [using a configuration 
file](cluster-overview.html#loading-configurations-from-a-file)
+which avoids hard-coding the master name in your application.
 
-In the Spark shell, a special interpreter-aware SparkContext is already 
created for you, in the variable called `sc`. Making your own SparkContext will 
not work. You can set which master the context connects to using the `MASTER` 
environment variable, and you can add JARs to the classpath with the `ADD_JARS` 
variable. For example, to run `bin/spark-shell` on exactly four cores, use
+In the Spark shell, a special interpreter-aware SparkContext is already 
created for you, in the
+variable called `sc`. Making your own SparkContext will not work. You can set 
which master the
+context connects to using the `MASTER` environment variable, and you can add 
JARs to the classpath
+with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly 
four cores, use
 
 {% highlight bash %}
 $ MASTER=local[4] ./bin/spark-shell
@@ -83,21 +90,16 @@ The master URL passed to Spark can be in one of the 
following formats:
         The host parameter is the hostname of the Mesos master. The port must 
be whichever one the master is configured to use,
         which is 5050 by default.
 </td></tr>
+<tr><td> yarn-client </td><td> Connect to a <a href="running-on-yarn.html"> 
YARN </a> cluster in
+client mode. The cluster location will be inferred based on the local Hadoop 
configuration.
+</td></tr>
+<tr><td> yarn-cluster </td><td> Connect to a <a href="running-on-yarn.html"> 
YARN </a> cluster in
+cluster mode. The cluster location will be inferred based on the local Hadoop 
configuration.
+</td></tr>
 </table>
 
 If no master URL is specified, the spark shell defaults to "local[*]".
 
-For running on YARN, Spark launches an instance of the standalone deploy 
cluster within YARN; see [running on YARN](running-on-yarn.html) for details.
-
-### Deploying Code on a Cluster
-
-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 
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 `bin/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 ./bin/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.
-
 # Resilient Distributed Datasets (RDDs)
 
 Spark revolves around the concept of a _resilient distributed dataset_ (RDD), 
which is a fault-tolerant collection of elements that can be operated on in 
parallel. There are currently two types of RDDs: *parallelized collections*, 
which take an existing Scala collection and run functions on it in parallel, 
and *Hadoop datasets*, which run functions on each record of a file in Hadoop 
distributed file system or any other storage system supported by Hadoop. Both 
types of RDDs can be operated on through the same methods.

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/docs/sql-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 6f616fb..8e98cc0 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -69,7 +69,7 @@ import sqlContext._
 
 </div>
 
-<div data-lang="java"  markdown="1">
+<div data-lang="java" markdown="1">
 
 The entry point into all relational functionality in Spark is the
 
[JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext)
 class, or one

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git 
a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
 
b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index f078d06..2f74965 100644
--- 
a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ 
b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -39,6 +39,9 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, 
SparkContext}
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.util.Utils
 
+/**
+ * An application master that runs the users driver program and allocates 
executors.
+ */
 class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
                         sparkConf: SparkConf) extends Logging {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --git 
a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala 
b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 0179b06..00c7649 100644
--- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -33,7 +33,9 @@ import org.apache.hadoop.yarn.util.{Apps, Records}
 
 import org.apache.spark.{Logging, SparkConf}
 
-
+/**
+ * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's 
alpha API.
+ */
 class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: 
SparkConf)
   extends YarnClientImpl with ClientBase with Logging {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
----------------------------------------------------------------------
diff --git 
a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala 
b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
index 21f1457..ea356f3 100644
--- 
a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
+++ 
b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
@@ -34,6 +34,12 @@ import org.apache.spark.util.{Utils, AkkaUtils}
 import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
 import org.apache.spark.scheduler.SplitInfo
 
+/**
+ * An application master that allocates executors on behalf of a driver that 
is running outside
+ * the cluster.
+ *
+ * This is used only in yarn-client mode.
+ */
 class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, 
sparkConf: SparkConf)
   extends Logging {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------
diff --git 
a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
 
b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index d6d46a5..95f0f9d 100644
--- 
a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ 
b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -54,6 +54,10 @@ object AllocationType extends Enumeration {
 // Note that right now, we assume all node asks as uniform in terms of 
capabilities and priority
 // Refer to 
http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/
 for
 // more info on how we are requesting for containers.
+
+/**
+ * Acquires resources for executors from a ResourceManager and launches 
executors in new containers.
+ */
 private[yarn] class YarnAllocationHandler(
     val conf: Configuration,
     val resourceManager: AMRMProtocol, 

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala
----------------------------------------------------------------------
diff --git 
a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala 
b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala
index 566de71..c00b636 100644
--- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala
@@ -22,16 +22,13 @@ import java.net.{InetAddress, UnknownHostException, URI, 
URISyntaxException}
 import java.nio.ByteBuffer
 
 import scala.collection.JavaConversions._
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
+import scala.collection.mutable.{HashMap, ListBuffer, Map}
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs._
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.fs.permission.FsPermission
 import org.apache.hadoop.mapred.Master
 import org.apache.hadoop.mapreduce.MRJobConfig
-import org.apache.hadoop.net.NetUtils
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.util.StringUtils
 import org.apache.hadoop.yarn.api._
@@ -39,19 +36,18 @@ import 
org.apache.hadoop.yarn.api.ApplicationConstants.Environment
 import org.apache.hadoop.yarn.api.protocolrecords._
 import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.ipc.YarnRPC
-import org.apache.hadoop.yarn.util.{Records, Apps}
-
-import org.apache.spark.{Logging, SparkConf}
-import org.apache.spark.util.Utils
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-
+import org.apache.hadoop.yarn.util.{Apps, Records}
+import org.apache.spark.{Logging, SparkConf, SparkContext}
 
 /**
  * The entry point (starting in Client#main() and Client#run()) for launching 
Spark on YARN. The
- * Client submits an application to the global ResourceManager to launch 
Spark's ApplicationMaster,
- * which will launch a Spark master process and negotiate resources throughout 
its duration.
+ * Client submits an application to the YARN ResourceManager.
+ *
+ * Depending on the deployment mode this will launch one of two application 
master classes:
+ * 1. In standalone mode, it will launch an 
[[org.apache.spark.deploy.yarn.ApplicationMaster]]
+ *      which launches a driver program inside of the cluster.
+ * 2. In client mode, it will launch an 
[[org.apache.spark.deploy.yarn.ExecutorLauncher]] to
+ *      request executors on behalf of a driver running outside of the cluster.
  */
 trait ClientBase extends Logging {
   val args: ClientArguments
@@ -70,7 +66,6 @@ trait ClientBase extends Logging {
   // TODO(harvey): This could just go in ClientArguments.
   def validateArgs() = {
     Map(
-      (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR 
environment variable!",
       ((args.userJar == null && args.amClass == 
classOf[ApplicationMaster].getName) ->
           "Error: You must specify a user jar when running in standalone 
mode!"),
       (args.userClass == null) -> "Error: You must specify a user class!",
@@ -208,7 +203,7 @@ trait ClientBase extends Logging {
     val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
 
     Map(
-      ClientBase.SPARK_JAR -> System.getenv("SPARK_JAR"), ClientBase.APP_JAR 
-> args.userJar,
+      ClientBase.SPARK_JAR -> ClientBase.getSparkJar, ClientBase.APP_JAR -> 
args.userJar,
       ClientBase.LOG4J_PROP -> System.getenv(ClientBase.LOG4J_CONF_ENV_KEY)
     ).foreach { case(destName, _localPath) =>
       val localPath: String = if (_localPath != null) _localPath.trim() else ""
@@ -251,8 +246,10 @@ trait ClientBase extends Logging {
     logInfo("Setting up the launch environment")
 
     val env = new HashMap[String, String]()
+
+    val extraCp = sparkConf.getOption("spark.driver.extraClassPath")
     val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY)
-    ClientBase.populateClasspath(args, yarnConf, sparkConf, log4jConf, env)
+    ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp)
     env("SPARK_YARN_MODE") = "true"
     env("SPARK_YARN_STAGING_DIR") = stagingDir
     env("SPARK_USER") = 
UserGroupInformation.getCurrentUser().getShortUserName()
@@ -268,9 +265,6 @@ trait ClientBase extends Logging {
     YarnSparkHadoopUtil.setEnvFromInputString(env, 
System.getenv("SPARK_YARN_USER_ENV"),
       File.pathSeparator)
 
-    // Add each SPARK_* key to the environment.
-    System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => 
env(k) = v }
-
     env
   }
 
@@ -299,13 +293,13 @@ trait ClientBase extends Logging {
 
     val amMemory = calculateAMMemory(newApp)
 
-    var JAVA_OPTS = ""
+    val JAVA_OPTS = ListBuffer[String]()
 
     // Add Xmx for AM memory
     JAVA_OPTS += "-Xmx" + amMemory + "m"
 
     val tmpDir = new Path(Environment.PWD.$(), 
YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR)
-    JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir
+    JAVA_OPTS += "-Djava.io.tmpdir=" + tmpDir
 
     // TODO: Remove once cpuset version is pushed out.
     // The context is, default gc for server class machines ends up using all 
cores to do gc -
@@ -319,35 +313,48 @@ trait ClientBase extends Logging {
     if (useConcurrentAndIncrementalGC) {
       // In our expts, using (default) throughput collector has severe perf 
ramifications in
       // multi-tenant machines
-      JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
-      JAVA_OPTS += " -XX:+CMSIncrementalMode "
-      JAVA_OPTS += " -XX:+CMSIncrementalPacing "
-      JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
-      JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
+      JAVA_OPTS += "-XX:+UseConcMarkSweepGC"
+      JAVA_OPTS += "-XX:+CMSIncrementalMode"
+      JAVA_OPTS += "-XX:+CMSIncrementalPacing"
+      JAVA_OPTS += "-XX:CMSIncrementalDutyCycleMin=0"
+      JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10"
     }
 
-    if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
-      JAVA_OPTS += " " + env("SPARK_JAVA_OPTS")
+    // TODO: it might be nicer to pass these as an internal environment 
variable rather than
+    // as Java options, due to complications with string parsing of nested 
quotes.
+    if (args.amClass == classOf[ExecutorLauncher].getName) {
+      // If we are being launched in client mode, forward the spark-conf 
options
+      // onto the executor launcher
+      for ((k, v) <- sparkConf.getAll) {
+        JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\""
+      }
+    } else {
+      // If we are being launched in standalone mode, capture and forward any 
spark
+      // system properties (e.g. set by spark-class).
+      for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) {
+        JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\""
+      }
+      sys.props.get("spark.driver.extraJavaOptions").foreach(opts => JAVA_OPTS 
+= opts)
+      sys.props.get("spark.driver.libraryPath").foreach(p => JAVA_OPTS += 
s"-Djava.library.path=$p")
     }
     JAVA_OPTS += ClientBase.getLog4jConfiguration(localResources)
 
     // Command for the ApplicationMaster
-    val commands = List[String](
-      Environment.JAVA_HOME.$() + "/bin/java" +
-        " -server " +
-        JAVA_OPTS +
-        " " + args.amClass +
-        " --class " + args.userClass +
-        " --jar " + args.userJar +
-        userArgsToString(args) +
-        " --executor-memory " + args.executorMemory +
-        " --executor-cores " + args.executorCores +
-        " --num-executors " + args.numExecutors +
-        " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
-        " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
-
-    logInfo("Command for starting the Spark ApplicationMaster: " + commands(0))
-    amContainer.setCommands(commands)
+    val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++
+      JAVA_OPTS ++
+      Seq(args.amClass, "--class", args.userClass, "--jar ", args.userJar,
+        userArgsToString(args),
+        "--executor-memory", args.executorMemory.toString,
+        "--executor-cores", args.executorCores.toString,
+        "--num-executors ", args.numExecutors.toString,
+        "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout",
+        "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+
+    logInfo("Command for starting the Spark ApplicationMaster: " + commands)
+
+    // TODO: it would be nicer to just make sure there are no null commands 
here
+    val printableCommands = commands.map(s => if (s == null) "null" else 
s).toList
+    amContainer.setCommands(printableCommands)
 
     setupSecurityToken(amContainer)
     amContainer
@@ -361,6 +368,8 @@ object ClientBase {
   val LOG4J_CONF_ENV_KEY: String = "SPARK_LOG4J_CONF"
   val LOCAL_SCHEME = "local"
 
+  def getSparkJar = 
sys.env.get("SPARK_JAR").getOrElse(SparkContext.jarOfClass(this.getClass).head)
+
   // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
   def populateHadoopClasspath(conf: Configuration, env: HashMap[String, 
String]) {
     val classpathEntries = Option(conf.getStrings(
@@ -433,10 +442,9 @@ object ClientBase {
     " -Dlog4j.configuration=" + log4jConf
   }
 
-  def populateClasspath(args: ClientArguments, conf: Configuration, sparkConf: 
SparkConf,
-      log4jConf: String, env: HashMap[String, String]) {
-    YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, 
Environment.PWD.$(),
-      File.pathSeparator)
+  def populateClasspath(conf: Configuration, sparkConf: SparkConf, log4jConf: 
String,
+      env: HashMap[String, String], extraClassPath: Option[String] = None) {
+
     if (log4jConf != null) {
       // If a custom log4j config file is provided as a local: URI, add its 
parent directory to the
       // classpath. Note that this only works if the custom config's file name 
is
@@ -448,19 +456,26 @@ object ClientBase {
           File.pathSeparator)
       }
     }
+
+    /** Add entry to the classpath. */
+    def addClasspathEntry(path: String) = Apps.addToEnvironment(env, 
Environment.CLASSPATH.name, path)
+    /** Add entry to the classpath. Interpreted as a path relative to the 
working directory. */
+    def addPwdClasspathEntry(entry: String) = 
addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry)
+
+    extraClassPath.foreach(addClasspathEntry)
+
+    addClasspathEntry(Environment.PWD.$())
     // Normally the users app.jar is last in case conflicts with spark jars
-    val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", 
"false")
-      .toBoolean
-    if (userClasspathFirst) {
-      addUserClasspath(args, env)
-    }
-    addClasspathEntry(System.getenv("SPARK_JAR"), SPARK_JAR, env);
-    ClientBase.populateHadoopClasspath(conf, env)
-    if (!userClasspathFirst) {
-      addUserClasspath(args, env)
+    if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) {
+      addPwdClasspathEntry(APP_JAR)
+      addPwdClasspathEntry(SPARK_JAR)
+      ClientBase.populateHadoopClasspath(conf, env)
+    } else {
+      addPwdClasspathEntry(SPARK_JAR)
+      ClientBase.populateHadoopClasspath(conf, env)
+      addPwdClasspathEntry(APP_JAR)
     }
-    YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name,
-      Environment.PWD.$() + Path.SEPARATOR + "*", File.pathSeparator)
+    addPwdClasspathEntry("*")
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
----------------------------------------------------------------------
diff --git 
a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
 
b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
index 40b3866..7d07f6f 100644
--- 
a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
+++ 
b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
@@ -19,26 +19,18 @@ package org.apache.spark.deploy.yarn
 
 import java.io.File
 import java.net.URI
-import java.nio.ByteBuffer
-import java.security.PrivilegedExceptionAction
 
 import scala.collection.JavaConversions._
 import scala.collection.mutable.HashMap
 
-import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
-import org.apache.hadoop.io.DataOutputBuffer
-import org.apache.hadoop.net.NetUtils
-import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.yarn.api._
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
 import org.apache.hadoop.yarn.api.records._
-import org.apache.hadoop.yarn.api.protocolrecords._
-import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records}
-
-import org.apache.spark.{SparkConf, Logging}
 import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records}
 
+import org.apache.spark.{Logging, SparkConf}
 
 trait ExecutorRunnableUtil extends Logging {
 
@@ -58,8 +50,10 @@ trait ExecutorRunnableUtil extends Logging {
     // Set the JVM memory
     val executorMemoryString = executorMemory + "m"
     JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + 
executorMemoryString + " "
-    if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
-      JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
+
+    // Set extra Java options for the executor, if defined
+    sys.props.get("spark.executor.extraJavaOptions").foreach { opts =>
+      JAVA_OPTS += opts
     }
 
     JAVA_OPTS += " -Djava.io.tmpdir=" +
@@ -162,8 +156,9 @@ trait ExecutorRunnableUtil extends Logging {
   def prepareEnvironment: HashMap[String, String] = {
     val env = new HashMap[String, String]()
 
+    val extraCp = sparkConf.getOption("spark.executor.extraClassPath")
     val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY)
-    ClientBase.populateClasspath(null, yarnConf, sparkConf, log4jConf, env)
+    ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp)
     if (log4jConf != null) {
       env(ClientBase.LOG4J_CONF_ENV_KEY) = log4jConf
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
----------------------------------------------------------------------
diff --git 
a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
 
b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
index 1619188..ce2dde0 100644
--- 
a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
+++ 
b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
@@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster
 
 import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState}
 import org.apache.spark.{SparkException, Logging, SparkContext}
-import org.apache.spark.deploy.yarn.{Client, ClientArguments}
+import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher}
 import org.apache.spark.scheduler.TaskSchedulerImpl
 
 import scala.collection.mutable.ArrayBuffer
@@ -54,7 +54,7 @@ private[spark] class YarnClientSchedulerBackend(
       "--class", "notused",
       "--jar", null,
       "--args", hostport,
-      "--am-class", "org.apache.spark.deploy.yarn.ExecutorLauncher"
+      "--am-class", classOf[ExecutorLauncher].getName
     )
 
     // process any optional arguments, given either as environment variables

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git 
a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
 
b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index b225be6..90e8071 100644
--- 
a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ 
b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -42,6 +42,9 @@ import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.util.Utils
 
 
+/**
+ * An application master that runs the user's driver program and allocates 
executors.
+ */
 class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
                         sparkConf: SparkConf) extends Logging {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --git 
a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala 
b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 77eb127..2e2fb5d 100644
--- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -34,9 +34,7 @@ import org.apache.spark.{Logging, SparkConf}
 
 
 /**
- * The entry point (starting in Client#main() and Client#run()) for launching 
Spark on YARN. The
- * Client submits an application to the global ResourceManager to launch 
Spark's ApplicationMaster,
- * which will launch a Spark master process and negotiate resources throughout 
its duration.
+ * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's 
stable API.
  */
 class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: 
SparkConf)
   extends YarnClientImpl with ClientBase with Logging {

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
----------------------------------------------------------------------
diff --git 
a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
 
b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
index 67ed591..a14bb37 100644
--- 
a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
+++ 
b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
@@ -35,6 +35,12 @@ import org.apache.spark.scheduler.SplitInfo
 import org.apache.hadoop.yarn.client.api.AMRMClient
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
 
+/**
+ * An application master that allocates executors on behalf of a driver that 
is running outside
+ * the cluster.
+ *
+ * This is used only in yarn-client mode.
+ */
 class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, 
sparkConf: SparkConf)
   extends Logging {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fb98488f/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------
diff --git 
a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
 
b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index 4fafae1..a979fe4 100644
--- 
a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ 
b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -56,6 +56,10 @@ object AllocationType extends Enumeration {
 // Note that right now, we assume all node asks as uniform in terms of 
capabilities and priority
 // Refer to 
http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/
 for
 // more info on how we are requesting for containers.
+
+/**
+ * Acquires resources for executors from a ResourceManager and launches 
executors in new containers.
+ */
 private[yarn] class YarnAllocationHandler(
     val conf: Configuration,
     val amClient: AMRMClient[ContainerRequest],

Reply via email to