You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2013/12/07 20:38:00 UTC

[01/13] git commit: Merge pull request #189 from tgravescs/sparkYarnErrorHandling

Updated Branches:
  refs/heads/branch-0.8 d77c3371b -> d6e5eab2f


Merge pull request #189 from tgravescs/sparkYarnErrorHandling

Impove Spark on Yarn Error handling

Improve cli error handling and only allow a certain number of worker failures before failing the application.  This will help prevent users from doing foolish things and their jobs running forever.  For instance using 32 bit java but trying to allocate 8G containers. This loops forever without this change, now it errors out after a certain number of retries.  The number of tries is configurable.  Also increase the frequency we ping the RM to increase speed at which we get containers if they die. The Yarn MR app defaults to pinging the RM every 1 seconds, so the default of 5 seconds here is fine. But that is configurable as well in case people want to change it.

I do want to make sure there aren't any cases that calling stopExecutors in CoarseGrainedSchedulerBackend would cause problems?  I couldn't think of any and testing on standalone cluster as well as yarn.
(cherry picked from commit aa638ed9c140174a47df082ed5631ffe8e624ee6)

Signed-off-by: Patrick Wendell <pw...@gmail.com>


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

Branch: refs/heads/branch-0.8
Commit: ee22be0e6c302fb2cdb24f83365c2b8a43a1baab
Parents: d77c337
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Tue Nov 19 16:05:44 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Fri Dec 6 23:29:38 2013 -0800

----------------------------------------------------------------------
 .../cluster/CoarseGrainedSchedulerBackend.scala |  1 +
 .../cluster/SimrSchedulerBackend.scala          |  1 -
 docs/running-on-yarn.md                         |  2 +
 .../spark/deploy/yarn/ApplicationMaster.scala   | 39 +++++++++++++-------
 .../org/apache/spark/deploy/yarn/Client.scala   | 32 ++++++++++------
 .../deploy/yarn/YarnAllocationHandler.scala     | 16 ++++++--
 6 files changed, 61 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ee22be0e/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index a45bee5..d0ba5bf 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -199,6 +199,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
   }
 
   override def stop() {
+    stopExecutors()
     try {
       if (driverActor != null) {
         val future = driverActor.ask(StopDriver)(timeout)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ee22be0e/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
index 0ea35e2..e000531 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
@@ -62,7 +62,6 @@ private[spark] class SimrSchedulerBackend(
     val conf = new Configuration()
     val fs = FileSystem.get(conf)
     fs.delete(new Path(driverFilePath), false)
-    super.stopExecutors()
     super.stop()
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ee22be0e/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 6fd1d0d..4056e9c 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -37,6 +37,8 @@ System Properties:
 * 'spark.yarn.applicationMaster.waitTries', property to set the number of times the ApplicationMaster waits for the the spark master and then also the number of tries it waits for the Spark Context to be intialized. Default is 10.
 * 'spark.yarn.submit.file.replication', the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives.
 * 'spark.yarn.preserve.staging.files', set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them.
+* 'spark.yarn.scheduler.heartbeat.interval-ms', the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds. 
+* 'spark.yarn.max.worker.failures', the maximum number of worker failures before failing the application. Default is the number of workers requested times 2 with minimum of 3.
 
 # Launching Spark on YARN
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ee22be0e/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
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 4302ef4..2afc1d9 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
@@ -54,7 +54,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   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 
+  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
@@ -227,12 +229,13 @@ 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)
+            ", numTries = " + numTries)
+          this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager,
+            appAttemptId, args)
         }
       }
     } finally {
@@ -251,8 +254,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       while(yarnAllocator.getNumWorkersRunning < args.numWorkers &&
         // If user thread exists, then quit !
         userThread.isAlive) {
-
-          this.yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
+          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)
       }
@@ -268,21 +274,27 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       // 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))
+
+      // 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)
     }
   }
 
-  // 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 (userThread.isAlive) {
+          if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+            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")
@@ -321,7 +333,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   }
   */
 
-  def finishApplicationMaster(status: FinalApplicationStatus) {
+  def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") {
 
     synchronized {
       if (isFinished) {
@@ -335,6 +347,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
       .asInstanceOf[FinishApplicationMasterRequest]
     finishReq.setAppAttemptId(appAttemptId)
     finishReq.setFinishApplicationStatus(status)
+    finishReq.setDiagnostics(diagnostics)
     // set tracking url to empty since we don't have a history server
     finishReq.setTrackingUrl("")
     resourceManager.finishApplicationMaster(finishReq)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ee22be0e/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 4e0e060..15b3480 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
@@ -60,6 +60,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) 
 
   def run() {
+    validateArgs()
+
     init(yarnConf)
     start()
     logClusterResourceDetails()
@@ -84,6 +86,23 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     System.exit(0)
   }
 
+  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 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) => 
+      if (cond) {
+        logError(errStr)
+        args.printUsageAndExit(1)
+      }
+    }
+  }
+
   def getAppStagingDir(appId: ApplicationId): String = {
     SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR
   }
@@ -97,7 +116,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       ", queueMaxCapacity=" + queueInfo.getMaximumCapacity + ", queueApplicationCount=" + queueInfo.getApplications.size +
       ", queueChildQueueCount=" + queueInfo.getChildQueues.size)
   }
-
   
   def verifyClusterResources(app: GetNewApplicationResponse) = { 
     val maxMem = app.getMaximumResourceCapability().getMemory()
@@ -215,11 +233,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
 
-    if (System.getenv("SPARK_JAR") == null || args.userJar == null) {
-      logError("Error: You must set SPARK_JAR environment variable and specify a user jar!")
-      System.exit(1)
-    }
-
     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) =>
@@ -334,7 +347,6 @@ 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)
@@ -360,11 +372,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
     }
 
