diff --git a/.gitignore b/.gitignore
index b3c4363af038e21e670ea5530a9ddfe0b8de8c69..399362f7d3e46c9e225c846659f850e71bb69c94 100644
--- a/.gitignore
+++ b/.gitignore
@@ -1,6 +1,8 @@
 *~
 *.swp
+*.ipr
 *.iml
+*.iws
 .idea/
 .settings
 .cache
diff --git a/README.md b/README.md
index 873ec9882f9a8befd103363214338dfb2c6f2ad7..6daa4633ae5762fd6cbb335dd0ac37971c0b41c6 100644
--- a/README.md
+++ b/README.md
@@ -19,14 +19,14 @@ which can be obtained [here](http://www.scala-sbt.org). To build Spark and its e
 
 Once you've built Spark, the easiest way to start using it is the shell:
 
-    ./spark-shell
+    ./bin/spark-shell
 
-Or, for the Python API, the Python shell (`./pyspark`).
+Or, for the Python API, the Python shell (`./bin/pyspark`).
 
 Spark also comes with several sample programs in the `examples` directory.
-To run one of them, use `./run-example <class> <params>`. For example:
+To run one of them, use `./bin/run-example <class> <params>`. For example:
 
-    ./run-example org.apache.spark.examples.SparkLR local[2]
+    ./bin/run-example org.apache.spark.examples.SparkLR local[2]
 
 will run the Logistic Regression example locally on 2 CPUs.
 
diff --git a/assembly/pom.xml b/assembly/pom.xml
index c19e4bc9863492dd47dbcbbd5aad981966012a76..9b70812c64b1ed5171a19d5858997aaaa18f95d2 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -124,7 +124,17 @@
 
   <profiles>
     <profile>
-      <id>hadoop2-yarn</id>
+      <id>yarn-alpha</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.spark</groupId>
+          <artifactId>spark-yarn-alpha_${scala.binary.version}</artifactId>
+          <version>${project.version}</version>
+        </dependency>
+      </dependencies>
+    </profile>
+    <profile>
+      <id>yarn</id>
       <dependencies>
         <dependency>
           <groupId>org.apache.spark</groupId>
diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml
index 47d3fa93d07657dc2401a4f2a8365b0a83fb8edb..6af383db65d47fae74fc85063eb5bd06a490a98c 100644
--- a/assembly/src/main/assembly/assembly.xml
+++ b/assembly/src/main/assembly/assembly.xml
@@ -39,23 +39,20 @@
     </fileSet>
     <fileSet>
       <directory>
-        ${project.parent.basedir}/bin/
+        ${project.parent.basedir}/sbin/
       </directory>
-      <outputDirectory>/bin</outputDirectory>
+      <outputDirectory>/sbin</outputDirectory>
       <includes>
         <include>**/*</include>
       </includes>
     </fileSet>
     <fileSet>
       <directory>
-        ${project.parent.basedir}
+        ${project.parent.basedir}/bin/
       </directory>
       <outputDirectory>/bin</outputDirectory>
       <includes>
-        <include>run-example*</include>
-        <include>spark-class*</include>
-        <include>spark-shell*</include>
-        <include>spark-executor*</include>
+        <include>**/*</include>
       </includes>
     </fileSet>
   </fileSets>
diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd
index 9e3e10ecaad982f8a64a510168f2340ab3b07020..4f60bff19cb93adcef2355f3a22aa89e38b9c51c 100644
--- a/bin/compute-classpath.cmd
+++ b/bin/compute-classpath.cmd
@@ -29,7 +29,7 @@ rem Load environment variables from conf\spark-env.cmd, if it exists
 if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
 
 rem Build up classpath
-set CLASSPATH=%SPARK_CLASSPATH%;%FWDIR%conf
+set CLASSPATH=%FWDIR%conf
 if exist "%FWDIR%RELEASE" (
   for %%d in ("%FWDIR%jars\spark-assembly*.jar") do (
     set ASSEMBLY_JAR=%%d
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
index 40555089fcfe4608f53130c0744094a06090c2c4..0c823104215aa353f8087af65a2d2b6d29d20d21 100755
--- a/bin/compute-classpath.sh
+++ b/bin/compute-classpath.sh
@@ -26,7 +26,7 @@ SCALA_VERSION=2.10
 FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
diff --git a/pyspark b/bin/pyspark
similarity index 96%
rename from pyspark
rename to bin/pyspark
index 1d003e2a008feca6c4044ffde8b688907e6ffff4..f97dfa7e2ff464890202bcdef0ad9079a489e58b 100755
--- a/pyspark
+++ b/bin/pyspark
@@ -18,7 +18,7 @@
 #
 
 # Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Export this as SPARK_HOME
 export SPARK_HOME="$FWDIR"
@@ -37,7 +37,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then
 fi
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
diff --git a/pyspark.cmd b/bin/pyspark.cmd
similarity index 100%
rename from pyspark.cmd
rename to bin/pyspark.cmd
diff --git a/pyspark2.cmd b/bin/pyspark2.cmd
similarity index 98%
rename from pyspark2.cmd
rename to bin/pyspark2.cmd
index 21f9a3438878ad8e99b062ee2ee171b1bb587dbb..95791095ec932e175f8b2b0e137cbb485219d2b9 100644
--- a/pyspark2.cmd
+++ b/bin/pyspark2.cmd
@@ -20,7 +20,7 @@ rem
 set SCALA_VERSION=2.10
 
 rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
 
 rem Export this as SPARK_HOME
 set SPARK_HOME=%FWDIR%
diff --git a/run-example b/bin/run-example
similarity index 97%
rename from run-example
rename to bin/run-example
index fbd81fe6f331fb70da03cc67236f2b4c3b5a6a72..dfb4bf7baf0c2349037e42b7968658c4cd5edb14 100755
--- a/run-example
+++ b/bin/run-example
@@ -25,13 +25,13 @@ esac
 SCALA_VERSION=2.10
 
 # Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Export this as SPARK_HOME
 export SPARK_HOME="$FWDIR"
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
diff --git a/run-example.cmd b/bin/run-example.cmd
similarity index 100%
rename from run-example.cmd
rename to bin/run-example.cmd
diff --git a/run-example2.cmd b/bin/run-example2.cmd
similarity index 97%
rename from run-example2.cmd
rename to bin/run-example2.cmd
index d4ad98d6e7ba85510ddec713e6d89f51073a4ab9..6861334cb07e505d01ce90e6eba968c7e5f43892 100644
--- a/run-example2.cmd
+++ b/bin/run-example2.cmd
@@ -20,7 +20,7 @@ rem
 set SCALA_VERSION=2.10
 
 rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
 
 rem Export this as SPARK_HOME
 set SPARK_HOME=%FWDIR%
@@ -49,7 +49,7 @@ if "x%SPARK_EXAMPLES_JAR%"=="x" (
 
 rem Compute Spark classpath using external script
 set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
+call "%FWDIR%sbin\compute-classpath.cmd"
 set DONT_PRINT_CLASSPATH=0
 set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH%
 
diff --git a/spark-class b/bin/spark-class
similarity index 97%
rename from spark-class
rename to bin/spark-class
index 254ddee04ae7bf50349ed676eb806df3ee825fb5..49b0bef0bd3daf701d453651b3ce3e7c7de55813 100755
--- a/spark-class
+++ b/bin/spark-class
@@ -25,13 +25,13 @@ esac
 SCALA_VERSION=2.10
 
 # Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 # Export this as SPARK_HOME
 export SPARK_HOME="$FWDIR"
 
 # Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
@@ -92,7 +92,7 @@ JAVA_OPTS="$OUR_JAVA_OPTS"
 JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
 JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM"
 # Load extra JAVA_OPTS from conf/java-opts, if it exists
-if [ -e $FWDIR/conf/java-opts ] ; then
+if [ -e "$FWDIR/conf/java-opts" ] ; then
   JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
 fi
 export JAVA_OPTS
diff --git a/spark-class.cmd b/bin/spark-class.cmd
similarity index 100%
rename from spark-class.cmd
rename to bin/spark-class.cmd
diff --git a/spark-class2.cmd b/bin/spark-class2.cmd
similarity index 98%
rename from spark-class2.cmd
rename to bin/spark-class2.cmd
index dc9dadf356e2626f251abac3fec0925147c03a92..460e6614766f8220e02efd2dc037f461df8b98e0 100644
--- a/spark-class2.cmd
+++ b/bin/spark-class2.cmd
@@ -20,7 +20,7 @@ rem
 set SCALA_VERSION=2.10
 
 rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
 
 rem Export this as SPARK_HOME
 set SPARK_HOME=%FWDIR%
@@ -73,7 +73,7 @@ for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar"
 
 rem Compute classpath using external script
 set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
+call "%FWDIR%sbin\compute-classpath.cmd"
 set DONT_PRINT_CLASSPATH=0
 set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR%
 
diff --git a/spark-shell b/bin/spark-shell
similarity index 93%
rename from spark-shell
rename to bin/spark-shell
index d20af0fb39f2b6ae39f3d085734ac511d0ecb8b1..e6885b51ef5676c834c6c13340a95b44875a07de 100755
--- a/spark-shell
+++ b/bin/spark-shell
@@ -32,7 +32,7 @@ esac
 # Enter posix mode for bash
 set -o posix
 
-FWDIR="`dirname $0`"
+FWDIR="$(cd `dirname $0`/..; pwd)"
 
 for o in "$@"; do
   if [ "$1" = "-c" -o "$1" = "--cores" ]; then
@@ -90,10 +90,10 @@ if $cygwin; then
     # "Backspace sends ^H" setting in "Keys" section of the Mintty options
     # (see https://github.com/sbt/sbt/issues/562).
     stty -icanon min 1 -echo > /dev/null 2>&1
-    $FWDIR/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@"
+    $FWDIR/bin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@"
     stty icanon echo > /dev/null 2>&1
 else
-    $FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
+    $FWDIR/bin/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
 fi
 
 # record the exit status lest it be overwritten:
diff --git a/spark-shell.cmd b/bin/spark-shell.cmd
similarity index 87%
rename from spark-shell.cmd
rename to bin/spark-shell.cmd
index 3e52bf835e5beac773336bf3ec160e3ad0a5d703..23973e3e3dd432c0e19dc48332ebbf0f6217a3ea 100644
--- a/spark-shell.cmd
+++ b/bin/spark-shell.cmd
@@ -17,6 +17,7 @@ rem See the License for the specific language governing permissions and
 rem limitations under the License.
 rem
 
-set FWDIR=%~dp0
+rem Find the path of sbin
+set SBIN=%~dp0..\sbin\
 
-cmd /V /E /C %FWDIR%spark-class2.cmd org.apache.spark.repl.Main %*
+cmd /V /E /C %SBIN%spark-class2.cmd org.apache.spark.repl.Main %*
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 08811520cfefc76b023d4cf6dd96c221bd0f0001..d46fceba8918a1cc76deea05b938315072f86ff3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -127,7 +127,7 @@ private[spark] class CoarseMesosSchedulerBackend(
       CoarseGrainedSchedulerBackend.ACTOR_NAME)
     val uri = conf.get("spark.executor.uri", null)
     if (uri == null) {
-      val runScript = new File(sparkHome, "spark-class").getCanonicalPath
+      val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath
       command.setValue(
         "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format(
           runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
@@ -136,7 +136,7 @@ private[spark] class CoarseMesosSchedulerBackend(
       // glob the directory "correctly".
       val basename = uri.split('/').last.split('.').head
       command.setValue(
-        "cd %s*; ./spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
+        "cd %s*; ./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
           .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
       command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
     }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index bb278fb155c79de55e965eaace32dcf4a0407524..ae8d527352f733b5f8e002372b0a422e0b33c8c6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -102,12 +102,12 @@ private[spark] class MesosSchedulerBackend(
       .setEnvironment(environment)
     val uri = sc.conf.get("spark.executor.uri", null)
     if (uri == null) {
-      command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath)
+      command.setValue(new File(sparkHome, "/sbin/spark-executor").getCanonicalPath)
     } else {
       // Grab everything to the first '.'. We'll use that and '*' to
       // glob the directory "correctly".
       val basename = uri.split('/').last.split('.').head
-      command.setValue("cd %s*; ./spark-executor".format(basename))
+      command.setValue("cd %s*; ./sbin/spark-executor".format(basename))
       command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
     }
     val memory = Resource.newBuilder()
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 58d47a201dd31fab5fa7f0cb32a0065746f8f27d..6ba15187d9f637a496513f58bbc0a9b276e60bf8 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -27,7 +27,7 @@ import org.apache.spark.scheduler.SchedulingMode
 /**
  * Continuously generates jobs that expose various features of the WebUI (internal testing tool).
  *
- * Usage: ./run spark.ui.UIWorkloadGenerator [master]
+ * Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]
  */
 private[spark] object UIWorkloadGenerator {
 
@@ -36,7 +36,7 @@ private[spark] object UIWorkloadGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
-      println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
+      println("usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
       System.exit(1)
     }
 
diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
index 7e1e55fa3b29962426d4289399e2195ea63f107e..fb89537258542245c0d0584f3d69c33f9204a875 100644
--- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@ -36,7 +36,7 @@ class DriverSuite extends FunSuite with Timeouts {
     forAll(masters) { (master: String) =>
       failAfter(60 seconds) {
         Utils.executeAndGetOutput(
-          Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master),
+          Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master),
           new File(sparkHome), 
           Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
       }
diff --git a/kmeans_data.txt b/data/kmeans_data.txt
similarity index 100%
rename from kmeans_data.txt
rename to data/kmeans_data.txt
diff --git a/lr_data.txt b/data/lr_data.txt
similarity index 100%
rename from lr_data.txt
rename to data/lr_data.txt
diff --git a/pagerank_data.txt b/data/pagerank_data.txt
similarity index 100%
rename from pagerank_data.txt
rename to data/pagerank_data.txt
diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md
index de001e6c52f85e39c6ca62afd7e699039221ec35..c4f1f6d6ade8cc0f2a870b4eb6fb5f1735ca9323 100644
--- a/docs/bagel-programming-guide.md
+++ b/docs/bagel-programming-guide.md
@@ -157,8 +157,8 @@ trait Message[K] {
 
 # Where to Go from Here
 
-Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark; e.g.:
+Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `bin/run-example` script included in Spark; e.g.:
 
-    ./run-example org.apache.spark.examples.bagel.WikipediaPageRank
+    ./bin/run-example org.apache.spark.examples.bagel.WikipediaPageRank
 
 Each example program prints usage help when run without any arguments.
diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md
index c709001632261c1d0443fdcb0ac2d94211d0add3..b9ff0af76f6478f49fe37a0e6a360796eb1d967a 100644
--- a/docs/building-with-maven.md
+++ b/docs/building-with-maven.md
@@ -37,20 +37,16 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit
     # Cloudera CDH 4.2.0 with MapReduce v1
     $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package
 
-For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "hadoop2-yarn" profile and set the "yarn.version" property:
+For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property:
 
     # Apache Hadoop 2.0.5-alpha
-    $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package
+    $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package
 
     # Cloudera CDH 4.2.0 with MapReduce v2
-    $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
+    $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
 
-Hadoop versions 2.2.x and newer can be built by setting the ```new-yarn``` and the ```yarn.version``` as follows:
- 
-    # Apache Hadoop 2.2.X and newer
-    $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn
-
-The build process handles Hadoop 2.2.x as a special case that uses the directory ```new-yarn```, which supports the new YARN API. Furthermore, for this version, the build depends on artifacts published by the spark-project to enable Akka 2.0.5 to work with protobuf 2.5. 
+    # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer
+    $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package
 
 ## Spark Tests in Maven ##
 
diff --git a/docs/index.md b/docs/index.md
index 5278e33e1c054459568d3b57b5c2a92dc6c4505b..bf8d1c3375e8526e1f1df14e22cf35918346eff3 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -24,9 +24,9 @@ For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VE
 # Running the Examples and Shell
 
 Spark comes with several sample programs in the `examples` directory.
-To run one of the samples, use `./run-example <class> <params>` in the top-level Spark directory
-(the `run-example` script sets up the appropriate paths and launches that program).
-For example, try `./run-example org.apache.spark.examples.SparkPi local`.
+To run one of the samples, use `./bin/run-example <class> <params>` in the top-level Spark directory
+(the `bin/run-example` script sets up the appropriate paths and launches that program).
+For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`.
 Each example prints usage help when run with no parameters.
 
 Note that all of the sample programs take a `<master>` parameter specifying the cluster URL
@@ -34,8 +34,8 @@ to connect to. This can be a [URL for a distributed cluster](scala-programming-g
 or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using
 `local` for testing.
 
-Finally, you can run Spark interactively through modified versions of the Scala shell (`./spark-shell`) or
-Python interpreter (`./pyspark`). These are a great way to learn the framework.
+Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or
+Python interpreter (`./bin/pyspark`). These are a great way to learn the framework.
 
 # Launching on a Cluster
 
diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md
index 53085cc6719b07ea185ab1c8e39a9d987cafa315..07732fa1229f3bfef0ad729121efadd25c1f471e 100644
--- a/docs/java-programming-guide.md
+++ b/docs/java-programming-guide.md
@@ -190,9 +190,9 @@ We hope to generate documentation with Java-style syntax in the future.
 
 Spark includes several sample programs using the Java API in
 [`examples/src/main/java`](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/examples).  You can run them by passing the class name to the
-`run-example` script included in Spark; for example:
+`bin/run-example` script included in Spark; for example:
 
-    ./run-example org.apache.spark.examples.JavaWordCount
+    ./bin/run-example org.apache.spark.examples.JavaWordCount
 
 Each example program prints usage help when run
 without any arguments.
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md
index c1ff9c417c35396ffc71bf7cf56faaad9c136af9..95537ef18520fc778f51fbf26857c70d0440d19a 100644
--- a/docs/mllib-guide.md
+++ b/docs/mllib-guide.md
@@ -87,7 +87,7 @@ svmAlg.optimizer.setNumIterations(200)
 val modelL1 = svmAlg.run(parsedData)
 {% endhighlight %}
 
-Both of the code snippets above can be executed in `spark-shell` to generate a
+Both of the code snippets above can be executed in `bin/spark-shell` to generate a
 classifier for the provided dataset.
 
 Available algorithms for binary classification:
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 45a6250ecab9deab9a69f1d5cae025588fd9323e..5d48cb676a1422480811a76b7079b52745e99a70 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -47,7 +47,7 @@ PySpark will automatically ship these functions to workers, along with any objec
 Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers.
 The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers.
 
-In addition, PySpark fully supports interactive use---simply run `./pyspark` to launch an interactive shell.
+In addition, PySpark fully supports interactive use---simply run `./bin/pyspark` to launch an interactive shell.
 
 
 # Installing and Configuring PySpark
@@ -60,17 +60,17 @@ By default, PySpark requires `python` to be available on the system `PATH` and u
 
 All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported.
 
-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`.
+Standalone PySpark applications should be run using the `bin/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 `bin/pyspark` package to the `PYTHONPATH`.
 
 
 # Interactive Use
 
-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:
+The `bin/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 assembly
-$ ./pyspark
+$ ./bin/pyspark
 {% endhighlight %}
 
 The Python shell can be used explore data interactively and is a simple way to learn the API:
@@ -82,35 +82,35 @@ 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 applications locally on a single core.
+By default, the `bin/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):
+For example, to use the `bin/pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
 
 {% highlight bash %}
-$ MASTER=spark://IP:PORT ./pyspark
+$ MASTER=spark://IP:PORT ./bin/pyspark
 {% endhighlight %}
 
 Or, to use four cores on the local machine:
 
 {% highlight bash %}
-$ MASTER=local[4] ./pyspark
+$ MASTER=local[4] ./bin/pyspark
 {% endhighlight %}
 
 
 ## IPython
 
 It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter.
-To do this, set the `IPYTHON` variable to `1` when running `pyspark`:
+To do this, set the `IPYTHON` variable to `1` when running `bin/pyspark`:
 
 {% highlight bash %}
-$ IPYTHON=1 ./pyspark
+$ IPYTHON=1 ./bin/pyspark
 {% endhighlight %}
 
 Alternatively, you can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch
 the [IPython Notebook](http://ipython.org/notebook.html) with PyLab graphing support:
 
 {% highlight bash %}
-$ IPYTHON_OPTS="notebook --pylab inline" ./pyspark
+$ IPYTHON_OPTS="notebook --pylab inline" ./bin/pyspark
 {% endhighlight %}
 
 IPython also works on a cluster or on multiple cores if you set the `MASTER` environment variable.
@@ -118,7 +118,7 @@ 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`.
+PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/pyspark`.
 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:
@@ -153,6 +153,6 @@ Many of the methods also contain [doctests](http://docs.python.org/2/library/doc
 PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/incubator-spark/tree/master/python/examples).
 You can run them by passing the files to `pyspark`; e.g.:
 
-    ./pyspark python/examples/wordcount.py
+    ./bin/pyspark python/examples/wordcount.py
 
 Each program prints usage help when run without arguments.
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 1882ea75c03ad38c8494ff46b8a5c7e6d03c2de9..9b9261cfff8eabdf9083449cdae08cd821663f1f 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -20,7 +20,7 @@ $ sbt assembly
 ## Basics
 
 Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively.
-Start the shell by running `./spark-shell` in the Spark directory.
+Start the shell by running `./bin/spark-shell` in the Spark directory.
 
 Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory:
 
@@ -99,7 +99,7 @@ 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 `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.
@@ -277,11 +277,11 @@ We can pass Python functions to Spark, which are automatically serialized along
 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 `pyspark` script:
+We can run this application using the `bin/pyspark` script:
 
 {% highlight python %}
 $ cd $SPARK_HOME
-$ ./pyspark SimpleApp.py
+$ ./bin/pyspark SimpleApp.py
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight python %}
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index aded6430b376742ac4a1d76491c469f7bad836bf..a35e003cdc1ee5130f0e472a520a98499f40b716 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -54,7 +54,7 @@ There are two scheduler mode that can be used to launch spark application on YAR
 
 The command to launch the YARN Client is as follows:
 
-    SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./spark-class org.apache.spark.deploy.yarn.Client \
+    SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./bin/spark-class org.apache.spark.deploy.yarn.Client \
       --jar <YOUR_APP_JAR_FILE> \
       --class <APP_MAIN_CLASS> \
       --args <APP_MAIN_ARGUMENTS> \
@@ -79,7 +79,7 @@ For example:
 
     # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example
     $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
-        ./spark-class org.apache.spark.deploy.yarn.Client \
+        ./bin/spark-class org.apache.spark.deploy.yarn.Client \
           --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
           --class org.apache.spark.examples.SparkPi \
           --args yarn-standalone \
@@ -107,17 +107,15 @@ For example:
 
     SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
     SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
-    ./run-example org.apache.spark.examples.SparkPi yarn-client
+    ./bin/run-example org.apache.spark.examples.SparkPi yarn-client
 
 
     SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
     SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
-    MASTER=yarn-client ./spark-shell
+    MASTER=yarn-client ./bin/spark-shell
 
 # Building Spark for Hadoop/YARN 2.2.x
 
-Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5) that is not compatible with the Akka version (2.0.5) that Spark uses. Therefore, if the Hadoop version (e.g. set through ```SPARK_HADOOP_VERSION```) starts with 2.2.0 or higher then the build process will depend on Akka artifacts distributed by the Spark project compatible with Protobuf 2.5. Furthermore, the build process then uses the directory ```new-yarn``` (instead of ```yarn```), which supports the new YARN API. The build process should seamlessly work out of the box. 
-
 See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process.
 
 # Important Notes
@@ -126,4 +124,3 @@ See [Building Spark with Maven](building-with-maven.html) for instructions on ho
 - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored.
 - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN.
 - The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files.
-- YARN 2.2.x users cannot simply depend on the Spark packages without building Spark, as the published Spark artifacts are compiled to work with the pre 2.2 API. Those users must build Spark and publish it locally.  
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index fe1bca789f89ddc7911c3047332ea46310b0bbce..3d0e8923d54279f9e319d55211d4f379c67ed971 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -13,7 +13,7 @@ At a high level, every Spark application consists of a *driver program* that run
 
 A second abstraction in Spark is *shared variables* that can be used in parallel operations. By default, when Spark runs a function in parallel as a set of tasks on different nodes, it ships a copy of each variable used in the function to each task. Sometimes, a variable needs to be shared across tasks, or between tasks and the driver program. Spark supports two types of shared variables: *broadcast variables*, which can be used to cache a value in memory on all nodes, and *accumulators*, which are variables that are only "added" to, such as counters and sums.
 
-This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `spark-shell` script. We highly recommend doing that to follow along!
+This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `bin/spark-shell` script. We highly recommend doing that to follow along!
 
 # Linking with Spark
 
@@ -54,16 +54,16 @@ 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.
 
-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 `spark-shell` on 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 four cores, use
 
 {% highlight bash %}
-$ MASTER=local[4] ./spark-shell
+$ MASTER=local[4] ./bin/spark-shell
 {% endhighlight %}
 
 Or, to also add `code.jar` to its classpath, use:
 
 {% highlight bash %}
-$ MASTER=local[4] ADD_JARS=code.jar ./spark-shell
+$ MASTER=local[4] ADD_JARS=code.jar ./bin/spark-shell
 {% endhighlight %}
 
 ### Master URLs
@@ -95,7 +95,7 @@ If you want to run your application on a cluster, you will need to specify the t
 * `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 `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.
+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)
 
@@ -366,9 +366,9 @@ res2: Int = 10
 # Where to Go from Here
 
 You can see some [example Spark programs](http://spark.incubator.apache.org/examples.html) on the Spark website.
-In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark; for example:
+In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `bin/run-example` script included in Spark; for example:
 
-    ./run-example org.apache.spark.examples.SparkPi
+    ./bin/run-example org.apache.spark.examples.SparkPi
 
 Each example program prints usage help when run without any arguments.
 
diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md
index d6315d97f4d590d47c6e85cb108bc70368ea2323..11c51d5cde7c929fc44a6d6f8368d5e657737b2a 100644
--- a/docs/spark-debugger.md
+++ b/docs/spark-debugger.md
@@ -39,7 +39,7 @@ where `path/to/event-log` is where you want the event log to go relative to `$SP
 
 ### Loading the event log into the debugger
 
-1. Run a Spark shell with `MASTER=<i>host</i> ./spark-shell`.
+1. Run a Spark shell with `MASTER=<i>host</i> ./bin/spark-shell`.
 2. Use `EventLogReader` to load the event log as follows:
     {% highlight scala %}
 spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log"))
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index f7f0b789081d4005fcb140dda775032ed5286efa..c851833a18dd9252c507476eb5a745fe790c1092 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -20,7 +20,7 @@ then modify `conf/spark-env.sh` in the `dist/` directory before deploying to all
 
 You can start a standalone master server by executing:
 
-    ./bin/start-master.sh
+    ./sbin/start-master.sh
 
 Once started, the master will print out a `spark://HOST:PORT` URL for itself, which you can use to connect workers to it,
 or pass as the "master" argument to `SparkContext`. You can also find this URL on
@@ -28,7 +28,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by
 
 Similarly, you can start one or more workers and connect them to the master via:
 
-    ./spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT
+    ./bin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT
 
 Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default).
 You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS).
@@ -70,12 +70,12 @@ To launch a Spark standalone cluster with the launch scripts, you need to create
 
 Once you've set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
 
-- `bin/start-master.sh` - Starts a master instance on the machine the script is executed on.
-- `bin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
-- `bin/start-all.sh` - Starts both a master and a number of slaves as described above.
-- `bin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
-- `bin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
-- `bin/stop-all.sh` - Stops both the master and the slaves as described above.
+- `sbin/start-master.sh` - Starts a master instance on the machine the script is executed on.
+- `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
+- `sbin/start-all.sh` - Starts both a master and a number of slaves as described above.
+- `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
+- `sbin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
+- `sbin/stop-all.sh` - Stops both the master and the slaves as described above.
 
 Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine.
 
@@ -143,9 +143,9 @@ constructor](scala-programming-guide.html#initializing-spark).
 
 To run an interactive Spark shell against the cluster, run the following command:
 
-    MASTER=spark://IP:PORT ./spark-shell
+    MASTER=spark://IP:PORT ./bin/spark-shell
 
-Note that if you are running spark-shell from one of the spark cluster machines, the `spark-shell` script will
+Note that if you are running spark-shell from one of the spark cluster machines, the `bin/spark-shell` script will
 automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`.
 
 You can also pass an option `-c <numCores>` to control the number of cores that spark-shell uses on the cluster.
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 6f97db6a3e73395407c7b98e1697296311b32d97..1c9ece62707818cfc403103269a81c2448dc3552 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -245,7 +245,7 @@ $ nc -lk 9999
 Then, in a different terminal, you can start NetworkWordCount by using
 
 {% highlight bash %}
-$ ./run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
 {% endhighlight %}
 
 This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen.
@@ -283,7 +283,7 @@ Time: 1357008430000 ms
 </td>
 </table>
 
-You can find more examples in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
+You can find more examples in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./bin/run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
 
 # DStream Persistence
 Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`.
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index a2b0e7e7f47480f2bfd42fbc4951067a161c82d1..d82a1e1490cc0e7125030a7b93a10bfd1ddca208 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -436,7 +436,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
 def setup_standalone_cluster(master, slave_nodes, opts):
   slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes])
   ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips))
-  ssh(master, opts, "/root/spark/bin/start-all.sh")
+  ssh(master, opts, "/root/spark/sbin/start-all.sh")
 
 def setup_spark_cluster(master, opts):
   ssh(master, opts, "chmod u+x spark-ec2/setup.sh")
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
index 22994fb2ec71c70261b604b569124e0e0330fc1c..75b588e4b85a927a7326ba9c798d620349cea5fe 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
@@ -41,7 +41,7 @@ import scala.Tuple2;
  *   <numThreads> is the number of threads the kafka consumer should use
  *
  * Example:
- *    `./run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
+ *    `./bin/run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
  *    zoo03 my-consumer-group topic1,topic2 1`
  */
 
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
index 2402409e6ef87552dc3e8835584180cfa199f9f7..3641517934b74b54410256fb217cc49b29c8bee9 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
@@ -134,9 +134,9 @@ object FeederActor {
  *   <hostname> and <port> describe the AkkaSystem that Spark Sample feeder is running on.
  *
  * To run this example locally, you may run Feeder Actor as
- *    `$ ./run-example spark.streaming.examples.FeederActor 127.0.1.1 9999`
+ *    `$ ./bin/run-example spark.streaming.examples.FeederActor 127.0.1.1 9999`
  * and then run the example
- *    `$ ./run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
+ *    `$ ./bin/run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
  */
 object ActorWordCount {
   def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
index bc8564b3ba080aa3ed3f367fb1211736d9035411..61be1ce4b172605f0edcaae815f860c68f5e07db 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
@@ -28,7 +28,7 @@ import org.apache.spark.streaming.StreamingContext._
  *   <directory> is the directory that Spark Streaming will use to find and read new text files.
  *
  * To run this on your local machine on directory `localdir`, run this example
- *    `$ ./run-example spark.streaming.examples.HdfsWordCount local[2] localdir`
+ *    `$ ./bin/run-example spark.streaming.examples.HdfsWordCount local[2] localdir`
  * Then create a text file in `localdir` and the words in the file will get counted.
  */
 object HdfsWordCount {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
index 570ba4c81a1d2c3cc00144727f04bc8a45dc071f..8dc8a3531a014c97db8cb6e3eeb725e2c1a81ddd 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
@@ -35,7 +35,7 @@ import org.apache.spark.streaming.util.RawTextHelper._
  *   <numThreads> is the number of threads the kafka consumer should use
  *
  * Example:
- *    `./run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
+ *    `./bin/run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
  */
 object KafkaWordCount {
   def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
index ff332a02821290f98faebcbf6a06ec6c091bc31b..ea138f55e872ba9ad1036eb4446b378544417635 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
@@ -79,9 +79,9 @@ object MQTTPublisher {
  *   <MqttbrokerUrl> and <topic> describe where Mqtt publisher is running.
  *
  * To run this example locally, you may run publisher as
- *    `$ ./run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
  * and run the example as
- *    `$ ./run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
+ *    `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
  */
 object MQTTWordCount {
 
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
index e2487dca5f2cc894098c2e1bb08aee761c85bbea..ce8df8c5023ee19b9b9ee4886a4bb29f566a4e82 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
@@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._
  * To run this on your local machine, you need to first run a Netcat server
  *    `$ nc -lk 9999`
  * and then run the example
- *    `$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
+ *    `$ ./bin/run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
  */
 object NetworkWordCount {
   def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
index cb30c4edb30ce408faff73dc504c50f3a426f882..e55d71edfcdc922c9d3ffd5a2628a50497ffd018 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
@@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._
  * To run this on your local machine, you need to first run a Netcat server
  *    `$ nc -lk 9999`
  * and then run the example
- *    `$ ./run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999`
+ *    `$ ./bin/run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999`
  */
 object StatefulNetworkWordCount {
   def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
index e83ce78aa5f1b6e602f7891727ab811f21c4a13c..2948aa7cc494436181a20a755728bc70432b9bad 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
@@ -62,9 +62,9 @@ object SimpleZeroMQPublisher {
  *   <zeroMQurl> and <topic> describe where zeroMq publisher is running.
  *
  * To run this example locally, you may run publisher as
- *    `$ ./run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
+ *    `$ ./bin/run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
  * and run the example as
- *    `$ ./run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
+ *    `$ ./bin/run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
  */
 object ZeroMQWordCount {
   def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
index de70c50473558a5cf07df0eceacbef6f55fb785b..1a40fdb9a3153ea906eb45496e40e51421b97fff 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -39,8 +39,8 @@ object PageView extends Serializable {
 /** Generates streaming events to simulate page views on a website.
   *
   * This should be used in tandem with PageViewStream.scala. Example:
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+  * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+  * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
   *
   * When running this, you may want to set the root logging level to ERROR in
   * conf/log4j.properties to reduce the verbosity of the output.
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
index 8282cc9269c13fc287e0e7dc38c0e921139efce5..968b578487a5eafec9986fd6dce1d74b8238f5b4 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -25,8 +25,8 @@ import org.apache.spark.SparkContext._
   * operators available in Spark streaming.
   *
   * This should be used in tandem with PageViewStream.scala. Example:
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
-  * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+  * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+  * $ ./bin/run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
   */
 object PageViewStream {
   def main(args: Array[String]) {
diff --git a/make-distribution.sh b/make-distribution.sh
index 8ae8a4cf742a161451ca13c12c08282b17f69382..6c466c8a06a6da151a1ddd07265ca086366c852c 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -31,10 +31,10 @@
 #
 # Recommended deploy/testing procedure (standalone mode):
 # 1) Rsync / deploy the dist/ dir to one host
-# 2) cd to deploy dir; ./bin/start-master.sh
+# 2) cd to deploy dir; ./sbin/start-master.sh
 # 3) Verify master is up by visiting web page, ie http://master-ip:8080.  Note the spark:// URL.
-# 4) ./bin/start-slave.sh 1 <<spark:// URL>>
-# 5) MASTER="spark://my-master-ip:7077" ./spark-shell
+# 4) ./sbin/start-slave.sh 1 <<spark:// URL>>
+# 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell
 #
 
 # Figure out where the Spark framework is installed
@@ -107,10 +107,7 @@ mkdir "$DISTDIR"/conf
 cp "$FWDIR"/conf/*.template "$DISTDIR"/conf
 cp -r "$FWDIR/bin" "$DISTDIR"
 cp -r "$FWDIR/python" "$DISTDIR"
-cp "$FWDIR/spark-class" "$DISTDIR"
-cp "$FWDIR/spark-shell" "$DISTDIR"
-cp "$FWDIR/spark-executor" "$DISTDIR"
-cp "$FWDIR/pyspark" "$DISTDIR"
+cp -r "$FWDIR/sbin" "$DISTDIR"
 
 
 if [ "$MAKE_TGZ" == "true" ]; then
diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml
deleted file mode 100644
index 4cd28f34e3cbd743e7883f1a20faaf1c5ff2ba2a..0000000000000000000000000000000000000000
--- a/new-yarn/pom.xml
+++ /dev/null
@@ -1,161 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one or more
-  ~ contributor license agreements.  See the NOTICE file distributed with
-  ~ this work for additional information regarding copyright ownership.
-  ~ The ASF licenses this file to You under the Apache License, Version 2.0
-  ~ (the "License"); you may not use this file except in compliance with
-  ~ the License.  You may obtain a copy of the License at
-  ~
-  ~    http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.spark</groupId>
-    <artifactId>spark-parent</artifactId>
-    <version>0.9.0-incubating-SNAPSHOT</version>
-    <relativePath>../pom.xml</relativePath>
-  </parent>
-
-  <groupId>org.apache.spark</groupId>
-  <artifactId>spark-yarn_2.10</artifactId>
-  <packaging>jar</packaging>
-  <name>Spark Project YARN Support</name>
-  <url>http://spark.incubator.apache.org/</url>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_2.10</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-      <version>${yarn.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-ipc</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_2.10</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-shade-plugin</artifactId>
-        <configuration>
-          <shadedArtifactAttached>false</shadedArtifactAttached>
-          <outputFile>${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar</outputFile>
-          <artifactSet>
-            <includes>
-              <include>*:*</include>
-            </includes>
-          </artifactSet>
-          <filters>
-            <filter>
-              <artifact>*:*</artifact>
-              <excludes>
-                <exclude>META-INF/*.SF</exclude>
-                <exclude>META-INF/*.DSA</exclude>
-                <exclude>META-INF/*.RSA</exclude>
-              </excludes>
-            </filter>
-          </filters>
-        </configuration>
-        <executions>
-          <execution>
-            <phase>package</phase>
-            <goals>
-              <goal>shade</goal>
-            </goals>
-            <configuration>
-              <transformers>
-                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
-                <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
-                  <resource>reference.conf</resource>
-                </transformer>
-              </transformers>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>test</phase>
-            <goals>
-              <goal>run</goal>
-            </goals>
-            <configuration>
-              <exportAntProperties>true</exportAntProperties>
-              <tasks>
-                <property name="spark.classpath" refid="maven.test.classpath" />
-                <property environment="env" />
-                <fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry.">  
-                  <condition>
-                    <not>
-                      <or>
-                        <isset property="env.SCALA_HOME" />
-                        <isset property="env.SCALA_LIBRARY_PATH" />
-                      </or>
-                    </not>
-                  </condition>
-                </fail>
-              </tasks>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.scalatest</groupId>
-        <artifactId>scalatest-maven-plugin</artifactId>
-        <configuration>
-          <environmentVariables>
-            <SPARK_HOME>${basedir}/..</SPARK_HOME>
-            <SPARK_TESTING>1</SPARK_TESTING>
-            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
-          </environmentVariables>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-</project>
diff --git a/pom.xml b/pom.xml
index 3a8eb882ccd21a1462d7a66c39c7796b2471d316..aa2f076aacc2423fba4e319378f1bf6e304eadfb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -722,7 +722,7 @@
 
   <profiles>
     <profile>
-      <id>hadoop2-yarn</id>
+      <id>yarn-alpha</id>
       <properties>
         <hadoop.major.version>2</hadoop.major.version>
         <!-- 0.23.* is same as 2.0.* - except hardened to run production jobs -->
@@ -735,57 +735,20 @@
         <module>yarn</module>
       </modules>
 
-      <repositories>
-        <repository>
-          <id>maven-root</id>
-          <name>Maven root repository</name>
-          <url>http://repo1.maven.org/maven2</url>
-          <releases>
-            <enabled>true</enabled>
-          </releases>
-          <snapshots>
-            <enabled>false</enabled>
-          </snapshots>
-        </repository>
-      </repositories>
-
-      <dependencyManagement>
-        <dependencies>
-        </dependencies>
-      </dependencyManagement>
     </profile>
 
     <profile>
-      <id>new-yarn</id>
-       <properties>
-         <hadoop.major.version>2</hadoop.major.version>
-         <hadoop.version>2.2.0</hadoop.version>
-         <protobuf.version>2.5.0</protobuf.version>
-       </properties>
-
-       <modules>
-         <module>new-yarn</module>
-       </modules>
-
-       <repositories>
-         <repository>
-           <id>maven-root</id>
-           <name>Maven root repository</name>
-           <url>http://repo1.maven.org/maven2</url>
-           <releases>
-             <enabled>true</enabled>
-           </releases>
-           <snapshots>
-             <enabled>false</enabled>
-           </snapshots>
-         </repository>
-       </repositories>
+      <id>yarn</id>
+      <properties>
+        <hadoop.major.version>2</hadoop.major.version>
+        <hadoop.version>2.2.0</hadoop.version>
+        <protobuf.version>2.5.0</protobuf.version>
+      </properties>
+      <modules>
+        <module>yarn</module>
+      </modules>
 
-       <dependencyManagement>
-         <dependencies>
-         </dependencies>
-       </dependencyManagement>
-     </profile>
+    </profile>
 
     <profile>
       <id>repl-bin</id>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 5f57c964bde615b5323d62da2cb703b5b2a7a7a8..a6c560d5c6bae60d5968fbdbfba48102ec23dc34 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -89,12 +89,11 @@ object SparkBuild extends Build {
   }
 
   // Conditionally include the yarn sub-project
-  lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "yarn"), settings = yarnSettings) dependsOn(core)
+  lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core)
+  lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core)
 
-  //lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core)
-
-  lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
-  lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
+  lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn else yarnAlpha) else Seq[ClasspathDependency]()
+  lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn else yarnAlpha) else Seq[ProjectReference]()
 
   // Everything except assembly, tools and examples belong to packageProjects
   lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef
@@ -325,10 +324,29 @@ object SparkBuild extends Build {
     )
   )
 
-  def yarnSettings = sharedSettings ++ Seq(
-    name := "spark-yarn"
+  def yarnCommonSettings = sharedSettings ++ Seq(
+    unmanagedSourceDirectories in Compile <++= baseDirectory { base =>
+      Seq(
+         base / "../common/src/main/scala"
+      )
+    },
+
+    unmanagedSourceDirectories in Test <++= baseDirectory { base =>
+      Seq(
+         base / "../common/src/test/scala"
+      )
+    }
+
   ) ++ extraYarnSettings
 
+  def yarnAlphaSettings = yarnCommonSettings ++ Seq(
+    name := "spark-yarn-alpha"
+  )
+
+  def yarnSettings = yarnCommonSettings ++ Seq(
+    name := "spark-yarn"
+  )
+
   // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain
   // if we refer to nonexistent dependencies (e.g. hadoop-yarn-api from a Hadoop version without YARN).
   def extraYarnSettings = if(isYarnEnabled) yarnEnabledSettings else Seq()
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index d8ca9fce0037b4bc33653e7dbf9513973edf546b..c15add52375072bd1e5167c7b26f086727e8ccef 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -31,7 +31,7 @@ def launch_gateway():
     # Launch the Py4j gateway using Spark's run command so that we pick up the
     # proper classpath and SPARK_MEM settings from spark-env.sh
     on_windows = platform.system() == "Windows"
-    script = "spark-class.cmd" if on_windows else "spark-class"
+    script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class"
     command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",
                "--die-on-broken-pipe", "0"]
     if not on_windows:
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index ef07eb437baac423945a41ccdbb8f6b3652e7730..1602227a273e78d325933dace3d6b5c1e124615c 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -47,7 +47,7 @@ print "Spark context available as sc."
 if add_files != None:
     print "Adding files: [%s]" % ", ".join(add_files)
 
-# The ./pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
+# The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
 # which allows us to execute the user's PYTHONSTARTUP file:
 _pythonstartup = os.environ.get('OLD_PYTHONSTARTUP')
 if _pythonstartup and os.path.isfile(_pythonstartup):
diff --git a/python/run-tests b/python/run-tests
index 4b71fff7c12711c856abc95dd322561f6604ecb4..feba97cee0a8e926d1f08009316e2bbf2a340d5a 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -29,7 +29,7 @@ FAILED=0
 rm -f unit-tests.log
 
 function run_test() {
-    SPARK_TESTING=0 $FWDIR/pyspark $1 2>&1 | tee -a unit-tests.log
+    SPARK_TESTING=0 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log
     FAILED=$((PIPESTATUS[0]||$FAILED))
 }
 
diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run
index 47bb654baf44784565dc6130e0f0b534ff24c905..3a6f22f41fca5406a615f57b9d1c96187ce6d928 100755
--- a/repl-bin/src/deb/bin/run
+++ b/repl-bin/src/deb/bin/run
@@ -48,8 +48,7 @@ fi
 export JAVA_OPTS
 
 # Build up classpath
-CLASSPATH="$SPARK_CLASSPATH"
-CLASSPATH+=":$FWDIR/conf"
+CLASSPATH=":$FWDIR/conf"
 for jar in `find $FWDIR -name '*jar'`; do
   CLASSPATH+=":$jar"
 done
diff --git a/repl/pom.xml b/repl/pom.xml
index b0e7877bbbac4a11b9af25ac0ca0ba6a1426d4aa..2dfe7ac900b831854b087e36f21db4d4cdc74f78 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -127,7 +127,6 @@
           <environmentVariables>
             <SPARK_HOME>${basedir}/..</SPARK_HOME>
             <SPARK_TESTING>1</SPARK_TESTING>
-            <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
           </environmentVariables>
         </configuration>
       </plugin>
diff --git a/bin/slaves.sh b/sbin/slaves.sh
similarity index 97%
rename from bin/slaves.sh
rename to sbin/slaves.sh
index c367c2fd8ed9ae152607e54eb51b162788521613..a5bc2183d87f3a1a14233bf488e25feef516cbb4 100755
--- a/bin/slaves.sh
+++ b/sbin/slaves.sh
@@ -36,10 +36,10 @@ if [ $# -le 0 ]; then
   exit 1
 fi
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 # If the slaves file is specified in the command line,
 # then it takes precedence over the definition in
diff --git a/bin/spark-config.sh b/sbin/spark-config.sh
similarity index 100%
rename from bin/spark-config.sh
rename to sbin/spark-config.sh
diff --git a/bin/spark-daemon.sh b/sbin/spark-daemon.sh
similarity index 96%
rename from bin/spark-daemon.sh
rename to sbin/spark-daemon.sh
index a0c0d44b58d9991f2d94ed16c41269baabe15900..2be2b3d7c09339087fb553a7b4e244f70b392def 100755
--- a/bin/spark-daemon.sh
+++ b/sbin/spark-daemon.sh
@@ -37,10 +37,10 @@ if [ $# -le 1 ]; then
   exit 1
 fi
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 # get arguments
 
@@ -147,7 +147,7 @@ case $startStop in
     spark_rotate_log "$log"
     echo starting $command, logging to $log
     cd "$SPARK_PREFIX"
-    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
+    nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
     newpid=$!
     echo $newpid > $pid
     sleep 2
diff --git a/bin/spark-daemons.sh b/sbin/spark-daemons.sh
similarity index 88%
rename from bin/spark-daemons.sh
rename to sbin/spark-daemons.sh
index 64286cb2da4f5acee66db6dd19ed442190d6466c..5d9f2bb51cae0547b053967fd5521a6ecb53ca04 100755
--- a/bin/spark-daemons.sh
+++ b/sbin/spark-daemons.sh
@@ -27,9 +27,9 @@ if [ $# -le 1 ]; then
   exit 1
 fi
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
-exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/spark-daemon.sh" "$@"
+exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/spark-daemon.sh" "$@"
diff --git a/spark-executor b/sbin/spark-executor
similarity index 88%
rename from spark-executor
rename to sbin/spark-executor
index 2c07c5484338fc9acddf4a4c74130566efba5176..de5bfab563125689b7c5c5a2e1b9f9d68f4b4aa1 100755
--- a/spark-executor
+++ b/sbin/spark-executor
@@ -17,6 +17,7 @@
 # limitations under the License.
 #
 
-FWDIR="`dirname $0`"
+FWDIR="$(cd `dirname $0`/..; pwd)"
+
 echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/spark-class org.apache.spark.executor.MesosExecutorBackend
+exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend
diff --git a/bin/start-all.sh b/sbin/start-all.sh
similarity index 89%
rename from bin/start-all.sh
rename to sbin/start-all.sh
index 0182f1ab240b744ec097c74dd1065029188c7f99..2daf49db359df4d62c9b3ddee2d9787523801345 100755
--- a/bin/start-all.sh
+++ b/sbin/start-all.sh
@@ -21,14 +21,14 @@
 # Starts the master on this node.
 # Starts a worker on each node specified in conf/slaves
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
 # Load the Spark configuration
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 # Start Master
-"$bin"/start-master.sh
+"$sbin"/start-master.sh
 
 # Start Workers
-"$bin"/start-slaves.sh
+"$sbin"/start-slaves.sh
diff --git a/bin/start-master.sh b/sbin/start-master.sh
similarity index 88%
rename from bin/start-master.sh
rename to sbin/start-master.sh
index 648c7ae75fe81ddafbd9394ee6f1f60f00f7c263..3dcf7cc3483a785c21fd3266b16426a40cbdbb54 100755
--- a/bin/start-master.sh
+++ b/sbin/start-master.sh
@@ -19,10 +19,10 @@
 
 # Starts the master on the machine this script is executed on.
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
   . "${SPARK_CONF_DIR}/spark-env.sh"
@@ -49,4 +49,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
     fi
 fi
 
-"$bin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
+"$sbin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
diff --git a/bin/start-slave.sh b/sbin/start-slave.sh
similarity index 92%
rename from bin/start-slave.sh
rename to sbin/start-slave.sh
index 4eefa209443385881905ee0c7208ea56fa79e9e5..524be38c629681b8acf1df845feabe020e25ee5c 100755
--- a/bin/start-slave.sh
+++ b/sbin/start-slave.sh
@@ -20,8 +20,8 @@
 # Usage: start-slave.sh <worker#> <master-spark-URL>
 #   where <master-spark-URL> is like "spark://localhost:7077"
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
 # Set SPARK_PUBLIC_DNS so slaves can be linked in master web UI
 if [ "$SPARK_PUBLIC_DNS" = "" ]; then
@@ -32,4 +32,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
     fi
 fi
 
-"$bin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"
+"$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"
diff --git a/bin/start-slaves.sh b/sbin/start-slaves.sh
similarity index 78%
rename from bin/start-slaves.sh
rename to sbin/start-slaves.sh
index 00dc4888b2e95220b2ea8ec4bcc32810a7235bfc..fd5cdeb1e6788ab0e7550621781883b31dc8f3cd 100755
--- a/bin/start-slaves.sh
+++ b/sbin/start-slaves.sh
@@ -17,10 +17,10 @@
 # limitations under the License.
 #
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
   . "${SPARK_CONF_DIR}/spark-env.sh"
@@ -37,12 +37,12 @@ fi
 
 # Launch the slaves
 if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
-  exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
+  exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
 else
   if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then
     SPARK_WORKER_WEBUI_PORT=8081
   fi
   for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
-    "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" $(( $i + 1 ))  spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
+    "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 ))  spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
   done
 fi
diff --git a/bin/stop-all.sh b/sbin/stop-all.sh
similarity index 89%
rename from bin/stop-all.sh
rename to sbin/stop-all.sh
index b6c83a7ba4c7df08aafa58f7e7a772e0d98452c9..60b358d3745658dad80f3a71c58410253d5a7455 100755
--- a/bin/stop-all.sh
+++ b/sbin/stop-all.sh
@@ -21,12 +21,12 @@
 # Run this on the master nde
 
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
 # Load the Spark configuration
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 # Stop the slaves, then the master
-"$bin"/stop-slaves.sh
-"$bin"/stop-master.sh
+"$sbin"/stop-slaves.sh
+"$sbin"/stop-master.sh
diff --git a/bin/stop-master.sh b/sbin/stop-master.sh
similarity index 86%
rename from bin/stop-master.sh
rename to sbin/stop-master.sh
index 310e33bedc057194d4b410f2335234f5e2f30613..2adabd426563c7f5bf4f67f73da10cc60e753fc2 100755
--- a/bin/stop-master.sh
+++ b/sbin/stop-master.sh
@@ -19,9 +19,9 @@
 
 # Starts the master on the machine this script is executed on.
 
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
-"$bin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1
+"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1
diff --git a/bin/stop-slaves.sh b/sbin/stop-slaves.sh
similarity index 83%
rename from bin/stop-slaves.sh
rename to sbin/stop-slaves.sh
index fcb8555d4e5706a6f4f2ec9670525437a3a2acae..c6b0b6ab6695c505f9368d5862579a1b5bc952dc 100755
--- a/bin/stop-slaves.sh
+++ b/sbin/stop-slaves.sh
@@ -18,18 +18,18 @@
 #
 
 bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+bin=`cd "$sbin"; pwd`
 
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
 
 if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
   . "${SPARK_CONF_DIR}/spark-env.sh"
 fi
 
 if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
-  "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
+  "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
 else
   for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
-    "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
+    "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
   done
 fi
diff --git a/yarn/README.md b/yarn/README.md
new file mode 100644
index 0000000000000000000000000000000000000000..65ee85447e04a7eca0843e18a3a4896476ddcb8c
--- /dev/null
+++ b/yarn/README.md
@@ -0,0 +1,12 @@
+# YARN DIRECTORY LAYOUT
+
+Hadoop Yarn related codes are organized in separate directories to minimize duplicated code.
+
+ * common : Common codes that do not depending on specific version of Hadoop.
+
+ * alpha / stable : Codes that involve specific version of Hadoop YARN API.
+
+  alpha represents 0.23 and 2.0.x
+  stable represents 2.2 and later, until the API changes again.
+
+alpha / stable will build together with common dir into a single jar
diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..8291e9e7a36ce8ab42d742bd3e3e13ffde99e7b8
--- /dev/null
+++ b/yarn/alpha/pom.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>yarn-parent_2.10</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-yarn-alpha_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project YARN Alpha API</name>
+
+</project>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
similarity index 97%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 7cf120d3eb8a47a966f1263610b2678262f85652..2bb11e54c549af037fc75a2dd92300e3588828b4 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -39,11 +39,15 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
 import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.util.Utils
 
-class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
+                        sparkConf: SparkConf) extends Logging {
 
-  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+  def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
 
-  private var rpc: YarnRPC = YarnRPC.create(conf)
+  def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
+
+  private val rpc: YarnRPC = YarnRPC.create(conf)
   private var resourceManager: AMRMProtocol = _
   private var appAttemptId: ApplicationAttemptId = _
   private var userThread: Thread = _
@@ -57,7 +61,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES)
   private var isLastAMRetry: Boolean = true
 
-  private val sparkConf = new SparkConf()
   // Default to numWorkers * 2, with minimum of 3
   private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures",
     math.max(args.numWorkers * 2, 3))
@@ -125,7 +128,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X
     val localDirs = Option(System.getenv("YARN_LOCAL_DIRS"))
       .getOrElse(Option(System.getenv("LOCAL_DIRS"))
-        .getOrElse(""))
+      .getOrElse(""))
 
     if (localDirs.isEmpty()) {
       throw new Exception("Yarn Local dirs can't be empty")
@@ -164,11 +167,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     resourceManager.registerApplicationMaster(appMasterRequest)
   }
 
-  private def startUserClass(): Thread  = {
+  private def startUserClass(): Thread = {
     logInfo("Starting the user JAR in a separate Thread")
     val mainMethod = Class.forName(
       args.userClass,
-      false /* initialize */,
+      false /* initialize */ ,
       Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
     val t = new Thread {
       override def run() {
@@ -230,7 +233,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
             yarnConf,
             resourceManager,
             appAttemptId,
-            args, 
+            args,
             sparkContext.getConf)
         }
       }
@@ -285,7 +288,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   }
 
   private def launchReporterThread(_sleepTime: Long): Thread = {
-    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+    val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime
 
     val t = new Thread {
       override def run() {
@@ -384,6 +387,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
     }
   }
+
 }
 
 object ApplicationMaster {
@@ -393,6 +397,7 @@ object ApplicationMaster {
   // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
   // optimal as more containers are available. Might need to handle this better.
   private val ALLOCATOR_LOOP_WAIT_COUNT = 30
+
   def incrementAllocatorLoop(by: Int) {
     val count = yarnAllocatorLoop.getAndAdd(by)
     if (count >= ALLOCATOR_LOOP_WAIT_COUNT) {
@@ -431,6 +436,7 @@ object ApplicationMaster {
         // This is not only logs, but also ensures that log system is initialized for this instance
         // when we are actually 'run'-ing.
         logInfo("Adding shutdown hook for context " + sc)
+
         override def run() {
           logInfo("Invoking sc stop from shutdown hook")
           sc.stop()
@@ -439,7 +445,7 @@ object ApplicationMaster {
             master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           }
         }
-      } )
+      })
     }
 
     // Wait for initialization to complete and atleast 'some' nodes can get allocated.
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
similarity index 95%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 2bd047c97adbab917c069c83dc2abc9844e421c6..6abb4d501779fd3193c3050d9eb1991576db0178 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -45,16 +45,19 @@ import org.apache.spark.util.Utils
 import org.apache.spark.deploy.SparkHadoopUtil
 
 
-class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
+class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
+  extends YarnClientImpl with Logging {
 
-  def this(args: ClientArguments) = this(new Configuration(), args)
+  def this(args: ClientArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
+
+  def this(args: ClientArguments) = this(args, new SparkConf())
 
   var rpc: YarnRPC = YarnRPC.create(conf)
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
   val credentials = UserGroupInformation.getCurrentUser().getCredentials()
   private val SPARK_STAGING: String = ".sparkStaging"
   private val distCacheMgr = new ClientDistributedCacheManager()
-  private val sparkConf = new SparkConf
 
   // Staging directory is private! -> rwx--------
   val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short)
@@ -122,7 +125,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       clusterMetrics.getNumNodeManagers)
 
     val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
-    logInfo("""Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s,
+    logInfo( """Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s,
       queueApplicationCount = %s, queueChildQueueCount = %s""".format(
         queueInfo.getQueueName,
         queueInfo.getCurrentCapacity,
@@ -142,7 +145,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     }
     val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
     if (amMem > maxMem) {
-      logError("AM size is to large to run on this cluster "  + amMem)
+      logError("AM size is to large to run on this cluster " + amMem)
       System.exit(1)
     }
 
@@ -307,7 +310,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     val env = new HashMap[String, String]()
 
-    Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env)
+    Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env)
     env("SPARK_YARN_MODE") = "true"
     env("SPARK_YARN_STAGING_DIR") = stagingDir
 
@@ -327,7 +330,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     val prefix = " --args "
     val args = clientArgs.userArgs
     val retval = new StringBuilder()
-    for (arg <- args){
+    for (arg <- args) {
       retval.append(prefix).append(" '").append(arg).append("' ")
     }
     retval.toString
@@ -466,9 +469,10 @@ object Client {
     // Note that anything with SPARK prefix gets propagated to all (remote) processes
     System.setProperty("SPARK_YARN_MODE", "true")
 
-    val args = new ClientArguments(argStrings)
+    val sparkConf = new SparkConf
+    val args = new ClientArguments(argStrings, sparkConf)
 
-    new Client(args).run
+    new Client(args, sparkConf).run
   }
 
   // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
@@ -478,7 +482,7 @@ object Client {
     }
   }
 
-  def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) {
+  def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) {
     Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
     // If log4j present, ensure ours overrides all others
     if (addLog4j) {
@@ -486,7 +490,7 @@ object Client {
         Path.SEPARATOR + LOG4J_PROP)
     }
     // Normally the users app.jar is last in case conflicts with spark jars
-    val userClasspathFirst = new SparkConf().get("spark.yarn.user.classpath.first", "false").toBoolean
+    val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean
     if (userClasspathFirst) {
       Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
         Path.SEPARATOR + APP_JAR)
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
similarity index 93%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
index a8de89c67081187843d7af7dfa708dd6afa997e3..ddfec1a4ac6728e8dd0030a6b5fcf164a1dff42c 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
@@ -34,33 +34,36 @@ import org.apache.spark.util.{Utils, AkkaUtils}
 import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
 import org.apache.spark.scheduler.SplitInfo
 
-class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf)
+  extends Logging {
 
-  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+  def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = this(args, new Configuration(), sparkConf)
+
+  def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
 
   private val rpc: YarnRPC = YarnRPC.create(conf)
-  private var resourceManager: AMRMProtocol = null
-  private var appAttemptId: ApplicationAttemptId = null
-  private var reporterThread: Thread = null
+  private var resourceManager: AMRMProtocol = _
+  private var appAttemptId: ApplicationAttemptId = _
+  private var reporterThread: Thread = _
   private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
 
-  private var yarnAllocator: YarnAllocationHandler = null
+  private var yarnAllocator: YarnAllocationHandler = _
   private var driverClosed:Boolean = false
-  private val sparkConf = new SparkConf
 
   val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0,
     conf = sparkConf)._1
-  var actor: ActorRef = null
+  var actor: ActorRef = _
 
   // This actor just working as a monitor to watch on Driver Actor.
   class MonitorActor(driverUrl: String) extends Actor {
 
-    var driver: ActorSelection = null
+    var driver: ActorSelection = _
 
     override def preStart() {
       logInfo("Listen to driver: " + driverUrl)
       driver = context.actorSelection(driverUrl)
-      driver ! "hello"
+      // Send a hello message thus the connection is actually established, thus we can monitor Lifecycle Events.
+      driver ! "Hello"
       context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
     }
 
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
similarity index 97%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index 6a90cc51cfbaf71ef5683703dd84fa6d5b872ccb..132630e5ef04cbb09bc400dacef37a1ea2b0d28d 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -37,12 +37,13 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils}
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 
 
 class WorkerRunnable(
     container: Container,
     conf: Configuration,
+    sparkConf: SparkConf,
     masterAddress: String,
     slaveId: String,
     hostname: String,
@@ -51,7 +52,7 @@ class WorkerRunnable(
   extends Runnable with Logging {
 
   var rpc: YarnRPC = YarnRPC.create(conf)
-  var cm: ContainerManager = null
+  var cm: ContainerManager = _
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
 
   def run = {
@@ -200,7 +201,7 @@ class WorkerRunnable(
   def prepareEnvironment: HashMap[String, String] = {
     val env = new HashMap[String, String]()
 
-    Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
+    Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
 
     // Allow users to specify some environment variables
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
similarity index 99%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index c8af653b3ffab9191b586a015902b616e1952eff..e91257be8ed008585c0491fc0c1ce13a6258cd15 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -261,7 +261,7 @@ private[yarn] class YarnAllocationHandler(
           }
 
           new Thread(
-            new WorkerRunnable(container, conf, driverUrl, workerId,
+            new WorkerRunnable(container, conf, sparkConf, driverUrl, workerId,
               workerHostname, workerMemory, workerCores)
           ).start()
         }
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
similarity index 97%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index 7aac2328dad60794c35142a64fc18b87a5a97973..1419f215c78e5831e7134adbb6e10d5fb6420960 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -26,7 +26,7 @@ import org.apache.spark.util.MemoryParam
 
 
 // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware !
-class ClientArguments(val args: Array[String]) {
+class ClientArguments(val args: Array[String], val sparkConf: SparkConf) {
   var addJars: String = null
   var files: String = null
   var archives: String = null
@@ -36,7 +36,7 @@ class ClientArguments(val args: Array[String]) {
   var workerMemory = 1024 // MB
   var workerCores = 1
   var numWorkers = 2
-  var amQueue = new SparkConf().get("QUEUE", "default")
+  var amQueue = sparkConf.get("QUEUE", "default")
   var amMemory: Int = 512 // MB
   var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster"
   var appName: String = "Spark"
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
rename to yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
similarity index 97%
rename from new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
rename to yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
index 4b69f5078b0ab10818f1d6ecd5d9655cd00327c8..324ef4616fe268b4005a2ec014fb0a8636d4e2e2 100644
--- a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
@@ -67,8 +67,8 @@ private[spark] class YarnClientSchedulerBackend(
       "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher"
     )
 
-    val args = new ClientArguments(argsArray)
-    client = new Client(args)
+    val args = new ClientArguments(argsArray, conf)
+    client = new Client(args, conf)
     appId = client.runApp()
     waitForApp()
   }
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
rename to yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
diff --git a/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
similarity index 100%
rename from new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
rename to yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
diff --git a/yarn/pom.xml b/yarn/pom.xml
index bc64a190fd284cebc6ab993ea6de6254841402c0..aea8b0cddefa29d82dc6ceff60ec1d74b4de1100 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -25,11 +25,10 @@
   </parent>
 
   <groupId>org.apache.spark</groupId>
-  <artifactId>spark-yarn_2.10</artifactId>
-  <packaging>jar</packaging>
-  <name>Spark Project YARN Support</name>
-  <url>http://spark.incubator.apache.org/</url>
-
+  <artifactId>yarn-parent_2.10</artifactId>
+  <packaging>pom</packaging>
+  <name>Spark Project YARN Parent POM</name>
+  
   <dependencies>
     <dependency>
       <groupId>org.apache.spark</groupId>
@@ -73,45 +72,52 @@
     </dependency>
   </dependencies>
 
+  <profiles>
+    <profile>
+      <id>yarn-alpha</id>
+      <modules>
+        <module>alpha</module>
+      </modules>
+    </profile>
+
+    <profile>
+      <id>yarn</id>
+      <modules>
+        <module>stable</module>
+      </modules>
+    </profile>
+  </profiles>
+
   <build>
-    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
-    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
     <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-shade-plugin</artifactId>
-        <configuration>
-          <shadedArtifactAttached>false</shadedArtifactAttached>
-          <outputFile>${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar</outputFile>
-          <artifactSet>
-            <includes>
-              <include>*:*</include>
-            </includes>
-          </artifactSet>
-          <filters>
-            <filter>
-              <artifact>*:*</artifact>
-              <excludes>
-                <exclude>META-INF/*.SF</exclude>
-                <exclude>META-INF/*.DSA</exclude>
-                <exclude>META-INF/*.RSA</exclude>
-              </excludes>
-            </filter>
-          </filters>
-        </configuration>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
         <executions>
           <execution>
-            <phase>package</phase>
+           <id>add-scala-sources</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>src/main/scala</source>
+                <source>../common/src/main/scala</source>
+              </sources>
+            </configuration>
+          </execution>
+          <execution>
+            <id>add-scala-test-sources</id>
+            <phase>generate-test-sources</phase>
             <goals>
-              <goal>shade</goal>
+              <goal>add-test-source</goal>
             </goals>
             <configuration>
-              <transformers>
-                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
-                <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
-                  <resource>reference.conf</resource>
-                </transformer>
-              </transformers>
+              <sources>
+                <source>src/test/scala</source>
+                <source>../common/src/test/scala</source>
+              </sources>
             </configuration>
           </execution>
         </executions>
@@ -150,12 +156,16 @@
         <artifactId>scalatest-maven-plugin</artifactId>
         <configuration>
           <environmentVariables>
-            <SPARK_HOME>${basedir}/..</SPARK_HOME>
+            <SPARK_HOME>${basedir}/../..</SPARK_HOME>
             <SPARK_TESTING>1</SPARK_TESTING>
             <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
           </environmentVariables>
         </configuration>
       </plugin>
     </plugins>
+
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
   </build>
+
 </project>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
deleted file mode 100644
index f76a5ddd39e90d4998d7712113dd0949823d5e18..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.deploy.yarn
-
-import org.apache.spark.util.IntParam
-import collection.mutable.ArrayBuffer
-
-class ApplicationMasterArguments(val args: Array[String]) {
-  var userJar: String = null
-  var userClass: String = null
-  var userArgs: Seq[String] = Seq[String]()
-  var workerMemory = 1024
-  var workerCores = 1
-  var numWorkers = 2
-
-  parseArgs(args.toList)
-  
-  private def parseArgs(inputArgs: List[String]): Unit = {
-    val userArgsBuffer = new ArrayBuffer[String]()
-
-    var args = inputArgs
-
-    while (! args.isEmpty) {
-
-      args match {
-        case ("--jar") :: value :: tail =>
-          userJar = value
-          args = tail
-
-        case ("--class") :: value :: tail =>
-          userClass = value
-          args = tail
-
-        case ("--args") :: value :: tail =>
-          userArgsBuffer += value
-          args = tail
-
-        case ("--num-workers") :: IntParam(value) :: tail =>
-          numWorkers = value
-          args = tail
-
-        case ("--worker-memory") :: IntParam(value) :: tail =>
-          workerMemory = value
-          args = tail
-
-        case ("--worker-cores") :: IntParam(value) :: tail =>
-          workerCores = value
-          args = tail
-
-        case Nil =>
-          if (userJar == null || userClass == null) {
-            printUsageAndExit(1)
-          }
-
-        case _ =>
-          printUsageAndExit(1, args)
-      }
-    }
-
-    userArgs = userArgsBuffer.readOnly
-  }
-  
-  def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
-    if (unknownParam != null) {
-      System.err.println("Unknown/unsupported param " + unknownParam)
-    }
-    System.err.println(
-      "Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options] \n" +
-      "Options:\n" +
-      "  --jar JAR_PATH       Path to your application's JAR file (required)\n" +
-      "  --class CLASS_NAME   Name of your application's main class (required)\n" +
-      "  --args ARGS          Arguments to be passed to your application's main class.\n" +
-      "                       Mutliple invocations are possible, each will be passed in order.\n" +
-      "  --num-workers NUM    Number of workers to start (Default: 2)\n" +
-      "  --worker-cores NUM   Number of cores for the workers (Default: 1)\n" +
-      "  --worker-memory MEM  Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n")
-    System.exit(exitCode)
-  }
-}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
deleted file mode 100644
index 9075ca71e7efc38fa83068e91ffa78db400b4ec7..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.deploy.yarn
-
-import org.apache.spark.SparkConf
-import org.apache.spark.util.MemoryParam
-import org.apache.spark.util.IntParam
-import collection.mutable.{ArrayBuffer, HashMap}
-import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
-
-// TODO: Add code and support for ensuring that yarn resource 'asks' are location aware !
-class ClientArguments(val args: Array[String]) {
-  var addJars: String = null
-  var files: String = null
-  var archives: String = null
-  var userJar: String = null
-  var userClass: String = null
-  var userArgs: Seq[String] = Seq[String]()
-  var workerMemory = 1024
-  var workerCores = 1
-  var numWorkers = 2
-  var amQueue = new SparkConf().get("QUEUE", "default")
-  var amMemory: Int = 512
-  var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster"
-  var appName: String = "Spark"
-  // TODO
-  var inputFormatInfo: List[InputFormatInfo] = null
-
-  parseArgs(args.toList)
-
-  private def parseArgs(inputArgs: List[String]): Unit = {
-    val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]()
-    val inputFormatMap: HashMap[String, InputFormatInfo] = new HashMap[String, InputFormatInfo]()
-
-    var args = inputArgs
-
-    while (! args.isEmpty) {
-
-      args match {
-        case ("--jar") :: value :: tail =>
-          userJar = value
-          args = tail
-
-        case ("--class") :: value :: tail =>
-          userClass = value
-          args = tail
-
-        case ("--args") :: value :: tail =>
-          userArgsBuffer += value
-          args = tail
-
-        case ("--master-class") :: value :: tail =>
-          amClass = value
-          args = tail
-
-        case ("--master-memory") :: MemoryParam(value) :: tail =>
-          amMemory = value
-          args = tail
-
-        case ("--worker-memory") :: MemoryParam(value) :: tail =>
-          workerMemory = value
-          args = tail
-
-        case ("--num-workers") :: IntParam(value) :: tail =>
-          numWorkers = value
-          args = tail
-
-        case ("--worker-cores") :: IntParam(value) :: tail =>
-          workerCores = value
-          args = tail
-
-        case ("--queue") :: value :: tail =>
-          amQueue = value
-          args = tail
-
-        case ("--name") :: value :: tail =>
-          appName = value
-          args = tail
-
-        case ("--addJars") :: value :: tail =>
-          addJars = value
-          args = tail
-
-        case ("--files") :: value :: tail =>
-          files = value
-          args = tail
-
-        case ("--archives") :: value :: tail =>
-          archives = value
-          args = tail
-
-        case Nil =>
-          if (userJar == null || userClass == null) {
-            printUsageAndExit(1)
-          }
-
-        case _ =>
-          printUsageAndExit(1, args)
-      }
-    }
-
-    userArgs = userArgsBuffer.readOnly
-    inputFormatInfo = inputFormatMap.values.toList
-  }
-
-
-  def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
-    if (unknownParam != null) {
-      System.err.println("Unknown/unsupported param " + unknownParam)
-    }
-    System.err.println(
-      "Usage: org.apache.spark.deploy.yarn.Client [options] \n" +
-      "Options:\n" +
-      "  --jar JAR_PATH             Path to your application's JAR file (required)\n" +
-      "  --class CLASS_NAME         Name of your application's main class (required)\n" +
-      "  --args ARGS                Arguments to be passed to your application's main class.\n" +
-      "                             Mutliple invocations are possible, each will be passed in order.\n" +
-      "  --num-workers NUM          Number of workers to start (Default: 2)\n" +
-      "  --worker-cores NUM         Number of cores for the workers (Default: 1). This is unsused right now.\n" +
-      "  --master-class CLASS_NAME  Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" +
-      "  --master-memory MEM        Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" +
-      "  --worker-memory MEM        Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" +
-      "  --name NAME                The name of your application (Default: Spark)\n" +
-      "  --queue QUEUE              The hadoop queue to use for allocation requests (Default: 'default')\n" +
-      "  --addJars jars             Comma separated list of local jars that want SparkContext.addJar to work with.\n" +
-      "  --files files              Comma separated list of files to be distributed with the job.\n" +
-      "  --archives archives        Comma separated list of archives to be distributed with the job."
-      )
-    System.exit(exitCode)
-  }
-
-}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
deleted file mode 100644
index 5f159b073f5372dc9e1f73347f88ae8aa48e0cbb..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.deploy.yarn
-
-import java.net.URI
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileStatus
-import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.fs.permission.FsAction
-import org.apache.hadoop.yarn.api.records.LocalResource
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
-import org.apache.hadoop.yarn.api.records.LocalResourceType
-import org.apache.hadoop.yarn.util.{Records, ConverterUtils}
-
-import org.apache.spark.Logging 
-
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.LinkedHashMap
-import scala.collection.mutable.Map
-
-
-/** Client side methods to setup the Hadoop distributed cache */
-class ClientDistributedCacheManager() extends Logging {
-  private val distCacheFiles: Map[String, Tuple3[String, String, String]] = 
-    LinkedHashMap[String, Tuple3[String, String, String]]()
-  private val distCacheArchives: Map[String, Tuple3[String, String, String]] = 
-    LinkedHashMap[String, Tuple3[String, String, String]]()
-
-
-  /**
-   * Add a resource to the list of distributed cache resources. This list can
-   * be sent to the ApplicationMaster and possibly the workers so that it can 
-   * be downloaded into the Hadoop distributed cache for use by this application.
-   * Adds the LocalResource to the localResources HashMap passed in and saves 
-   * the stats of the resources to they can be sent to the workers and verified.
-   *
-   * @param fs FileSystem
-   * @param conf Configuration
-   * @param destPath path to the resource
-   * @param localResources localResource hashMap to insert the resource into
-   * @param resourceType LocalResourceType 
-   * @param link link presented in the distributed cache to the destination
-   * @param statCache cache to store the file/directory stats 
-   * @param appMasterOnly Whether to only add the resource to the app master
-   */
-  def addResource(
-      fs: FileSystem,
-      conf: Configuration,
-      destPath: Path, 
-      localResources: HashMap[String, LocalResource],
-      resourceType: LocalResourceType,
-      link: String,
-      statCache: Map[URI, FileStatus],
-      appMasterOnly: Boolean = false) = {
-    val destStatus = fs.getFileStatus(destPath)
-    val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
-    amJarRsrc.setType(resourceType)
-    val visibility = getVisibility(conf, destPath.toUri(), statCache)
-    amJarRsrc.setVisibility(visibility)
-    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath))
-    amJarRsrc.setTimestamp(destStatus.getModificationTime())
-    amJarRsrc.setSize(destStatus.getLen())
-    if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name")
-    localResources(link) = amJarRsrc
-    
-    if (appMasterOnly == false) {
-      val uri = destPath.toUri()
-      val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link)
-      if (resourceType == LocalResourceType.FILE) {
-        distCacheFiles(pathURI.toString()) = (destStatus.getLen().toString(), 
-          destStatus.getModificationTime().toString(), visibility.name())
-      } else {
-        distCacheArchives(pathURI.toString()) = (destStatus.getLen().toString(), 
-          destStatus.getModificationTime().toString(), visibility.name())
-      }
-    }
-  }
-
-  /**
-   * Adds the necessary cache file env variables to the env passed in
-   * @param env
-   */
-  def setDistFilesEnv(env: Map[String, String]) = {
-    val (keys, tupleValues) = distCacheFiles.unzip
-    val (sizes, timeStamps, visibilities) = tupleValues.unzip3
-
-    if (keys.size > 0) {
-      env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = 
-        timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = 
-        sizes.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_FILES_VISIBILITIES") = 
-        visibilities.reduceLeft[String] { (acc,n) => acc + "," + n }
-    }
-  }
-
-  /**
-   * Adds the necessary cache archive env variables to the env passed in
-   * @param env
-   */
-  def setDistArchivesEnv(env: Map[String, String]) = {
-    val (keys, tupleValues) = distCacheArchives.unzip
-    val (sizes, timeStamps, visibilities) = tupleValues.unzip3
-
-    if (keys.size > 0) {
-      env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = 
-        timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") =
-        sizes.reduceLeft[String] { (acc,n) => acc + "," + n }
-      env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") = 
-        visibilities.reduceLeft[String] { (acc,n) => acc + "," + n }
-    }
-  }
-
-  /**
-   * Returns the local resource visibility depending on the cache file permissions
-   * @param conf
-   * @param uri
-   * @param statCache
-   * @return LocalResourceVisibility
-   */
-  def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]):
-      LocalResourceVisibility = {
-    if (isPublic(conf, uri, statCache)) {
-      return LocalResourceVisibility.PUBLIC 
-    } 
-    return LocalResourceVisibility.PRIVATE
-  }
-
-  /**
-   * Returns a boolean to denote whether a cache file is visible to all(public)
-   * or not
-   * @param conf
-   * @param uri
-   * @param statCache
-   * @return true if the path in the uri is visible to all, false otherwise
-   */
-  def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = {
-    val fs = FileSystem.get(uri, conf)
-    val current = new Path(uri.getPath())
-    //the leaf level file should be readable by others
-    if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) {
-      return false
-    }
-    return ancestorsHaveExecutePermissions(fs, current.getParent(), statCache)
-  }
-
-  /**
-   * Returns true if all ancestors of the specified path have the 'execute'
-   * permission set for all users (i.e. that other users can traverse
-   * the directory heirarchy to the given path)
-   * @param fs
-   * @param path
-   * @param statCache
-   * @return true if all ancestors have the 'execute' permission set for all users
-   */
-  def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, 
-      statCache: Map[URI, FileStatus]): Boolean =  {
-    var current = path
-    while (current != null) {
-      //the subdirs in the path should have execute permissions for others
-      if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) {
-        return false
-      }
-      current = current.getParent()
-    }
-    return true
-  }
-
-  /**
-   * Checks for a given path whether the Other permissions on it 
-   * imply the permission in the passed FsAction
-   * @param fs
-   * @param path
-   * @param action
-   * @param statCache
-   * @return true if the path in the uri is visible to all, false otherwise
-   */
-  def checkPermissionOfOther(fs: FileSystem, path: Path,
-      action: FsAction, statCache: Map[URI, FileStatus]): Boolean = {
-    val status = getFileStatus(fs, path.toUri(), statCache)
-    val perms = status.getPermission()
-    val otherAction = perms.getOtherAction()
-    if (otherAction.implies(action)) {
-      return true
-    }
-    return false
-  }
-
-  /**
-   * Checks to see if the given uri exists in the cache, if it does it 
-   * returns the existing FileStatus, otherwise it stats the uri, stores
-   * it in the cache, and returns the FileStatus.
-   * @param fs
-   * @param uri
-   * @param statCache
-   * @return FileStatus
-   */
-  def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = {
-    val stat = statCache.get(uri) match {
-      case Some(existstat) => existstat
-      case None => 
-        val newStat = fs.getFileStatus(new Path(uri))
-        statCache.put(uri, newStat)
-        newStat
-    }
-    return stat
-  }
-}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
deleted file mode 100644
index 2ba2366ead17113c764663ed934a5221f0eeb0ee..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.deploy.yarn
-
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.security.UserGroupInformation
-import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.conf.Configuration
-
-/**
- * Contains util methods to interact with Hadoop from spark.
- */
-class YarnSparkHadoopUtil extends SparkHadoopUtil {
-
-  // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true.
-  override def isYarnMode(): Boolean = { true }
-
-  // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
-  // Always create a new config, dont reuse yarnConf.
-  override def newConfiguration(): Configuration = new YarnConfiguration(new Configuration())
-
-  // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster
-  override def addCredentials(conf: JobConf) {
-    val jobCreds = conf.getCredentials()
-    jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials())
-  }
-}
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
deleted file mode 100644
index 522e0a9ad7eeb50f4c2b6b781a68ea998639b30a..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.scheduler.cluster
-
-import org.apache.spark._
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.deploy.yarn.YarnAllocationHandler
-import org.apache.spark.scheduler.TaskSchedulerImpl
-import org.apache.spark.util.Utils
-
-/**
- *
- * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM.
- */
-private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) {
-
-  def this(sc: SparkContext) = this(sc, new Configuration())
-
-  // By default, rack is unknown
-  override def getRackForHost(hostPort: String): Option[String] = {
-    val host = Utils.parseHostPort(hostPort)._1
-    val retval = YarnAllocationHandler.lookupRack(conf, host)
-    if (retval != null) Some(retval) else None
-  }
-
-  override def postStartHook() {
-
-    // The yarn application is running, but the worker might not yet ready
-    // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt
-    Thread.sleep(2000L)
-    logInfo("YarnClientClusterScheduler.postStartHook done")
-  }
-}
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
deleted file mode 100644
index 4b69f5078b0ab10818f1d6ecd5d9655cd00327c8..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-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.scheduler.TaskSchedulerImpl
-
-private[spark] class YarnClientSchedulerBackend(
-    scheduler: TaskSchedulerImpl,
-    sc: SparkContext)
-  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
-  with Logging {
-
-  var client: Client = null
-  var appId: ApplicationId = null
-
-  override def start() {
-    super.start()
-
-    val defalutWorkerCores = "2"
-    val defalutWorkerMemory = "512m"
-    val defaultWorkerNumber = "1"
-
-    val userJar = System.getenv("SPARK_YARN_APP_JAR")
-    var workerCores = System.getenv("SPARK_WORKER_CORES")
-    var workerMemory = System.getenv("SPARK_WORKER_MEMORY")
-    var workerNumber = System.getenv("SPARK_WORKER_INSTANCES")
-
-    if (userJar == null)
-      throw new SparkException("env SPARK_YARN_APP_JAR is not set")
-
-    if (workerCores == null)
-      workerCores = defalutWorkerCores
-    if (workerMemory == null)
-      workerMemory = defalutWorkerMemory
-    if (workerNumber == null)
-      workerNumber = defaultWorkerNumber
-
-    val driverHost = conf.get("spark.driver.host")
-    val driverPort = conf.get("spark.driver.port")
-    val hostport = driverHost + ":" + driverPort
-
-    val argsArray = Array[String](
-      "--class", "notused",
-      "--jar", userJar,
-      "--args", hostport,
-      "--worker-memory", workerMemory,
-      "--worker-cores", workerCores,
-      "--num-workers", workerNumber,
-      "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher"
-    )
-
-    val args = new ClientArguments(argsArray)
-    client = new Client(args)
-    appId = client.runApp()
-    waitForApp()
-  }
-
-  def waitForApp() {
-
-    // TODO : need a better way to find out whether the workers are ready or not
-    // maybe by resource usage report?
-    while(true) {
-      val report = client.getApplicationReport(appId)
-
-      logInfo("Application report from ASM: \n" +
-        "\t appMasterRpcPort: " + report.getRpcPort() + "\n" +
-        "\t appStartTime: " + report.getStartTime() + "\n" +
-        "\t yarnAppState: " + report.getYarnApplicationState() + "\n"
-      )
-
-      // Ready to go, or already gone.
-      val state = report.getYarnApplicationState()
-      if (state == YarnApplicationState.RUNNING) {
-        return
-      } else if (state == YarnApplicationState.FINISHED ||
-        state == YarnApplicationState.FAILED ||
-        state == YarnApplicationState.KILLED) {
-        throw new SparkException("Yarn application already ended," +
-          "might be killed or not able to launch application master.")
-      }
-
-      Thread.sleep(1000)
-    }
-  }
-
-  override def stop() {
-    super.stop()
-    client.stop()
-    logInfo("Stoped")
-  }
-
-}
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
deleted file mode 100644
index 2d9fbcb400e5bc07e1af665116910909d7edd118..0000000000000000000000000000000000000000
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.scheduler.cluster
-
-import org.apache.hadoop.conf.Configuration
-
-import org.apache.spark._
-import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler}
-import org.apache.spark.scheduler.TaskSchedulerImpl
-import org.apache.spark.util.Utils
-
-/**
- *
- * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of
- * ApplicationMaster, etc. is done
- */
-private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration)
-  extends TaskSchedulerImpl(sc) {
-
-  logInfo("Created YarnClusterScheduler")
-
-  def this(sc: SparkContext) = this(sc, new Configuration())
-
-  // Nothing else for now ... initialize application master : which needs sparkContext to determine how to allocate
-  // Note that only the first creation of SparkContext influences (and ideally, there must be only one SparkContext, right ?)
-  // Subsequent creations are ignored - since nodes are already allocated by then.
-
-
-  // By default, rack is unknown
-  override def getRackForHost(hostPort: String): Option[String] = {
-    val host = Utils.parseHostPort(hostPort)._1
-    val retval = YarnAllocationHandler.lookupRack(conf, host)
-    if (retval != null) Some(retval) else None
-  }
-
-  override def postStartHook() {
-    val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc)
-    if (sparkContextInitialized){
-      // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt
-      Thread.sleep(3000L)
-    }
-    logInfo("YarnClusterScheduler.postStartHook done")
-  }
-}
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
deleted file mode 100644
index 2941356bc55f9f85ca176c3ef0d23a1a08c6a8e5..0000000000000000000000000000000000000000
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.deploy.yarn
-
-import java.net.URI
-
-import org.scalatest.FunSuite
-import org.scalatest.mock.MockitoSugar
-import org.mockito.Mockito.when
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.FileStatus
-import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.fs.permission.FsAction
-import org.apache.hadoop.yarn.api.records.LocalResource
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
-import org.apache.hadoop.yarn.api.records.LocalResourceType
-import org.apache.hadoop.yarn.util.{Records, ConverterUtils}
-
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
-
-
-class ClientDistributedCacheManagerSuite extends FunSuite with MockitoSugar {
-
-  class MockClientDistributedCacheManager extends ClientDistributedCacheManager {
-    override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): 
-        LocalResourceVisibility = {
-      return LocalResourceVisibility.PRIVATE
-    }
-  }
-  
-  test("test getFileStatus empty") {
-    val distMgr = new ClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val uri = new URI("/tmp/testing")
-    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    val stat = distMgr.getFileStatus(fs, uri, statCache)
-    assert(stat.getPath() === null)
-  }
-
-  test("test getFileStatus cached") {
-    val distMgr = new ClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val uri = new URI("/tmp/testing")
-    val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner", 
-      null, new Path("/tmp/testing"))
-    when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus())
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus)
-    val stat = distMgr.getFileStatus(fs, uri, statCache)
-    assert(stat.getPath().toString() === "/tmp/testing")
-  }
-
-  test("test addResource") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
-
-    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link", 
-      statCache, false)
-    val resource = localResources("link")
-    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
-    assert(resource.getTimestamp() === 0)
-    assert(resource.getSize() === 0)
-    assert(resource.getType() === LocalResourceType.FILE)
-
-    val env = new HashMap[String, String]()
-    distMgr.setDistFilesEnv(env)
-    assert(env("SPARK_YARN_CACHE_FILES") === "file:/foo.invalid.com:8080/tmp/testing#link")
-    assert(env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === "0")
-    assert(env("SPARK_YARN_CACHE_FILES_FILE_SIZES") === "0")
-    assert(env("SPARK_YARN_CACHE_FILES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name())
-
-    distMgr.setDistArchivesEnv(env)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None)
-
-    //add another one and verify both there and order correct
-    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
-      null, new Path("/tmp/testing2"))
-    val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2")
-    when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus)
-    distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2", 
-      statCache, false)
-    val resource2 = localResources("link2")
-    assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2)
-    assert(resource2.getTimestamp() === 10)
-    assert(resource2.getSize() === 20)
-    assert(resource2.getType() === LocalResourceType.FILE)
-
-    val env2 = new HashMap[String, String]()
-    distMgr.setDistFilesEnv(env2)
-    val timestamps = env2("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
-    val files = env2("SPARK_YARN_CACHE_FILES").split(',') 
-    val sizes = env2("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
-    val visibilities = env2("SPARK_YARN_CACHE_FILES_VISIBILITIES") .split(',')
-    assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link")
-    assert(timestamps(0)  === "0")
-    assert(sizes(0)  === "0")
-    assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name())
-
-    assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2")
-    assert(timestamps(1)  === "10")
-    assert(sizes(1)  === "20")
-    assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name())
-  }
-
-  test("test addResource link null") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    when(fs.getFileStatus(destPath)).thenReturn(new FileStatus())
-
-    intercept[Exception] {
-      distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null, 
-        statCache, false)
-    }
-    assert(localResources.get("link") === None)
-    assert(localResources.size === 0)
-  }
-
-  test("test addResource appmaster only") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
-      null, new Path("/tmp/testing"))
-    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
-
-    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", 
-      statCache, true)
-    val resource = localResources("link")
-    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
-    assert(resource.getTimestamp() === 10)
-    assert(resource.getSize() === 20)
-    assert(resource.getType() === LocalResourceType.ARCHIVE)
-
-    val env = new HashMap[String, String]()
-    distMgr.setDistFilesEnv(env)
-    assert(env.get("SPARK_YARN_CACHE_FILES") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None)
-
-    distMgr.setDistArchivesEnv(env)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None)
-    assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None)
-  }
-
-  test("test addResource archive") {
-    val distMgr = new MockClientDistributedCacheManager()
-    val fs = mock[FileSystem]
-    val conf = new Configuration()
-    val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing")
-    val localResources = HashMap[String, LocalResource]()
-    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-    val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", 
-      null, new Path("/tmp/testing"))
-    when(fs.getFileStatus(destPath)).thenReturn(realFileStatus)
-
-    distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", 
-      statCache, false)
-    val resource = localResources("link")
-    assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE)
-    assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath)
-    assert(resource.getTimestamp() === 10)
-    assert(resource.getSize() === 20)
-    assert(resource.getType() === LocalResourceType.ARCHIVE)
-
-    val env = new HashMap[String, String]()
-
-    distMgr.setDistArchivesEnv(env)
-    assert(env("SPARK_YARN_CACHE_ARCHIVES") === "file:/foo.invalid.com:8080/tmp/testing#link")
-    assert(env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === "10")
-    assert(env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === "20")
-    assert(env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name())
-
-    distMgr.setDistFilesEnv(env)
-    assert(env.get("SPARK_YARN_CACHE_FILES") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None)
-    assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None)
-  }
-
-
-}
diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..62fe3e274250f2ad13d37adfa9f353666b0d9ac2
--- /dev/null
+++ b/yarn/stable/pom.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>yarn-parent_2.10</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-yarn_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project YARN Stable API</name>
+
+</project>
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
similarity index 97%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 7c32e0ab9bcb5c41ecfad1688062d66abce86241..69ae14ce8385cfb3b978481d90b2681d02a2ae80 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -42,11 +42,14 @@ import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.apache.spark.util.Utils
 
 
-class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
+                        sparkConf: SparkConf) extends Logging {
 
-  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+  def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
+
+  def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
 
-  private var rpc: YarnRPC = YarnRPC.create(conf)
   private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
   private var appAttemptId: ApplicationAttemptId = _
   private var userThread: Thread = _
@@ -60,7 +63,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   private var isLastAMRetry: Boolean = true
   private var amClient: AMRMClient[ContainerRequest] = _
 
-  private val sparkConf = new SparkConf()
   // Default to numWorkers * 2, with minimum of 3
   private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures",
     math.max(args.numWorkers * 2, 3))
@@ -115,7 +117,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X
     val localDirs = Option(System.getenv("YARN_LOCAL_DIRS"))
       .getOrElse(Option(System.getenv("LOCAL_DIRS"))
-        .getOrElse(""))
+      .getOrElse(""))
 
     if (localDirs.isEmpty()) {
       throw new Exception("Yarn Local dirs can't be empty")
@@ -137,11 +139,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
   }
 
-  private def startUserClass(): Thread  = {
+  private def startUserClass(): Thread = {
     logInfo("Starting the user JAR in a separate Thread")
     val mainMethod = Class.forName(
       args.userClass,
-      false /* initialize */,
+      false /* initialize */ ,
       Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
     val t = new Thread {
       override def run() {
@@ -257,7 +259,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   }
 
   private def launchReporterThread(_sleepTime: Long): Thread = {
-    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+    val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime
 
     val t = new Thread {
       override def run() {
@@ -316,7 +318,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
     logInfo("finishApplicationMaster with " + status)
     // Set tracking URL to empty since we don't have a history server.
-    amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */)
+    amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */)
   }
 
   /**
@@ -351,6 +353,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
     }
   }
+
 }
 
 object ApplicationMaster {
@@ -401,6 +404,7 @@ object ApplicationMaster {
         // This is not only logs, but also ensures that log system is initialized for this instance
         // when we are actually 'run'-ing.
         logInfo("Adding shutdown hook for context " + sc)
+
         override def run() {
           logInfo("Invoking sc stop from shutdown hook")
           sc.stop()
@@ -409,7 +413,7 @@ object ApplicationMaster {
             master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           }
         }
-      } )
+      })
     }
 
     // Wait for initialization to complete and atleast 'some' nodes can get allocated.
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
similarity index 96%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index a75066888c22f63ae374399cfbcb239abcdc5808..440ad5cde54eb4c8812bbb510329a2928c0a051e 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -50,23 +50,25 @@ import org.apache.spark.deploy.SparkHadoopUtil
  * 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.
  */
-class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
+class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
+  extends YarnClientImpl with Logging {
+
+  def this(args: ClientArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
+
+  def this(args: ClientArguments) = this(args, new SparkConf())
 
   var rpc: YarnRPC = YarnRPC.create(conf)
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
   val credentials = UserGroupInformation.getCurrentUser().getCredentials()
   private val SPARK_STAGING: String = ".sparkStaging"
   private val distCacheMgr = new ClientDistributedCacheManager()
-  private val sparkConf = new SparkConf
-
 
   // Staging directory is private! -> rwx--------
   val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short)
   // App files are world-wide readable and owner writable -> rw-r--r--
   val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short)
 
-  def this(args: ClientArguments) = this(new Configuration(), args)
-
   def runApp(): ApplicationId = {
     validateArgs()
     // Initialize and start the client service.
@@ -143,7 +145,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       clusterMetrics.getNumNodeManagers)
 
     val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
-    logInfo("""Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s,
+    logInfo( """Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s,
       queueApplicationCount = %s, queueChildQueueCount = %s""".format(
         queueInfo.getQueueName,
         queueInfo.getCurrentCapacity,
@@ -326,7 +328,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     val env = new HashMap[String, String]()
 
-    Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env)
+    Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env)
     env("SPARK_YARN_MODE") = "true"
     env("SPARK_YARN_STAGING_DIR") = stagingDir
 
@@ -347,7 +349,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     val prefix = " --args "
     val args = clientArgs.userArgs
     val retval = new StringBuilder()
-    for (arg <- args){
+    for (arg <- args) {
       retval.append(prefix).append(" '").append(arg).append("' ")
     }
     retval.toString
@@ -482,10 +484,10 @@ object Client {
     // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes -
     // see Client#setupLaunchEnv().
     System.setProperty("SPARK_YARN_MODE", "true")
+    val sparkConf = new SparkConf()
+    val args = new ClientArguments(argStrings, sparkConf)
 
-    val args = new ClientArguments(argStrings)
-
-    (new Client(args)).run()
+    new Client(args, sparkConf).run()
   }
 
   // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
@@ -495,7 +497,7 @@ object Client {
     }
   }
 
-  def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) {
+  def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) {
     Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
     // If log4j present, ensure ours overrides all others
     if (addLog4j) {
@@ -503,7 +505,7 @@ object Client {
         Path.SEPARATOR + LOG4J_PROP)
     }
     // Normally the users app.jar is last in case conflicts with spark jars
-    val userClasspathFirst = new SparkConf().get("spark.yarn.user.classpath.first", "false")
+    val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false")
       .toBoolean
     if (userClasspathFirst) {
       Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
similarity index 89%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
index 99b824e1295a7d87b9e7753eb353eaf32c1f2673..49248a8516b9cef6941524500965354d9ac30d08 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
@@ -35,9 +35,13 @@ import org.apache.spark.scheduler.SplitInfo
 import org.apache.hadoop.yarn.client.api.AMRMClient
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
 
-class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf)
+  extends Logging {
 
-  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+  def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
+    this(args, new Configuration(), sparkConf)
+
+  def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
 
   private var appAttemptId: ApplicationAttemptId = _
   private var reporterThread: Thread = _
@@ -47,26 +51,27 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
   private var driverClosed:Boolean = false
 
   private var amClient: AMRMClient[ContainerRequest] = _
-  private val sparkConf = new SparkConf
 
-  val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0,
+  val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0,
     conf = sparkConf)._1
   var actor: ActorRef = _
 
   // This actor just working as a monitor to watch on Driver Actor.
   class MonitorActor(driverUrl: String) extends Actor {
 
-    var driver: ActorSelection = null
+    var driver: ActorSelection = _
 
     override def preStart() {
       logInfo("Listen to driver: " + driverUrl)
       driver = context.actorSelection(driverUrl)
+      // Send a hello message thus the connection is actually established, thus we can monitor Lifecycle Events.
+      driver ! "Hello"
       context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
     }
 
     override def receive = {
       case x: DisassociatedEvent =>
-        logInfo("Driver terminated or disconnected! Shutting down.")
+        logInfo(s"Driver terminated or disconnected! Shutting down. $x")
         driverClosed = true
     }
   }
@@ -92,7 +97,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
     // must be <= timeoutInterval/ 2.
     // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM.
     // so atleast 1 minute or timeoutInterval / 10 - whichever is higher.
-    val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L))
+    val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval / 10, 60000L))
     reporterThread = launchReporterThread(interval)
 
     // Wait for the reporter thread to Finish.
@@ -138,8 +143,8 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
         Thread.sleep(100)
       }
     }
-    sparkConf.set("spark.driver.host",  driverHost)
-    sparkConf.set("spark.driver.port",  driverPort.toString)
+    sparkConf.set("spark.driver.host", driverHost)
+    sparkConf.set("spark.driver.port", driverPort.toString)
 
     val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
       driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME)
@@ -168,7 +173,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
     // TODO: Handle container failure
 
     yarnAllocator.addResourceRequests(args.numWorkers)
-    while(yarnAllocator.getNumWorkersRunning < args.numWorkers) {
+    while (yarnAllocator.getNumWorkersRunning < args.numWorkers) {
       yarnAllocator.allocateResources()
       Thread.sleep(100)
     }
@@ -179,7 +184,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
 
   // TODO: We might want to extend this to allocate more containers in case they die !
   private def launchReporterThread(_sleepTime: Long): Thread = {
-    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+    val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime
 
     val t = new Thread {
       override def run() {
@@ -211,7 +216,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
 
   def finishApplicationMaster(status: FinalApplicationStatus) {
     logInfo("finish ApplicationMaster with " + status)
-    amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */)
+    amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */)
   }
 
 }
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
similarity index 97%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index 9f5523c4b97a8811c14a19f9e34c6cd1e32028e7..b7699050bbaeb0f8dbf63b2f9a942eb3b1ab1bab 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -39,12 +39,13 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records}
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 
 
 class WorkerRunnable(
     container: Container,
     conf: Configuration,
+    sparkConf: SparkConf,
     masterAddress: String,
     slaveId: String,
     hostname: String,
@@ -197,7 +198,7 @@ class WorkerRunnable(
   def prepareEnvironment: HashMap[String, String] = {
     val env = new HashMap[String, String]()
 
-    Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
+    Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
 
     // Allow users to specify some environment variables
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
similarity index 99%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index 85ab08ef34db4738f53f95d3d68dadde555df3cb..738ff986d85a58152e4b874a698f380b2b645977 100644
--- a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
 import org.apache.hadoop.yarn.util.{RackResolver, Records}
 
 
-object AllocationType extends Enumeration ("HOST", "RACK", "ANY") {
+object AllocationType extends Enumeration {
   type AllocationType = Value
   val HOST, RACK, ANY = Value
 }
@@ -280,6 +280,7 @@ private[yarn] class YarnAllocationHandler(
           val workerRunnable = new WorkerRunnable(
             container,
             conf,
+            sparkConf,
             driverUrl,
             workerId,
             workerHostname,