From 3dc379ce5aa51cc9c41f590d79c350b5dea33fc3 Mon Sep 17 00:00:00 2001
From: Raymond Liu <raymond.liu@intel.com>
Date: Wed, 4 Dec 2013 13:20:27 +0800
Subject: [PATCH] Reorganize yarn related codes into sub projects to remove
 duplicate files.

---
 project/SparkBuild.scala                      |  23 +-
 {new-yarn => yarn/2.0}/pom.xml                |   0
 .../spark/deploy/yarn/ApplicationMaster.scala |   0
 .../org/apache/spark/deploy/yarn/Client.scala |   0
 .../spark/deploy/yarn/WorkerLauncher.scala    |   0
 .../spark/deploy/yarn/WorkerRunnable.scala    |   0
 .../deploy/yarn/YarnAllocationHandler.scala   |   0
 yarn/2.2/pom.xml                              | 161 +++++++++++++
 .../spark/deploy/yarn/ApplicationMaster.scala |   0
 .../org/apache/spark/deploy/yarn/Client.scala |   0
 .../spark/deploy/yarn/WorkerLauncher.scala    |   0
 .../spark/deploy/yarn/WorkerRunnable.scala    |   0
 .../deploy/yarn/YarnAllocationHandler.scala   |   0
 .../yarn/ApplicationMasterArguments.scala     |   0
 .../spark/deploy/yarn/ClientArguments.scala   |   0
 .../yarn/ClientDistributedCacheManager.scala  |   0
 .../deploy/yarn/YarnSparkHadoopUtil.scala     |   0
 .../ClientDistributedCacheManagerSuite.scala  |   0
 yarn/scheduler/pom.xml                        | 161 +++++++++++++
 .../cluster/YarnClientClusterScheduler.scala  |   0
 .../cluster/YarnClientSchedulerBackend.scala  |   0
 .../cluster/YarnClusterScheduler.scala        |   0
 .../yarn/ApplicationMasterArguments.scala     |  94 --------
 .../spark/deploy/yarn/ClientArguments.scala   | 147 -----------
 .../yarn/ClientDistributedCacheManager.scala  | 228 ------------------
 .../deploy/yarn/YarnSparkHadoopUtil.scala     |  43 ----
 .../cluster/YarnClientClusterScheduler.scala  |  48 ----
 .../cluster/YarnClientSchedulerBackend.scala  | 110 ---------
 .../cluster/YarnClusterScheduler.scala        |  59 -----
 .../ClientDistributedCacheManagerSuite.scala  | 220 -----------------
 30 files changed, 337 insertions(+), 957 deletions(-)
 rename {new-yarn => yarn/2.0}/pom.xml (100%)
 rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (100%)
 rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (100%)
 rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala (100%)
 rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala (100%)
 rename yarn/{ => 2.0}/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala (100%)
 create mode 100644 yarn/2.2/pom.xml
 rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (100%)
 rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (100%)
 rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala (100%)
 rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala (100%)
 rename {new-yarn => yarn/2.2}/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala (100%)
 rename {new-yarn => yarn/common}/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala (100%)
 rename {new-yarn => yarn/common}/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala (100%)
 rename {new-yarn => yarn/common}/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala (100%)
 rename {new-yarn => yarn/common}/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala (100%)
 rename {new-yarn => yarn/common}/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala (100%)
 create mode 100644 yarn/scheduler/pom.xml
 rename {new-yarn => yarn/scheduler}/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala (100%)
 rename {new-yarn => yarn/scheduler}/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala (100%)
 rename {new-yarn => yarn/scheduler}/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala (100%)
 delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
 delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
 delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
 delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
 delete mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
 delete mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
 delete mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
 delete mode 100644 yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala

diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 2eef2dfa5e..b34d42da16 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -85,12 +85,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 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 yarnCommon = Project("yarn-common", file("yarn/common"), settings = yarnCommonSettings) dependsOn(core)
+  lazy val yarnAPI = Project("yarn-api", file(if (isNewHadoop) "yarn/2.2" else "yarn/2.0"), settings = yarnAPISettings) dependsOn(yarnCommon)
+  lazy val yarnScheduler = Project("yarn", file("yarn/scheduler"), settings = yarnSchedulerSettings) dependsOn(yarnAPI)
+  lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarnCommon, yarnAPI, yarnScheduler) else Seq[ClasspathDependency]()
+  lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarnCommon, yarnAPI, yarnScheduler) else Seq[ProjectReference]()
 
   // Everything except assembly, tools and examples belong to packageProjects
   lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef
@@ -320,10 +319,18 @@ object SparkBuild extends Build {
     )
   )
 
-  def yarnSettings = sharedSettings ++ Seq(
-    name := "spark-yarn"
+  def yarnAPISettings = sharedSettings ++ Seq(
+    name := "spark-yarn-api"
   ) ++ extraYarnSettings
 
+  def yarnCommonSettings = sharedSettings ++ Seq(
+    name := "spark-yarn-common"
+  )
+
+  def yarnSchedulerSettings = sharedSettings ++ 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/new-yarn/pom.xml b/yarn/2.0/pom.xml
similarity index 100%
rename from new-yarn/pom.xml
rename to yarn/2.0/pom.xml
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
similarity index 100%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
similarity index 100%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
similarity index 100%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
similarity index 100%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
similarity index 100%
rename from yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
rename to yarn/2.0/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
diff --git a/yarn/2.2/pom.xml b/yarn/2.2/pom.xml
new file mode 100644
index 0000000000..8a065c6d7d
--- /dev/null
+++ b/yarn/2.2/pom.xml
@@ -0,0 +1,161 @@
+<?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.9.3</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.9.3</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.9.3</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/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
rename to yarn/2.2/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
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 100%
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
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/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/scheduler/pom.xml b/yarn/scheduler/pom.xml
new file mode 100644
index 0000000000..4847d52694
--- /dev/null
+++ b/yarn/scheduler/pom.xml
@@ -0,0 +1,161 @@
+<?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.9.3</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project YARN Scheduler</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_2.9.3</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.9.3</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/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/scheduler/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/scheduler/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/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
similarity index 100%
rename from new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
rename to yarn/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/scheduler/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/scheduler/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
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 f76a5ddd39..0000000000
--- 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 9075ca71e7..0000000000
--- 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 5f159b073f..0000000000
--- 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 2ba2366ead..0000000000
--- 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 522e0a9ad7..0000000000
--- 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 4b69f5078b..0000000000
--- 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 2d9fbcb400..0000000000
--- 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 2941356bc5..0000000000
--- 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)
-  }
-
-
-}
-- 
GitLab