-    if (args.userClass == null) {
-      logError("Error: You must specify a user class!")
-      System.exit(1)
-    }
-
     val commands = List[String](javaCommand + 
       " -server " +
       JAVA_OPTS +
@@ -442,6 +449,7 @@ object Client {
     System.setProperty("SPARK_YARN_MODE", "true")
 
     val args = new ClientArguments(argStrings)
+
     new Client(args).run
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ee22be0e/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------
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 25da9aa..507a074 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
@@ -72,9 +72,11 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
   // Used to generate a unique id per worker
   private val workerIdCounter = new AtomicInteger()
   private val lastResponseId = new AtomicInteger()
+  private val numWorkersFailed = new AtomicInteger()
 
   def getNumWorkersRunning: Int = numWorkersRunning.intValue
 
+  def getNumWorkersFailed: Int = numWorkersFailed.intValue
 
   def isResourceConstraintSatisfied(container: Container): Boolean = {
     container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
@@ -253,8 +255,16 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
         else {
           // simply decrement count - next iteration of ReporterThread will take care of allocating !
           numWorkersRunning.decrementAndGet()
-          logInfo("Container completed ? nodeId: " + containerId + ", state " + completedContainer.getState +
-            " httpaddress: " + completedContainer.getDiagnostics)
+          logInfo("Container completed not by us ? nodeId: " + containerId + ", state " + completedContainer.getState +
+            " httpaddress: " + completedContainer.getDiagnostics + " exit status: " + 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 {
@@ -378,8 +388,6 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
     val releasedContainerList = createReleasedContainerList()
     req.addAllReleases(releasedContainerList)
 
-
-
     if (numWorkers > 0) {
       logInfo("Allocating " + numWorkers + " worker containers with " + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + " of memory each.")
     }


[02/13] git commit: Merge pull request #178 from hsaputra/simplecleanupcode

Posted by pw...@apache.org.
Merge pull request #178 from hsaputra/simplecleanupcode

Simple cleanup on Spark's Scala code

Simple cleanup on Spark's Scala code while testing some modules:
-) Remove some of unused imports as I found them
-) Remove ";" in the imports statements
-) Remove () at the end of method calls like size that does not have size effect.
(cherry picked from commit 1b5b358309a5adfc12b75b0ebb4254ad8e69f5a0)

Signed-off-by: Patrick Wendell <pw...@gmail.com>


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

Branch: refs/heads/branch-0.8
Commit: 2b7631555ca0e8a6812bcbf008c6a719bbd5b4b7
Parents: ee22be0
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Sat Nov 16 11:44:10 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 00:02:14 2013 -0800

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/SparkContext.scala      | 1 -
 .../scala/org/apache/spark/deploy/LocalSparkCluster.scala    | 6 +++---
 .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 2 +-
 .../scala/org/apache/spark/executor/ExecutorSource.scala     | 2 --
 .../org/apache/spark/deploy/yarn/ApplicationMaster.scala     | 8 +++-----
 .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +-
 .../spark/deploy/yarn/ClientDistributedCacheManager.scala    | 2 +-
 .../deploy/yarn/ClientDistributedCacheManagerSuite.scala     | 2 +-
 8 files changed, 10 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2b763155/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index ff5b2e0..9ccf70f 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger
 
 import scala.collection.Map
 import scala.collection.generic.Growable
-import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2b763155/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index 308a2bf..a724900 100644
--- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -17,12 +17,12 @@
 
 package org.apache.spark.deploy
 
-import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
+import akka.actor.ActorSystem
 
 import org.apache.spark.deploy.worker.Worker
 import org.apache.spark.deploy.master.Master
-import org.apache.spark.util.{Utils, AkkaUtils}
-import org.apache.spark.{Logging}
+import org.apache.spark.util.Utils
+import org.apache.spark.Logging
 
 import scala.collection.mutable.ArrayBuffer
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2b763155/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index caee6b0..8332631 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer
 import akka.actor.{ActorRef, Actor, Props, Terminated}
 import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
 
-import org.apache.spark.{Logging, SparkEnv}
+import org.apache.spark.Logging
 import org.apache.spark.TaskState.TaskState
 import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
 import org.apache.spark.util.{Utils, AkkaUtils}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2b763155/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
index 18c9dc1..a537b2c 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
@@ -20,8 +20,6 @@ package org.apache.spark.executor
 import com.codahale.metrics.{Gauge, MetricRegistry}
 
 import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.hdfs.DistributedFileSystem
-import org.apache.hadoop.fs.LocalFileSystem
 
 import scala.collection.JavaConversions._
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2b763155/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
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 2afc1d9..89b0041 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
@@ -17,9 +17,8 @@
 
 package org.apache.spark.deploy.yarn
 
-import java.io.IOException;
+import java.io.IOException
 import java.net.Socket
-import java.security.PrivilegedExceptionAction
 import java.util.concurrent.CopyOnWriteArrayList
 import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
 import org.apache.hadoop.conf.Configuration
@@ -34,7 +33,6 @@ 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 org.apache.hadoop.security.UserGroupInformation
 import scala.collection.JavaConversions._
 
 class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
@@ -188,8 +186,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
         var successed = false
         try {
           // Copy
-          var mainArgs: Array[String] = new Array[String](args.userArgs.size())
-          args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size())
+          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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2b763155/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 15b3480..1078d5b 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
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer
 
 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.fs.permission.FsPermission
 import org.apache.hadoop.mapred.Master
 import org.apache.hadoop.net.NetUtils
 import org.apache.hadoop.io.DataOutputBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2b763155/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
index 07686fe..674c8f8 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.deploy.yarn
 
-import java.net.URI;
+import java.net.URI
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.FileStatus

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2b763155/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
----------------------------------------------------------------------
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
index c0a2af0..2941356 100644
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
+++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.deploy.yarn
 
-import java.net.URI;
+import java.net.URI
 
 import org.scalatest.FunSuite
 import org.scalatest.mock.MockitoSugar


[06/13] git commit: Merge pull request #199 from harveyfeng/yarn-2.2

Posted by pw...@apache.org.
Merge pull request #199 from harveyfeng/yarn-2.2

Hadoop 2.2 migration

Includes support for the YARN API stabilized in the Hadoop 2.2 release, and a few style patches.

Short description for each set of commits:

a98f5a0 - "Misc style changes in the 'yarn' package"
a67ebf4 - "A few more style fixes in the 'yarn' package"
Both of these are some minor style changes, such as fixing lines over 100 chars, to the existing YARN code.

ab8652f - "Add a 'new-yarn' directory ... "
Copies everything from `SPARK_HOME/yarn` to `SPARK_HOME/new-yarn`. No actual code changes here.

4f1c3fa - "Hadoop 2.2 YARN API migration ..."
API patches to code in the `SPARK_HOME/new-yarn` directory. There are a few more small style changes mixed in, too.
Based on @colorant's Hadoop 2.2 support for the scala-2.10 branch in #141.

a1a1c62 - "Add optional Hadoop 2.2 settings in sbt build ... "
If Spark should be built against Hadoop 2.2, then:
a) the `org.apache.spark.deploy.yarn` package will be compiled from the `new-yarn` directory.
b) Protobuf v2.5 will be used as a Spark dependency, since Hadoop 2.2 depends on it. Also, Spark will be built against a version of Akka v2.0.5 that's built against Protobuf 2.5, named `akka-2.0.5-protobuf-2.5`. The patched Akka is here: https://github.com/harveyfeng/akka/tree/2.0.5-protobuf-2.5, and was published to local Ivy during testing.

There's also a new boolean environment variable, `SPARK_IS_NEW_HADOOP`, that users can manually set if their `SPARK_HADOOP_VERSION` specification does not start with `2.2`, which is how the build file tries to detect a 2.2 version. Not sure if this is necessary or done in the best way, though...
(cherry picked from commit 72b696156c8662cae2cef4b943520b4be86148ea)

Conflicts:

	project/SparkBuild.scala
	streaming/pom.xml


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

Branch: refs/heads/branch-0.8
Commit: 2d3eae244bc1b5e6a22798cd307fba3026053edc
Parents: 1e9d084
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Wed Dec 4 23:33:04 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 01:15:19 2013 -0800

----------------------------------------------------------------------
 core/pom.xml                                    |  10 +-
 .../scala/org/apache/spark/SparkContext.scala   |   2 +-
 new-yarn/pom.xml                                | 161 +++++
 .../spark/deploy/yarn/ApplicationMaster.scala   | 446 ++++++++++++
 .../yarn/ApplicationMasterArguments.scala       |  94 +++
 .../org/apache/spark/deploy/yarn/Client.scala   | 519 ++++++++++++++
 .../spark/deploy/yarn/ClientArguments.scala     | 148 ++++
 .../yarn/ClientDistributedCacheManager.scala    | 228 ++++++
 .../spark/deploy/yarn/WorkerLauncher.scala      | 223 ++++++
 .../spark/deploy/yarn/WorkerRunnable.scala      | 209 ++++++
 .../deploy/yarn/YarnAllocationHandler.scala     | 687 +++++++++++++++++++
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala |  43 ++
 .../cluster/YarnClientClusterScheduler.scala    |  47 ++
 .../cluster/YarnClientSchedulerBackend.scala    | 109 +++
 .../cluster/YarnClusterScheduler.scala          |  55 ++
 .../ClientDistributedCacheManagerSuite.scala    | 220 ++++++
 pom.xml                                         |  61 +-
 project/SparkBuild.scala                        |  35 +-
 streaming/pom.xml                               |   3 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   | 172 ++---
 .../org/apache/spark/deploy/yarn/Client.scala   | 151 ++--
 .../spark/deploy/yarn/WorkerRunnable.scala      |  85 ++-
 .../deploy/yarn/YarnAllocationHandler.scala     | 346 ++++++----
 23 files changed, 3717 insertions(+), 337 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 5e9136f..1264bc4 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>
@@ -127,10 +127,6 @@
       <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>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 82be558..eb5bb17 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]] =

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml
new file mode 100644
index 0000000..8a065c6
--- /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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
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 0000000..eeeca3e
--- /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()
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
----------------------------------------------------------------------
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 0000000..f76a5dd
--- /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)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 0000000..9467881
--- /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 + "*")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
----------------------------------------------------------------------
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 0000000..9efb28a
--- /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)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
----------------------------------------------------------------------
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 0000000..5f159b0
--- /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
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
----------------------------------------------------------------------
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 0000000..c38f33e
--- /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()
+  }
+}


