diff --git a/core/pom.xml b/core/pom.xml
index 6af229c71d3ad4742dab73dfe9cfab5fac03fe16..38f4be128019c13d177ff3fe6cc4d11f7b5dff5f 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -95,15 +95,15 @@
       <version>0.3.1</version>
     </dependency>
     <dependency>
-      <groupId>com.typesafe.akka</groupId>
+      <groupId>${akka.group}</groupId>
       <artifactId>akka-actor</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.typesafe.akka</groupId>
+      <groupId>${akka.group}</groupId>
       <artifactId>akka-remote</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.typesafe.akka</groupId>
+      <groupId>${akka.group}</groupId>
       <artifactId>akka-slf4j</artifactId>
     </dependency>
     <dependency>
@@ -126,10 +126,6 @@
       <groupId>colt</groupId>
       <artifactId>colt</artifactId>
     </dependency>
-    <dependency>
-      <groupId>com.github.scala-incubator.io</groupId>
-      <artifactId>scala-io-file_2.9.2</artifactId>
-    </dependency>
     <dependency>
       <groupId>org.apache.mesos</groupId>
       <artifactId>mesos</artifactId>
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 3da13133dacde4e342118ab5b28cc1c4593e1d02..66006bf21238d7676ee3827db7cc6b9d695a1cdf 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -81,7 +81,7 @@ class SparkContext(
     val sparkHome: String = null,
     val jars: Seq[String] = Nil,
     val environment: Map[String, String] = Map(),
-    // This is used only by yarn for now, but should be relevant to other cluster types (mesos, etc)
+    // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc)
     // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set
     // of data-local splits on host
     val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] =
diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..8a065c6d7d1d7f458789f3ab50f0054c8a947dfb
--- /dev/null
+++ b/new-yarn/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/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
new file mode 100644
index 0000000000000000000000000000000000000000..eeeca3ea8a33e4f77562383dfef50a7507628894
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -0,0 +1,446 @@
+/*
+ * 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.io.IOException
+import java.net.Socket
+import java.util.concurrent.CopyOnWriteArrayList
+import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
+
+import scala.collection.JavaConversions._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.util.ShutdownHookManager
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+
+import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.util.Utils
+
+
+class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+
+  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+  
+  private var rpc: YarnRPC = YarnRPC.create(conf)
+  private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+  private var appAttemptId: ApplicationAttemptId = _
+  private var userThread: Thread = _
+  private val fs = FileSystem.get(yarnConf)
+
+  private var yarnAllocator: YarnAllocationHandler = _
+  private var isFinished: Boolean = false
+  private var uiAddress: String = _
+  private val maxAppAttempts: Int = conf.getInt(
+    YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)
+  private var isLastAMRetry: Boolean = true
+  private var amClient: AMRMClient[ContainerRequest] = _
+
+  // Default to numWorkers * 2, with minimum of 3
+  private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures",
+    math.max(args.numWorkers * 2, 3).toString()).toInt
+
+  def run() {
+    // Setup the directories so things go to YARN approved directories rather
+    // than user specified and /tmp.
+    System.setProperty("spark.local.dir", getLocalDirs())
+
+    // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using.
+    ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
+
+    appAttemptId = getApplicationAttemptId()
+    isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts
+    amClient = AMRMClient.createAMRMClient()
+    amClient.init(yarnConf)
+    amClient.start()
+
+    // Workaround until hadoop moves to something which has
+    // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line)
+    // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf)
+    
+    ApplicationMaster.register(this)
+
+    // Start the user's JAR
+    userThread = startUserClass()
+    
+    // This a bit hacky, but we need to wait until the spark.driver.port property has
+    // been set by the Thread executing the user class.
+    waitForSparkMaster()
+
+    waitForSparkContextInitialized()
+
+    // Do this after Spark master is up and SparkContext is created so that we can register UI Url.
+    val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
+
+    // Allocate all containers
+    allocateWorkers()
+
+    // Wait for the user class to Finish     
+    userThread.join()
+
+    System.exit(0)
+  }
+
+  /** Get the Yarn approved local directories. */
+  private def getLocalDirs(): String = {
+    // Hadoop 0.23 and 2.x have different Environment variable names for the
+    // local dirs, so lets check both. We assume one of the 2 is set.
+    // 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(""))
+
+    if (localDirs.isEmpty()) {
+      throw new Exception("Yarn Local dirs can't be empty")
+    }
+    localDirs
+  }
+  
+  private def getApplicationAttemptId(): ApplicationAttemptId = {
+    val envs = System.getenv()
+    val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name())
+    val containerId = ConverterUtils.toContainerId(containerIdString)
+    val appAttemptId = containerId.getApplicationAttemptId()
+    logInfo("ApplicationAttemptId: " + appAttemptId)
+    appAttemptId
+  }
+  
+  private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
+    logInfo("Registering the ApplicationMaster")
+    amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
+  }
+  
+  private def waitForSparkMaster() {
+    logInfo("Waiting for Spark driver to be reachable.")
+    var driverUp = false
+    var tries = 0
+    val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt
+    while (!driverUp && tries < numTries) {
+      val driverHost = System.getProperty("spark.driver.host")
+      val driverPort = System.getProperty("spark.driver.port")
+      try {
+        val socket = new Socket(driverHost, driverPort.toInt)
+        socket.close()
+        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
+        driverUp = true
+      } catch {
+        case e: Exception => {
+          logWarning("Failed to connect to driver at %s:%s, retrying ...".
+            format(driverHost, driverPort))
+          Thread.sleep(100)
+          tries = tries + 1
+        }
+      }
+    }
+  }
+
+  private def startUserClass(): Thread  = {
+    logInfo("Starting the user JAR in a separate Thread")
+    val mainMethod = Class.forName(
+      args.userClass,
+      false /* initialize */,
+      Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
+    val t = new Thread {
+      override def run() {
+        var successed = false
+        try {
+          // Copy
+          var mainArgs: Array[String] = new Array[String](args.userArgs.size)
+          args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size)
+          mainMethod.invoke(null, mainArgs)
+          // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR
+          // userThread will stop here unless it has uncaught exception thrown out
+          // It need shutdown hook to set SUCCEEDED
+          successed = true
+        } finally {
+          logDebug("finishing main")
+          isLastAMRetry = true
+          if (successed) {
+            ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+          } else {
+            ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED)
+          }
+        }
+      }
+    }
+    t.start()
+    t
+  }
+
+  // This need to happen before allocateWorkers()
+  private def waitForSparkContextInitialized() {
+    logInfo("Waiting for Spark context initialization")
+    try {
+      var sparkContext: SparkContext = null
+      ApplicationMaster.sparkContextRef.synchronized {
+        var numTries = 0
+        val waitTime = 10000L
+        val maxNumTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt
+        while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) {
+          logInfo("Waiting for Spark context initialization ... " + numTries)
+          numTries = numTries + 1
+          ApplicationMaster.sparkContextRef.wait(waitTime)
+        }
+        sparkContext = ApplicationMaster.sparkContextRef.get()
+        assert(sparkContext != null || numTries >= maxNumTries)
+
+        if (sparkContext != null) {
+          uiAddress = sparkContext.ui.appUIAddress
+          this.yarnAllocator = YarnAllocationHandler.newAllocator(
+            yarnConf,
+            amClient,
+            appAttemptId,
+            args, 
+            sparkContext.preferredNodeLocationData)
+        } else {
+          logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d".
+            format(numTries * waitTime, maxNumTries))
+          this.yarnAllocator = YarnAllocationHandler.newAllocator(
+            yarnConf,
+            amClient,
+            appAttemptId,
+            args)
+        }
+      }
+    } finally {
+      // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT :
+      // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks.
+      ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT)
+    }
+  }
+
+  private def allocateWorkers() {
+    try {
+      logInfo("Allocating " + args.numWorkers + " workers.")
+      // Wait until all containers have finished
+      // TODO: This is a bit ugly. Can we make it nicer?
+      // TODO: Handle container failure
+      yarnAllocator.addResourceRequests(args.numWorkers)
+      // Exits the loop if the user thread exits.
+      while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) {
+        if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+          finishApplicationMaster(FinalApplicationStatus.FAILED,
+            "max number of worker failures reached")
+        }
+        yarnAllocator.allocateResources()
+        ApplicationMaster.incrementAllocatorLoop(1)
+        Thread.sleep(100)
+      }
+    } finally {
+      // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT,
+      // so that the loop in ApplicationMaster#sparkContextInitialized() breaks.
+      ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT)
+    }
+    logInfo("All workers have launched.")
+
+    // Launch a progress reporter thread, else the app will get killed after expiration
+    // (def: 10mins) timeout.
+    if (userThread.isAlive) {
+      // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
+      val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+
+      // we want to be reasonably responsive without causing too many requests to RM.
+      val schedulerInterval =
+        System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong
+
+      // must be <= timeoutInterval / 2.
+      val interval = math.min(timeoutInterval / 2, schedulerInterval)
+
+      launchReporterThread(interval)
+    }
+  }
+
+  private def launchReporterThread(_sleepTime: Long): Thread = {
+    val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+
+    val t = new Thread {
+      override def run() {
+        while (userThread.isAlive) {
+          if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+            finishApplicationMaster(FinalApplicationStatus.FAILED,
+              "max number of worker failures reached")
+          }
+          val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning -
+            yarnAllocator.getNumPendingAllocate
+          if (missingWorkerCount > 0) {
+            logInfo("Allocating %d containers to make up for (potentially) lost containers".
+              format(missingWorkerCount))
+            yarnAllocator.addResourceRequests(missingWorkerCount)
+          }
+          sendProgress()
+          Thread.sleep(sleepTime)
+        }
+      }
+    }
+    // Setting to daemon status, though this is usually not a good idea.
+    t.setDaemon(true)
+    t.start()
+    logInfo("Started progress reporter thread - sleep time : " + sleepTime)
+    t
+  }
+
+  private def sendProgress() {
+    logDebug("Sending progress")
+    // Simulated with an allocate request with no nodes requested.
+    yarnAllocator.allocateResources()
+  }
+
+  /*
+  def printContainers(containers: List[Container]) = {
+    for (container <- containers) {
+      logInfo("Launching shell command on a new container."
+        + ", containerId=" + container.getId()
+        + ", containerNode=" + container.getNodeId().getHost() 
+        + ":" + container.getNodeId().getPort()
+        + ", containerNodeURI=" + container.getNodeHttpAddress()
+        + ", containerState" + container.getState()
+        + ", containerResourceMemory"  
+        + container.getResource().getMemory())
+    }
+  }
+  */
+
+  def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") {
+    synchronized {
+      if (isFinished) {
+        return
+      }
+      isFinished = true
+    }
+
+    logInfo("finishApplicationMaster with " + status)
+    // Set tracking URL to empty since we don't have a history server.
+    amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */)
+  }
+
+  /**
+   * Clean up the staging directory. 
+   */
+  private def cleanupStagingDir() { 
+    var stagingDirPath: Path = null
+    try {
+      val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean
+      if (!preserveFiles) {
+        stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
+        if (stagingDirPath == null) {
+          logError("Staging directory is null")
+          return
+        }
+        logInfo("Deleting staging directory " + stagingDirPath)
+        fs.delete(stagingDirPath, true)
+      }
+    } catch {
+      case ioe: IOException =>
+        logError("Failed to cleanup staging dir " + stagingDirPath, ioe)
+    }
+  }
+
+  // The shutdown hook that runs when a signal is received AND during normal close of the JVM. 
+  class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
+
+    def run() {
+      logInfo("AppMaster received a signal.")
+      // we need to clean up staging dir before HDFS is shut down
+      // make sure we don't delete it until this is the last AM
+      if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
+    }
+  }
+}
+
+object ApplicationMaster {
+  // Number of times to wait for the allocator loop to complete.
+  // Each loop iteration waits for 100ms, so maximum of 3 seconds.
+  // This is to ensure that we have reasonable number of containers before we start
+  // 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
+
+  private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]()
+
+  val sparkContextRef: AtomicReference[SparkContext] =
+    new AtomicReference[SparkContext](null /* initialValue */)
+
+  val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0)
+
+  def incrementAllocatorLoop(by: Int) {
+    val count = yarnAllocatorLoop.getAndAdd(by)
+    if (count >= ALLOCATOR_LOOP_WAIT_COUNT) {
+      yarnAllocatorLoop.synchronized {
+        // to wake threads off wait ...
+        yarnAllocatorLoop.notifyAll()
+      }
+    }
+  }
+
+  def register(master: ApplicationMaster) {
+    applicationMasters.add(master)
+  }
+
+  // TODO(harvey): See whether this should be discarded - it isn't used anywhere atm...
+  def sparkContextInitialized(sc: SparkContext): Boolean = {
+    var modified = false
+    sparkContextRef.synchronized {
+      modified = sparkContextRef.compareAndSet(null, sc)
+      sparkContextRef.notifyAll()
+    }
+
+    // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do
+    // System.exit.
+    // Should not really have to do this, but it helps YARN to evict resources earlier.
+    // Not to mention, prevent the Client from declaring failure even though we exited properly.
+    // Note that this will unfortunately not properly clean up the staging files because it gets
+    // called too late, after the filesystem is already shutdown.
+    if (modified) {
+      Runtime.getRuntime().addShutdownHook(new Thread with Logging { 
+        // 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() 
+          // Best case ...
+          for (master <- applicationMasters) {
+            master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+          }
+        } 
+      } )
+    }
+
+    // Wait for initialization to complete and atleast 'some' nodes can get allocated.
+    yarnAllocatorLoop.synchronized {
+      while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) {
+        yarnAllocatorLoop.wait(1000L)
+      }
+    }
+    modified
+  }
+
+  def main(argStrings: Array[String]) {
+    val args = new ApplicationMasterArguments(argStrings)
+    new ApplicationMaster(args).run()
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f76a5ddd39e90d4998d7712113dd0949823d5e18
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -0,0 +1,94 @@
+/*
+ * 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/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
new file mode 100644
index 0000000000000000000000000000000000000000..94678815e806ad66680880e59ac4f7c01d1fd67c
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -0,0 +1,519 @@
+/*
+ * 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.{InetAddress, UnknownHostException, URI}
+import java.nio.ByteBuffer
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil}
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{Apps, Records}
+
+import org.apache.spark.Logging 
+import org.apache.spark.util.Utils
+import org.apache.spark.deploy.SparkHadoopUtil
+
+
+/**
+ * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The
+ * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster,
+ * which will launch a Spark master process and negotiate resources throughout its duration.
+ */
+class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
+
+  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()
+
+  // 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.
+    init(yarnConf)
+    start()
+
+    // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers).
+    logClusterResourceDetails()
+
+    // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM)
+    // interface).
+
+    // Get a new client application.
+    val newApp = super.createApplication()
+    val newAppResponse = newApp.getNewApplicationResponse()
+    val appId = newAppResponse.getApplicationId()
+
+    verifyClusterResources(newAppResponse)
+
+    // Set up resource and environment variables.
+    val appStagingDir = getAppStagingDir(appId)
+    val localResources = prepareLocalResources(appStagingDir)
+    val launchEnv = setupLaunchEnv(localResources, appStagingDir)
+    val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv)
+
+    // Set up an application submission context.
+    val appContext = newApp.getApplicationSubmissionContext()
+    appContext.setApplicationName(args.appName)
+    appContext.setQueue(args.amQueue)
+    appContext.setAMContainerSpec(amContainer)
+
+    // Memory for the ApplicationMaster.
+    val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
+    memoryResource.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+    appContext.setResource(memoryResource)
+
+    // Finally, submit and monitor the application.
+    submitApp(appContext)
+    appId
+  }
+
+  def run() {
+    val appId = runApp()
+    monitorApplication(appId)
+    System.exit(0)
+  }
+
+  // TODO(harvey): This could just go in ClientArguments.
+  def validateArgs() = {
+    Map(
+      (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
+      (args.userJar == null) -> "Error: You must specify a user jar!",
+      (args.userClass == null) -> "Error: You must specify a user class!",
+      (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!",
+      (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" +
+        "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
+      (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" +
+        "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString)
+    ).foreach { case(cond, errStr) =>
+      if (cond) {
+        logError(errStr)
+        args.printUsageAndExit(1)
+      }
+    }
+  }
+
+  def getAppStagingDir(appId: ApplicationId): String = {
+    SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR
+  }
+
+  def logClusterResourceDetails() {
+    val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
+    logInfo("Got Cluster metric info from ApplicationsManager (ASM), number of NodeManagers: " +
+      clusterMetrics.getNumNodeManagers)
+
+    val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
+    logInfo("""Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s,
+      queueApplicationCount = %s, queueChildQueueCount = %s""".format(
+        queueInfo.getQueueName,
+        queueInfo.getCurrentCapacity,
+        queueInfo.getMaximumCapacity,
+        queueInfo.getApplications.size,
+        queueInfo.getChildQueues.size))
+  }
+
+  def verifyClusterResources(app: GetNewApplicationResponse) = { 
+    val maxMem = app.getMaximumResourceCapability().getMemory()
+    logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
+
+    // If we have requested more then the clusters max for a single resource then exit.
+    if (args.workerMemory > maxMem) {
+      logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.".
+        format(args.workerMemory, maxMem))
+      System.exit(1)
+    }
+    val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
+    if (amMem > maxMem) {
+      logError("Required AM memory (%d) is above the max threshold (%d) of this cluster".
+        format(args.amMemory, maxMem))
+      System.exit(1)
+    }
+
+    // We could add checks to make sure the entire cluster has enough resources but that involves
+    // getting all the node reports and computing ourselves.
+  }
+
+  /** See if two file systems are the same or not. */
+  private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
+    val srcUri = srcFs.getUri()
+    val dstUri = destFs.getUri()
+    if (srcUri.getScheme() == null) {
+      return false
+    }
+    if (!srcUri.getScheme().equals(dstUri.getScheme())) {
+      return false
+    }
+    var srcHost = srcUri.getHost()
+    var dstHost = dstUri.getHost()
+    if ((srcHost != null) && (dstHost != null)) {
+      try {
+        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
+        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
+      } catch {
+        case e: UnknownHostException =>
+          return false
+      }
+      if (!srcHost.equals(dstHost)) {
+        return false
+      }
+    } else if (srcHost == null && dstHost != null) {
+      return false
+    } else if (srcHost != null && dstHost == null) {
+      return false
+    }
+    //check for ports
+    if (srcUri.getPort() != dstUri.getPort()) {
+      return false
+    }
+    return true
+  }
+
+  /** Copy the file into HDFS if needed. */
+  private def copyRemoteFile(
+      dstDir: Path,
+      originalPath: Path,
+      replication: Short,
+      setPerms: Boolean = false): Path = {
+    val fs = FileSystem.get(conf)
+    val remoteFs = originalPath.getFileSystem(conf)
+    var newPath = originalPath
+    if (! compareFs(remoteFs, fs)) {
+      newPath = new Path(dstDir, originalPath.getName())
+      logInfo("Uploading " + originalPath + " to " + newPath)
+      FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
+      fs.setReplication(newPath, replication)
+      if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
+    } 
+    // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
+    // version shows the specific version in the distributed cache configuration
+    val qualPath = fs.makeQualified(newPath)
+    val fc = FileContext.getFileContext(qualPath.toUri(), conf)
+    val destPath = fc.resolvePath(qualPath)
+    destPath
+  }
+
+  def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = {
+    logInfo("Preparing Local resources")
+    // Upload Spark and the application JAR to the remote file system if necessary. Add them as
+    // local resources to the application master.
+    val fs = FileSystem.get(conf)
+
+    val delegTokenRenewer = Master.getMasterPrincipal(conf)
+    if (UserGroupInformation.isSecurityEnabled()) {
+      if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+        logError("Can't get Master Kerberos principal for use as renewer")
+        System.exit(1)
+      }
+    }
+    val dst = new Path(fs.getHomeDirectory(), appStagingDir)
+    val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      val dstFs = dst.getFileSystem(conf)
+      dstFs.addDelegationTokens(delegTokenRenewer, credentials)
+    }
+
+    val localResources = HashMap[String, LocalResource]()
+    FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION))
+
+    val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+
+    Map(
+      Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar,
+      Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF")
+    ).foreach { case(destName, _localPath) =>
+      val localPath: String = if (_localPath != null) _localPath.trim() else ""
+      if (! localPath.isEmpty()) {
+        var localURI = new URI(localPath)
+        // If not specified assume these are in the local filesystem to keep behavior like Hadoop
+        if (localURI.getScheme() == null) {
+          localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString)
+        }
+        val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
+        val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+          destName, statCache)
+      }
+    }
+
+    // Handle jars local to the ApplicationMaster.
+    if ((args.addJars != null) && (!args.addJars.isEmpty())){
+      args.addJars.split(',').foreach { case file: String =>
+        val localURI = new URI(file.trim())
+        val localPath = new Path(localURI)
+        val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+        val destPath = copyRemoteFile(dst, localPath, replication)
+        // Only add the resource to the Spark ApplicationMaster.
+        val appMasterOnly = true
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+          linkname, statCache, appMasterOnly)
+      }
+    }
+
+    // Handle any distributed cache files
+    if ((args.files != null) && (!args.files.isEmpty())){
+      args.files.split(',').foreach { case file: String =>
+        val localURI = new URI(file.trim())
+        val localPath = new Path(localURI)
+        val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+        val destPath = copyRemoteFile(dst, localPath, replication)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, 
+          linkname, statCache)
+      }
+    }
+
+    // Handle any distributed cache archives
+    if ((args.archives != null) && (!args.archives.isEmpty())) {
+      args.archives.split(',').foreach { case file:String =>
+        val localURI = new URI(file.trim())
+        val localPath = new Path(localURI)
+        val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+        val destPath = copyRemoteFile(dst, localPath, replication)
+        distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, 
+          linkname, statCache)
+      }
+    }
+
+    UserGroupInformation.getCurrentUser().addCredentials(credentials)
+    localResources
+  }
+
+  def setupLaunchEnv(
+      localResources: HashMap[String, LocalResource], 
+      stagingDir: String): HashMap[String, String] = {
+    logInfo("Setting up the launch environment")
+    val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
+
+    val env = new HashMap[String, String]()
+
+    Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env)
+    env("SPARK_YARN_MODE") = "true"
+    env("SPARK_YARN_STAGING_DIR") = stagingDir
+
+    // Set the environment variables to be passed on to the Workers.
+    distCacheMgr.setDistFilesEnv(env)
+    distCacheMgr.setDistArchivesEnv(env)
+
+    // Allow users to specify some environment variables.
+    Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
+
+    // Add each SPARK_* key to the environment.
+    System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
+
+    env
+  }
+
+  def userArgsToString(clientArgs: ClientArguments): String = {
+    val prefix = " --args "
+    val args = clientArgs.userArgs
+    val retval = new StringBuilder()
+    for (arg <- args){
+      retval.append(prefix).append(" '").append(arg).append("' ")
+    }
+    retval.toString
+  }
+
+  def createContainerLaunchContext(
+      newApp: GetNewApplicationResponse,
+      localResources: HashMap[String, LocalResource],
+      env: HashMap[String, String]): ContainerLaunchContext = {
+    logInfo("Setting up container launch context")
+    val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
+    amContainer.setLocalResources(localResources)
+    amContainer.setEnvironment(env)
+
+    // TODO: Need a replacement for the following code to fix -Xmx?
+    // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
+    // var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
+    //  ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
+    //    YarnAllocationHandler.MEMORY_OVERHEAD)
+
+    // Extra options for the JVM
+    var JAVA_OPTS = ""
+
+    // Add Xmx for AM memory
+    JAVA_OPTS += "-Xmx" + args.amMemory + "m"
+
+    val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR)
+    JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir
+
+    // TODO: Remove once cpuset version is pushed out.
+    // The context is, default gc for server class machines ends up using all cores to do gc -
+    // hence if there are multiple containers in same node, Spark GC affects all other containers'
+    // performance (which can be that of other Spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in
+    // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset
+    // of cores on a node.
+    val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") &&
+      java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))
+    if (useConcurrentAndIncrementalGC) {
+      // In our expts, using (default) throughput collector has severe perf ramifications in
+      // multi-tenant machines
+      JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
+      JAVA_OPTS += " -XX:+CMSIncrementalMode "
+      JAVA_OPTS += " -XX:+CMSIncrementalPacing "
+      JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
+      JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
+    }
+
+    if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
+      JAVA_OPTS += " " + env("SPARK_JAVA_OPTS")
+    }
+
+    // Command for the ApplicationMaster
+    var javaCommand = "java"
+    val javaHome = System.getenv("JAVA_HOME")
+    if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
+      javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
+    }
+
+    val commands = List[String](
+      javaCommand + 
+      " -server " +
+      JAVA_OPTS +
+      " " + args.amClass +
+      " --class " + args.userClass + 
+      " --jar " + args.userJar +
+      userArgsToString(args) +
+      " --worker-memory " + args.workerMemory +
+      " --worker-cores " + args.workerCores +
+      " --num-workers " + args.numWorkers +
+      " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
+      " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+
+    logInfo("Command for starting the Spark ApplicationMaster: " + commands(0))
+    amContainer.setCommands(commands)
+
+    // Setup security tokens.
+    val dob = new DataOutputBuffer()
+    credentials.writeTokenStorageToStream(dob)
+    amContainer.setTokens(ByteBuffer.wrap(dob.getData()))
+
+    amContainer
+  }
+
+  def submitApp(appContext: ApplicationSubmissionContext) = {
+    // Submit the application to the applications manager.
+    logInfo("Submitting application to ASM")
+    super.submitApplication(appContext)
+  }
+
+  def monitorApplication(appId: ApplicationId): Boolean = {  
+    while (true) {
+      Thread.sleep(1000)
+      val report = super.getApplicationReport(appId)
+
+      logInfo("Application report from ASM: \n" +
+        "\t application identifier: " + appId.toString() + "\n" +
+        "\t appId: " + appId.getId() + "\n" +
+        "\t clientToAMToken: " + report.getClientToAMToken() + "\n" +
+        "\t appDiagnostics: " + report.getDiagnostics() + "\n" +
+        "\t appMasterHost: " + report.getHost() + "\n" +
+        "\t appQueue: " + report.getQueue() + "\n" +
+        "\t appMasterRpcPort: " + report.getRpcPort() + "\n" +
+        "\t appStartTime: " + report.getStartTime() + "\n" +
+        "\t yarnAppState: " + report.getYarnApplicationState() + "\n" +
+        "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" +
+        "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" +
+        "\t appUser: " + report.getUser()
+      )
+
+      val state = report.getYarnApplicationState()
+      val dsStatus = report.getFinalApplicationStatus()
+      if (state == YarnApplicationState.FINISHED || 
+        state == YarnApplicationState.FAILED ||
+        state == YarnApplicationState.KILLED) {
+        return true
+      }
+    }
+    true
+  }
+}
+
+object Client {
+  val SPARK_JAR: String = "spark.jar"
+  val APP_JAR: String = "app.jar"
+  val LOG4J_PROP: String = "log4j.properties"
+
+  def main(argStrings: Array[String]) {
+    // Set an env variable indicating we are running in YARN mode.
+    // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes -
+    // see Client#setupLaunchEnv().
+    System.setProperty("SPARK_YARN_MODE", "true")
+
+    val args = new ClientArguments(argStrings)
+
+    (new Client(args)).run()
+  }
+
+  // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
+  def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) {
+    for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim)
+    }
+  }
+
+  def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) {
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
+    // If log4j present, ensure ours overrides all others
+    if (addLog4j) {
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+        Path.SEPARATOR + LOG4J_PROP)
+    }
+    // Normally the users app.jar is last in case conflicts with spark jars
+    val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false")
+      .toBoolean
+    if (userClasspathFirst) {
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+        Path.SEPARATOR + APP_JAR)
+    }
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Path.SEPARATOR + SPARK_JAR)
+    Client.populateHadoopClasspath(conf, env)
+
+    if (!userClasspathFirst) {
+      Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+        Path.SEPARATOR + APP_JAR)
+    }
+    Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
+      Path.SEPARATOR + "*")
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
new file mode 100644
index 0000000000000000000000000000000000000000..9efb28a9426726c4e41e4287291b8ab939ba942e
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -0,0 +1,148 @@
+/*
+ * 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 scala.collection.mutable.{ArrayBuffer, HashMap}
+
+import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
+import org.apache.spark.util.IntParam
+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]) {
+  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 // MB
+  var workerCores = 1
+  var numWorkers = 2
+  var amQueue = System.getProperty("QUEUE", "default")
+  var amMemory: Int = 512 // MB
+  var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster"
+  var appName: String = "Spark"
+  // TODO
+  var inputFormatInfo: List[InputFormatInfo] = null
+  // TODO(harvey)
+  var priority = 0
+
+  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 ("--num-workers") :: IntParam(value) :: tail =>
+          numWorkers = value
+          args = tail
+
+        case ("--worker-memory") :: MemoryParam(value) :: tail =>
+          workerMemory = 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
+
+        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/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5f159b073f5372dc9e1f73347f88ae8aa48e0cbb
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
@@ -0,0 +1,228 @@
+/*
+ * 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/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c38f33e212fbf2e2bf60fa1dd4793012ea288f81
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
@@ -0,0 +1,223 @@
+/*
+ * 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.Socket
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+import akka.actor._
+import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent}
+import akka.actor.Terminated
+import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.util.{Utils, AkkaUtils}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+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 {
+
+  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+
+  private var appAttemptId: ApplicationAttemptId = _
+  private var reporterThread: Thread = _
+  private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+  private var yarnAllocator: YarnAllocationHandler = _
+  private var driverClosed:Boolean = false
+
+  private var amClient: AMRMClient[ContainerRequest] = _
+
+  val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1
+  var actor: ActorRef = _
+
+  // This actor just working as a monitor to watch on Driver Actor.
+  class MonitorActor(driverUrl: String) extends Actor {
+
+    var driver: ActorRef = null
+
+    override def preStart() {
+      logInfo("Listen to driver: " + driverUrl)
+      driver = context.actorFor(driverUrl)
+      context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+      context.watch(driver) // Doesn't work with remote actors, but useful for testing
+    }
+
+    override def receive = {
+      case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
+        logInfo("Driver terminated or disconnected! Shutting down.")
+        driverClosed = true
+    }
+  }
+
+  def run() {
+
+    amClient = AMRMClient.createAMRMClient()
+    amClient.init(yarnConf)
+    amClient.start()
+
+    appAttemptId = getApplicationAttemptId()
+    registerApplicationMaster()
+
+    waitForSparkMaster()
+
+    // Allocate all containers
+    allocateWorkers()
+
+    // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
+    // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
+
+    val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+    // 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))
+    reporterThread = launchReporterThread(interval)
+
+    // Wait for the reporter thread to Finish.
+    reporterThread.join()
+
+    finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+    actorSystem.shutdown()
+
+    logInfo("Exited")
+    System.exit(0)
+  }
+
+  private def getApplicationAttemptId(): ApplicationAttemptId = {
+    val envs = System.getenv()
+    val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name())
+    val containerId = ConverterUtils.toContainerId(containerIdString)
+    val appAttemptId = containerId.getApplicationAttemptId()
+    logInfo("ApplicationAttemptId: " + appAttemptId)
+    appAttemptId
+  }
+
+  private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
+    logInfo("Registering the ApplicationMaster")
+    // TODO:(Raymond) Find out Spark UI address and fill in here?
+    amClient.registerApplicationMaster(Utils.localHostName(), 0, "")
+  }
+
+  private def waitForSparkMaster() {
+    logInfo("Waiting for Spark driver to be reachable.")
+    var driverUp = false
+    val hostport = args.userArgs(0)
+    val (driverHost, driverPort) = Utils.parseHostPort(hostport)
+    while(!driverUp) {
+      try {
+        val socket = new Socket(driverHost, driverPort)
+        socket.close()
+        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
+        driverUp = true
+      } catch {
+        case e: Exception =>
+          logError("Failed to connect to driver at %s:%s, retrying ...".
+            format(driverHost, driverPort))
+        Thread.sleep(100)
+      }
+    }
+    System.setProperty("spark.driver.host", driverHost)
+    System.setProperty("spark.driver.port", driverPort.toString)
+
+    val driverUrl = "akka://spark@%s:%s/user/%s".format(
+      driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
+    actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM")
+  }
+
+
+  private def allocateWorkers() {
+
+    // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now.
+    val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] =
+      scala.collection.immutable.Map()
+
+    yarnAllocator = YarnAllocationHandler.newAllocator(
+      yarnConf,
+      amClient,
+      appAttemptId,
+      args,
+      preferredNodeLocationData)
+
+    logInfo("Allocating " + args.numWorkers + " workers.")
+    // Wait until all containers have finished
+    // TODO: This is a bit ugly. Can we make it nicer?
+    // TODO: Handle container failure
+
+    yarnAllocator.addResourceRequests(args.numWorkers)
+    while(yarnAllocator.getNumWorkersRunning < args.numWorkers) {
+      yarnAllocator.allocateResources()
+      Thread.sleep(100)
+    }
+
+    logInfo("All workers have launched.")
+
+  }
+
+  // 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 t = new Thread {
+      override def run() {
+        while (!driverClosed) {
+          val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning -
+            yarnAllocator.getNumPendingAllocate
+          if (missingWorkerCount > 0) {
+            logInfo("Allocating %d containers to make up for (potentially) lost containers".
+              format(missingWorkerCount))
+            yarnAllocator.addResourceRequests(missingWorkerCount)
+          }
+          sendProgress()
+          Thread.sleep(sleepTime)
+        }
+      }
+    }
+    // setting to daemon status, though this is usually not a good idea.
+    t.setDaemon(true)
+    t.start()
+    logInfo("Started progress reporter thread - sleep time : " + sleepTime)
+    t
+  }
+
+  private def sendProgress() {
+    logDebug("Sending progress")
+    // simulated with an allocate request with no nodes requested ...
+    yarnAllocator.allocateResources()
+  }
+
+  def finishApplicationMaster(status: FinalApplicationStatus) {
+    logInfo("finish ApplicationMaster with " + status)
+    amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */)
+  }
+
+}
+
+
+object WorkerLauncher {
+  def main(argStrings: Array[String]) {
+    val args = new ApplicationMasterArguments(argStrings)
+    new WorkerLauncher(args).run()
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
new file mode 100644
index 0000000000000000000000000000000000000000..9f5523c4b97a8811c14a19f9e34c6cd1e32028e7
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -0,0 +1,209 @@
+/*
+ * 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 java.nio.ByteBuffer
+import java.security.PrivilegedExceptionAction
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.client.api.NMClient
+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
+
+
+class WorkerRunnable(
+    container: Container,
+    conf: Configuration,
+    masterAddress: String,
+    slaveId: String,
+    hostname: String,
+    workerMemory: Int,
+    workerCores: Int) 
+  extends Runnable with Logging {
+
+  var rpc: YarnRPC = YarnRPC.create(conf)
+  var nmClient: NMClient = _
+  val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+  def run = {
+    logInfo("Starting Worker Container")
+    nmClient = NMClient.createNMClient()
+    nmClient.init(yarnConf)
+    nmClient.start()
+    startContainer
+  }
+
+  def startContainer = {
+    logInfo("Setting up ContainerLaunchContext")
+
+    val ctx = Records.newRecord(classOf[ContainerLaunchContext])
+      .asInstanceOf[ContainerLaunchContext]
+
+    val localResources = prepareLocalResources
+    ctx.setLocalResources(localResources)
+
+    val env = prepareEnvironment
+    ctx.setEnvironment(env)
+
+    // Extra options for the JVM
+    var JAVA_OPTS = ""
+    // Set the JVM memory
+    val workerMemoryString = workerMemory + "m"
+    JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " "
+    if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
+      JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
+    }
+
+    JAVA_OPTS += " -Djava.io.tmpdir=" + 
+      new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
+
+    // Commenting it out for now - so that people can refer to the properties if required. Remove
+    // it once cpuset version is pushed out.
+    // The context is, default gc for server class machines end up using all cores to do gc - hence
+    // if there are multiple containers in same node, spark gc effects all other containers
+    // performance (which can also be other spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+    // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+    // of cores on a node.
+/*
+    else {
+      // If no java_opts specified, default to using -XX:+CMSIncrementalMode
+      // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
+      // want to mess with it.
+      // In our expts, using (default) throughput collector has severe perf ramnifications in
+      // multi-tennent machines
+      // The options are based on
+      // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline
+      JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
+      JAVA_OPTS += " -XX:+CMSIncrementalMode "
+      JAVA_OPTS += " -XX:+CMSIncrementalPacing "
+      JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
+      JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
+    }
+*/
+
+    val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+    val dob = new DataOutputBuffer()
+    credentials.writeTokenStorageToStream(dob)
+    ctx.setTokens(ByteBuffer.wrap(dob.getData()))
+
+    var javaCommand = "java"
+    val javaHome = System.getenv("JAVA_HOME")
+    if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
+      javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
+    }
+
+    val commands = List[String](javaCommand +
+      " -server " +
+      // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
+      // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in
+      // an inconsistent state.
+      // TODO: If the OOM is not recoverable by rescheduling it on different node, then do
+      // 'something' to fail job ... akin to blacklisting trackers in mapred ?
+      " -XX:OnOutOfMemoryError='kill %p' " +
+      JAVA_OPTS +
+      " org.apache.spark.executor.CoarseGrainedExecutorBackend " +
+      masterAddress + " " +
+      slaveId + " " +
+      hostname + " " +
+      workerCores +
+      " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
+      " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+    logInfo("Setting up worker with commands: " + commands)
+    ctx.setCommands(commands)
+
+    // Send the start request to the ContainerManager
+    nmClient.startContainer(container, ctx)
+  }
+
+  private def setupDistributedCache(
+      file: String,
+      rtype: LocalResourceType,
+      localResources: HashMap[String, LocalResource],
+      timestamp: String,
+      size: String, 
+      vis: String) = {
+    val uri = new URI(file)
+    val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+    amJarRsrc.setType(rtype)
+    amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
+    amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
+    amJarRsrc.setTimestamp(timestamp.toLong)
+    amJarRsrc.setSize(size.toLong)
+    localResources(uri.getFragment()) = amJarRsrc
+  }
+
+  def prepareLocalResources: HashMap[String, LocalResource] = {
+    logInfo("Preparing Local resources")
+    val localResources = HashMap[String, LocalResource]()
+
+    if (System.getenv("SPARK_YARN_CACHE_FILES") != null) {
+      val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
+      val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
+      val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',')
+      val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',')
+      for( i <- 0 to distFiles.length - 1) {
+        setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i),
+          fileSizes(i), visibilities(i))
+      }
+    }
+
+    if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) {
+      val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',')
+      val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',')
+      val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',')
+      val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',')
+      for( i <- 0 to distArchives.length - 1) {
+        setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, 
+          timeStamps(i), fileSizes(i), visibilities(i))
+      }
+    }
+
+    logInfo("Prepared Local resources " + localResources)
+    localResources
+  }
+
+  def prepareEnvironment: HashMap[String, String] = {
+    val env = new HashMap[String, String]()
+
+    Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
+
+    // Allow users to specify some environment variables
+    Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
+
+    System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
+    env
+  }
+
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..dba0f7640e67cc88bbe432ec12e218c6103244ff
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -0,0 +1,687 @@
+/*
+ * 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.lang.{Boolean => JBoolean}
+import java.util.{Collections, Set => JSet}
+import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
+import java.util.concurrent.atomic.AtomicInteger
+
+import scala.collection
+import scala.collection.JavaConversions._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
+import org.apache.spark.Logging
+import org.apache.spark.scheduler.SplitInfo
+import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend}
+import org.apache.spark.util.Utils
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId
+import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus}
+import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest}
+import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse}
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.util.{RackResolver, Records}
+
+
+object AllocationType extends Enumeration ("HOST", "RACK", "ANY") {
+  type AllocationType = Value
+  val HOST, RACK, ANY = Value
+}
+
+// TODO:
+// Too many params.
+// Needs to be mt-safe
+// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should
+// make it more proactive and decoupled.
+
+// Note that right now, we assume all node asks as uniform in terms of capabilities and priority
+// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for
+// more info on how we are requesting for containers.
+private[yarn] class YarnAllocationHandler(
+    val conf: Configuration,
+    val amClient: AMRMClient[ContainerRequest],
+    val appAttemptId: ApplicationAttemptId,
+    val maxWorkers: Int,
+    val workerMemory: Int,
+    val workerCores: Int,
+    val preferredHostToCount: Map[String, Int], 
+    val preferredRackToCount: Map[String, Int])
+  extends Logging {
+  // These three are locked on allocatedHostToContainersMap. Complementary data structures
+  // allocatedHostToContainersMap : containers which are running : host, Set<containerid>
+  // allocatedContainerToHostMap: container to host mapping.
+  private val allocatedHostToContainersMap =
+    new HashMap[String, collection.mutable.Set[ContainerId]]()
+
+  private val allocatedContainerToHostMap = new HashMap[ContainerId, String]()
+
+  // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an
+  // allocated node)
+  // As with the two data structures above, tightly coupled with them, and to be locked on
+  // allocatedHostToContainersMap
+  private val allocatedRackCount = new HashMap[String, Int]()
+
+  // Containers which have been released.
+  private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]()
+  // Containers to be released in next request to RM
+  private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean]
+
+  // Number of container requests that have been sent to, but not yet allocated by the
+  // ApplicationMaster.
+  private val numPendingAllocate = new AtomicInteger()
+  private val numWorkersRunning = new AtomicInteger()
+  // Used to generate a unique id per worker
+  private val workerIdCounter = new AtomicInteger()
+  private val lastResponseId = new AtomicInteger()
+  private val numWorkersFailed = new AtomicInteger()
+
+  def getNumPendingAllocate: Int = numPendingAllocate.intValue
+
+  def getNumWorkersRunning: Int = numWorkersRunning.intValue
+
+  def getNumWorkersFailed: Int = numWorkersFailed.intValue
+
+  def isResourceConstraintSatisfied(container: Container): Boolean = {
+    container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+  }
+
+  def releaseContainer(container: Container) {
+    val containerId = container.getId
+    pendingReleaseContainers.put(containerId, true)
+    amClient.releaseAssignedContainer(containerId)
+  }
+
+  def allocateResources() {
+    // We have already set the container request. Poll the ResourceManager for a response.
+    // This doubles as a heartbeat if there are no pending container requests.
+    val progressIndicator = 0.1f
+    val allocateResponse = amClient.allocate(progressIndicator)
+
+    val allocatedContainers = allocateResponse.getAllocatedContainers()
+    if (allocatedContainers.size > 0) {
+      var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size)
+
+      if (numPendingAllocateNow < 0) {
+        numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow)
+      }
+
+      logDebug("""
+        Allocated containers: %d
+        Current worker count: %d
+        Containers released: %s
+        Containers to-be-released: %s
+        Cluster resources: %s
+        """.format(
+          allocatedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers,
+          allocateResponse.getAvailableResources))
+
+      val hostToContainers = new HashMap[String, ArrayBuffer[Container]]()
+
+      for (container <- allocatedContainers) {
+        if (isResourceConstraintSatisfied(container)) {
+          // Add the accepted `container` to the host's list of already accepted,
+          // allocated containers
+          val host = container.getNodeId.getHost
+          val containersForHost = hostToContainers.getOrElseUpdate(host,
+            new ArrayBuffer[Container]())
+          containersForHost += container
+        } else {
+          // Release container, since it doesn't satisfy resource constraints.
+          releaseContainer(container)
+        }
+      }
+
+       // Find the appropriate containers to use.
+      // TODO: Cleanup this group-by...
+      val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
+      val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
+      val offRackContainers = new HashMap[String, ArrayBuffer[Container]]()
+
+      for (candidateHost <- hostToContainers.keySet) {
+        val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0)
+        val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost)
+
+        val remainingContainersOpt = hostToContainers.get(candidateHost)
+        assert(remainingContainersOpt.isDefined)
+        var remainingContainers = remainingContainersOpt.get
+
+        if (requiredHostCount >= remainingContainers.size) {
+          // Since we have <= required containers, add all remaining containers to
+          // `dataLocalContainers`.
+          dataLocalContainers.put(candidateHost, remainingContainers)
+          // There are no more free containers remaining.
+          remainingContainers = null
+        } else if (requiredHostCount > 0) {
+          // Container list has more containers than we need for data locality.
+          // Split the list into two: one based on the data local container count,
+          // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining
+          // containers.
+          val (dataLocal, remaining) = remainingContainers.splitAt(
+            remainingContainers.size - requiredHostCount)
+          dataLocalContainers.put(candidateHost, dataLocal)
+
+          // Invariant: remainingContainers == remaining
+
+          // YARN has a nasty habit of allocating a ton of containers on a host - discourage this.
+          // Add each container in `remaining` to list of containers to release. If we have an
+          // insufficient number of containers, then the next allocation cycle will reallocate
+          // (but won't treat it as data local).
+          // TODO(harvey): Rephrase this comment some more.
+          for (container <- remaining) releaseContainer(container)
+          remainingContainers = null
+        }
+
+        // For rack local containers
+        if (remainingContainers != null) {
+          val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
+          if (rack != null) {
+            val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0)
+            val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) -
+              rackLocalContainers.getOrElse(rack, List()).size
+
+            if (requiredRackCount >= remainingContainers.size) {
+              // Add all remaining containers to to `dataLocalContainers`.
+              dataLocalContainers.put(rack, remainingContainers)
+              remainingContainers = null
+            } else if (requiredRackCount > 0) {
+              // Container list has more containers that we need for data locality.
+              // Split the list into two: one based on the data local container count,
+              // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining
+              // containers.
+              val (rackLocal, remaining) = remainingContainers.splitAt(
+                remainingContainers.size - requiredRackCount)
+              val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack,
+                new ArrayBuffer[Container]())
+
+              existingRackLocal ++= rackLocal
+
+              remainingContainers = remaining
+            }
+          }
+        }
+
+        if (remainingContainers != null) {
+          // Not all containers have been consumed - add them to the list of off-rack containers.
+          offRackContainers.put(candidateHost, remainingContainers)
+        }
+      }
+
+      // Now that we have split the containers into various groups, go through them in order:
+      // first host-local, then rack-local, and finally off-rack.
+      // Note that the list we create below tries to ensure that not all containers end up within
+      // a host if there is a sufficiently large number of hosts/containers.
+      val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size)
+      allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(dataLocalContainers)
+      allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(rackLocalContainers)
+      allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(offRackContainers)
+
+      // Run each of the allocated containers.
+      for (container <- allocatedContainersToProcess) {
+        val numWorkersRunningNow = numWorkersRunning.incrementAndGet()
+        val workerHostname = container.getNodeId.getHost
+        val containerId = container.getId
+
+        val workerMemoryOverhead = (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+        assert(container.getResource.getMemory >= workerMemoryOverhead)
+
+        if (numWorkersRunningNow > maxWorkers) {
+          logInfo("""Ignoring container %s at host %s, since we already have the required number of
+            containers for it.""".format(containerId, workerHostname))
+          releaseContainer(container)
+          numWorkersRunning.decrementAndGet()
+        } else {
+          val workerId = workerIdCounter.incrementAndGet().toString
+          val driverUrl = "akka://spark@%s:%s/user/%s".format(
+            System.getProperty("spark.driver.host"),
+            System.getProperty("spark.driver.port"),
+            CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
+          logInfo("Launching container %s for on host %s".format(containerId, workerHostname))
+
+          // To be safe, remove the container from `pendingReleaseContainers`.
+          pendingReleaseContainers.remove(containerId)
+
+          val rack = YarnAllocationHandler.lookupRack(conf, workerHostname)
+          allocatedHostToContainersMap.synchronized {
+            val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname,
+              new HashSet[ContainerId]())
+
+            containerSet += containerId
+            allocatedContainerToHostMap.put(containerId, workerHostname)
+
+            if (rack != null) {
+              allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+            }
+          }
+          logInfo("Launching WorkerRunnable. driverUrl: %s,  workerHostname: %s".format(driverUrl, workerHostname))
+          val workerRunnable = new WorkerRunnable(
+            container,
+            conf,
+            driverUrl,
+            workerId,
+            workerHostname,
+            workerMemory,
+            workerCores)
+          new Thread(workerRunnable).start()
+        }
+      }
+      logDebug("""
+        Finished allocating %s containers (from %s originally).
+        Current number of workers running: %d,
+        releasedContainerList: %s,
+        pendingReleaseContainers: %s
+        """.format(
+          allocatedContainersToProcess,
+          allocatedContainers,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers))
+    }
+
+    val completedContainers = allocateResponse.getCompletedContainersStatuses()
+    if (completedContainers.size > 0) {
+      logDebug("Completed %d containers".format(completedContainers.size))
+
+      for (completedContainer <- completedContainers) {
+        val containerId = completedContainer.getContainerId
+
+        if (pendingReleaseContainers.containsKey(containerId)) {
+          // YarnAllocationHandler already marked the container for release, so remove it from
+          // `pendingReleaseContainers`.
+          pendingReleaseContainers.remove(containerId)
+        } else {
+          // Decrement the number of workers running. The next iteration of the ApplicationMaster's
+          // reporting thread will take care of allocating.
+          numWorkersRunning.decrementAndGet()
+          logInfo("Completed container %s (state: %s, exit status: %s)".format(
+            containerId,
+            completedContainer.getState,
+            completedContainer.getExitStatus()))
+          // Hadoop 2.2.X added a ContainerExitStatus we should switch to use
+          // there are some exit status' we shouldn't necessarily count against us, but for
+          // now I think its ok as none of the containers are expected to exit
+          if (completedContainer.getExitStatus() != 0) {
+            logInfo("Container marked as failed: " + containerId)
+            numWorkersFailed.incrementAndGet()
+          }
+        }
+
+        allocatedHostToContainersMap.synchronized {
+          if (allocatedContainerToHostMap.containsKey(containerId)) {
+            val hostOpt = allocatedContainerToHostMap.get(containerId)
+            assert(hostOpt.isDefined)
+            val host = hostOpt.get
+
+            val containerSetOpt = allocatedHostToContainersMap.get(host)
+            assert(containerSetOpt.isDefined)
+            val containerSet = containerSetOpt.get
+
+            containerSet.remove(containerId)
+            if (containerSet.isEmpty) {
+              allocatedHostToContainersMap.remove(host)
+            } else {
+              allocatedHostToContainersMap.update(host, containerSet)
+            }
+
+            allocatedContainerToHostMap.remove(containerId)
+
+            // TODO: Move this part outside the synchronized block?
+            val rack = YarnAllocationHandler.lookupRack(conf, host)
+            if (rack != null) {
+              val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1
+              if (rackCount > 0) {
+                allocatedRackCount.put(rack, rackCount)
+              } else {
+                allocatedRackCount.remove(rack)
+              }
+            }
+          }
+        }
+      }
+      logDebug("""
+        Finished processing %d completed containers.
+        Current number of workers running: %d,
+        releasedContainerList: %s,
+        pendingReleaseContainers: %s
+        """.format(
+          completedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers))
+    }
+  }
+
+  def createRackResourceRequests(
+      hostContainers: ArrayBuffer[ContainerRequest]
+    ): ArrayBuffer[ContainerRequest] = {
+    // Generate modified racks and new set of hosts under it before issuing requests.
+    val rackToCounts = new HashMap[String, Int]()
+
+    for (container <- hostContainers) {
+      val candidateHost = container.getNodes.last
+      assert(YarnAllocationHandler.ANY_HOST != candidateHost)
+
+      val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
+      if (rack != null) {
+        var count = rackToCounts.getOrElse(rack, 0)
+        count += 1
+        rackToCounts.put(rack, count)
+      }
+    }
+
+    val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size)
+    for ((rack, count) <- rackToCounts) {
+      requestedContainers ++= createResourceRequests(
+        AllocationType.RACK,
+        rack,
+        count,
+        YarnAllocationHandler.PRIORITY)
+    }
+
+    requestedContainers
+  }
+
+  def allocatedContainersOnHost(host: String): Int = {
+    var retval = 0
+    allocatedHostToContainersMap.synchronized {
+      retval = allocatedHostToContainersMap.getOrElse(host, Set()).size
+    }
+    retval
+  }
+
+  def allocatedContainersOnRack(rack: String): Int = {
+    var retval = 0
+    allocatedHostToContainersMap.synchronized {
+      retval = allocatedRackCount.getOrElse(rack, 0)
+    }
+    retval
+  }
+
+  def addResourceRequests(numWorkers: Int) {
+    val containerRequests: List[ContainerRequest] =
+      if (numWorkers <= 0 || preferredHostToCount.isEmpty) {
+        logDebug("numWorkers: " + numWorkers + ", host preferences: " +
+          preferredHostToCount.isEmpty)
+        createResourceRequests(
+          AllocationType.ANY,
+          resource = null,
+          numWorkers,
+          YarnAllocationHandler.PRIORITY).toList
+      } else {
+        // Request for all hosts in preferred nodes and for numWorkers - 
+        // candidates.size, request by default allocation policy.
+        val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size)
+        for ((candidateHost, candidateCount) <- preferredHostToCount) {
+          val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost)
+
+          if (requiredCount > 0) {
+            hostContainerRequests ++= createResourceRequests(
+              AllocationType.HOST,
+              candidateHost,
+              requiredCount,
+              YarnAllocationHandler.PRIORITY)
+          }
+        }
+        val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests(
+          hostContainerRequests).toList
+
+        val anyContainerRequests = createResourceRequests(
+          AllocationType.ANY,
+          resource = null,
+          numWorkers,
+          YarnAllocationHandler.PRIORITY)
+
+        val containerRequestBuffer = new ArrayBuffer[ContainerRequest](
+          hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size)
+
+        containerRequestBuffer ++= hostContainerRequests
+        containerRequestBuffer ++= rackContainerRequests
+        containerRequestBuffer ++= anyContainerRequests
+        containerRequestBuffer.toList
+      }
+
+    for (request <- containerRequests) {
+      amClient.addContainerRequest(request)
+    }
+
+    if (numWorkers > 0) {
+      numPendingAllocate.addAndGet(numWorkers)
+      logInfo("Will Allocate %d worker containers, each with %d memory".format(
+        numWorkers,
+        (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)))
+    } else {
+      logDebug("Empty allocation request ...")
+    }
+
+    for (request <- containerRequests) {
+      val nodes = request.getNodes
+      var hostStr = if (nodes == null || nodes.isEmpty) {
+        "Any"
+      } else {
+        nodes.last
+      }
+      logInfo("Container request (host: %s, priority: %s, capability: %s".format(
+        hostStr,
+        request.getPriority().getPriority,
+        request.getCapability))
+    }
+  }
+
+  private def createResourceRequests(
+      requestType: AllocationType.AllocationType,
+      resource: String,
+      numWorkers: Int,
+      priority: Int
+    ): ArrayBuffer[ContainerRequest] = {
+
+    // If hostname is specified, then we need at least two requests - node local and rack local.
+    // There must be a third request, which is ANY. That will be specially handled.
+    requestType match {
+      case AllocationType.HOST => {
+        assert(YarnAllocationHandler.ANY_HOST != resource)
+        val hostname = resource
+        val nodeLocal = constructContainerRequests(
+          Array(hostname),
+          racks = null,
+          numWorkers,
+          priority)
+
+        // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler.
+        YarnAllocationHandler.populateRackInfo(conf, hostname)
+        nodeLocal
+      }
+      case AllocationType.RACK => {
+        val rack = resource
+        constructContainerRequests(hosts = null, Array(rack), numWorkers, priority)
+      }
+      case AllocationType.ANY => constructContainerRequests(
+        hosts = null, racks = null, numWorkers, priority)
+      case _ => throw new IllegalArgumentException(
+        "Unexpected/unsupported request type: " + requestType)
+    }
+  }
+
+  private def constructContainerRequests(
+      hosts: Array[String],
+      racks: Array[String],
+      numWorkers: Int,
+      priority: Int
+    ): ArrayBuffer[ContainerRequest] = {
+
+    val memoryResource = Records.newRecord(classOf[Resource])
+    memoryResource.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+
+    val prioritySetting = Records.newRecord(classOf[Priority])
+    prioritySetting.setPriority(priority)
+
+    val requests = new ArrayBuffer[ContainerRequest]()
+    for (i <- 0 until numWorkers) {
+      requests += new ContainerRequest(memoryResource, hosts, racks, prioritySetting)
+    }
+    requests
+  }
+}
+
+object YarnAllocationHandler {
+
+  val ANY_HOST = "*"
+  // All requests are issued with same priority : we do not (yet) have any distinction between 
+  // request types (like map/reduce in hadoop for example)
+  val PRIORITY = 1
+
+  // Additional memory overhead - in mb.
+  val MEMORY_OVERHEAD = 384
+
+  // Host to rack map - saved from allocation requests. We are expecting this not to change.
+  // Note that it is possible for this to change : and ResurceManager will indicate that to us via
+  // update response to allocate. But we are punting on handling that for now.
+  private val hostToRack = new ConcurrentHashMap[String, String]()
+  private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]()
+
+
+  def newAllocator(
+      conf: Configuration,
+      amClient: AMRMClient[ContainerRequest],
+      appAttemptId: ApplicationAttemptId,
+      args: ApplicationMasterArguments
+    ): YarnAllocationHandler = {
+    new YarnAllocationHandler(
+      conf,
+      amClient,
+      appAttemptId,
+      args.numWorkers, 
+      args.workerMemory,
+      args.workerCores,
+      Map[String, Int](),
+      Map[String, Int]())
+  }
+
+  def newAllocator(
+      conf: Configuration,
+      amClient: AMRMClient[ContainerRequest],
+      appAttemptId: ApplicationAttemptId,
+      args: ApplicationMasterArguments,
+      map: collection.Map[String,
+      collection.Set[SplitInfo]]
+    ): YarnAllocationHandler = {
+    val (hostToSplitCount, rackToSplitCount) = generateNodeToWeight(conf, map)
+    new YarnAllocationHandler(
+      conf,
+      amClient,
+      appAttemptId,
+      args.numWorkers, 
+      args.workerMemory,
+      args.workerCores,
+      hostToSplitCount,
+      rackToSplitCount)
+  }
+
+  def newAllocator(
+      conf: Configuration,
+      amClient: AMRMClient[ContainerRequest],
+      appAttemptId: ApplicationAttemptId,
+      maxWorkers: Int,
+      workerMemory: Int,
+      workerCores: Int,
+      map: collection.Map[String, collection.Set[SplitInfo]]
+    ): YarnAllocationHandler = {
+    val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
+    new YarnAllocationHandler(
+      conf,
+      amClient,
+      appAttemptId,
+      maxWorkers,
+      workerMemory,
+      workerCores,
+      hostToCount,
+      rackToCount)
+  }
+
+  // A simple method to copy the split info map.
+  private def generateNodeToWeight(
+      conf: Configuration,
+      input: collection.Map[String, collection.Set[SplitInfo]]
+    ): (Map[String, Int], Map[String, Int]) = {
+
+    if (input == null) {
+      return (Map[String, Int](), Map[String, Int]())
+    }
+
+    val hostToCount = new HashMap[String, Int]
+    val rackToCount = new HashMap[String, Int]
+
+    for ((host, splits) <- input) {
+      val hostCount = hostToCount.getOrElse(host, 0)
+      hostToCount.put(host, hostCount + splits.size)
+
+      val rack = lookupRack(conf, host)
+      if (rack != null){
+        val rackCount = rackToCount.getOrElse(host, 0)
+        rackToCount.put(host, rackCount + splits.size)
+      }
+    }
+
+    (hostToCount.toMap, rackToCount.toMap)
+  }
+
+  def lookupRack(conf: Configuration, host: String): String = {
+    if (!hostToRack.contains(host)) {
+      populateRackInfo(conf, host)
+    }
+    hostToRack.get(host)
+  }
+
+  def fetchCachedHostsForRack(rack: String): Option[Set[String]] = {
+    Option(rackToHostSet.get(rack)).map { set =>
+      val convertedSet: collection.mutable.Set[String] = set
+      // TODO: Better way to get a Set[String] from JSet.
+      convertedSet.toSet
+    }
+  }
+
+  def populateRackInfo(conf: Configuration, hostname: String) {
+    Utils.checkHost(hostname)
+
+    if (!hostToRack.containsKey(hostname)) {
+      // If there are repeated failures to resolve, all to an ignore list.
+      val rackInfo = RackResolver.resolve(conf, hostname)
+      if (rackInfo != null && rackInfo.getNetworkLocation != null) {
+        val rack = rackInfo.getNetworkLocation
+        hostToRack.put(hostname, rack)
+        if (! rackToHostSet.containsKey(rack)) {
+          rackToHostSet.putIfAbsent(rack,
+            Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]()))
+        }
+        rackToHostSet.get(rack).add(hostname)
+
+        // TODO(harvey): Figure out what this comment means...
+        // Since RackResolver caches, we are disabling this for now ...
+      } /* else {
+        // right ? Else we will keep calling rack resolver in case we cant resolve rack info ...
+        hostToRack.put(hostname, null)
+      } */
+    }
+  }
+}
diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2ba2366ead17113c764663ed934a5221f0eeb0ee
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -0,0 +1,43 @@
+/*
+ * 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/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..63a0449e5a0730085554d2b8ae86067135fa8dba
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
@@ -0,0 +1,47 @@
+/*
+ * 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.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 ClusterScheduler(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/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b206780c7806e15c84944db05876f89c8f848040
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
@@ -0,0 +1,109 @@
+/*
+ * 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}
+
+private[spark] class YarnClientSchedulerBackend(
+    scheduler: ClusterScheduler,
+    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 = System.getProperty("spark.driver.host")
+    val driverPort = System.getProperty("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/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..29b3f22e13697b38bc501e2f914d8fc0a202d722
--- /dev/null
+++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler}
+import org.apache.spark.util.Utils
+import org.apache.hadoop.conf.Configuration
+
+/**
+ *
+ * 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 ClusterScheduler(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/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2941356bc55f9f85ca176c3ef0d23a1a08c6a8e5
--- /dev/null
+++ b/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
@@ -0,0 +1,220 @@
+/*
+ * 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/pom.xml b/pom.xml
index 42c1e00e9d11da954ea90e26f5a48a5ff1d09db7..9348c770fb523877db19fc4dca81c63e71962840 100644
--- a/pom.xml
+++ b/pom.xml
@@ -102,7 +102,9 @@
     <java.version>1.5</java.version>
     <scala.version>2.9.3</scala.version>
     <mesos.version>0.13.0</mesos.version>
+    <akka.group>com.typesafe.akka</akka.group>
     <akka.version>2.0.5</akka.version>
+    <protobuf.version>2.4.1</protobuf.version>
     <slf4j.version>1.7.2</slf4j.version>
     <log4j.version>1.2.17</log4j.version>
     <hadoop.version>1.0.4</hadoop.version>
@@ -245,7 +247,7 @@
       <dependency>
         <groupId>com.google.protobuf</groupId>
         <artifactId>protobuf-java</artifactId>
-        <version>2.4.1</version>
+        <version>${protobuf.version}</version>
       </dependency>
       <dependency>
         <groupId>com.twitter</groupId>
@@ -258,7 +260,7 @@
         <version>0.3.1</version>
       </dependency>
       <dependency>
-        <groupId>com.typesafe.akka</groupId>
+        <groupId>${akka.group}</groupId>
         <artifactId>akka-actor</artifactId>
         <version>${akka.version}</version>
         <exclusions>
@@ -269,7 +271,7 @@
         </exclusions>
       </dependency>
       <dependency>
-        <groupId>com.typesafe.akka</groupId>
+        <groupId>${akka.group}</groupId>
         <artifactId>akka-remote</artifactId>
         <version>${akka.version}</version>
         <exclusions>
@@ -280,7 +282,7 @@
         </exclusions>
       </dependency>
       <dependency>
-        <groupId>com.typesafe.akka</groupId>
+        <groupId>${akka.group}</groupId>
         <artifactId>akka-slf4j</artifactId>
         <version>${akka.version}</version>
         <exclusions>
@@ -290,6 +292,17 @@
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <groupId>${akka.group}</groupId>
+        <artifactId>akka-zeromq</artifactId>
+        <version>${akka.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jboss.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
       <dependency>
         <groupId>it.unimi.dsi</groupId>
         <artifactId>fastutil</artifactId>
@@ -300,11 +313,6 @@
         <artifactId>colt</artifactId>
         <version>1.2.0</version>
       </dependency>
-      <dependency>
-        <groupId>com.github.scala-incubator.io</groupId>
-        <artifactId>scala-io-file_2.9.2</artifactId>
-        <version>0.4.1</version>
-      </dependency>
       <dependency>
         <groupId>org.apache.mesos</groupId>
         <artifactId>mesos</artifactId>
@@ -770,6 +778,41 @@
         </dependencies>
       </dependencyManagement>
     </profile>
+
+    <profile>
+      <id>new-yarn</id>
+      <properties>
+        <akka.group>org.spark-project</akka.group>
+        <akka.version>2.0.5-protobuf-2.5-java-1.5</akka.version>
+        <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>
+
+      <dependencyManagement>
+        <dependencies>
+        </dependencies>
+      </dependencyManagement>
+    </profile>
+
     <profile>
       <id>repl-bin</id>
       <activation>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 819dfa29a9e02c714a5260f8a9f4560d7e27f8fe..ac87cffd9fbd434875fad4d205b5c8ecb72310dd 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -28,6 +28,11 @@ object SparkBuild extends Build {
   // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set
   // through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN.
   val DEFAULT_HADOOP_VERSION = "1.0.4"
+
+  // Whether the Hadoop version to build against is 2.2.x, or a variant of it. This can be set
+  // through the SPARK_IS_NEW_HADOOP environment variable.
+  val DEFAULT_IS_NEW_HADOOP = false
+
   val DEFAULT_YARN = false
 
   // HBase version; set as appropriate.
@@ -55,8 +60,6 @@ object SparkBuild extends Build {
 
   lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core)
 
-  lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core)
-
   lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings)
     .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*)
 
@@ -68,14 +71,27 @@ object SparkBuild extends Build {
 
   // Allows build configuration to be set through environment variables
   lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION)
+  lazy val isNewHadoop = scala.util.Properties.envOrNone("SPARK_IS_NEW_HADOOP") match {
+    case None => {
+      val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined
+      (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP)
+    }
+    case Some(v) => v.toBoolean
+  }
   lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match {
     case None => DEFAULT_YARN
     case Some(v) => v.toBoolean
   }
 
+  // Build against a protobuf-2.5 compatible Akka if Hadoop 2 is used.
+  lazy val protobufVersion = if (isNewHadoop) "2.5.0" else "2.4.1"
+  lazy val akkaVersion = if (isNewHadoop) "2.0.5-protobuf-2.5-java-1.5" else "2.0.5"
+  lazy val akkaGroup = if (isNewHadoop) "org.spark-project" else "com.typesafe.akka"
+
   // Conditionally include the yarn sub-project
-  lazy val maybeYarn = if(isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
-  lazy val maybeYarnRef = if(isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
+  lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "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]()
 
   // Everything except assembly, tools and examples belong to packageProjects
   lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef
@@ -211,10 +227,10 @@ object SparkBuild extends Build {
       "com.ning" % "compress-lzf" % "0.8.4",
       "org.xerial.snappy" % "snappy-java" % "1.0.5",
       "org.ow2.asm" % "asm" % "4.0",
-      "com.google.protobuf" % "protobuf-java" % "2.4.1",
-      "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty),
-      "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty),
-      "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty),
+      "com.google.protobuf" % "protobuf-java" % protobufVersion,
+      akkaGroup % "akka-actor" % akkaVersion excludeAll(excludeNetty),
+      akkaGroup % "akka-remote" % akkaVersion excludeAll(excludeNetty),
+      akkaGroup % "akka-slf4j" % akkaVersion excludeAll(excludeNetty),
       "it.unimi.dsi" % "fastutil" % "6.4.4",
       "colt" % "colt" % "1.2.0",
       "net.liftweb" % "lift-json_2.9.2" % "2.5",
@@ -290,7 +306,7 @@ object SparkBuild extends Build {
       "org.eclipse.paho" % "mqtt-client" % "0.4.0",
       "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy),
       "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty),
-      "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty),
+      akkaGroup % "akka-zeromq" % akkaVersion excludeAll(excludeNetty),
       "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1"
         exclude("com.sun.jdmk", "jmxtools")
         exclude("com.sun.jmx", "jmxri")
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 7a9ae6a97ba7ff615e3e24c76923dc941d76b8f3..40892937b8ca5f031a6844938739bc75324f29ff 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -110,15 +110,8 @@
       <artifactId>scala-library</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.typesafe.akka</groupId>
+      <groupId>${akka.group}</groupId>
       <artifactId>akka-zeromq</artifactId>
-      <version>2.0.3</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.jboss.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index a7baf0c36cfd4db3ab2831c073b0af0f830a834e..240ed8b32ae93b7099b7d83adb2fcbf600a444b8 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -22,9 +22,12 @@ import java.net.Socket
 import java.util.concurrent.CopyOnWriteArrayList
 import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
 
+import scala.collection.JavaConversions._
+
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.util.ShutdownHookManager
 import org.apache.hadoop.yarn.api._
 import org.apache.hadoop.yarn.api.records._
@@ -32,38 +35,38 @@ import org.apache.hadoop.yarn.api.protocolrecords._
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+
 import org.apache.spark.{SparkContext, Logging}
 import org.apache.spark.util.Utils
 
-import scala.collection.JavaConversions._
 
 class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
 
   def this(args: ApplicationMasterArguments) = this(args, new Configuration())
   
   private var rpc: YarnRPC = YarnRPC.create(conf)
-  private var resourceManager: AMRMProtocol = null
-  private var appAttemptId: ApplicationAttemptId = null
-  private var userThread: Thread = null
+  private var resourceManager: AMRMProtocol = _
+  private var appAttemptId: ApplicationAttemptId = _
+  private var userThread: Thread = _
   private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
   private val fs = FileSystem.get(yarnConf)
 
-  private var yarnAllocator: YarnAllocationHandler = null
-  private var isFinished:Boolean = false
-  private var uiAddress: String = ""
+  private var yarnAllocator: YarnAllocationHandler = _
+  private var isFinished: Boolean = false
+  private var uiAddress: String = _
   private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES,
     YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES)
   private var isLastAMRetry: Boolean = true
-  // default to numWorkers * 2, with minimum of 3 
+  // default to numWorkers * 2, with minimum of 3
   private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures",
     math.max(args.numWorkers * 2, 3).toString()).toInt
 
   def run() {
-    // setup the directories so things go to yarn approved directories rather
-    // then user specified and /tmp
+    // Setup the directories so things go to yarn approved directories rather
+    // then user specified and /tmp.
     System.setProperty("spark.local.dir", getLocalDirs())
 
-    // use priority 30 as its higher then HDFS. Its same priority as MapReduce is using
+    // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using.
     ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
     
     appAttemptId = getApplicationAttemptId()
@@ -72,9 +75,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
     // Workaround until hadoop moves to something which has
     // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line)
-    // ignore result
+    // ignore result.
     // This does not, unfortunately, always work reliably ... but alleviates the bug a lot of times
-    // Hence args.workerCores = numCore disabled above. Any better option ?
+    // Hence args.workerCores = numCore disabled above. Any better option?
 
     // Compute number of threads for akka
     //val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory()
@@ -100,7 +103,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
     waitForSparkContextInitialized()
 
-    // do this after spark master is up and SparkContext is created so that we can register UI Url
+    // Do this after spark master is up and SparkContext is created so that we can register UI Url
     val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
     
     // Allocate all containers
@@ -119,12 +122,12 @@ 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")
     }
-    return localDirs
+    localDirs
   }
   
   private def getApplicationAttemptId(): ApplicationAttemptId = {
@@ -133,7 +136,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     val containerId = ConverterUtils.toContainerId(containerIdString)
     val appAttemptId = containerId.getApplicationAttemptId()
     logInfo("ApplicationAttemptId: " + appAttemptId)
-    return appAttemptId
+    appAttemptId
   }
   
   private def registerWithResourceManager(): AMRMProtocol = {
@@ -141,7 +144,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       YarnConfiguration.RM_SCHEDULER_ADDRESS,
       YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS))
     logInfo("Connecting to ResourceManager at " + rmAddress)
-    return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
+    rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
   }
   
   private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
@@ -149,12 +152,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest])
       .asInstanceOf[RegisterApplicationMasterRequest]
     appMasterRequest.setApplicationAttemptId(appAttemptId)
-    // Setting this to master host,port - so that the ApplicationReport at client has some sensible info. 
+    // Setting this to master host,port - so that the ApplicationReport at client has some
+    // sensible info. 
     // Users can then monitor stderr/stdout on that node if required.
     appMasterRequest.setHost(Utils.localHostName())
     appMasterRequest.setRpcPort(0)
     appMasterRequest.setTrackingUrl(uiAddress)
-    return resourceManager.registerApplicationMaster(appMasterRequest)
+    resourceManager.registerApplicationMaster(appMasterRequest)
   }
   
   private def waitForSparkMaster() {
@@ -168,21 +172,25 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       try {
         val socket = new Socket(driverHost, driverPort.toInt)
         socket.close()
-        logInfo("Driver now available: " + driverHost + ":" + driverPort)
+        logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
         driverUp = true
       } catch {
-        case e: Exception =>
-          logWarning("Failed to connect to driver at " + driverHost + ":" + driverPort + ", retrying")
-        Thread.sleep(100)
-        tries = tries + 1
+        case e: Exception => {
+          logWarning("Failed to connect to driver at %s:%s, retrying ...".
+            format(driverHost, driverPort))
+          Thread.sleep(100)
+          tries = tries + 1
+        }
       }
     }
   }
 
   private def startUserClass(): Thread  = {
     logInfo("Starting the user JAR in a separate Thread")
-    val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader)
-      .getMethod("main", classOf[Array[String]])
+    val mainMethod = Class.forName(
+      args.userClass,
+      false /* initialize */,
+      Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
     val t = new Thread {
       override def run() {
         var successed = false
@@ -207,7 +215,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       }
     }
     t.start()