[12/13] git commit: Merge pull request #234 from alig/master

Posted by pw...@apache.org.
Merge pull request #234 from alig/master

Updated documentation about the YARN v2.2 build process
(cherry picked from commit 241336add5be07fca5ff6c17eed368df7d0c3e3c)

Signed-off-by: Patrick Wendell <pw...@gmail.com>


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

Branch: refs/heads/branch-0.8
Commit: 264231293915480d63af7fc71b1c822692c36c49
Parents: 07470d1
Author: Patrick Wendell <pw...@gmail.com>
Authored: Fri Dec 6 17:29:03 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 01:15:20 2013 -0800

----------------------------------------------------------------------
 docs/building-with-maven.md | 4 ++++
 docs/cluster-overview.md    | 2 +-
 docs/index.md               | 6 ++++--
 docs/running-on-yarn.md     | 8 ++++++++
 4 files changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/26423129/docs/building-with-maven.md
----------------------------------------------------------------------
diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md
index 19c01e1..a508786 100644
--- a/docs/building-with-maven.md
+++ b/docs/building-with-maven.md
@@ -45,6 +45,10 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with
     # Cloudera CDH 4.2.0 with MapReduce v2
     $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
 
+Hadoop versions 2.2.x and newer can be built by setting the ```new-yarn``` and the ```yarn.version``` as follows:
+       mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn
+
+The build process handles Hadoop 2.2.x as a special case that uses the directory ```new-yarn```, which supports the new YARN API. Furthermore, for this version, the build depends on artifacts published by the spark-project to enable Akka 2.0.5 to work with protobuf 2.5. 
 
 ## Spark Tests in Maven ##
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/26423129/docs/cluster-overview.md
----------------------------------------------------------------------
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index 5927f73..e167032 100644
--- a/docs/cluster-overview.md
+++ b/docs/cluster-overview.md
@@ -45,7 +45,7 @@ The system currently supports three cluster managers:
   easy to set up a cluster.
 * [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce
   and service applications.
-* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.0.
+* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.
 
 In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone
 cluster on Amazon EC2.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/26423129/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index bd386a8..bbb2733 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -56,14 +56,16 @@ Hadoop, you must build Spark against the same version that your cluster uses.
 By default, Spark links to Hadoop 1.0.4. You can change this by setting the
 `SPARK_HADOOP_VERSION` variable when compiling:
 
-    SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly
+    SPARK_HADOOP_VERSION=2.2.0 sbt/sbt assembly
 
 In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set
 `SPARK_YARN` to `true`:
 
     SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly
 
-(Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`.)
+Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`.
+
+For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.md). This is needed because Hadoop 2.2 has non backwards compatible API changes.
 
 # Where to Go from Here
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/26423129/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 68fd6c2..ae65127 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -17,6 +17,7 @@ This can be built by setting the Hadoop version and `SPARK_YARN` environment var
 The assembled JAR will be something like this:
 `./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`.
 
+The build process now also supports new YARN versions (2.2.x). See below.
 
 # Preparations
 
@@ -111,9 +112,16 @@ For example:
     SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
     MASTER=yarn-client ./spark-shell
 
+# Building Spark for Hadoop/YARN 2.2.x
+
+Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5) that is not compatible with the Akka version (2.0.5) that Spark uses. Therefore, if the Hadoop version (e.g. set through ```SPARK_HADOOP_VERSION```) starts with 2.2.0 or higher then the build process will depend on Akka artifacts distributed by the Spark project compatible with Protobuf 2.5. Furthermore, the build process then uses the directory ```new-yarn``` (instead of ```yarn```), which supports the new YARN API. The build process should seamlessly work out of the box. 
+
+See [Building Spark with Maven](building-with-maven.md) for instructions on how to build Spark using the Maven process.
+
 # Important Notes
 
 - We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed.
 - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored.
 - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN.
 - The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files.
+- YARN 2.2.x users cannot simply depend on the Spark packages without building Spark, as the published Spark artifacts are compiled to work with the pre 2.2 API. Those users must build Spark and publish it locally.  
\ No newline at end of file


[10/13] git commit: Merge pull request #237 from pwendell/formatting-fix

Posted by pw...@apache.org.
Merge pull request #237 from pwendell/formatting-fix

Formatting fix

This is a single-line change. The diff appears larger here due to github being out of sync.
(cherry picked from commit 10c3c0c6524d0cf6c59b6f2227bf316cdeb7d06c)

Signed-off-by: Patrick Wendell <pw...@gmail.com>


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

Branch: refs/heads/branch-0.8
Commit: 80cc4ff94f1cbf53314ee6117733864b690e0249
Parents: 4a6aae3
Author: Patrick Wendell <pw...@gmail.com>
Authored: Fri Dec 6 20:29:45 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 01:15:20 2013 -0800

----------------------------------------------------------------------
 docs/configuration.md | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/80cc4ff9/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 22abe1c..0edbac2 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -335,7 +335,6 @@ Apart from these, the following properties are also available, and may be useful
   </td>
 </tr>
 <tr>
-<tr>
   <td>spark.speculation</td>
   <td>false</td>
   <td>


[05/13] Merge pull request #199 from harveyfeng/yarn-2.2

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
----------------------------------------------------------------------
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 0000000..9f5523c
--- /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
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------
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 0000000..dba0f76
--- /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)
+      } */
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
----------------------------------------------------------------------
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 0000000..2ba2366
--- /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())
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
----------------------------------------------------------------------
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 0000000..63a0449
--- /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")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
----------------------------------------------------------------------
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 0000000..b206780
--- /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")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
----------------------------------------------------------------------
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 0000000..29b3f22
--- /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")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
----------------------------------------------------------------------
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 0000000..2941356
--- /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)
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 07213f9..83f9b23 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>
@@ -234,7 +236,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>
@@ -247,21 +249,32 @@
         <version>0.3.1</version>
       </dependency>
       <dependency>
-        <groupId>com.typesafe.akka</groupId>
+        <groupId>${akka.group}</groupId>
         <artifactId>akka-actor</artifactId>
         <version>${akka.version}</version>
       </dependency>
       <dependency>
-        <groupId>com.typesafe.akka</groupId>
+        <groupId>${akka.group}</groupId>
         <artifactId>akka-remote</artifactId>
         <version>${akka.version}</version>
       </dependency>
       <dependency>
-        <groupId>com.typesafe.akka</groupId>
+        <groupId>${akka.group}</groupId>
         <artifactId>akka-slf4j</artifactId>
         <version>${akka.version}</version>
       </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>
         <version>6.4.4</version>
@@ -272,11 +285,6 @@
         <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>
         <version>${mesos.version}</version>
@@ -768,6 +776,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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 660f0e2..03d4cae 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: _*)
 
@@ -66,14 +69,28 @@ 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]()
+
   lazy val allProjects = Seq[ProjectReference](
     core, repl, examples, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef
 
@@ -203,10 +220,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",
@@ -280,7 +297,7 @@ object SparkBuild extends Build {
       "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy),
       "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty),
       "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)
     )
   )
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
index cf5de8f..320d532 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -86,9 +86,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>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
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 a7baf0c..240ed8b 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)


[07/13] git commit: Merge pull request #101 from colorant/yarn-client-scheduler

Posted by pw...@apache.org.
Merge pull request #101 from colorant/yarn-client-scheduler

For SPARK-527, Support spark-shell when running on YARN

sync to trunk and resubmit here

In current YARN mode approaching, the application is run in the Application Master as a user program thus the whole spark context is on remote.

This approaching won't support application that involve local interaction and need to be run on where it is launched.

So In this pull request I have a YarnClientClusterScheduler and backend added.

With this scheduler, the user application is launched locally,While the executor will be launched by YARN on remote nodes with a thin AM which only launch the executor and monitor the Driver Actor status, so that when client app is done, it can finish the YARN Application as well.

This enables spark-shell to run upon YARN.

This also enable other Spark applications to have the spark context to run locally with a master-url "yarn-client". Thus e.g. SparkPi could have the result output locally on console instead of output in the log of the remote machine where AM is running on.

Docs also updated to show how to use this yarn-client mode.
(cherry picked from commit eb4296c8f7561aaf8782479dd5cd7c9320b7fa6b)

Conflicts:

	core/src/main/scala/org/apache/spark/SparkContext.scala


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