-    return t
+    t
   }
 
   // this need to happen before allocateWorkers
@@ -229,13 +237,20 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
         if (null != sparkContext) {
           uiAddress = sparkContext.ui.appUIAddress
-          this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, 
-            appAttemptId, args, sparkContext.preferredNodeLocationData) 
+          this.yarnAllocator = YarnAllocationHandler.newAllocator(
+            yarnConf,
+            resourceManager,
+            appAttemptId,
+            args, 
+            sparkContext.preferredNodeLocationData) 
         } else {
-          logWarning("Unable to retrieve sparkContext inspite of waiting for " + count * waitTime + 
-            ", numTries = " + numTries)
-          this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager,
-            appAttemptId, args)
+          logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d".
+            format(count * waitTime, numTries))
+          this.yarnAllocator = YarnAllocationHandler.newAllocator(
+            yarnConf,
+            resourceManager,
+            appAttemptId,
+            args)
         }
       }
     } finally {
@@ -251,36 +266,39 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       // Wait until all containers have finished
       // TODO: This is a bit ugly. Can we make it nicer?
       // TODO: Handle container failure
-      while(yarnAllocator.getNumWorkersRunning < args.numWorkers &&
-        // If user thread exists, then quit !
-        userThread.isAlive) {
-          if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
-            finishApplicationMaster(FinalApplicationStatus.FAILED,
-              "max number of worker failures reached")
-          }
-          yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
-          ApplicationMaster.incrementAllocatorLoop(1)
-          Thread.sleep(100)
+
+      // Exists the loop if the user thread exits.
+      while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) {
+        if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+          finishApplicationMaster(FinalApplicationStatus.FAILED,
+            "max number of worker failures reached")
+        }
+        yarnAllocator.allocateContainers(
+          math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
+        ApplicationMaster.incrementAllocatorLoop(1)
+        Thread.sleep(100)
       }
     } finally {
-      // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT : 
-      // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks
+      // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT,
+      // so that the loop in ApplicationMaster#sparkContextInitialized() breaks.
       ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT)
     }
     logInfo("All workers have launched.")
 
-    // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
+    // Launch a progress reporter thread, else the app will get killed after expiration
+    // (def: 10mins) timeout.
+    // TODO(harvey): Verify the timeout
     if (userThread.isAlive) {
-      // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
-
+      // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
       val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
 
       // we want to be reasonably responsive without causing too many requests to RM.
-      val schedulerInterval = 
+      val schedulerInterval =
         System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong
 
       // must be <= timeoutInterval / 2.
       val interval = math.min(timeoutInterval / 2, schedulerInterval)
+
       launchReporterThread(interval)
     }
   }
@@ -292,12 +310,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       override def run() {
         while (userThread.isAlive) {
           if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
-            finishApplicationMaster(FinalApplicationStatus.FAILED, 
+            finishApplicationMaster(FinalApplicationStatus.FAILED,
               "max number of worker failures reached")
           }
           val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning
           if (missingWorkerCount > 0) {
-            logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers")
+            logInfo("Allocating %d containers to make up for (potentially) lost containers".
+              format(missingWorkerCount))
             yarnAllocator.allocateContainers(missingWorkerCount)
           }
           else sendProgress()
@@ -305,16 +324,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
         }
       }
     }
-    // setting to daemon status, though this is usually not a good idea.
+    // Setting to daemon status, though this is usually not a good idea.
     t.setDaemon(true)
     t.start()
     logInfo("Started progress reporter thread - sleep time : " + sleepTime)