Branch: refs/heads/branch-0.8
Commit: 1e9d08408a964ff0ad2db65c9f67040bac00c284
Parents: 20d1f8b
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Mon Nov 25 15:25:29 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 01:15:19 2013 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/SparkContext.scala   |  25 ++
 docs/running-on-yarn.md                         |  27 +-
 .../org/apache/spark/deploy/yarn/Client.scala   |  13 +-
 .../spark/deploy/yarn/ClientArguments.scala     |  40 +--
 .../spark/deploy/yarn/WorkerLauncher.scala      | 246 +++++++++++++++++++
 .../cluster/YarnClientClusterScheduler.scala    |  47 ++++
 .../cluster/YarnClientSchedulerBackend.scala    | 109 ++++++++
 7 files changed, 484 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1e9d0840/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 9ccf70f..82be558 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -226,6 +226,31 @@ class SparkContext(
         scheduler.initialize(backend)
         scheduler
 
+      case "yarn-client" =>
+        val scheduler = try {
+          val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler")
+          val cons = clazz.getConstructor(classOf[SparkContext])
+          cons.newInstance(this).asInstanceOf[ClusterScheduler]
+
+        } catch {
+          case th: Throwable => {
+            throw new SparkException("YARN mode not available ?", th)
+          }
+        }
+
+        val backend = try {
+          val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend")
+          val cons = clazz.getConstructor(classOf[ClusterScheduler], classOf[SparkContext])
+          cons.newInstance(scheduler, this).asInstanceOf[CoarseGrainedSchedulerBackend]
+        } catch {
+          case th: Throwable => {
+            throw new SparkException("YARN mode not available ?", th)
+          }
+        }
+
+        scheduler.initialize(backend)
+        scheduler
+
       case _ =>
         if (MESOS_REGEX.findFirstIn(master).isEmpty) {
           logWarning("Master %s does not match expected format, parsing as Mesos URL".format(master))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1e9d0840/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 4056e9c..68fd6c2 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -45,6 +45,10 @@ System Properties:
 Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the hadoop cluster.
 This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager.
 
+There are two scheduler mode that can be used to launch spark application on YARN.
+
+## Launch spark application by YARN Client with yarn-standalone mode.
+
 The command to launch the YARN Client is as follows:
 
     SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./spark-class org.apache.spark.deploy.yarn.Client \
@@ -52,6 +56,7 @@ The command to launch the YARN Client is as follows:
       --class <APP_MAIN_CLASS> \
       --args <APP_MAIN_ARGUMENTS> \
       --num-workers <NUMBER_OF_WORKER_MACHINES> \
+      --master-class <ApplicationMaster_CLASS>
       --master-memory <MEMORY_FOR_MASTER> \
       --worker-memory <MEMORY_PER_WORKER> \
       --worker-cores <CORES_PER_WORKER> \
@@ -85,11 +90,29 @@ For example:
     $ cat $YARN_APP_LOGS_DIR/$YARN_APP_ID/container*_000001/stdout
     Pi is roughly 3.13794
 
-The above starts a YARN Client programs which periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running.
+The above starts a YARN Client programs which start the default Application Master. Then SparkPi will be run as a child thread of Application Master, YARN Client will  periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running.
+
+With this mode, your application is actually run on the remote machine where the Application Master is run upon. Thus application that involve local interaction will not work well, e.g. spark-shell.
+
+## Launch spark application with yarn-client mode.
+
+With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR and SPARK_YARN_APP_JAR
+
+In order to tune worker core/number/memory etc. You need to export SPARK_WORKER_CORES, SPARK_WORKER_MEMORY, SPARK_WORKER_INSTANCES e.g. by ./conf/spark-env.sh
+
+For example:
+
+    SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
+    SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
+    ./run-example org.apache.spark.examples.SparkPi yarn-client
+
+
+    SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
+    SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
+    MASTER=yarn-client ./spark-shell
 
 # Important Notes
 
-- When your application instantiates a Spark context it must use a special "yarn-standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "yarn-standalone" as an argument to your program, as shown in the example above.
 - We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed.
 - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored.
 - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1e9d0840/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 94e353a..bb73f6d 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
@@ -54,9 +54,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
   // 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) 
+  val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short)
 
-  def run() {
+  // for client user who want to monitor app status by itself.
+  def runApp() = {
     validateArgs()
 
     init(yarnConf)
@@ -78,7 +79,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName())
 
     submitApp(appContext)
-    
+    appId
+  }
+
+  def run() {
+    val appId = runApp()
     monitorApplication(appId)
     System.exit(0)
   }
@@ -372,7 +377,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     val commands = List[String](javaCommand + 
       " -server " +
       JAVA_OPTS +
-      " org.apache.spark.deploy.yarn.ApplicationMaster" +
+      " " + args.amClass +
       " --class " + args.userClass + 
       " --jar " + args.userJar +
       userArgsToString(args) +

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1e9d0840/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index 852dbd7..b9dbc3f 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -35,6 +35,7 @@ class ClientArguments(val args: Array[String]) {
   var numWorkers = 2
   var amQueue = System.getProperty("QUEUE", "default")
   var amMemory: Int = 512
+  var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster"
   var appName: String = "Spark"
   // TODO
   var inputFormatInfo: List[InputFormatInfo] = null
@@ -62,18 +63,22 @@ class ClientArguments(val args: Array[String]) {
           userArgsBuffer += value
           args = tail
 
-        case ("--master-memory") :: MemoryParam(value) :: tail =>
-          amMemory = value
+        case ("--master-class") :: value :: tail =>
+          amClass = value
           args = tail
 
-        case ("--num-workers") :: IntParam(value) :: tail =>
-          numWorkers = value
+        case ("--master-memory") :: MemoryParam(value) :: tail =>
+          amMemory = value
           args = tail
 
         case ("--worker-memory") :: MemoryParam(value) :: tail =>
           workerMemory = value
           args = tail
 
+        case ("--num-workers") :: IntParam(value) :: tail =>
+          numWorkers = value
+          args = tail
+
         case ("--worker-cores") :: IntParam(value) :: tail =>
           workerCores = value
           args = tail
@@ -119,19 +124,20 @@ class ClientArguments(val args: Array[String]) {
     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-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."
+      "  --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)
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1e9d0840/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
new file mode 100644
index 0000000..421a83c
--- /dev/null
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
@@ -0,0 +1,246 @@
+/*
+ * 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.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+import akka.actor._
+import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent}
+import akka.remote.RemoteClientShutdown
+import akka.actor.Terminated
+import akka.remote.RemoteClientDisconnected
+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
+
+class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+
+  def this(args: ApplicationMasterArguments) = this(args, new Configuration())
+
+  private val rpc: YarnRPC = YarnRPC.create(conf)
+  private var resourceManager: AMRMProtocol = null
+  private var appAttemptId: ApplicationAttemptId = null
+  private var reporterThread: Thread = null
+  private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+  private var yarnAllocator: YarnAllocationHandler = null
+  private var driverClosed:Boolean = false
+
+  val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1
+  var actor: ActorRef = null
+
+  // 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)
+      driver ! "hello"
+      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() {
+
+    appAttemptId = getApplicationAttemptId()
+    resourceManager = registerWithResourceManager()
+    val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
+
+    // Compute number of threads for akka
+    val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory()
+
+    if (minimumMemory > 0) {
+      val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD
+      val numCore = (mem  / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0)
+
+      if (numCore > 0) {
+        // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406
+        // TODO: Uncomment when hadoop is on a version which has this fixed.
+        // args.workerCores = numCore
+      }
+    }
+
+    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.AM_CONTAINER_ID_ENV)
+    val containerId = ConverterUtils.toContainerId(containerIdString)
+    val appAttemptId = containerId.getApplicationAttemptId()
+    logInfo("ApplicationAttemptId: " + appAttemptId)
+    return appAttemptId
+  }
+
+  private def registerWithResourceManager(): AMRMProtocol = {
+    val rmAddress = NetUtils.createSocketAddr(yarnConf.get(
+      YarnConfiguration.RM_SCHEDULER_ADDRESS,
+      YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS))
+    logInfo("Connecting to ResourceManager at " + rmAddress)
+    return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
+  }
+
+  private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
+    logInfo("Registering the ApplicationMaster")
+    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.
+    // Users can then monitor stderr/stdout on that node if required.
+    appMasterRequest.setHost(Utils.localHostName())
+    appMasterRequest.setRpcPort(0)
+    // What do we provide here ? Might make sense to expose something sensible later ?
+    appMasterRequest.setTrackingUrl("")
+    return resourceManager.registerApplicationMaster(appMasterRequest)
+  }
+
+  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("Master now available: " + driverHost + ":" + driverPort)
+        driverUp = true
+      } catch {
+        case e: Exception =>
+          logError("Failed to connect to driver at " + 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, resourceManager, 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
+    while(yarnAllocator.getNumWorkersRunning < args.numWorkers) {
+      yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
+      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
+          if (missingWorkerCount > 0) {
+            logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers")
+            yarnAllocator.allocateContainers(missingWorkerCount)
+          }
+          else 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)
+    return t
+  }
+
+  private def sendProgress() {
+    logDebug("Sending progress")
+    // simulated with an allocate request with no nodes requested ...
+    yarnAllocator.allocateContainers(0)
+  }
+
+  def finishApplicationMaster(status: FinalApplicationStatus) {
+
+    logInfo("finish ApplicationMaster with " + status)
+    val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest])
+      .asInstanceOf[FinishApplicationMasterRequest]
+    finishReq.setAppAttemptId(appAttemptId)
+    finishReq.setFinishApplicationStatus(status)
+    resourceManager.finishApplicationMaster(finishReq)
+  }
+
+}
+
+
+object WorkerLauncher {
+  def main(argStrings: Array[String]) {
+    val args = new ApplicationMasterArguments(argStrings)
+    new WorkerLauncher(args).run()
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1e9d0840/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
new file mode 100644
index 0000000..63a0449
--- /dev/null
+++ b/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")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1e9d0840/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
new file mode 100644
index 0000000..b206780
--- /dev/null
+++ b/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")
+  }
+
+}


[09/13] git commit: Small fix for Harvey's patch

Posted by pw...@apache.org.
Small fix for Harvey's patch


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

Branch: refs/heads/branch-0.8
Commit: 07470d130bba48e90e86f9cffb4b5af271a394ac
Parents: 2d3eae2
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sat Dec 7 00:15:00 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 01:15:20 2013 -0800

----------------------------------------------------------------------
 new-yarn/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/07470d13/new-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml
index 8a065c6..d9168e3 100644
--- a/new-yarn/pom.xml
+++ b/new-yarn/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
-    <version>0.9.0-incubating-SNAPSHOT</version>
+    <version>0.8.1-incubating-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 


[13/13] git commit: Merge pull request #235 from pwendell/master

Posted by pw...@apache.org.
Merge pull request #235 from pwendell/master

Minor doc fixes and updating README
(cherry picked from commit e5d5728b72e58046cc175ab06b5f1c7be4957711)

Signed-off-by: Patrick Wendell <pw...@gmail.com>


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

Branch: refs/heads/branch-0.8
Commit: 4a6aae3bb0936dd2b6592e787ff7ffe9c5236251
Parents: 2642312
Author: Patrick Wendell <pw...@gmail.com>
Authored: Fri Dec 6 20:14:56 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 01:15:20 2013 -0800

----------------------------------------------------------------------
 README.md                   | 7 ++++++-
 docs/building-with-maven.md | 4 +++-
 docs/index.md               | 2 +-
 3 files changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4a6aae3b/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 5875505..37d9c0f 100644
--- a/README.md
+++ b/README.md
@@ -55,7 +55,7 @@ versions without YARN, use:
     # Cloudera CDH 4.2.0 with MapReduce v1
     $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly
 
-For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions
+For Apache Hadoop 2.0.X, 2.1.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions
 with YARN, also set `SPARK_YARN=true`:
 
     # Apache Hadoop 2.0.5-alpha
@@ -64,6 +64,11 @@ with YARN, also set `SPARK_YARN=true`:
     # Cloudera CDH 4.2.0 with MapReduce v2
     $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly
 
+When building for Hadoop 2.2.X and newer, you'll need to include the additional `new-yarn` profile:
+
+    # Apache Hadoop 2.2.X and newer
+    $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn
+
 For convenience, these variables may also be set through the `conf/spark-env.sh` file
 described below.
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4a6aae3b/docs/building-with-maven.md
----------------------------------------------------------------------
diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md
index a508786..c709001 100644
--- a/docs/building-with-maven.md
+++ b/docs/building-with-maven.md
@@ -46,7 +46,9 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with
     $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
 
 Hadoop versions 2.2.x and newer can be built by setting the ```new-yarn``` and the ```yarn.version``` as follows:
-       mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn
+ 
+    # Apache Hadoop 2.2.X and newer
+    $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn
 
 The build process handles Hadoop 2.2.x as a special case that uses the directory ```new-yarn```, which supports the new YARN API. Furthermore, for this version, the build depends on artifacts published by the spark-project to enable Akka 2.0.5 to work with protobuf 2.5. 
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4a6aae3b/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index bbb2733..45616f7 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -65,7 +65,7 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set
 
 Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`.
 
-For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.md). This is needed because Hadoop 2.2 has non backwards compatible API changes.
+For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.html). This is needed because Hadoop 2.2 has non backwards compatible API changes.
 
 # Where to Go from Here
 


[08/13] git commit: typo fix

Posted by pw...@apache.org.
typo fix


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

Branch: refs/heads/branch-0.8
Commit: d6e5eab2f68d260cf89fa859ad6d288b33149648
Parents: cfca70e
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sat Dec 7 00:15:41 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 01:15:20 2013 -0800

----------------------------------------------------------------------
 docs/configuration.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d6e5eab2/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index e86b9ea..2accfe8 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -331,7 +331,7 @@ Apart from these, the following properties are also available, and may be useful
   <td>spark.shuffle.consolidateFiles</td>
   <td>false</td>
   <td>
-    If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is reccomended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations.
+    If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is recomended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations.
   </td>
 </tr>
 <tr>


[04/13] Merge pull request #199 from harveyfeng/yarn-2.2

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 bb73f6d..79dd038 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) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
----------------------------------------------------------------------
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 a4d6e1d..6a90cc5 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
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2d3eae24/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------
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 507a074..f15f3c7 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 ...


[03/13] git commit: Merge pull request #191 from hsaputra/removesemicolonscala

Posted by pw...@apache.org.
Merge pull request #191 from hsaputra/removesemicolonscala

Cleanup to remove semicolons (;) from Scala code

-) The main reason for this PR is to remove semicolons from single statements of Scala code.
-) Remove unused imports as I see them
-) Fix ASF comment header from some of files (bad copy paste I suppose)
(cherry picked from commit 4b895013cc965b37d44fd255656da470a3d2c222)

Conflicts:

	examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala

Squash into 191


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

Branch: refs/heads/branch-0.8
Commit: 20d1f8b4551b5d7e4df37248d3183131119cbd22
Parents: 2b76315
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Wed Nov 20 10:36:10 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 01:15:09 2013 -0800

----------------------------------------------------------------------
 .../spark/deploy/FaultToleranceTest.scala       | 28 +++++++++----------
 .../spark/network/netty/ShuffleCopier.scala     |  2 +-
 .../org/apache/spark/rdd/CartesianRDD.scala     |  2 +-
 .../org/apache/spark/ui/jobs/StagePage.scala    |  2 +-
 .../org/apache/spark/LocalSparkContext.scala    |  2 +-
 .../org/apache/spark/PartitioningSuite.scala    | 10 +++----
 .../org/apache/spark/examples/LocalALS.scala    |  2 +-
 .../org/apache/spark/examples/SparkTC.scala     |  2 +-
 .../streaming/examples/ActorWordCount.scala     |  2 +-
 .../org/apache/spark/streaming/Checkpoint.scala |  6 ++--
 .../api/java/JavaStreamingContext.scala         |  7 ++---
 .../streaming/dstream/FlumeInputDStream.scala   |  4 +--
 .../spark/streaming/InputStreamsSuite.scala     |  4 +--
 .../apache/spark/streaming/TestSuiteBase.scala  |  2 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |  6 ++--
 .../org/apache/spark/deploy/yarn/Client.scala   | 29 +++++++++-----------
 .../yarn/ClientDistributedCacheManager.scala    |  4 +--
 .../spark/deploy/yarn/WorkerRunnable.scala      | 13 ++++-----
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala |  5 +---
 19 files changed, 64 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
index 668032a..0aa8852 100644
--- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
@@ -1,19 +1,19 @@
 /*
  *
- *  * 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.
+ *  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.
  *
  */
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
index 481ff8c..b1e1576 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
@@ -76,7 +76,7 @@ private[spark] object ShuffleCopier extends Logging {
     extends FileClientHandler with Logging {
 
     override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) {
-      logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)");
+      logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)")
       resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen))
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
index 9b0c882..0de22f0 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
@@ -70,7 +70,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest](
   override def compute(split: Partition, context: TaskContext) = {
     val currSplit = split.asInstanceOf[CartesianPartition]
     for (x <- rdd1.iterator(currSplit.s1, context);
-      y <- rdd2.iterator(currSplit.s2, context)) yield (x, y)
+         y <- rdd2.iterator(currSplit.s2, context)) yield (x, y)
   }
 
   override def getDependencies: Seq[Dependency[_]] = List(

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index c862571..baccc42 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -133,7 +133,7 @@ private[spark] class StagePage(parent: JobProgressUI) {
         summary ++
         <h4>Summary Metrics for {numCompleted} Completed Tasks</h4> ++
         <div>{summaryTable.getOrElse("No tasks have reported metrics yet.")}</div> ++
-        <h4>Tasks</h4> ++ taskTable;
+        <h4>Tasks</h4> ++ taskTable
 
       headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages)
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
index 6ec124d..03f7c0b 100644
--- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
@@ -30,7 +30,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self
   @transient var sc: SparkContext = _
 
   override def beforeAll() {
-    InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory());
+    InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory())
     super.beforeAll()
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
index 7d93891..1374d01 100644
--- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
+++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
@@ -142,11 +142,11 @@ class PartitioningSuite extends FunSuite with SharedSparkContext {
         .filter(_ >= 0.0)
 
     // Run the partitions, including the consecutive empty ones, through StatCounter
-    val stats: StatCounter = rdd.stats();
-    assert(abs(6.0 - stats.sum) < 0.01);
-    assert(abs(6.0/2 - rdd.mean) < 0.01);
-    assert(abs(1.0 - rdd.variance) < 0.01);
-    assert(abs(1.0 - rdd.stdev) < 0.01);
+    val stats: StatCounter = rdd.stats()
+    assert(abs(6.0 - stats.sum) < 0.01)
+    assert(abs(6.0/2 - rdd.mean) < 0.01)
+    assert(abs(1.0 - rdd.variance) < 0.01)
+    assert(abs(1.0 - rdd.stdev) < 0.01)
 
     // Add other tests here for classes that should be able to handle empty partitions correctly
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
index 4af45b2..83db8b9 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
@@ -120,7 +120,7 @@ object LocalALS {
         System.exit(1)
       }
     }
-    printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS);
+    printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS)
     
     val R = generateR()
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
index 5a7a9d1..8543ce0 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
@@ -65,7 +65,7 @@ object SparkTC {
       oldCount = nextCount
       // Perform the join, obtaining an RDD of (y, (z, x)) pairs,
       // then project the result to obtain the new (x, z) paths.
-      tc = tc.union(tc.join(edges).map(x => (x._2._2, x._2._1))).distinct().cache();
+      tc = tc.union(tc.join(edges).map(x => (x._2._2, x._2._1))).distinct().cache()
       nextCount = tc.count()
     } while (nextCount != oldCount)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