-    return t
+    t
   }
 
   private def sendProgress() {
     logDebug("Sending progress")
-    // simulated with an allocate request with no nodes requested ...
+    // Simulated with an allocate request with no nodes requested ...
     yarnAllocator.allocateContainers(0)
   }
 
@@ -334,7 +353,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   */
 
   def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") {
-
     synchronized {
       if (isFinished) {
         return
@@ -348,14 +366,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     finishReq.setAppAttemptId(appAttemptId)
     finishReq.setFinishApplicationStatus(status)
     finishReq.setDiagnostics(diagnostics)
-    // set tracking url to empty since we don't have a history server
+    // Set tracking url to empty since we don't have a history server.
     finishReq.setTrackingUrl("")
     resourceManager.finishApplicationMaster(finishReq)
-
   }
 
   /**
-   * clean up the staging directory. 
+   * Clean up the staging directory. 
    */
   private def cleanupStagingDir() { 
     var stagingDirPath: Path = null
@@ -371,13 +388,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
         fs.delete(stagingDirPath, true)
       }
     } catch {
-      case e: IOException =>
-        logError("Failed to cleanup staging dir " + stagingDirPath, e)
+      case ioe: IOException =>
+        logError("Failed to cleanup staging dir " + stagingDirPath, ioe)
     }
   }
 