index cd3423a..af52b7e 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
@@ -120,7 +120,7 @@ object FeederActor {
 
     println("Feeder started as:" + feeder)
 
-    actorSystem.awaitTermination();
+    actorSystem.awaitTermination()
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index 2d8f072..7406986 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -92,7 +92,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
           fs.delete(file, false)
           fs.rename(writeFile, file)
 
-          val finishTime = System.currentTimeMillis();
+          val finishTime = System.currentTimeMillis()
           logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file +
             "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds")
           return
@@ -122,7 +122,9 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
 
   def stop() {
     synchronized {
-      if (stopped) return ;
+      if (stopped) {
+        return
+      }
       stopped = true
     }
     executor.shutdown()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index cfc1c26..4f6d479 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.streaming.api.java
 
-import java.lang.{Long => JLong, Integer => JInt}
+import java.lang.{Integer => JInt}
 import java.io.InputStream
 import java.util.{Map => JMap, List => JList}
 
@@ -33,10 +33,9 @@ import twitter4j.auth.Authorization
 import org.apache.spark.rdd.RDD
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
-import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaSparkContext, JavaRDD}
+import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD}
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.dstream._
-import org.apache.spark.streaming.receivers.{ActorReceiver, ReceiverSupervisorStrategy}
 
 /**
  * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -302,7 +301,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
     implicit val cmf: ClassManifest[F] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[F]]
-    ssc.fileStream[K, V, F](directory);
+    ssc.fileStream[K, V, F](directory)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
index 18de772..a0189ec 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
@@ -137,8 +137,8 @@ class FlumeReceiver(
 
   protected override def onStart() {
     val responder = new SpecificResponder(
-      classOf[AvroSourceProtocol], new FlumeEventServer(this));
-    val server = new NettyServer(responder, new InetSocketAddress(host, port));
+      classOf[AvroSourceProtocol], new FlumeEventServer(this))
+    val server = new NettyServer(responder, new InetSocketAddress(host, port))
     blockGenerator.start()
     server.start()
     logInfo("Flume receiver started")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index ca2da68..f398263 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -124,9 +124,9 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
     val input = Seq(1, 2, 3, 4, 5)
     Thread.sleep(1000)
-    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort));
+    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
     val client = SpecificRequestor.getClient(
-      classOf[AvroSourceProtocol], transceiver);
+      classOf[AvroSourceProtocol], transceiver)
 
     for (i <- 0 until input.size) {
       val event = new AvroFlumeEvent

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index be14069..8c8c359 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -251,7 +251,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
 
       Thread.sleep(500) // Give some time for the forgetting old RDDs to complete
     } catch {
-      case e: Exception => e.printStackTrace(); throw e;
+      case e: Exception => {e.printStackTrace(); throw e}
     } finally {
       ssc.stop()
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
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 89b0041..a7baf0c 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
@@ -21,6 +21,7 @@ import java.io.IOException
 import java.net.Socket
 import java.util.concurrent.CopyOnWriteArrayList
 import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
+
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.hadoop.net.NetUtils
@@ -33,6 +34,7 @@ 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 {
@@ -65,7 +67,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
     
     appAttemptId = getApplicationAttemptId()
-    isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts;
+    isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts
     resourceManager = registerWithResourceManager()
 
     // Workaround until hadoop moves to something which has
@@ -195,7 +197,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
           successed = true
         } finally {
           logDebug("finishing main")
-          isLastAMRetry = true;
+          isLastAMRetry = true
           if (successed) {
             ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
           } else {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
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 1078d5b..94e353a 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
@@ -17,14 +17,13 @@
 
 package org.apache.spark.deploy.yarn
 
-import java.net.{InetAddress, InetSocketAddress, UnknownHostException, URI}
+import java.net.{InetAddress, UnknownHostException, URI}
 import java.nio.ByteBuffer
 
 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.net.NetUtils
 import org.apache.hadoop.io.DataOutputBuffer
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.yarn.api._
@@ -40,9 +39,7 @@ 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 {
   
@@ -123,7 +120,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     
     // 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);
+      logError("the worker size is to large to run on this cluster " + args.workerMemory)
       System.exit(1)
     }
     val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
@@ -160,8 +157,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     var dstHost = dstUri.getHost()
     if ((srcHost != null) && (dstHost != null)) {
       try {
-        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName();
-        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName();
+        srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
+        dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
       } catch {
         case e: UnknownHostException =>
           return false
@@ -178,7 +175,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     if (srcUri.getPort() != dstUri.getPort()) {
       return false
     }
-    return true;
+    return true
   }
 
   /**
@@ -190,13 +187,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       replication: Short,
       setPerms: Boolean = false): Path = {
     val fs = FileSystem.get(conf)
-    val remoteFs = originalPath.getFileSystem(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);
+      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
@@ -214,7 +211,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     // Add them as local resources to the AM
     val fs = FileSystem.get(conf)
 
-    val delegTokenRenewer = Master.getMasterPrincipal(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")
@@ -226,7 +223,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
 
     if (UserGroupInformation.isSecurityEnabled()) {
       val dstFs = dst.getFileSystem(conf)
-      dstFs.addDelegationTokens(delegTokenRenewer, credentials);
+      dstFs.addDelegationTokens(delegTokenRenewer, credentials)
     }
     val localResources = HashMap[String, LocalResource]()
     FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION))
@@ -286,7 +283,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
       }
     }
 
-    UserGroupInformation.getCurrentUser().addCredentials(credentials);
+    UserGroupInformation.getCurrentUser().addCredentials(credentials)
     return localResources
   }
   
@@ -366,7 +363,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
     }
 
     // Command for the ApplicationMaster
-    var javaCommand = "java";
+    var javaCommand = "java"
     val javaHome = System.getenv("JAVA_HOME")
     if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
       javaCommand = Environment.JAVA_HOME.$() + "/bin/java"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
index 674c8f8..5f159b0 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
@@ -197,11 +197,11 @@ class ClientDistributedCacheManager() extends Logging {
    */
   def checkPermissionOfOther(fs: FileSystem, path: Path,
       action: FsAction, statCache: Map[URI, FileStatus]): Boolean = {
-    val status = getFileStatus(fs, path.toUri(), statCache);
+    val status = getFileStatus(fs, path.toUri(), statCache)
     val perms = status.getPermission()
     val otherAction = perms.getOtherAction()
     if (otherAction.implies(action)) {
-      return true;
+      return true
     }
     return false
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
----------------------------------------------------------------------
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 7a66532..a4d6e1d 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
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer
 import java.security.PrivilegedExceptionAction
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
+import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.DataOutputBuffer
 import org.apache.hadoop.net.NetUtils
 import org.apache.hadoop.security.UserGroupInformation
@@ -38,7 +38,6 @@ import scala.collection.JavaConversions._
 import scala.collection.mutable.HashMap
 
 import org.apache.spark.Logging
-import org.apache.spark.util.Utils
 
 class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String,
     slaveId: String, hostname: String, workerMemory: Int, workerCores: Int) 