-  // The shutdown hook that runs when a signal is received AND during normal
-  // close of the JVM. 
+  // The shutdown hook that runs when a signal is received AND during normal close of the JVM. 
   class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
 
     def run() {
@@ -387,15 +403,14 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
     }
   }
- 
 }
 
 object ApplicationMaster {
-  // number of times to wait for the allocator loop to complete.
-  // each loop iteration waits for 100ms, so maximum of 3 seconds.
+  // Number of times to wait for the allocator loop to complete.
+  // Each loop iteration waits for 100ms, so maximum of 3 seconds.
   // This is to ensure that we have reasonable number of containers before we start
-  // 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.
+  // 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)
@@ -413,7 +428,8 @@ object ApplicationMaster {
     applicationMasters.add(master)
   }
 
-  val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null)
+  val sparkContextRef: AtomicReference[SparkContext] =
+    new AtomicReference[SparkContext](null /* initialValue */)
   val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0)
 
   def sparkContextInitialized(sc: SparkContext): Boolean = {
@@ -423,19 +439,21 @@ object ApplicationMaster {
       sparkContextRef.notifyAll()
     }
 
-    // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do System.exit
-    // Should not really have to do this, but it helps yarn to evict resources earlier.
-    // not to mention, prevent Client declaring failure even though we exit'ed properly.
-    // Note that this will unfortunately not properly clean up the staging files because it gets called to 
-    // late and the filesystem is already shutdown.
+    // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do
+    // System.exit.
+    // Should not really have to do this, but it helps YARN to evict resources earlier.
+    // Not to mention, prevent the Client from declaring failure even though we exited properly.
+    // Note that this will unfortunately not properly clean up the staging files because it gets
+    // called too late, after the filesystem is already shutdown.
     if (modified) {
       Runtime.getRuntime().addShutdownHook(new Thread with Logging { 
-        // This is not just to log, but also to ensure that log system is initialized for this instance when we actually are 'run'
+        // 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() 
-          // best case ...
+          // Best case ...
           for (master <- applicationMasters) {
             master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           }
@@ -443,7 +461,7 @@ object ApplicationMaster {
       } )
     }
 
-    // Wait for initialization to complete and atleast 'some' nodes can get allocated
+    // Wait for initialization to complete and atleast 'some' nodes can get allocated.
     yarnAllocatorLoop.synchronized {
       while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) {
         yarnAllocatorLoop.wait(1000L)
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index bb73f6d337ba04f47bd407c24b6311f0a50e10b6..79dd03806523bcea066fcafd05160fe732032d22 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -20,41 +20,46 @@ package org.apache.spark.deploy.yarn
 import java.net.{InetAddress, UnknownHostException, URI}
 import java.nio.ByteBuffer
 
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil}
-import org.apache.hadoop.fs.permission.FsPermission
-import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.net.NetUtils
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.yarn.api._
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.client.YarnClientImpl
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, Records}
 
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
-import scala.collection.JavaConversions._
+import org.apache.spark.Logging 
+import org.apache.spark.util.Utils
+import org.apache.spark.deploy.SparkHadoopUtil
 
-import org.apache.spark.Logging
 
 class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
-  
+
   def this(args: ClientArguments) = this(new Configuration(), args)
-  
+
   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()
 
-  // staging directory is private! -> rwx--------
+  // 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)
+
+  // App files are world-wide readable and owner writable -> rw-r--r--
+  val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) 
 
   // for client user who want to monitor app status by itself.
   def runApp() = {
@@ -89,15 +94,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   }
 
   def validateArgs() = {
-    Map((System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
+    Map(
+      (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
       (args.userJar == null) -> "Error: You must specify a user jar!",
       (args.userClass == null) -> "Error: You must specify a user class!",
       (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!",
-      (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) ->
-        ("Error: AM memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD),
-      (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) ->
-        ("Error: Worker memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString()))
-    .foreach { case(cond, errStr) => 
+      (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be " +
+        "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
+      (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " +
+        "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD)
+    ).foreach { case(cond, errStr) => 
       if (cond) {
         logError(errStr)
         args.printUsageAndExit(1)
@@ -111,19 +117,24 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
   def logClusterResourceDetails() {
     val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
-    logInfo("Got Cluster metric info from ASM, numNodeManagers=" + clusterMetrics.getNumNodeManagers)
+    logInfo("Got Cluster metric info from ASM, numNodeManagers = " +
+      clusterMetrics.getNumNodeManagers)
 
     val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
-    logInfo("Queue info .. queueName=" + queueInfo.getQueueName + ", queueCurrentCapacity=" + queueInfo.getCurrentCapacity +
-      ", queueMaxCapacity=" + queueInfo.getMaximumCapacity + ", queueApplicationCount=" + queueInfo.getApplications.size +
-      ", queueChildQueueCount=" + queueInfo.getChildQueues.size)
+    logInfo("""Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s,
+      queueApplicationCount = %s, queueChildQueueCount = %s""".format(
+        queueInfo.getQueueName,
+        queueInfo.getCurrentCapacity,
+        queueInfo.getMaximumCapacity,
+        queueInfo.getApplications.size,
+        queueInfo.getChildQueues.size))
   }
-  
+
   def verifyClusterResources(app: GetNewApplicationResponse) = { 
     val maxMem = app.getMaximumResourceCapability().getMemory()
     logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
-    
-    // if we have requested more then the clusters max for a single resource then exit.
+
+    // If we have requested more then the clusters max for a single resource then exit.
     if (args.workerMemory > maxMem) {
       logError("the worker size is to large to run on this cluster " + args.workerMemory)
       System.exit(1)
@@ -134,10 +145,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       System.exit(1)
     }
 
-    // We could add checks to make sure the entire cluster has enough resources but that involves getting
-    // all the node reports and computing ourselves 
+    // We could add checks to make sure the entire cluster has enough resources but that involves
+    // getting all the node reports and computing ourselves 
   }
-  
+
   def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = {
     logInfo("Setting up application submission context for ASM")
     val appContext = Records.newRecord(classOf[ApplicationSubmissionContext])
@@ -146,9 +157,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     return appContext
   }
 
-  /*
-   * see if two file systems are the same or not.
-   */
+  /** See if two file systems are the same or not. */
   private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
     val srcUri = srcFs.getUri()
     val dstUri = destFs.getUri()
@@ -183,9 +192,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     return true
   }
 
-  /**
-   * Copy the file into HDFS if needed.
-   */
+  /** Copy the file into HDFS if needed. */
   private def copyRemoteFile(
       dstDir: Path,
       originalPath: Path,
@@ -201,9 +208,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       fs.setReplication(newPath, replication)
       if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
     } 
-    // resolve any symlinks in the URI path so using a "current" symlink
-    // to point to a specific version shows the specific version
-    // in the distributed cache configuration
+    // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
+    // version shows the specific version in the distributed cache configuration
     val qualPath = fs.makeQualified(newPath)
     val fc = FileContext.getFileContext(qualPath.toUri(), conf)
     val destPath = fc.resolvePath(qualPath)
@@ -212,8 +218,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
   def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = {
     logInfo("Preparing Local resources")
-    // Upload Spark and the application JAR to the remote file system if necessary
-    // Add them as local resources to the AM
+    // Upload Spark and the application JAR to the remote file system if necessary. Add them as
+    // local resources to the AM.
     val fs = FileSystem.get(conf)
 
     val delegTokenRenewer = Master.getMasterPrincipal(conf)
@@ -243,7 +249,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         var localURI = new URI(localPath)
         // if not specified assume these are in the local filesystem to keep behavior like Hadoop
         if (localURI.getScheme() == null) {
-          localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString())
+          localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString)
         }
         val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
         val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
@@ -291,7 +297,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     UserGroupInformation.getCurrentUser().addCredentials(credentials)
     return localResources
   }
-  
+
   def setupLaunchEnv(
       localResources: HashMap[String, LocalResource], 
       stagingDir: String): HashMap[String, String] = {
@@ -304,16 +310,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     env("SPARK_YARN_MODE") = "true"
     env("SPARK_YARN_STAGING_DIR") = stagingDir
 
-    // set the environment variables to be passed on to the Workers
+    // Set the environment variables to be passed on to the Workers.
     distCacheMgr.setDistFilesEnv(env)
     distCacheMgr.setDistArchivesEnv(env)
 
-    // allow users to specify some environment variables
+    // Allow users to specify some environment variables.
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
 
-    // Add each SPARK-* key to the environment
+    // Add each SPARK-* key to the environment.
     System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
-    return env
+    env
   }
 
   def userArgsToString(clientArgs: ClientArguments): String = {
@@ -323,13 +329,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     for (arg <- args){
       retval.append(prefix).append(" '").append(arg).append("' ")
     }
-
     retval.toString
   }
 
-  def createContainerLaunchContext(newApp: GetNewApplicationResponse,
-                                   localResources: HashMap[String, LocalResource],
-                                   env: HashMap[String, String]): ContainerLaunchContext = {
+  def createContainerLaunchContext(
+      newApp: GetNewApplicationResponse,
+      localResources: HashMap[String, LocalResource],
+      env: HashMap[String, String]): ContainerLaunchContext = {
     logInfo("Setting up container launch context")
     val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
     amContainer.setLocalResources(localResources)
@@ -337,8 +343,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
 
+    // TODO(harvey): This can probably be a val.
     var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
-        (if (0 != (args.amMemory % minResMemory)) minResMemory else 0) - YarnAllocationHandler.MEMORY_OVERHEAD
+      ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
+        YarnAllocationHandler.MEMORY_OVERHEAD)
 
     // Extra options for the JVM
     var JAVA_OPTS = ""
@@ -349,13 +357,18 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     JAVA_OPTS += " -Djava.io.tmpdir=" + 
       new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
 
-    // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out.
-    // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same
-    // node, spark gc effects all other containers performance (which can also be other spark containers)
-    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is
-    // limited to subset of cores on a node.
-    if (env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))) {
-      // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tenant machines
+    // Commenting it out for now - so that people can refer to the properties if required. Remove
+    // it once cpuset version is pushed out. The context is, default gc for server class machines
+    // end up using all cores to do gc - hence if there are multiple containers in same node,
+    // spark gc effects all other containers performance (which can also be other spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+    // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+    // of cores on a node.
+    val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") &&
+      java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))
+    if (useConcurrentAndIncrementalGC) {
+      // In our expts, using (default) throughput collector has severe perf ramnifications in
+      // multi-tenant machines
       JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
       JAVA_OPTS += " -XX:+CMSIncrementalMode "
       JAVA_OPTS += " -XX:+CMSIncrementalPacing "
@@ -388,28 +401,28 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
     logInfo("Command for the ApplicationMaster: " + commands(0))
     amContainer.setCommands(commands)
-    
+
     val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
-    // Memory for the ApplicationMaster
+    // Memory for the ApplicationMaster.
     capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
     amContainer.setResource(capability)
 
-    // Setup security tokens
+    // Setup security tokens.
     val dob = new DataOutputBuffer()
     credentials.writeTokenStorageToStream(dob)
     amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData()))
 
-    return amContainer
+    amContainer
   }
-  
+
   def submitApp(appContext: ApplicationSubmissionContext) = {
-    // Submit the application to the applications manager
+    // Submit the application to the applications manager.
     logInfo("Submitting application to ASM")
     super.submitApplication(appContext)
   }
-  
+
   def monitorApplication(appId: ApplicationId): Boolean = {  
-    while(true) {
+    while (true) {
       Thread.sleep(1000)
       val report = super.getApplicationReport(appId)
 
@@ -427,16 +440,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
         "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" +
         "\t appUser: " + report.getUser()
       )
-      
+
       val state = report.getYarnApplicationState()
       val dsStatus = report.getFinalApplicationStatus()
       if (state == YarnApplicationState.FINISHED || 
         state == YarnApplicationState.FAILED ||
         state == YarnApplicationState.KILLED) {
-          return true
+        return true
       }
     }
-    return true
+    true
   }
 }
 
@@ -469,7 +482,7 @@ object Client {
       Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + 
         Path.SEPARATOR + LOG4J_PROP)
     }
-    // normally the users app.jar is last in case conflicts with spark jars
+    // Normally the users app.jar is last in case conflicts with spark jars
     val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false")
       .toBoolean
     if (userClasspathFirst) {
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index a4d6e1d87d1270fd48e8f0f6273f8cfd076e7e64..6a90cc51cfbaf71ef5683703dd84fa6d5b872ccb 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -21,52 +21,59 @@ import java.net.URI
 import java.nio.ByteBuffer
 import java.security.PrivilegedExceptionAction
 
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.DataOutputBuffer
 import org.apache.hadoop.net.NetUtils
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
 import org.apache.hadoop.yarn.api.records._
 import org.apache.hadoop.yarn.api.protocolrecords._
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils}
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-
-import scala.collection.JavaConversions._
-import scala.collection.mutable.HashMap
 
 import org.apache.spark.Logging
 
-class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String,
-    slaveId: String, hostname: String, workerMemory: Int, workerCores: Int) 
-    extends Runnable with Logging {
-  
+
+class WorkerRunnable(
+    container: Container,
+    conf: Configuration,
+    masterAddress: String,
+    slaveId: String,
+    hostname: String,
+    workerMemory: Int,
+    workerCores: Int) 
+  extends Runnable with Logging {
+
   var rpc: YarnRPC = YarnRPC.create(conf)
   var cm: ContainerManager = null
   val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
-  
+
   def run = {
     logInfo("Starting Worker Container")
     cm = connectToCM
     startContainer
   }
-  
+
   def startContainer = {
     logInfo("Setting up ContainerLaunchContext")
-    
+
     val ctx = Records.newRecord(classOf[ContainerLaunchContext])
       .asInstanceOf[ContainerLaunchContext]
-    
+
     ctx.setContainerId(container.getId())
     ctx.setResource(container.getResource())
     val localResources = prepareLocalResources
     ctx.setLocalResources(localResources)
-    
+
     val env = prepareEnvironment
     ctx.setEnvironment(env)
-    
+
     // Extra options for the JVM
     var JAVA_OPTS = ""
     // Set the JVM memory
@@ -79,17 +86,21 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
     JAVA_OPTS += " -Djava.io.tmpdir=" + 
       new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
 
-
-    // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out.
-    // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same
-    // node, spark gc effects all other containers performance (which can also be other spark containers)
-    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is
-    // limited to subset of cores on a node.
+    // Commenting it out for now - so that people can refer to the properties if required. Remove
+    // it once cpuset version is pushed out.
+    // The context is, default gc for server class machines end up using all cores to do gc - hence
+    // if there are multiple containers in same node, spark gc effects all other containers
+    // performance (which can also be other spark containers)
+    // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+    // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+    // of cores on a node.
 /*
     else {
       // If no java_opts specified, default to using -XX:+CMSIncrementalMode
-      // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont want to mess with it.
-      // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tennent machines
+      // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
+      // want to mess with it.
+      // In our expts, using (default) throughput collector has severe perf ramnifications in
+      // multi-tennent machines
       // The options are based on
       // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline
       JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
@@ -116,8 +127,10 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
     val commands = List[String](javaCommand +
       " -server " +
       // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
-      // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state.
-      // TODO: If the OOM is not recoverable by rescheduling it on different node, then do 'something' to fail job ... akin to blacklisting trackers in mapred ?
+      // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in
+      // an inconsistent state.
+      // TODO: If the OOM is not recoverable by rescheduling it on different node, then do
+      // 'something' to fail job ... akin to blacklisting trackers in mapred ?
       " -XX:OnOutOfMemoryError='kill %p' " +
       JAVA_OPTS +
       " org.apache.spark.executor.CoarseGrainedExecutorBackend " +
@@ -129,7 +142,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
       " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
     logInfo("Setting up worker with commands: " + commands)
     ctx.setCommands(commands)
-    
+
     // Send the start request to the ContainerManager
     val startReq = Records.newRecord(classOf[StartContainerRequest])
     .asInstanceOf[StartContainerRequest]
@@ -137,7 +150,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
     cm.startContainer(startReq)
   }
 
-  private def setupDistributedCache(file: String,
+  private def setupDistributedCache(
+      file: String,
       rtype: LocalResourceType,
       localResources: HashMap[String, LocalResource],
       timestamp: String,
@@ -152,12 +166,11 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
     amJarRsrc.setSize(size.toLong)
     localResources(uri.getFragment()) = amJarRsrc
   }
-  
-  
+
   def prepareLocalResources: HashMap[String, LocalResource] = {
     logInfo("Preparing Local resources")
     val localResources = HashMap[String, LocalResource]()
-    
+
     if (System.getenv("SPARK_YARN_CACHE_FILES") != null) {
       val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
       val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
@@ -179,30 +192,30 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
           timeStamps(i), fileSizes(i), visibilities(i))
       }
     }
-    
+
     logInfo("Prepared Local resources " + localResources)
     return localResources
   }
-  
+
   def prepareEnvironment: HashMap[String, String] = {
     val env = new HashMap[String, String]()
 
     Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
 
-    // allow users to specify some environment variables
+    // Allow users to specify some environment variables
     Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
 
     System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
     return env
   }
-  
+
   def connectToCM: ContainerManager = {
     val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort()
     val cmAddress = NetUtils.createSocketAddr(cmHostPortStr)
     logInfo("Connecting to ContainerManager at " + cmHostPortStr)
 
-    // use doAs and remoteUser here so we can add the container token and not 
-    // pollute the current users credentials with all of the individual container tokens
+    // Use doAs and remoteUser here so we can add the container token and not pollute the current
+    // users credentials with all of the individual container tokens
     val user = UserGroupInformation.createRemoteUser(container.getId().toString())
     val containerToken = container.getContainerToken()
     if (containerToken != null) {
@@ -218,5 +231,5 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
         })
     proxy
   }
-  
+
 }
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index 507a0743fd77abf8a7feed9aa953f1d52b4e9b35..f15f3c7c1195c61cf8ad31987e8879d0885c8233 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -17,55 +17,70 @@
 
 package org.apache.spark.deploy.yarn
 
+import java.lang.{Boolean => JBoolean}
+import java.util.{Collections, Set => JSet}
+import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
+import java.util.concurrent.atomic.AtomicInteger
+
+import scala.collection
+import scala.collection.JavaConversions._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
 import org.apache.spark.Logging
-import org.apache.spark.util.Utils
 import org.apache.spark.scheduler.SplitInfo
-import scala.collection
-import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId, ContainerId, Priority, Resource, ResourceRequest, ContainerStatus, Container}
 import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend}
+import org.apache.spark.util.Utils
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.AMRMProtocol
+import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId}
+import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus}
+import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest}
 import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse}
 import org.apache.hadoop.yarn.util.{RackResolver, Records}
-import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
-import java.util.concurrent.atomic.AtomicInteger
-import org.apache.hadoop.yarn.api.AMRMProtocol
-import collection.JavaConversions._
-import collection.mutable.{ArrayBuffer, HashMap, HashSet}
-import org.apache.hadoop.conf.Configuration
-import java.util.{Collections, Set => JSet}
-import java.lang.{Boolean => JBoolean}
+
 
 object AllocationType extends Enumeration ("HOST", "RACK", "ANY") {
   type AllocationType = Value
   val HOST, RACK, ANY = Value
 }
 
-// too many params ? refactor it 'somehow' ?
-// needs to be mt-safe
-// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive : should make it 
-// more proactive and decoupled.
+// TODO:
+// Too many params.
+// Needs to be mt-safe
+// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should
+// make it more proactive and decoupled.
+
 // Note that right now, we assume all node asks as uniform in terms of capabilities and priority
-// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for more info
-// on how we are requesting for containers.
-private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceManager: AMRMProtocol, 
-                                          val appAttemptId: ApplicationAttemptId,
-                                          val maxWorkers: Int, val workerMemory: Int, val workerCores: Int,
-                                          val preferredHostToCount: Map[String, Int], 
-                                          val preferredRackToCount: Map[String, Int])
+// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for
+// more info on how we are requesting for containers.
+private[yarn] class YarnAllocationHandler(
+    val conf: Configuration,
+    val resourceManager: AMRMProtocol, 
+    val appAttemptId: ApplicationAttemptId,
+    val maxWorkers: Int,
+    val workerMemory: Int,
+    val workerCores: Int,
+    val preferredHostToCount: Map[String, Int], 
+    val preferredRackToCount: Map[String, Int])
   extends Logging {
-
-
   // These three are locked on allocatedHostToContainersMap. Complementary data structures
   // allocatedHostToContainersMap : containers which are running : host, Set<containerid>
-  // allocatedContainerToHostMap: container to host mapping
-  private val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]()
+  // allocatedContainerToHostMap: container to host mapping.
+  private val allocatedHostToContainersMap =
+    new HashMap[String, collection.mutable.Set[ContainerId]]()
+
   private val allocatedContainerToHostMap = new HashMap[ContainerId, String]()
-  // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an allocated node)
-  // As with the two data structures above, tightly coupled with them, and to be locked on allocatedHostToContainersMap
+
+  // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an
+  // allocated node)
+  // As with the two data structures above, tightly coupled with them, and to be locked on
+  // allocatedHostToContainersMap
   private val allocatedRackCount = new HashMap[String, Int]()
 
-  // containers which have been released.
+  // Containers which have been released.
   private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]()
-  // containers to be released in next request to RM
+  // Containers to be released in next request to RM
   private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean]
 
   private val numWorkersRunning = new AtomicInteger()
@@ -83,23 +98,31 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
   }
 
   def allocateContainers(workersToRequest: Int) {
-    // We need to send the request only once from what I understand ... but for now, not modifying this much.
+    // We need to send the request only once from what I understand ... but for now, not modifying
+    // this much.
 
     // Keep polling the Resource Manager for containers
     val amResp = allocateWorkerResources(workersToRequest).getAMResponse
 
     val _allocatedContainers = amResp.getAllocatedContainers()
-    if (_allocatedContainers.size > 0) {
-
 
-      logDebug("Allocated " + _allocatedContainers.size + " containers, current count " + 
-        numWorkersRunning.get() + ", to-be-released " + releasedContainerList + 
-        ", pendingReleaseContainers : " + pendingReleaseContainers)
-      logDebug("Cluster Resources: " + amResp.getAvailableResources)
+    if (_allocatedContainers.size > 0) {
+      logDebug("""
+        Allocated containers: %d
+        Current worker count: %d
+        Containers released: %s
+        Containers to be released: %s
+        Cluster resources: %s
+        """.format(
+          _allocatedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers,
+          amResp.getAvailableResources))
 
       val hostToContainers = new HashMap[String, ArrayBuffer[Container]]()
 
-      // ignore if not satisfying constraints      {
+      // Ignore if not satisfying constraints      {
       for (container <- _allocatedContainers) {
         if (isResourceConstraintSatisfied(container)) {
           // allocatedContainers += container
@@ -113,8 +136,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
         else releasedContainerList.add(container.getId())
       }
 
-      // Find the appropriate containers to use
-      // Slightly non trivial groupBy I guess ...
+      // Find the appropriate containers to use. Slightly non trivial groupBy ...
       val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
       val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
       val offRackContainers = new HashMap[String, ArrayBuffer[Container]]()
@@ -134,21 +156,22 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
           remainingContainers = null
         }
         else if (requiredHostCount > 0) {
-          // container list has more containers than we need for data locality.
-          // Split into two : data local container count of (remainingContainers.size - requiredHostCount) 
-          // and rest as remainingContainer
-          val (dataLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredHostCount)
+          // Container list has more containers than we need for data locality.
+          // Split into two : data local container count of (remainingContainers.size -
+          // requiredHostCount) and rest as remainingContainer
+          val (dataLocal, remaining) = remainingContainers.splitAt(
+            remainingContainers.size - requiredHostCount)
           dataLocalContainers.put(candidateHost, dataLocal)
           // remainingContainers = remaining
 
           // yarn has nasty habit of allocating a tonne of containers on a host - discourage this :
-          // add remaining to release list. If we have insufficient containers, next allocation cycle 
-          // will reallocate (but wont treat it as data local)
+          // add remaining to release list. If we have insufficient containers, next allocation 
+          // cycle will reallocate (but wont treat it as data local)
           for (container <- remaining) releasedContainerList.add(container.getId())
           remainingContainers = null
         }
 
-        // now rack local
+        // Now rack local
         if (remainingContainers != null){
           val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
 
@@ -161,15 +184,17 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
             if (requiredRackCount >= remainingContainers.size){
               // Add all to dataLocalContainers
               dataLocalContainers.put(rack, remainingContainers)
-              // all consumed
+              // All consumed
               remainingContainers = null
             }
             else if (requiredRackCount > 0) {
               // container list has more containers than we need for data locality.
-              // Split into two : data local container count of (remainingContainers.size - requiredRackCount) 
-              // and rest as remainingContainer
-              val (rackLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredRackCount)
-              val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, new ArrayBuffer[Container]())
+              // Split into two : data local container count of (remainingContainers.size -
+              // requiredRackCount) and rest as remainingContainer
+              val (rackLocal, remaining) = remainingContainers.splitAt(
+                remainingContainers.size - requiredRackCount)
+              val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack,
+                new ArrayBuffer[Container]())
 
               existingRackLocal ++= rackLocal
               remainingContainers = remaining
@@ -185,8 +210,8 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
 
       // Now that we have split the containers into various groups, go through them in order : 
       // first host local, then rack local and then off rack (everything else).
-      // Note that the list we create below tries to ensure that not all containers end up within a host 
-      // if there are sufficiently large number of hosts/containers.
+      // Note that the list we create below tries to ensure that not all containers end up within a
+      // host if there are sufficiently large number of hosts/containers.
 
       val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size)
       allocatedContainers ++= ClusterScheduler.prioritizeContainers(dataLocalContainers)
@@ -199,33 +224,39 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
         val workerHostname = container.getNodeId.getHost
         val containerId = container.getId
 
-        assert (container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))
+        assert(
+          container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))
 
         if (numWorkersRunningNow > maxWorkers) {
-          logInfo("Ignoring container " + containerId + " at host " + workerHostname + 
-            " .. we already have required number of containers")
+          logInfo("""Ignoring container %s at host %s, since we already have the required number of
+            containers for it.""".format(containerId, workerHostname))
           releasedContainerList.add(containerId)
           // reset counter back to old value.
           numWorkersRunning.decrementAndGet()
         }
         else {
-          // deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter)
+          // Deallocate + allocate can result in reusing id's wrongly - so use a different counter
+          // (workerIdCounter)
           val workerId = workerIdCounter.incrementAndGet().toString
           val driverUrl = "akka://spark@%s:%s/user/%s".format(
             System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
             CoarseGrainedSchedulerBackend.ACTOR_NAME)
 
           logInfo("launching container on " + containerId + " host " + workerHostname)
-          // just to be safe, simply remove it from pendingReleaseContainers. Should not be there, but ..
+          // Just to be safe, simply remove it from pendingReleaseContainers.
+          // Should not be there, but ..
           pendingReleaseContainers.remove(containerId)
 
           val rack = YarnAllocationHandler.lookupRack(conf, workerHostname)
           allocatedHostToContainersMap.synchronized {
-            val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, new HashSet[ContainerId]())
+            val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname,
+              new HashSet[ContainerId]())
 
             containerSet += containerId
             allocatedContainerToHostMap.put(containerId, workerHostname)
-            if (rack != null) allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+            if (rack != null) {
+              allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+            }
           }
 
           new Thread(
@@ -234,17 +265,23 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
           ).start()
         }
       }
-      logDebug("After allocated " + allocatedContainers.size + " containers (orig : " + 
-        _allocatedContainers.size + "), current count " + numWorkersRunning.get() +
-        ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers)
+      logDebug("""
+        Finished processing %d containers.
+        Current number of workers running: %d,
+        releasedContainerList: %s,
+        pendingReleaseContainers: %s
+        """.format(
+          allocatedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers))
     }
 
 
     val completedContainers = amResp.getCompletedContainersStatuses()
     if (completedContainers.size > 0){
-      logDebug("Completed " + completedContainers.size + " containers, current count " + numWorkersRunning.get() +
-        ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers)
-
+      logDebug("Completed %d containers, to-be-released: %s".format(
+        completedContainers.size, releasedContainerList))
       for (completedContainer <- completedContainers){
         val containerId = completedContainer.getContainerId
 
@@ -253,16 +290,17 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
           pendingReleaseContainers.remove(containerId)
         }
         else {
-          // simply decrement count - next iteration of ReporterThread will take care of allocating !
+          // Simply decrement count - next iteration of ReporterThread will take care of allocating.
           numWorkersRunning.decrementAndGet()
-          logInfo("Container completed not by us ? nodeId: " + containerId + ", state " + completedContainer.getState +
-            " httpaddress: " + completedContainer.getDiagnostics + " exit status: " + completedContainer.getExitStatus())
-
+          logInfo("Completed container %s (state: %s, exit status: %s)".format(
+            containerId,
+            completedContainer.getState,
+            completedContainer.getExitStatus()))
           // Hadoop 2.2.X added a ContainerExitStatus we should switch to use
           // there are some exit status' we shouldn't necessarily count against us, but for
           // now I think its ok as none of the containers are expected to exit
           if (completedContainer.getExitStatus() != 0) {
-            logInfo("Container marked as failed: " + containerId) 
+            logInfo("Container marked as failed: " + containerId)
             numWorkersFailed.incrementAndGet()
           }
         }
@@ -281,7 +319,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
 
             allocatedContainerToHostMap -= containerId
 
-            // doing this within locked context, sigh ... move to outside ?
+            // Doing this within locked context, sigh ... move to outside ?
             val rack = YarnAllocationHandler.lookupRack(conf, host)
             if (rack != null) {
               val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1
@@ -291,9 +329,16 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
           }
         }
       }
-      logDebug("After completed " + completedContainers.size + " containers, current count " + 
-        numWorkersRunning.get() + ", to-be-released " + releasedContainerList + 
-        ", pendingReleaseContainers : " + pendingReleaseContainers)
+      logDebug("""
+        Finished processing %d completed containers.
+        Current number of workers running: %d,
+        releasedContainerList: %s,
+        pendingReleaseContainers: %s
+        """.format(
+          completedContainers.size,
+          numWorkersRunning.get(),
+          releasedContainerList,
+          pendingReleaseContainers))
     }
   }
 
@@ -347,7 +392,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
 
       // default.
     if (numWorkers <= 0 || preferredHostToCount.isEmpty) {
-      logDebug("numWorkers: " + numWorkers + ", host preferences ? " + preferredHostToCount.isEmpty)
+      logDebug("numWorkers: " + numWorkers + ", host preferences: " + preferredHostToCount.isEmpty)
       resourceRequests = List(
         createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY))
     }
@@ -360,17 +405,24 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
         val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost)
 
         if (requiredCount > 0) {
-          hostContainerRequests += 
-            createResourceRequest(AllocationType.HOST, candidateHost, requiredCount, YarnAllocationHandler.PRIORITY)
+          hostContainerRequests += createResourceRequest(
+            AllocationType.HOST,
+            candidateHost,
+            requiredCount,
+            YarnAllocationHandler.PRIORITY)
         }
       }
-      val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(hostContainerRequests.toList)
+      val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(
+        hostContainerRequests.toList)
 
-      val anyContainerRequests: ResourceRequest = 
-        createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)
+      val anyContainerRequests: ResourceRequest = createResourceRequest(
+        AllocationType.ANY,
+        resource = null,
+        numWorkers,
+        YarnAllocationHandler.PRIORITY)
 
-      val containerRequests: ArrayBuffer[ResourceRequest] =
-        new ArrayBuffer[ResourceRequest](hostContainerRequests.size() + rackContainerRequests.size() + 1)
+      val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](
+        hostContainerRequests.size + rackContainerRequests.size + 1)
 
       containerRequests ++= hostContainerRequests
       containerRequests ++= rackContainerRequests
@@ -389,52 +441,59 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
     req.addAllReleases(releasedContainerList)
 
     if (numWorkers > 0) {
-      logInfo("Allocating " + numWorkers + " worker containers with " + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + " of memory each.")
+      logInfo("Allocating %d worker containers with %d of memory each.".format(numWorkers,
+        workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))
     }
     else {
       logDebug("Empty allocation req ..  release : " + releasedContainerList)
     }
 
-    for (req <- resourceRequests) {
-      logInfo("rsrcRequest ... host : " + req.getHostName + ", numContainers : " + req.getNumContainers +
-        ", p = " + req.getPriority().getPriority + ", capability: "  + req.getCapability)
+    for (request <- resourceRequests) {
+      logInfo("ResourceRequest (host : %s, num containers: %d, priority = %s , capability : %s)".
+        format(
+          request.getHostName,
+          request.getNumContainers,
+          request.getPriority,
+          request.getCapability))
     }
     resourceManager.allocate(req)
   }
 
 
-  private def createResourceRequest(requestType: AllocationType.AllocationType, 
-                                    resource:String, numWorkers: Int, priority: Int): ResourceRequest = {
+  private def createResourceRequest(
+    requestType: AllocationType.AllocationType, 
+    resource:String,
+    numWorkers: Int,
+    priority: Int): ResourceRequest = {
 
     // If hostname specified, we need atleast two requests - node local and rack local.
     // There must be a third request - which is ANY : that will be specially handled.
     requestType match {
       case AllocationType.HOST => {
-        assert (YarnAllocationHandler.ANY_HOST != resource)
-
+        assert(YarnAllocationHandler.ANY_HOST != resource)
         val hostname = resource
         val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority)
 
-        // add to host->rack mapping
+        // Add to host->rack mapping
         YarnAllocationHandler.populateRackInfo(conf, hostname)
 
         nodeLocal
       }
-
       case AllocationType.RACK => {
         val rack = resource
         createResourceRequestImpl(rack, numWorkers, priority)
       }
-
-      case AllocationType.ANY => {
-        createResourceRequestImpl(YarnAllocationHandler.ANY_HOST, numWorkers, priority)
-      }
-
-      case _ => throw new IllegalArgumentException("Unexpected/unsupported request type .. " + requestType)
+      case AllocationType.ANY => createResourceRequestImpl(
+        YarnAllocationHandler.ANY_HOST, numWorkers, priority)
+      case _ => throw new IllegalArgumentException(
+        "Unexpected/unsupported request type: " + requestType)
     }
   }
 
-  private def createResourceRequestImpl(hostname:String, numWorkers: Int, priority: Int): ResourceRequest = {
+  private def createResourceRequestImpl(
+    hostname:String,
+    numWorkers: Int,
+    priority: Int): ResourceRequest = {
 
     val rsrcRequest = Records.newRecord(classOf[ResourceRequest])
     val memCapability = Records.newRecord(classOf[Resource])
@@ -455,11 +514,11 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
   def createReleasedContainerList(): ArrayBuffer[ContainerId] = {
 
     val retval = new ArrayBuffer[ContainerId](1)
-    // iterator on COW list ...
+    // Iterator on COW list ...
     for (container <- releasedContainerList.iterator()){
       retval += container
     }
-    // remove from the original list.
+    // Remove from the original list.
     if (! retval.isEmpty) {
       releasedContainerList.removeAll(retval)
       for (v <- retval) pendingReleaseContainers.put(v, true)
@@ -474,14 +533,14 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
 object YarnAllocationHandler {
 
   val ANY_HOST = "*"
-  // all requests are issued with same priority : we do not (yet) have any distinction between 
+  // All requests are issued with same priority : we do not (yet) have any distinction between 
   // request types (like map/reduce in hadoop for example)
   val PRIORITY = 1
 
   // Additional memory overhead - in mb
   val MEMORY_OVERHEAD = 384
 
-  // host to rack map - saved from allocation requests
+  // Host to rack map - saved from allocation requests
   // We are expecting this not to change.
   // Note that it is possible for this to change : and RM will indicate that to us via update 
   // response to allocate. But we are punting on handling that for now.
@@ -489,38 +548,69 @@ object YarnAllocationHandler {
   private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]()
 
 
-  def newAllocator(conf: Configuration,
-                   resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
-                   args: ApplicationMasterArguments): YarnAllocationHandler = {
-
-    new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers, 
-      args.workerMemory, args.workerCores, Map[String, Int](), Map[String, Int]())
+  def newAllocator(
+    conf: Configuration,
+    resourceManager: AMRMProtocol,
+    appAttemptId: ApplicationAttemptId,
+    args: ApplicationMasterArguments): YarnAllocationHandler = {
+
+    new YarnAllocationHandler(
+      conf,
+      resourceManager,
+      appAttemptId,
+      args.numWorkers, 
+      args.workerMemory,
+      args.workerCores,
+      Map[String, Int](),
+      Map[String, Int]())
   }
 
-  def newAllocator(conf: Configuration,
-                   resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
-                   args: ApplicationMasterArguments,
-                   map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = {
+  def newAllocator(
+    conf: Configuration,
+    resourceManager: AMRMProtocol,
+    appAttemptId: ApplicationAttemptId,
+    args: ApplicationMasterArguments,
+    map: collection.Map[String,
+    collection.Set[SplitInfo]]): YarnAllocationHandler = {
 
     val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
-
-    new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers, 
-      args.workerMemory, args.workerCores, hostToCount, rackToCount)
+    new YarnAllocationHandler(
+      conf,
+      resourceManager,
+      appAttemptId,
+      args.numWorkers, 
+      args.workerMemory,
+      args.workerCores,
+      hostToCount,
+      rackToCount)
   }
 
-  def newAllocator(conf: Configuration,
-                   resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
-                   maxWorkers: Int, workerMemory: Int, workerCores: Int,
-                   map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = {
+  def newAllocator(
+    conf: Configuration,
+    resourceManager: AMRMProtocol,
+    appAttemptId: ApplicationAttemptId,
+    maxWorkers: Int,
+    workerMemory: Int,
+    workerCores: Int,
+    map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = {
 
     val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
 
-    new YarnAllocationHandler(conf, resourceManager, appAttemptId, maxWorkers,
-      workerMemory, workerCores, hostToCount, rackToCount)
+    new YarnAllocationHandler(
+      conf,
+      resourceManager,
+      appAttemptId,
+      maxWorkers,
+      workerMemory,
+      workerCores,
+      hostToCount,
+      rackToCount)
   }
 
   // A simple method to copy the split info map.
-  private def generateNodeToWeight(conf: Configuration, input: collection.Map[String, collection.Set[SplitInfo]]) :
+  private def generateNodeToWeight(
+    conf: Configuration,
+    input: collection.Map[String, collection.Set[SplitInfo]]) :
   // host to count, rack to count
   (Map[String, Int], Map[String, Int]) = {
 
@@ -544,7 +634,7 @@ object YarnAllocationHandler {
   }
 
   def lookupRack(conf: Configuration, host: String): String = {
-    if (! hostToRack.contains(host)) populateRackInfo(conf, host)
+    if (!hostToRack.contains(host)) populateRackInfo(conf, host)
     hostToRack.get(host)
   }
 
@@ -567,10 +657,12 @@ object YarnAllocationHandler {
         val rack = rackInfo.getNetworkLocation
         hostToRack.put(hostname, rack)
         if (! rackToHostSet.containsKey(rack)) {
-          rackToHostSet.putIfAbsent(rack, Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]()))
+          rackToHostSet.putIfAbsent(rack,
+            Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]()))
         }
         rackToHostSet.get(rack).add(hostname)
 
+        // TODO(harvey): Figure out this comment...
         // Since RackResolver caches, we are disabling this for now ...
       } /* else {
         // right ? Else we will keep calling rack resolver in case we cant resolve rack info ...