@@ -108,7 +107,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
     credentials.writeTokenStorageToStream(dob)
     ctx.setContainerTokens(ByteBuffer.wrap(dob.getData()))
 
-    var javaCommand = "java";
+    var javaCommand = "java"
     val javaHome = System.getenv("JAVA_HOME")
     if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
       javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
@@ -204,8 +203,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
 
     // 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();
+    val user = UserGroupInformation.createRemoteUser(container.getId().toString())
+    val containerToken = container.getContainerToken()
     if (containerToken != null) {
       user.addToken(ProtoUtils.convertFromProtoFormat(containerToken, cmAddress))
     }
@@ -216,8 +215,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
             return rpc.getProxy(classOf[ContainerManager],
                 cmAddress, conf).asInstanceOf[ContainerManager]
           }
-        });
-    return proxy;
+        })
+    proxy
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20d1f8b4/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
----------------------------------------------------------------------
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
index ca2f1e2..2ba2366 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -18,13 +18,10 @@
 package org.apache.spark.deploy.yarn
 
 import org.apache.spark.deploy.SparkHadoopUtil
-import collection.mutable.HashMap
 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
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import java.security.PrivilegedExceptionAction
 
 /**
  * Contains util methods to interact with Hadoop from spark.
@@ -40,7 +37,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil {
 
   // 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();
+    val jobCreds = conf.getCredentials()
     jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials())
   }
 }


[11/13] git commit: Merge pull request #236 from pwendell/shuffle-docs

Posted by pw...@apache.org.
Merge pull request #236 from pwendell/shuffle-docs

Adding disclaimer for shuffle file consolidation
(cherry picked from commit 1b38f5f2774982d524742e987b6cef26ccaae676)

Signed-off-by: Patrick Wendell <pw...@gmail.com>


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

Branch: refs/heads/branch-0.8
Commit: cfca70e4b285a321e9a4543f3965430e6d72b2fd
Parents: 80cc4ff
Author: Patrick Wendell <pw...@gmail.com>
Authored: Fri Dec 6 20:16:15 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Dec 7 01:15:20 2013 -0800

----------------------------------------------------------------------
 docs/configuration.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cfca70e4/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 0edbac2..e86b9ea 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -331,7 +331,7 @@ Apart from these, the following properties are also available, and may be useful
   <td>spark.shuffle.consolidateFiles</td>
   <td>false</td>
   <td>
-    If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance if you run shuffles with large numbers of reduce tasks.
+    If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is reccomended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations.
   </td>
 </tr>
 <tr>